From 8005a2ebac1afbec6fcf43a4442f51f442f33590 Mon Sep 17 00:00:00 2001 From: Lakshmi Gururaja Rao Date: Tue, 10 Jul 2018 11:40:02 -0700 Subject: [PATCH 01/65] [FLINK-9692] [kinesis] Adaptive reads from Kinesis --- .../config/ConsumerConfigConstants.java | 6 + .../kinesis/internals/ShardConsumer.java | 38 ++++++- .../kinesis/internals/ShardConsumerTest.java | 45 ++++++++ .../FakeKinesisBehavioursFactory.java | 107 ++++++++++++++++++ 4 files changed, 195 insertions(+), 1 deletion(-) diff --git a/flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/config/ConsumerConfigConstants.java b/flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/config/ConsumerConfigConstants.java index e46f79e9b73a8..bcbc284853172 100644 --- a/flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/config/ConsumerConfigConstants.java +++ b/flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/config/ConsumerConfigConstants.java @@ -134,6 +134,10 @@ public SentinelSequenceNumber toSentinelSequenceNumber() { /** The interval between each attempt to discover new shards. */ public static final String SHARD_DISCOVERY_INTERVAL_MILLIS = "flink.shard.discovery.intervalmillis"; + /** The config to turn on adaptive reads from a shard. */ + public static final String SHARD_USE_ADAPTIVE_READS = "flink.shard.adaptive.read.records.enabled"; + + // ------------------------------------------------------------------------ // Default values for consumer configuration // ------------------------------------------------------------------------ @@ -179,6 +183,8 @@ public SentinelSequenceNumber toSentinelSequenceNumber() { public static final long DEFAULT_SHARD_DISCOVERY_INTERVAL_MILLIS = 10000L; + public static final boolean DEFAULT_SHARD_USE_ADAPTIVE_READS = false; + /** * To avoid shard iterator expires in {@link ShardConsumer}s, the value for the configured * getRecords interval can not exceed 5 minutes, which is the expire time for retrieved iterators. diff --git a/flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/internals/ShardConsumer.java b/flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/internals/ShardConsumer.java index 30f00163ca621..77d180cc3958e 100644 --- a/flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/internals/ShardConsumer.java +++ b/flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/internals/ShardConsumer.java @@ -56,6 +56,10 @@ public class ShardConsumer implements Runnable { private static final Logger LOG = LoggerFactory.getLogger(ShardConsumer.class); + // AWS Kinesis has a read limit of 2 Mb/sec + // https://docs.aws.amazon.com/kinesis/latest/APIReference/API_GetRecords.html + private static final long KINESIS_SHARD_BYTES_PER_SECOND_LIMIT = 2 * 1024L * 1024L; + private final KinesisDeserializationSchema deserializer; private final KinesisProxyInterface kinesis; @@ -66,8 +70,9 @@ public class ShardConsumer implements Runnable { private final StreamShardHandle subscribedShard; - private final int maxNumberOfRecordsPerFetch; + private int maxNumberOfRecordsPerFetch; private final long fetchIntervalMillis; + private final boolean useAdaptiveReads; private final ShardMetricsReporter shardMetricsReporter; @@ -125,6 +130,9 @@ protected ShardConsumer(KinesisDataFetcher fetcherRef, this.fetchIntervalMillis = Long.valueOf(consumerConfig.getProperty( ConsumerConfigConstants.SHARD_GETRECORDS_INTERVAL_MILLIS, Long.toString(ConsumerConfigConstants.DEFAULT_SHARD_GETRECORDS_INTERVAL_MILLIS))); + this.useAdaptiveReads = Boolean.valueOf(consumerConfig.getProperty( + ConsumerConfigConstants.SHARD_USE_ADAPTIVE_READS, + Boolean.toString(ConsumerConfigConstants.DEFAULT_SHARD_USE_ADAPTIVE_READS))); if (lastSequenceNum.equals(SentinelSequenceNumber.SENTINEL_AT_TIMESTAMP_SEQUENCE_NUM.get())) { String timestamp = consumerConfig.getProperty(ConsumerConfigConstants.STREAM_INITIAL_TIMESTAMP); @@ -224,10 +232,19 @@ public void run() { subscribedShard.getShard().getHashKeyRange().getStartingHashKey(), subscribedShard.getShard().getHashKeyRange().getEndingHashKey()); + long recordBatchSizeBytes = 0L; + long averageRecordSizeBytes = 0L; + for (UserRecord record : fetchedRecords) { + recordBatchSizeBytes += record.getData().remaining(); deserializeRecordForCollectionAndUpdateState(record); } + if (useAdaptiveReads && !fetchedRecords.isEmpty()) { + averageRecordSizeBytes = recordBatchSizeBytes / fetchedRecords.size(); + maxNumberOfRecordsPerFetch = getAdaptiveMaxRecordsPerFetch(averageRecordSizeBytes); + } + nextShardItr = getRecordsResult.getNextShardIterator(); } } @@ -330,4 +347,23 @@ private GetRecordsResult getRecords(String shardItr, int maxNumberOfRecords) thr protected static List deaggregateRecords(List records, String startingHashKey, String endingHashKey) { return UserRecord.deaggregate(records, new BigInteger(startingHashKey), new BigInteger(endingHashKey)); } + + /** + * Adapts the maxNumberOfRecordsPerFetch based on the current average record size + * to optimize 2 Mb / sec read limits. + * + * @param averageRecordSizeBytes + * @return adaptedMaxRecordsPerFetch + */ + + protected int getAdaptiveMaxRecordsPerFetch(long averageRecordSizeBytes) { + int adaptedMaxRecordsPerFetch = maxNumberOfRecordsPerFetch; + if (averageRecordSizeBytes != 0 && fetchIntervalMillis != 0) { + adaptedMaxRecordsPerFetch = (int) (KINESIS_SHARD_BYTES_PER_SECOND_LIMIT / (averageRecordSizeBytes * 1000L / fetchIntervalMillis)); + + // Ensure the value is not more than 10000L + adaptedMaxRecordsPerFetch = Math.min(adaptedMaxRecordsPerFetch, ConsumerConfigConstants.DEFAULT_SHARD_GETRECORDS_MAX); + } + return adaptedMaxRecordsPerFetch; + } } diff --git a/flink-connectors/flink-connector-kinesis/src/test/java/org/apache/flink/streaming/connectors/kinesis/internals/ShardConsumerTest.java b/flink-connectors/flink-connector-kinesis/src/test/java/org/apache/flink/streaming/connectors/kinesis/internals/ShardConsumerTest.java index 4478b2ff8e560..c2924e27035d0 100644 --- a/flink-connectors/flink-connector-kinesis/src/test/java/org/apache/flink/streaming/connectors/kinesis/internals/ShardConsumerTest.java +++ b/flink-connectors/flink-connector-kinesis/src/test/java/org/apache/flink/streaming/connectors/kinesis/internals/ShardConsumerTest.java @@ -168,6 +168,51 @@ public void testCorrectNumOfCollectedRecordsAndUpdatedStateWithUnexpectedExpired subscribedShardsStateUnderTest.get(0).getLastProcessedSequenceNum()); } + @Test + public void testCorrectNumOfCollectedRecordsAndUpdatedStateWithAdaptiveReads() { + Properties consumerProperties = new Properties(); + consumerProperties.put("flink.shard.adaptive.read.records.enabled", "true"); + + StreamShardHandle fakeToBeConsumedShard = getMockStreamShard("fakeStream", 0); + + LinkedList subscribedShardsStateUnderTest = new LinkedList<>(); + subscribedShardsStateUnderTest.add( + new KinesisStreamShardState(KinesisDataFetcher.convertToStreamShardMetadata(fakeToBeConsumedShard), + fakeToBeConsumedShard, new SequenceNumber("fakeStartingState"))); + + TestSourceContext sourceContext = new TestSourceContext<>(); + + TestableKinesisDataFetcher fetcher = + new TestableKinesisDataFetcher<>( + Collections.singletonList("fakeStream"), + sourceContext, + consumerProperties, + new KinesisDeserializationSchemaWrapper<>(new SimpleStringSchema()), + 10, + 2, + new AtomicReference<>(), + subscribedShardsStateUnderTest, + KinesisDataFetcher.createInitialSubscribedStreamsToLastDiscoveredShardsState(Collections.singletonList("fakeStream")), + Mockito.mock(KinesisProxyInterface.class)); + + new ShardConsumer<>( + fetcher, + 0, + subscribedShardsStateUnderTest.get(0).getStreamShardHandle(), + subscribedShardsStateUnderTest.get(0).getLastProcessedSequenceNum(), + // Initial number of records to fetch --> 10 + FakeKinesisBehavioursFactory.initialNumOfRecordsAfterNumOfGetRecordsCallsWithAdaptiveReads(10, 2, 500L), + new ShardMetricsReporter()).run(); + + // Avg record size for first batch --> 10 * 10 Kb/10 = 10 Kb + // Number of records fetched in second batch --> 2 Mb/10Kb * 5 = 40 + // Total number of records = 10 + 40 = 50 + assertEquals(50, sourceContext.getCollectedOutputs().size()); + assertEquals( + SentinelSequenceNumber.SENTINEL_SHARD_ENDING_SEQUENCE_NUM.get(), + subscribedShardsStateUnderTest.get(0).getLastProcessedSequenceNum()); + } + private static StreamShardHandle getMockStreamShard(String streamName, int shardId) { return new StreamShardHandle( streamName, diff --git a/flink-connectors/flink-connector-kinesis/src/test/java/org/apache/flink/streaming/connectors/kinesis/testutils/FakeKinesisBehavioursFactory.java b/flink-connectors/flink-connector-kinesis/src/test/java/org/apache/flink/streaming/connectors/kinesis/testutils/FakeKinesisBehavioursFactory.java index e40362337204a..eb3415572c0f3 100644 --- a/flink-connectors/flink-connector-kinesis/src/test/java/org/apache/flink/streaming/connectors/kinesis/testutils/FakeKinesisBehavioursFactory.java +++ b/flink-connectors/flink-connector-kinesis/src/test/java/org/apache/flink/streaming/connectors/kinesis/testutils/FakeKinesisBehavioursFactory.java @@ -18,6 +18,7 @@ package org.apache.flink.streaming.connectors.kinesis.testutils; import org.apache.flink.configuration.ConfigConstants; +import org.apache.flink.streaming.connectors.kinesis.config.ConsumerConfigConstants; import org.apache.flink.streaming.connectors.kinesis.model.StreamShardHandle; import org.apache.flink.streaming.connectors.kinesis.proxy.GetShardListResult; import org.apache.flink.streaming.connectors.kinesis.proxy.KinesisProxyInterface; @@ -93,6 +94,14 @@ public static KinesisProxyInterface totalNumOfRecordsAfterNumOfGetRecordsCallsWi numOfRecords, numOfGetRecordsCall, orderOfCallToExpire, millisBehindLatest); } + public static KinesisProxyInterface initialNumOfRecordsAfterNumOfGetRecordsCallsWithAdaptiveReads( + final int numOfRecords, + final int numOfGetRecordsCalls, + final long millisBehindLatest) { + return new SingleShardEmittingAdaptiveNumOfRecordsKinesis(numOfRecords, numOfGetRecordsCalls, + millisBehindLatest); + } + private static class SingleShardEmittingFixNumOfRecordsWithExpiredIteratorKinesis extends SingleShardEmittingFixNumOfRecordsKinesis { private final long millisBehindLatest; @@ -227,6 +236,104 @@ public static List createRecordBatchWithRange(int min, int max) { } + private static class SingleShardEmittingAdaptiveNumOfRecordsKinesis implements + KinesisProxyInterface { + + protected final int totalNumOfGetRecordsCalls; + + protected final int totalNumOfRecords; + + private final long millisBehindLatest; + + protected final Map> shardItrToRecordBatch; + + protected static long averageRecordSizeBytes; + + private static final long KINESIS_SHARD_BYTES_PER_SECOND_LIMIT = 2 * 1024L * 1024L; + + public SingleShardEmittingAdaptiveNumOfRecordsKinesis(final int numOfRecords, + final int numOfGetRecordsCalls, + final long millisBehindLatest) { + this.totalNumOfRecords = numOfRecords; + this.totalNumOfGetRecordsCalls = numOfGetRecordsCalls; + this.millisBehindLatest = millisBehindLatest; + this.averageRecordSizeBytes = 0L; + + // initialize the record batches that we will be fetched + this.shardItrToRecordBatch = new HashMap<>(); + + int numOfAlreadyPartitionedRecords = 0; + int numOfRecordsPerBatch = numOfRecords; + for (int batch = 0; batch < totalNumOfGetRecordsCalls; batch++) { + shardItrToRecordBatch.put( + String.valueOf(batch), + createRecordBatchWithRange( + numOfAlreadyPartitionedRecords, + numOfAlreadyPartitionedRecords + numOfRecordsPerBatch)); + numOfAlreadyPartitionedRecords += numOfRecordsPerBatch; + + numOfRecordsPerBatch = (int) (KINESIS_SHARD_BYTES_PER_SECOND_LIMIT / + (averageRecordSizeBytes * 1000L / ConsumerConfigConstants.DEFAULT_SHARD_GETRECORDS_INTERVAL_MILLIS)); + } + } + + @Override + public GetRecordsResult getRecords(String shardIterator, int maxRecordsToGet) { + // assuming that the maxRecordsToGet is always large enough + return new GetRecordsResult() + .withRecords(shardItrToRecordBatch.get(shardIterator)) + .withMillisBehindLatest(millisBehindLatest) + .withNextShardIterator( + (Integer.valueOf(shardIterator) == totalNumOfGetRecordsCalls - 1) + ? null : String + .valueOf(Integer.valueOf(shardIterator) + 1)); // last next shard iterator is null + } + + @Override + public String getShardIterator(StreamShardHandle shard, String shardIteratorType, + Object startingMarker) { + // this will be called only one time per ShardConsumer; + // so, simply return the iterator of the first batch of records + return "0"; + } + + @Override + public GetShardListResult getShardList(Map streamNamesWithLastSeenShardIds) { + return null; + } + + public static List createRecordBatchWithRange(int min, int max) { + List batch = new LinkedList<>(); + long sumRecordBatchBytes = 0L; + // Create record of size 10Kb + String data = createDataSize(10 * 1024L); + + for (int i = min; i < max; i++) { + Record record = new Record() + .withData( + ByteBuffer.wrap(String.valueOf(data).getBytes(ConfigConstants.DEFAULT_CHARSET))) + .withPartitionKey(UUID.randomUUID().toString()) + .withApproximateArrivalTimestamp(new Date(System.currentTimeMillis())) + .withSequenceNumber(String.valueOf(i)); + batch.add(record); + sumRecordBatchBytes += record.getData().remaining(); + + } + if (batch.size() != 0) { + averageRecordSizeBytes = sumRecordBatchBytes / batch.size(); + } + + return batch; + } + + private static String createDataSize(long msgSize) { + char[] data = new char[(int) msgSize]; + return new String(data); + + } + + } + private static class NonReshardedStreamsKinesis implements KinesisProxyInterface { private Map> streamsWithListOfShards = new HashMap<>(); From c3d3ff37de47e9302ba2a7397bef7933fab82f81 Mon Sep 17 00:00:00 2001 From: Stephan Ewen Date: Tue, 17 Jul 2018 10:16:50 +0200 Subject: [PATCH 02/65] [FLINK-9692] [kinesis] Harmonize style of config variable names --- .../connectors/kinesis/config/ConsumerConfigConstants.java | 2 +- .../connectors/kinesis/internals/ShardConsumerTest.java | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/config/ConsumerConfigConstants.java b/flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/config/ConsumerConfigConstants.java index bcbc284853172..48a0b3c955937 100644 --- a/flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/config/ConsumerConfigConstants.java +++ b/flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/config/ConsumerConfigConstants.java @@ -135,7 +135,7 @@ public SentinelSequenceNumber toSentinelSequenceNumber() { public static final String SHARD_DISCOVERY_INTERVAL_MILLIS = "flink.shard.discovery.intervalmillis"; /** The config to turn on adaptive reads from a shard. */ - public static final String SHARD_USE_ADAPTIVE_READS = "flink.shard.adaptive.read.records.enabled"; + public static final String SHARD_USE_ADAPTIVE_READS = "flink.shard.adaptivereads"; // ------------------------------------------------------------------------ diff --git a/flink-connectors/flink-connector-kinesis/src/test/java/org/apache/flink/streaming/connectors/kinesis/internals/ShardConsumerTest.java b/flink-connectors/flink-connector-kinesis/src/test/java/org/apache/flink/streaming/connectors/kinesis/internals/ShardConsumerTest.java index c2924e27035d0..dbc71182b0443 100644 --- a/flink-connectors/flink-connector-kinesis/src/test/java/org/apache/flink/streaming/connectors/kinesis/internals/ShardConsumerTest.java +++ b/flink-connectors/flink-connector-kinesis/src/test/java/org/apache/flink/streaming/connectors/kinesis/internals/ShardConsumerTest.java @@ -171,7 +171,7 @@ public void testCorrectNumOfCollectedRecordsAndUpdatedStateWithUnexpectedExpired @Test public void testCorrectNumOfCollectedRecordsAndUpdatedStateWithAdaptiveReads() { Properties consumerProperties = new Properties(); - consumerProperties.put("flink.shard.adaptive.read.records.enabled", "true"); + consumerProperties.put("flink.shard.adaptivereads", "true"); StreamShardHandle fakeToBeConsumedShard = getMockStreamShard("fakeStream", 0); From ffeb978260c0ce770911183ff042e502724a3fd3 Mon Sep 17 00:00:00 2001 From: Aljoscha Krettek Date: Mon, 16 Jul 2018 15:58:07 +0200 Subject: [PATCH 03/65] [FLINK-9857] Delay firing of processing-time timers by 1 ms --- .../runtime/tasks/SystemProcessingTimeService.java | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/SystemProcessingTimeService.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/SystemProcessingTimeService.java index 4e4208f390f9b..987ddd9155271 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/SystemProcessingTimeService.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/SystemProcessingTimeService.java @@ -109,7 +109,11 @@ public long getCurrentProcessingTime() { */ @Override public ScheduledFuture registerTimer(long timestamp, ProcessingTimeCallback target) { - long delay = Math.max(timestamp - getCurrentProcessingTime(), 0); + + // delay the firing of the timer by 1 ms to align the semantics with watermark. A watermark + // T says we won't see elements in the future with a timestamp smaller or equal to T. + // With processing time, we therefore need to delay firing the timer by one ms. + long delay = Math.max(timestamp - getCurrentProcessingTime(), 0) + 1; // we directly try to register the timer and only react to the status on exception // that way we save unnecessary volatile accesses for each timer From 0785edecb3a1c6f32a71fe4c49923381687db66f Mon Sep 17 00:00:00 2001 From: kkloudas Date: Tue, 17 Jul 2018 23:03:57 +0200 Subject: [PATCH 04/65] [FLINK-9880] Fix order in BucketerContext#update. --- .../api/functions/sink/filesystem/StreamingFileSink.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/sink/filesystem/StreamingFileSink.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/sink/filesystem/StreamingFileSink.java index b6fff03b1e799..0406afcf78be3 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/sink/filesystem/StreamingFileSink.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/sink/filesystem/StreamingFileSink.java @@ -332,7 +332,7 @@ public void invoke(IN value, Context context) throws Exception { final int subtaskIndex = getRuntimeContext().getIndexOfThisSubtask(); // setting the values in the bucketer context - bucketerContext.update(context.timestamp(), currentProcessingTime, context.currentWatermark()); + bucketerContext.update(context.timestamp(), context.currentWatermark(), currentProcessingTime); final String bucketId = bucketer.getBucketId(value, bucketerContext); From 343614cd8e73f2d10ba5cb99759b022dd829d03f Mon Sep 17 00:00:00 2001 From: yanghua Date: Tue, 10 Jul 2018 23:16:12 +0800 Subject: [PATCH 05/65] [FLINK-9777] YARN: JM and TM Memory must be specified with Units This closes #6297 --- docs/dev/scala_shell.md | 4 +- docs/ops/cli.md | 8 +- docs/ops/deployment/yarn_setup.md | 8 +- .../deployment/ClusterSpecification.java | 7 +- .../configuration/ConfigurationUtils.java | 38 +++++++ .../flink/yarn/YarnResourceManager.java | 4 +- .../flink/yarn/cli/FlinkYarnSessionCli.java | 21 ++-- .../flink/yarn/FlinkYarnSessionCliTest.java | 99 +++++++++++++++++++ 8 files changed, 167 insertions(+), 22 deletions(-) diff --git a/docs/dev/scala_shell.md b/docs/dev/scala_shell.md index b8d2b2c3b9baf..d236430ad57f6 100644 --- a/docs/dev/scala_shell.md +++ b/docs/dev/scala_shell.md @@ -175,7 +175,7 @@ Starts Flink scala shell connecting to a yarn cluster -n arg | --container arg Number of YARN container to allocate (= Number of TaskManagers) -jm arg | --jobManagerMemory arg - Memory for JobManager container [in MB] + Memory for JobManager container with optional unit (default: MB) -nm | --name Set a custom name for the application on YARN -qu | --queue @@ -183,7 +183,7 @@ Starts Flink scala shell connecting to a yarn cluster -s | --slots Number of slots per TaskManager -tm | --taskManagerMemory - Memory per TaskManager container [in MB] + Memory per TaskManager container with optional unit (default: MB) -a | --addclasspath Specifies additional jars to be used in Flink --configDir diff --git a/docs/ops/cli.md b/docs/ops/cli.md index 6974a2d211eab..f96ccf52ea9e2 100644 --- a/docs/ops/cli.md +++ b/docs/ops/cli.md @@ -272,8 +272,8 @@ Action "run" compiles and runs a program. -yh,--yarnhelp Help for the Yarn session CLI. -yid,--yarnapplicationId Attach to running YARN session -yj,--yarnjar Path to Flink jar file - -yjm,--yarnjobManagerMemory Memory for JobManager Container [in - MB] + -yjm,--yarnjobManagerMemory Memory for JobManager Container + with optional unit (default: MB) -yn,--yarncontainer Number of YARN container to allocate (=Number of Task Managers) -ynm,--yarnname Set a custom name for the application @@ -285,8 +285,8 @@ Action "run" compiles and runs a program. -yst,--yarnstreaming Start Flink in streaming mode -yt,--yarnship Ship files in the specified directory (t for transfer) - -ytm,--yarntaskManagerMemory Memory per TaskManager Container [in - MB] + -ytm,--yarntaskManagerMemory Memory per TaskManager Container + with optional unit (default: MB) -yz,--yarnzookeeperNamespace Namespace to create the Zookeeper sub-paths for high availability mode -ynl,--yarnnodeLabel Specify YARN node label for diff --git a/docs/ops/deployment/yarn_setup.md b/docs/ops/deployment/yarn_setup.md index d2fdad9460c2b..9c255d27b9299 100644 --- a/docs/ops/deployment/yarn_setup.md +++ b/docs/ops/deployment/yarn_setup.md @@ -38,7 +38,7 @@ Start a YARN session with 4 Task Managers (each with 4 GB of Heapspace): curl -O tar xvzf flink-{{ site.version }}-bin-hadoop2.tgz cd flink-{{ site.version }}/ -./bin/yarn-session.sh -n 4 -jm 1024 -tm 4096 +./bin/yarn-session.sh -n 4 -jm 1024m -tm 4096m {% endhighlight %} Specify the `-s` flag for the number of processing slots per Task Manager. We recommend to set the number of slots to the number of processors per machine. @@ -53,7 +53,7 @@ Once the session has been started, you can submit jobs to the cluster using the curl -O tar xvzf flink-{{ site.version }}-bin-hadoop2.tgz cd flink-{{ site.version }}/ -./bin/flink run -m yarn-cluster -yn 4 -yjm 1024 -ytm 4096 ./examples/batch/WordCount.jar +./bin/flink run -m yarn-cluster -yn 4 -yjm 1024m -ytm 4096m ./examples/batch/WordCount.jar {% endhighlight %} ## Flink YARN Session @@ -101,12 +101,12 @@ Usage: Optional -D Dynamic properties -d,--detached Start detached - -jm,--jobManagerMemory Memory for JobManager Container [in MB] + -jm,--jobManagerMemory Memory for JobManager Container with optional unit (default: MB) -nm,--name Set a custom name for the application on YARN -q,--query Display available YARN resources (memory, cores) -qu,--queue Specify YARN queue. -s,--slots Number of slots per TaskManager - -tm,--taskManagerMemory Memory per TaskManager Container [in MB] + -tm,--taskManagerMemory Memory per TaskManager Container with optional unit (default: MB) -z,--zookeeperNamespace Namespace to create the Zookeeper sub-paths for HA mode {% endhighlight %} diff --git a/flink-clients/src/main/java/org/apache/flink/client/deployment/ClusterSpecification.java b/flink-clients/src/main/java/org/apache/flink/client/deployment/ClusterSpecification.java index 90de955039fe4..72975d8b9b247 100644 --- a/flink-clients/src/main/java/org/apache/flink/client/deployment/ClusterSpecification.java +++ b/flink-clients/src/main/java/org/apache/flink/client/deployment/ClusterSpecification.java @@ -19,8 +19,7 @@ package org.apache.flink.client.deployment; import org.apache.flink.configuration.Configuration; -import org.apache.flink.configuration.JobManagerOptions; -import org.apache.flink.configuration.MemorySize; +import org.apache.flink.configuration.ConfigurationUtils; import org.apache.flink.configuration.TaskManagerOptions; /** @@ -68,8 +67,8 @@ public String toString() { public static ClusterSpecification fromConfiguration(Configuration configuration) { int slots = configuration.getInteger(TaskManagerOptions.NUM_TASK_SLOTS, 1); - int jobManagerMemoryMb = MemorySize.parse(configuration.getString(JobManagerOptions.JOB_MANAGER_HEAP_MEMORY)).getMebiBytes(); - int taskManagerMemoryMb = MemorySize.parse(configuration.getString(TaskManagerOptions.TASK_MANAGER_HEAP_MEMORY)).getMebiBytes(); + int jobManagerMemoryMb = ConfigurationUtils.getJobManagerHeapMemory(configuration).getMebiBytes(); + int taskManagerMemoryMb = ConfigurationUtils.getTaskManagerHeapMemory(configuration).getMebiBytes(); return new ClusterSpecificationBuilder() .setMasterMemoryMB(jobManagerMemoryMb) diff --git a/flink-core/src/main/java/org/apache/flink/configuration/ConfigurationUtils.java b/flink-core/src/main/java/org/apache/flink/configuration/ConfigurationUtils.java index 3d1d8300d87b1..1b3082177703b 100644 --- a/flink-core/src/main/java/org/apache/flink/configuration/ConfigurationUtils.java +++ b/flink-core/src/main/java/org/apache/flink/configuration/ConfigurationUtils.java @@ -31,6 +31,44 @@ public class ConfigurationUtils { private static final String[] EMPTY = new String[0]; + /** + * Get job manager's heap memory. This method will check the new key + * {@link JobManagerOptions#JOB_MANAGER_HEAP_MEMORY} and + * the old key {@link JobManagerOptions#JOB_MANAGER_HEAP_MEMORY_MB} for backwards compatibility. + * + * @param configuration the configuration object + * @return the memory size of job manager's heap memory. + */ + public static MemorySize getJobManagerHeapMemory(Configuration configuration) { + if (configuration.containsKey(JobManagerOptions.JOB_MANAGER_HEAP_MEMORY.key())) { + return MemorySize.parse(configuration.getString(JobManagerOptions.JOB_MANAGER_HEAP_MEMORY)); + } else if (configuration.containsKey(JobManagerOptions.JOB_MANAGER_HEAP_MEMORY_MB.key())) { + return MemorySize.parse(configuration.getInteger(JobManagerOptions.JOB_MANAGER_HEAP_MEMORY_MB) + "m"); + } else { + //use default value + return MemorySize.parse(configuration.getString(JobManagerOptions.JOB_MANAGER_HEAP_MEMORY)); + } + } + + /** + * Get task manager's heap memory. This method will check the new key + * {@link TaskManagerOptions#TASK_MANAGER_HEAP_MEMORY} and + * the old key {@link TaskManagerOptions#TASK_MANAGER_HEAP_MEMORY_MB} for backwards compatibility. + * + * @param configuration the configuration object + * @return the memory size of task manager's heap memory. + */ + public static MemorySize getTaskManagerHeapMemory(Configuration configuration) { + if (configuration.containsKey(TaskManagerOptions.TASK_MANAGER_HEAP_MEMORY.key())) { + return MemorySize.parse(configuration.getString(TaskManagerOptions.TASK_MANAGER_HEAP_MEMORY)); + } else if (configuration.containsKey(TaskManagerOptions.TASK_MANAGER_HEAP_MEMORY_MB.key())) { + return MemorySize.parse(configuration.getInteger(TaskManagerOptions.TASK_MANAGER_HEAP_MEMORY_MB) + "m"); + } else { + //use default value + return MemorySize.parse(configuration.getString(TaskManagerOptions.TASK_MANAGER_HEAP_MEMORY)); + } + } + /** * Extracts the task manager directories for temporary files as defined by * {@link org.apache.flink.configuration.CoreOptions#TMP_DIRS}. 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 96ec57e0a526c..0206ffb0f9e35 100644 --- a/flink-yarn/src/main/java/org/apache/flink/yarn/YarnResourceManager.java +++ b/flink-yarn/src/main/java/org/apache/flink/yarn/YarnResourceManager.java @@ -20,7 +20,7 @@ import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.configuration.Configuration; -import org.apache.flink.configuration.MemorySize; +import org.apache.flink.configuration.ConfigurationUtils; import org.apache.flink.configuration.TaskManagerOptions; import org.apache.flink.runtime.clusterframework.ApplicationStatus; import org.apache.flink.runtime.clusterframework.ContaineredTaskManagerParameters; @@ -163,7 +163,7 @@ public YarnResourceManager( this.webInterfaceUrl = webInterfaceUrl; this.numberOfTaskSlots = flinkConfig.getInteger(TaskManagerOptions.NUM_TASK_SLOTS); - this.defaultTaskManagerMemoryMB = MemorySize.parse(flinkConfig.getString(TaskManagerOptions.TASK_MANAGER_HEAP_MEMORY)).getMebiBytes(); + this.defaultTaskManagerMemoryMB = ConfigurationUtils.getTaskManagerHeapMemory(flinkConfig).getMebiBytes(); this.defaultCpus = flinkConfig.getInteger(YarnConfigOptions.VCORES, numberOfTaskSlots); } diff --git a/flink-yarn/src/main/java/org/apache/flink/yarn/cli/FlinkYarnSessionCli.java b/flink-yarn/src/main/java/org/apache/flink/yarn/cli/FlinkYarnSessionCli.java index 1ad1bcc931bef..c0180a83be11a 100644 --- a/flink-yarn/src/main/java/org/apache/flink/yarn/cli/FlinkYarnSessionCli.java +++ b/flink-yarn/src/main/java/org/apache/flink/yarn/cli/FlinkYarnSessionCli.java @@ -24,6 +24,7 @@ import org.apache.flink.client.deployment.ClusterSpecification; import org.apache.flink.client.program.ClusterClient; import org.apache.flink.configuration.Configuration; +import org.apache.flink.configuration.ConfigurationUtils; import org.apache.flink.configuration.CoreOptions; import org.apache.flink.configuration.GlobalConfiguration; import org.apache.flink.configuration.JobManagerOptions; @@ -190,8 +191,8 @@ public FlinkYarnSessionCli( queue = new Option(shortPrefix + "qu", longPrefix + "queue", true, "Specify YARN queue."); shipPath = new Option(shortPrefix + "t", longPrefix + "ship", true, "Ship files in the specified directory (t for transfer)"); flinkJar = new Option(shortPrefix + "j", longPrefix + "jar", true, "Path to Flink jar file"); - jmMemory = new Option(shortPrefix + "jm", longPrefix + "jobManagerMemory", true, "Memory for JobManager Container [in MB]"); - tmMemory = new Option(shortPrefix + "tm", longPrefix + "taskManagerMemory", true, "Memory per TaskManager Container [in MB]"); + jmMemory = new Option(shortPrefix + "jm", longPrefix + "jobManagerMemory", true, "Memory for JobManager Container with optional unit (default: MB)"); + tmMemory = new Option(shortPrefix + "tm", longPrefix + "taskManagerMemory", true, "Memory per TaskManager Container with optional unit (default: MB)"); container = new Option(shortPrefix + "n", longPrefix + "container", true, "Number of YARN container to allocate (=Number of Task Managers)"); slots = new Option(shortPrefix + "s", longPrefix + "slots", true, "Number of slots per TaskManager"); dynamicproperties = Option.builder(shortPrefix + "D") @@ -386,10 +387,10 @@ private ClusterSpecification createClusterSpecification(Configuration configurat } // JobManager Memory - final int jobManagerMemoryMB = MemorySize.parse(configuration.getString(JobManagerOptions.JOB_MANAGER_HEAP_MEMORY)).getMebiBytes(); + final int jobManagerMemoryMB = ConfigurationUtils.getJobManagerHeapMemory(configuration).getMebiBytes(); // Task Managers memory - final int taskManagerMemoryMB = MemorySize.parse(configuration.getString(TaskManagerOptions.TASK_MANAGER_HEAP_MEMORY)).getMebiBytes(); + final int taskManagerMemoryMB = ConfigurationUtils.getTaskManagerHeapMemory(configuration).getMebiBytes(); int slotsPerTaskManager = configuration.getInteger(TaskManagerOptions.NUM_TASK_SLOTS); @@ -500,11 +501,19 @@ protected Configuration applyCommandLineOptionsToConfiguration(CommandLine comma } if (commandLine.hasOption(jmMemory.getOpt())) { - effectiveConfiguration.setString(JobManagerOptions.JOB_MANAGER_HEAP_MEMORY, commandLine.getOptionValue(jmMemory.getOpt())); + String jmMemoryVal = commandLine.getOptionValue(jmMemory.getOpt()); + if (!MemorySize.MemoryUnit.hasUnit(jmMemoryVal)) { + jmMemoryVal += "m"; + } + effectiveConfiguration.setString(JobManagerOptions.JOB_MANAGER_HEAP_MEMORY, jmMemoryVal); } if (commandLine.hasOption(tmMemory.getOpt())) { - effectiveConfiguration.setString(TaskManagerOptions.TASK_MANAGER_HEAP_MEMORY, commandLine.getOptionValue(tmMemory.getOpt())); + String tmMemoryVal = commandLine.getOptionValue(tmMemory.getOpt()); + if (!MemorySize.MemoryUnit.hasUnit(tmMemoryVal)) { + tmMemoryVal += "m"; + } + effectiveConfiguration.setString(TaskManagerOptions.TASK_MANAGER_HEAP_MEMORY, tmMemoryVal); } if (commandLine.hasOption(slots.getOpt())) { diff --git a/flink-yarn/src/test/java/org/apache/flink/yarn/FlinkYarnSessionCliTest.java b/flink-yarn/src/test/java/org/apache/flink/yarn/FlinkYarnSessionCliTest.java index 977da455a520f..aa958e2514138 100644 --- a/flink-yarn/src/test/java/org/apache/flink/yarn/FlinkYarnSessionCliTest.java +++ b/flink-yarn/src/test/java/org/apache/flink/yarn/FlinkYarnSessionCliTest.java @@ -352,6 +352,105 @@ public void testConfigurationClusterSpecification() throws Exception { assertThat(clusterSpecification.getSlotsPerTaskManager(), is(slotsPerTaskManager)); } + /** + * Tests the specifying heap memory without unit for job manager and task manager. + */ + @Test + public void testHeapMemoryPropertyWithoutUnit() throws Exception { + final String[] args = new String[] { "-yn", "2", "-yjm", "1024", "-ytm", "2048" }; + final FlinkYarnSessionCli flinkYarnSessionCli = new FlinkYarnSessionCli( + new Configuration(), + tmp.getRoot().getAbsolutePath(), + "y", + "yarn"); + + final CommandLine commandLine = flinkYarnSessionCli.parseCommandLineOptions(args, false); + + final ClusterSpecification clusterSpecification = flinkYarnSessionCli.getClusterSpecification(commandLine); + + assertThat(clusterSpecification.getMasterMemoryMB(), is(1024)); + assertThat(clusterSpecification.getTaskManagerMemoryMB(), is(2048)); + } + + /** + * Tests the specifying heap memory with unit (MB) for job manager and task manager. + */ + @Test + public void testHeapMemoryPropertyWithUnitMB() throws Exception { + final String[] args = new String[] { "-yn", "2", "-yjm", "1024m", "-ytm", "2048m" }; + final FlinkYarnSessionCli flinkYarnSessionCli = new FlinkYarnSessionCli( + new Configuration(), + tmp.getRoot().getAbsolutePath(), + "y", + "yarn"); + final CommandLine commandLine = flinkYarnSessionCli.parseCommandLineOptions(args, false); + final ClusterSpecification clusterSpecification = flinkYarnSessionCli.getClusterSpecification(commandLine); + + assertThat(clusterSpecification.getMasterMemoryMB(), is(1024)); + assertThat(clusterSpecification.getTaskManagerMemoryMB(), is(2048)); + } + + /** + * Tests the specifying heap memory with arbitrary unit for job manager and task manager. + */ + @Test + public void testHeapMemoryPropertyWithArbitraryUnit() throws Exception { + final String[] args = new String[] { "-yn", "2", "-yjm", "1g", "-ytm", "2g" }; + final FlinkYarnSessionCli flinkYarnSessionCli = new FlinkYarnSessionCli( + new Configuration(), + tmp.getRoot().getAbsolutePath(), + "y", + "yarn"); + final CommandLine commandLine = flinkYarnSessionCli.parseCommandLineOptions(args, false); + final ClusterSpecification clusterSpecification = flinkYarnSessionCli.getClusterSpecification(commandLine); + + assertThat(clusterSpecification.getMasterMemoryMB(), is(1024)); + assertThat(clusterSpecification.getTaskManagerMemoryMB(), is(2048)); + } + + /** + * Tests the specifying heap memory with old config key for job manager and task manager. + */ + @Test + public void testHeapMemoryPropertyWithOldConfigKey() throws Exception { + Configuration configuration = new Configuration(); + configuration.setInteger(JobManagerOptions.JOB_MANAGER_HEAP_MEMORY_MB, 2048); + configuration.setInteger(TaskManagerOptions.TASK_MANAGER_HEAP_MEMORY_MB, 4096); + + final FlinkYarnSessionCli flinkYarnSessionCli = new FlinkYarnSessionCli( + configuration, + tmp.getRoot().getAbsolutePath(), + "y", + "yarn"); + + final CommandLine commandLine = flinkYarnSessionCli.parseCommandLineOptions(new String[0], false); + + final ClusterSpecification clusterSpecification = flinkYarnSessionCli.getClusterSpecification(commandLine); + + assertThat(clusterSpecification.getMasterMemoryMB(), is(2048)); + assertThat(clusterSpecification.getTaskManagerMemoryMB(), is(4096)); + } + + /** + * Tests the specifying heap memory with config default value for job manager and task manager. + */ + @Test + public void testHeapMemoryPropertyWithConfigDefaultValue() throws Exception { + final FlinkYarnSessionCli flinkYarnSessionCli = new FlinkYarnSessionCli( + new Configuration(), + tmp.getRoot().getAbsolutePath(), + "y", + "yarn"); + + final CommandLine commandLine = flinkYarnSessionCli.parseCommandLineOptions(new String[0], false); + + final ClusterSpecification clusterSpecification = flinkYarnSessionCli.getClusterSpecification(commandLine); + + assertThat(clusterSpecification.getMasterMemoryMB(), is(1024)); + assertThat(clusterSpecification.getTaskManagerMemoryMB(), is(1024)); + } + + /////////// // Utils // /////////// From 7e2ac0ac11ba69d9cac6dbdad6910bceb42af77d Mon Sep 17 00:00:00 2001 From: Till Rohrmann Date: Wed, 18 Jul 2018 09:48:49 +0200 Subject: [PATCH 06/65] [hotfix] Harden JarUploadHandlerTest#testUploadJar Use lastIndexOf("_") instead of indexOf("_") in order to find the file name part --- .../flink/runtime/webmonitor/handlers/JarUploadHandlerTest.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) 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 164171df2992f..648e7fc0451f9 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 @@ -106,7 +106,7 @@ public void testUploadJar() throws Exception { assertThat(jarUploadResponseBody.getStatus(), equalTo(JarUploadResponseBody.UploadStatus.success)); final String returnedFileNameWithUUID = jarUploadResponseBody.getFilename(); assertThat(returnedFileNameWithUUID, containsString("_")); - final String returnedFileName = returnedFileNameWithUUID.substring(returnedFileNameWithUUID.indexOf("_") + 1); + final String returnedFileName = returnedFileNameWithUUID.substring(returnedFileNameWithUUID.lastIndexOf("_") + 1); assertThat(returnedFileName, equalTo(uploadedFile.getFileName().toString())); } From 4022063ae26cafb5b5d2edf03a74d26f6daff3b1 Mon Sep 17 00:00:00 2001 From: zentol Date: Wed, 11 Jul 2018 18:37:15 +0200 Subject: [PATCH 07/65] [hotfix][rest] Simplify dispatcher host retrieval Adjusted to work like the JobSubmitHandler. --- .../webmonitor/handlers/JarRunHandler.java | 17 +---------------- 1 file changed, 1 insertion(+), 16 deletions(-) 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 1e620d4799e79..858a05ca00e5e 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 @@ -35,14 +35,11 @@ import org.apache.flink.runtime.rest.handler.RestHandlerException; import org.apache.flink.runtime.rest.messages.EmptyRequestBody; import org.apache.flink.runtime.rest.messages.MessageHeaders; -import org.apache.flink.runtime.util.ScalaUtils; import org.apache.flink.runtime.webmonitor.retriever.GatewayRetriever; import org.apache.flink.util.FlinkException; import org.apache.flink.shaded.netty4.io.netty.handler.codec.http.HttpResponseStatus; -import akka.actor.AddressFromURIString; - import javax.annotation.Nonnull; import javax.annotation.Nullable; @@ -51,7 +48,6 @@ import java.nio.file.Path; import java.util.List; import java.util.Map; -import java.util.Optional; import java.util.concurrent.CompletableFuture; import java.util.concurrent.CompletionException; import java.util.concurrent.Executor; @@ -112,7 +108,7 @@ protected CompletableFuture handleRequest( CompletableFuture blobServerPortFuture = gateway.getBlobServerPort(timeout); CompletableFuture jarUploadFuture = jobGraphFuture.thenCombine(blobServerPortFuture, (jobGraph, blobServerPort) -> { - final InetSocketAddress address = new InetSocketAddress(getDispatcherHost(gateway), blobServerPort); + final InetSocketAddress address = new InetSocketAddress(gateway.getHostname(), blobServerPort); try { ClientUtils.extractAndUploadJobGraphFiles(jobGraph, () -> new BlobClient(address, configuration)); } catch (FlinkException e) { @@ -182,15 +178,4 @@ private CompletableFuture getJobGraphAsync( return jobGraph; }, executor); } - - private static String getDispatcherHost(DispatcherGateway gateway) { - String dispatcherAddress = gateway.getAddress(); - final Optional host = ScalaUtils.toJava(AddressFromURIString.parse(dispatcherAddress).host()); - - return host.orElseGet(() -> { - // if the dispatcher address does not contain a host part, then assume it's running - // on the same machine as the handler - return "localhost"; - }); - } } From 1386f91b0ad50ee4900e221581f19608d7217994 Mon Sep 17 00:00:00 2001 From: zentol Date: Wed, 11 Jul 2018 18:41:06 +0200 Subject: [PATCH 08/65] [FLINK-9811][test] Add test for jar handler interactions This closes #6311. --- flink-runtime-web/pom.xml | 28 +++ .../handlers/JarDeleteMessageParameters.java | 2 +- .../webmonitor/handlers/JarListInfo.java | 6 +- .../handlers/JarPlanMessageParameters.java | 2 +- .../handlers/JarSubmissionITCase.java | 226 ++++++++++++++++++ .../handlers/utils/TestProgram.java | 31 +++ .../runtime/rest/messages/JobPlanInfo.java | 6 + .../runtime/util/BlobServerResource.java | 65 +++++ 8 files changed, 361 insertions(+), 5 deletions(-) create mode 100644 flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/JarSubmissionITCase.java create mode 100644 flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/utils/TestProgram.java create mode 100644 flink-runtime/src/test/java/org/apache/flink/runtime/util/BlobServerResource.java diff --git a/flink-runtime-web/pom.xml b/flink-runtime-web/pom.xml index 837aadb4b2513..943a737109256 100644 --- a/flink-runtime-web/pom.xml +++ b/flink-runtime-web/pom.xml @@ -135,8 +135,36 @@ under the License. test-jar + + test-program-jar + process-test-classes + + jar + + + + org/apache/flink/runtime/webmonitor/handlers/utils/TestProgram.java + + + + org.apache.flink.runtime.webmonitor.handlers.utils.TestProgram + + + test-program + + + + + org.apache.maven.plugins + maven-surefire-plugin + + + ${project.build.directory} + + + diff --git a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/JarDeleteMessageParameters.java b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/JarDeleteMessageParameters.java index 908040948de38..2b70602f79afc 100644 --- a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/JarDeleteMessageParameters.java +++ b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/JarDeleteMessageParameters.java @@ -30,7 +30,7 @@ */ public class JarDeleteMessageParameters extends MessageParameters { - private JarIdPathParameter jarIdPathParameter = new JarIdPathParameter(); + public JarIdPathParameter jarIdPathParameter = new JarIdPathParameter(); @Override public Collection> getPathParameters() { diff --git a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/JarListInfo.java b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/JarListInfo.java index 91686865b1397..4d8d4c9cd4739 100644 --- a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/JarListInfo.java +++ b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/JarListInfo.java @@ -41,7 +41,7 @@ public class JarListInfo implements ResponseBody { private String address; @JsonProperty(JAR_LIST_FIELD_FILES) - private List jarFileList; + public List jarFileList; @JsonCreator public JarListInfo( @@ -85,10 +85,10 @@ public static class JarFileInfo { public static final String JAR_FILE_FIELD_ENTRY = "entry"; @JsonProperty(JAR_FILE_FIELD_ID) - private String id; + public String id; @JsonProperty(JAR_FILE_FIELD_NAME) - private String name; + public String name; @JsonProperty(JAR_FILE_FIELD_UPLOADED) private long uploaded; diff --git a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/JarPlanMessageParameters.java b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/JarPlanMessageParameters.java index 7dd9950722374..8599a2ccf441d 100644 --- a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/JarPlanMessageParameters.java +++ b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/JarPlanMessageParameters.java @@ -31,7 +31,7 @@ */ public class JarPlanMessageParameters extends MessageParameters { - private final JarIdPathParameter jarIdPathParameter = new JarIdPathParameter(); + public final JarIdPathParameter jarIdPathParameter = new JarIdPathParameter(); private final EntryClassQueryParameter entryClassQueryParameter = new EntryClassQueryParameter(); diff --git a/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/JarSubmissionITCase.java b/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/JarSubmissionITCase.java new file mode 100644 index 0000000000000..e47a38a55133e --- /dev/null +++ b/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/JarSubmissionITCase.java @@ -0,0 +1,226 @@ +/* + * 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.webmonitor.handlers; + +import org.apache.flink.api.common.time.Time; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.runtime.dispatcher.DispatcherGateway; +import org.apache.flink.runtime.messages.Acknowledge; +import org.apache.flink.runtime.rest.handler.HandlerRequest; +import org.apache.flink.runtime.rest.messages.EmptyMessageParameters; +import org.apache.flink.runtime.rest.messages.EmptyRequestBody; +import org.apache.flink.runtime.rest.messages.JobPlanInfo; +import org.apache.flink.runtime.testingUtils.TestingUtils; +import org.apache.flink.runtime.util.BlobServerResource; +import org.apache.flink.runtime.webmonitor.RestfulGateway; +import org.apache.flink.runtime.webmonitor.TestingDispatcherGateway; +import org.apache.flink.runtime.webmonitor.retriever.GatewayRetriever; +import org.apache.flink.util.OperatingSystem; +import org.apache.flink.util.TestLogger; + +import org.junit.Assert; +import org.junit.Assume; +import org.junit.BeforeClass; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.TemporaryFolder; + +import java.nio.file.Files; +import java.nio.file.Path; +import java.nio.file.Paths; +import java.util.Collections; +import java.util.Map; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.Executor; + +import static org.hamcrest.Matchers.containsString; + +/** + * Tests the entire lifecycle of a jar submission. + */ +public class JarSubmissionITCase extends TestLogger { + + @Rule + public final TemporaryFolder temporaryFolder = new TemporaryFolder(); + + @Rule + public final BlobServerResource blobServerResource = new BlobServerResource(); + + @BeforeClass + public static void checkOS() { + Assume.assumeFalse("This test fails on Windows due to unclosed JarFiles, see FLINK-9844.", OperatingSystem.isWindows()); + } + + @Test + public void testJarSubmission() throws Exception { + final TestingDispatcherGateway restfulGateway = new TestingDispatcherGateway.Builder() + .setBlobServerPort(blobServerResource.getBlobServerPort()) + .setSubmitFunction(jobGraph -> CompletableFuture.completedFuture(Acknowledge.get())) + .build(); + final JarHandlers handlers = new JarHandlers(temporaryFolder.newFolder().toPath(), restfulGateway); + final JarUploadHandler uploadHandler = handlers.uploadHandler; + final JarListHandler listHandler = handlers.listHandler; + final JarPlanHandler planHandler = handlers.planHandler; + final JarRunHandler runHandler = handlers.runHandler; + final JarDeleteHandler deleteHandler = handlers.deleteHandler; + + // targetDir property is set via surefire configuration + final Path originalJar = Paths.get(System.getProperty("targetDir")).resolve("test-program.jar"); + final Path jar = Files.copy(originalJar, temporaryFolder.getRoot().toPath().resolve("test-program.jar")); + + final String storedJarPath = uploadJar(uploadHandler, jar, restfulGateway); + final String storedJarName = Paths.get(storedJarPath).getFileName().toString(); + + final JarListInfo postUploadListResponse = listJars(listHandler, restfulGateway); + Assert.assertEquals(1, postUploadListResponse.jarFileList.size()); + final JarListInfo.JarFileInfo listEntry = postUploadListResponse.jarFileList.iterator().next(); + Assert.assertEquals(jar.getFileName().toString(), listEntry.name); + Assert.assertEquals(storedJarName, listEntry.id); + + final JobPlanInfo planResponse = showPlan(planHandler, storedJarName, restfulGateway); + // we're only interested in the core functionality so checking for a small detail is sufficient + Assert.assertThat(planResponse.getJsonPlan(), containsString("TestProgram.java:29")); + + runJar(runHandler, storedJarName, restfulGateway); + + deleteJar(deleteHandler, storedJarName, restfulGateway); + + final JarListInfo postDeleteListResponse = listJars(listHandler, restfulGateway); + Assert.assertEquals(0, postDeleteListResponse.jarFileList.size()); + } + + private static String uploadJar(JarUploadHandler handler, Path jar, RestfulGateway restfulGateway) throws Exception { + HandlerRequest uploadRequest = new HandlerRequest<>( + EmptyRequestBody.getInstance(), + EmptyMessageParameters.getInstance(), + Collections.emptyMap(), + Collections.emptyMap(), + Collections.singletonList(jar.toFile())); + final JarUploadResponseBody uploadResponse = handler.handleRequest(uploadRequest, restfulGateway) + .get(); + return uploadResponse.getFilename(); + } + + private static JarListInfo listJars(JarListHandler handler, RestfulGateway restfulGateway) throws Exception { + HandlerRequest listRequest = new HandlerRequest<>( + EmptyRequestBody.getInstance(), + EmptyMessageParameters.getInstance()); + return handler.handleRequest(listRequest, restfulGateway) + .get(); + } + + private static JobPlanInfo showPlan(JarPlanHandler handler, String jarName, RestfulGateway restfulGateway) throws Exception { + JarPlanMessageParameters planParameters = JarPlanHeaders.getInstance().getUnresolvedMessageParameters(); + HandlerRequest planRequest = new HandlerRequest<>( + EmptyRequestBody.getInstance(), + planParameters, + Collections.singletonMap(planParameters.jarIdPathParameter.getKey(), jarName), + Collections.emptyMap(), + Collections.emptyList()); + return handler.handleRequest(planRequest, restfulGateway) + .get(); + } + + private static JarRunResponseBody runJar(JarRunHandler handler, String jarName, DispatcherGateway restfulGateway) throws Exception { + final JarRunMessageParameters runParameters = JarRunHeaders.getInstance().getUnresolvedMessageParameters(); + HandlerRequest runRequest = new HandlerRequest<>( + EmptyRequestBody.getInstance(), + runParameters, + Collections.singletonMap(runParameters.jarIdPathParameter.getKey(), jarName), + Collections.emptyMap(), + Collections.emptyList()); + return handler.handleRequest(runRequest, restfulGateway) + .get(); + } + + private static void deleteJar(JarDeleteHandler handler, String jarName, RestfulGateway restfulGateway) throws Exception { + JarDeleteMessageParameters deleteParameters = JarDeleteHeaders.getInstance().getUnresolvedMessageParameters(); + HandlerRequest deleteRequest = new HandlerRequest<>( + EmptyRequestBody.getInstance(), + deleteParameters, + Collections.singletonMap(deleteParameters.jarIdPathParameter.getKey(), jarName), + Collections.emptyMap(), + Collections.emptyList()); + handler.handleRequest(deleteRequest, restfulGateway) + .get(); + } + + private static class JarHandlers { + final JarUploadHandler uploadHandler; + final JarListHandler listHandler; + final JarPlanHandler planHandler; + final JarRunHandler runHandler; + final JarDeleteHandler deleteHandler; + + JarHandlers(final Path jarDir, final TestingDispatcherGateway restfulGateway) { + final GatewayRetriever gatewayRetriever = () -> CompletableFuture.completedFuture(restfulGateway); + final CompletableFuture localAddressFuture = CompletableFuture.completedFuture("shazam://localhost:12345"); + final Time timeout = Time.seconds(10); + final Map responseHeaders = Collections.emptyMap(); + final Executor executor = TestingUtils.defaultExecutor(); + + uploadHandler = new JarUploadHandler( + localAddressFuture, + gatewayRetriever, + timeout, + responseHeaders, + JarUploadHeaders.getInstance(), + jarDir, + executor); + + listHandler = new JarListHandler( + localAddressFuture, + gatewayRetriever, + timeout, + responseHeaders, + JarListHeaders.getInstance(), + jarDir.toFile(), + executor); + + planHandler = new JarPlanHandler( + localAddressFuture, + gatewayRetriever, + timeout, + responseHeaders, + JarPlanHeaders.getInstance(), + jarDir, + new Configuration(), + executor); + + runHandler = new JarRunHandler( + localAddressFuture, + gatewayRetriever, + timeout, + responseHeaders, + JarRunHeaders.getInstance(), + jarDir, + new Configuration(), + executor); + + deleteHandler = new JarDeleteHandler( + localAddressFuture, + gatewayRetriever, + timeout, + responseHeaders, + JarDeleteHeaders.getInstance(), + jarDir, + executor); + } + } +} diff --git a/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/utils/TestProgram.java b/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/utils/TestProgram.java new file mode 100644 index 0000000000000..19d4678f7ffd7 --- /dev/null +++ b/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/utils/TestProgram.java @@ -0,0 +1,31 @@ +/* + * 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.webmonitor.handlers.utils; + +import org.apache.flink.api.java.ExecutionEnvironment; + +/** + * Simple test program. + */ +public class TestProgram { + public static void main(String[] args) throws Exception { + ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); + env.fromElements("hello", "world").print(); + } +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/JobPlanInfo.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/JobPlanInfo.java index 7263b36735f04..965702d1bc619 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/JobPlanInfo.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/JobPlanInfo.java @@ -22,6 +22,7 @@ import org.apache.flink.util.Preconditions; import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonCreator; +import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonIgnore; import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonProperty; import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.core.JsonGenerator; import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.core.JsonParser; @@ -55,6 +56,11 @@ public JobPlanInfo(@JsonProperty(FIELD_NAME_PLAN) RawJson jsonPlan) { this.jsonPlan = jsonPlan; } + @JsonIgnore + public String getJsonPlan() { + return jsonPlan.json; + } + @Override public boolean equals(Object o) { if (this == o) { diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/util/BlobServerResource.java b/flink-runtime/src/test/java/org/apache/flink/runtime/util/BlobServerResource.java new file mode 100644 index 0000000000000..080ecf81e66bc --- /dev/null +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/util/BlobServerResource.java @@ -0,0 +1,65 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.runtime.util; + +import org.apache.flink.configuration.BlobServerOptions; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.runtime.blob.BlobServer; +import org.apache.flink.runtime.blob.VoidBlobStore; + +import org.junit.rules.ExternalResource; +import org.junit.rules.TemporaryFolder; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; + +/** + * A simple {@link ExternalResource} to be used by tests that require a {@link BlobServer}. + */ +public class BlobServerResource extends ExternalResource { + private static final Logger LOG = LoggerFactory.getLogger(BlobServerResource.class); + private final TemporaryFolder temporaryFolder = new TemporaryFolder(); + + private BlobServer blobServer; + + protected void before() throws Throwable { + temporaryFolder.create(); + + Configuration config = new Configuration(); + config.setString(BlobServerOptions.STORAGE_DIRECTORY, temporaryFolder.newFolder().getAbsolutePath()); + + blobServer = new BlobServer(config, new VoidBlobStore()); + blobServer.start(); + } + + protected void after() { + temporaryFolder.delete(); + + try { + blobServer.close(); + } catch (IOException e) { + LOG.error("Exception while shutting down blob server.", e); + } + } + + public int getBlobServerPort() { + return blobServer.getPort(); + } +} From ff2e7d59455ba0d3e01a2b48c186e0216bf8fdfc Mon Sep 17 00:00:00 2001 From: Till Rohrmann Date: Wed, 18 Jul 2018 11:11:25 +0200 Subject: [PATCH 09/65] [hotfix] Properly stop Flink Yarn application in YARNSessionFIFOITCase#testJavaAPI Calls ClusterClient#shutDownCluster in order to stop the Flink Yarn application --- .../flink/yarn/YARNSessionFIFOITCase.java | 58 ++++++++++--------- 1 file changed, 30 insertions(+), 28 deletions(-) diff --git a/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YARNSessionFIFOITCase.java b/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YARNSessionFIFOITCase.java index bb479ae1c6540..f027399be7fba 100644 --- a/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YARNSessionFIFOITCase.java +++ b/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YARNSessionFIFOITCase.java @@ -323,38 +323,40 @@ public void testJavaAPI() throws Exception { .setSlotsPerTaskManager(1) .createClusterSpecification(); // deploy - ClusterClient yarnCluster = null; + ClusterClient yarnClusterClient = null; try { - yarnCluster = clusterDescriptor.deploySessionCluster(clusterSpecification); - } catch (Exception e) { - LOG.warn("Failing test", e); - Assert.fail("Error while deploying YARN cluster: " + e.getMessage()); - } - GetClusterStatusResponse expectedStatus = new GetClusterStatusResponse(1, 1); - for (int second = 0; second < waitTime * 2; second++) { // run "forever" - try { - Thread.sleep(1000); - } catch (InterruptedException e) { - LOG.warn("Interrupted", e); - } - GetClusterStatusResponse status = yarnCluster.getClusterStatus(); - if (status != null && status.equals(expectedStatus)) { - LOG.info("ClusterClient reached status " + status); - break; // all good, cluster started + yarnClusterClient = clusterDescriptor.deploySessionCluster(clusterSpecification); + + GetClusterStatusResponse expectedStatus = new GetClusterStatusResponse(1, 1); + for (int second = 0; second < waitTime * 2; second++) { // run "forever" + try { + Thread.sleep(1000); + } catch (InterruptedException e) { + LOG.warn("Interrupted", e); + } + GetClusterStatusResponse status = yarnClusterClient.getClusterStatus(); + if (status != null && status.equals(expectedStatus)) { + LOG.info("ClusterClient reached status " + status); + break; // all good, cluster started + } + if (second > waitTime) { + // we waited for 15 seconds. cluster didn't come up correctly + Assert.fail("The custer didn't start after " + waitTime + " seconds"); + } } - if (second > waitTime) { - // we waited for 15 seconds. cluster didn't come up correctly - Assert.fail("The custer didn't start after " + waitTime + " seconds"); + + // use the cluster + Assert.assertNotNull(yarnClusterClient.getClusterConnectionInfo()); + Assert.assertNotNull(yarnClusterClient.getWebInterfaceURL()); + LOG.info("All tests passed."); + } finally { + if (yarnClusterClient != null) { + // shutdown cluster + LOG.info("Shutting down the Flink Yarn application."); + yarnClusterClient.shutDownCluster(); + yarnClusterClient.shutdown(); } } - - // use the cluster - Assert.assertNotNull(yarnCluster.getClusterConnectionInfo()); - Assert.assertNotNull(yarnCluster.getWebInterfaceURL()); - - LOG.info("Shutting down cluster. All tests passed"); - // shutdown cluster - yarnCluster.shutdown(); } LOG.info("Finished testJavaAPI()"); } From 8a247605bf41738900a4c8f6fa7a5949b7185e6d Mon Sep 17 00:00:00 2001 From: linjun <1428117789@qq.com> Date: Tue, 10 Jul 2018 22:55:11 +0800 Subject: [PATCH 10/65] [FLINK-9793][yarn] Fix flink-dist detection to prevent duplicate upload This closes #6296. --- .../apache/flink/yarn/AbstractYarnClusterDescriptor.java | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/flink-yarn/src/main/java/org/apache/flink/yarn/AbstractYarnClusterDescriptor.java b/flink-yarn/src/main/java/org/apache/flink/yarn/AbstractYarnClusterDescriptor.java index 089d825f2d894..c3ad9f7f42c2c 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 @@ -1189,9 +1189,9 @@ static List uploadAndRegisterFiles( @Override public FileVisitResult visitFile(java.nio.file.Path file, BasicFileAttributes attrs) throws IOException { - - if (!(file.getFileName().startsWith("flink-dist") && - file.getFileName().endsWith("jar"))) { + String fileName = file.getFileName().toString(); + if (!(fileName.startsWith("flink-dist") && + fileName.endsWith("jar"))) { java.nio.file.Path relativePath = parentPath.relativize(file); From 969ab84777437683b2b244db96d8e1436275427c Mon Sep 17 00:00:00 2001 From: Dawid Wysakowicz Date: Mon, 16 Jul 2018 17:30:17 +0200 Subject: [PATCH 11/65] [FLINK-9866] Allow passing command line arguments to standalone job This closes #6344 --- flink-container/docker/docker-compose.yml | 2 +- .../StandaloneJobClusterEntryPoint.java | 15 ++++++++++----- .../StandaloneJobClusterEntryPointTest.java | 5 +++-- .../flink/container/entrypoint/TestJob.java | 4 +++- 4 files changed, 17 insertions(+), 9 deletions(-) diff --git a/flink-container/docker/docker-compose.yml b/flink-container/docker/docker-compose.yml index 81e4c8c8a54b4..5fddff3a5dc6c 100644 --- a/flink-container/docker/docker-compose.yml +++ b/flink-container/docker/docker-compose.yml @@ -24,7 +24,7 @@ services: image: ${FLINK_DOCKER_IMAGE_NAME:-flink-job} ports: - "8081:8081" - command: job-cluster --job-classname ${FLINK_JOB} -Djobmanager.rpc.address=job-cluster + command: job-cluster --job-classname ${FLINK_JOB} -Djobmanager.rpc.address=job-cluster ${FLINK_JOB_ARGUMENTS} taskmanager: image: ${FLINK_DOCKER_IMAGE_NAME:-flink-job} diff --git a/flink-container/src/main/java/org/apache/flink/container/entrypoint/StandaloneJobClusterEntryPoint.java b/flink-container/src/main/java/org/apache/flink/container/entrypoint/StandaloneJobClusterEntryPoint.java index 47cca4c7d8509..57f7ca239b705 100644 --- a/flink-container/src/main/java/org/apache/flink/container/entrypoint/StandaloneJobClusterEntryPoint.java +++ b/flink-container/src/main/java/org/apache/flink/container/entrypoint/StandaloneJobClusterEntryPoint.java @@ -51,20 +51,23 @@ import java.util.concurrent.CompletableFuture; +import static org.apache.flink.util.Preconditions.checkNotNull; + /** * {@link JobClusterEntrypoint} which is started with a job in a predefined * location. */ public final class StandaloneJobClusterEntryPoint extends JobClusterEntrypoint { - private static final String[] EMPTY_ARGS = new String[0]; + private final String[] programArguments; @Nonnull private final String jobClassName; - StandaloneJobClusterEntryPoint(Configuration configuration, @Nonnull String jobClassName) { + StandaloneJobClusterEntryPoint(Configuration configuration, @Nonnull String jobClassName, @Nonnull String[] programArguments) { super(configuration); - this.jobClassName = jobClassName; + this.programArguments = checkNotNull(programArguments); + this.jobClassName = checkNotNull(jobClassName); } @Override @@ -84,7 +87,7 @@ protected JobGraph retrieveJobGraph(Configuration configuration) throws FlinkExc private PackagedProgram createPackagedProgram() throws FlinkException { try { final Class mainClass = getClass().getClassLoader().loadClass(jobClassName); - return new PackagedProgram(mainClass, EMPTY_ARGS); + return new PackagedProgram(mainClass, programArguments); } catch (ClassNotFoundException | ProgramInvocationException e) { throw new FlinkException("Could not load the provied entrypoint class.", e); } @@ -148,7 +151,9 @@ public static void main(String[] args) { configuration.setString(ClusterEntrypoint.EXECUTION_MODE, ExecutionMode.DETACHED.toString()); - StandaloneJobClusterEntryPoint entrypoint = new StandaloneJobClusterEntryPoint(configuration, clusterConfiguration.getJobClassName()); + StandaloneJobClusterEntryPoint entrypoint = new StandaloneJobClusterEntryPoint(configuration, + clusterConfiguration.getJobClassName(), + clusterConfiguration.getArgs()); entrypoint.startCluster(); } diff --git a/flink-container/src/test/java/org/apache/flink/container/entrypoint/StandaloneJobClusterEntryPointTest.java b/flink-container/src/test/java/org/apache/flink/container/entrypoint/StandaloneJobClusterEntryPointTest.java index 360799d19a823..d97d2b714d4c9 100644 --- a/flink-container/src/test/java/org/apache/flink/container/entrypoint/StandaloneJobClusterEntryPointTest.java +++ b/flink-container/src/test/java/org/apache/flink/container/entrypoint/StandaloneJobClusterEntryPointTest.java @@ -42,11 +42,12 @@ public void testJobGraphRetrieval() throws FlinkException { configuration.setInteger(CoreOptions.DEFAULT_PARALLELISM, parallelism); final StandaloneJobClusterEntryPoint standaloneJobClusterEntryPoint = new StandaloneJobClusterEntryPoint( configuration, - TestJob.class.getCanonicalName()); + TestJob.class.getCanonicalName(), + new String[] {"--arg", "suffix"}); final JobGraph jobGraph = standaloneJobClusterEntryPoint.retrieveJobGraph(configuration); - assertThat(jobGraph.getName(), is(equalTo(TestJob.class.getCanonicalName()))); + assertThat(jobGraph.getName(), is(equalTo(TestJob.class.getCanonicalName() + "-suffix"))); assertThat(jobGraph.getMaximumParallelism(), is(parallelism)); } diff --git a/flink-container/src/test/java/org/apache/flink/container/entrypoint/TestJob.java b/flink-container/src/test/java/org/apache/flink/container/entrypoint/TestJob.java index 5f8857fc35fb4..ada434dd8b7bf 100644 --- a/flink-container/src/test/java/org/apache/flink/container/entrypoint/TestJob.java +++ b/flink-container/src/test/java/org/apache/flink/container/entrypoint/TestJob.java @@ -18,6 +18,7 @@ package org.apache.flink.container.entrypoint; +import org.apache.flink.api.java.utils.ParameterTool; import org.apache.flink.streaming.api.datastream.DataStreamSource; import org.apache.flink.streaming.api.datastream.SingleOutputStreamOperator; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; @@ -35,6 +36,7 @@ public static void main(String[] args) throws Exception { final SingleOutputStreamOperator mapper = source.map(element -> 2 * element); mapper.addSink(new DiscardingSink<>()); - env.execute(TestJob.class.getCanonicalName()); + ParameterTool parameterTool = ParameterTool.fromArgs(args); + env.execute(TestJob.class.getCanonicalName() + "-" + parameterTool.getRequired("arg")); } } From f576381e5fb45bf3cb4aaa54ba13ae11b4df3ca1 Mon Sep 17 00:00:00 2001 From: Dawid Wysakowicz Date: Wed, 11 Jul 2018 09:52:48 +0200 Subject: [PATCH 12/65] [FLINK-9792] Added custom Description class for ConfigOptions to enable rich formatting. This closes #6312 --- .../generated/akka_configuration.html | 6 +- .../generated/metric_configuration.html | 24 ++-- .../generated/security_configuration.html | 2 +- .../flink/configuration/ConfigOption.java | 40 +++++- .../description/BlockElement.java | 26 ++++ .../description/Description.java | 109 +++++++++++++++++ .../description/DescriptionElement.java | 31 +++++ .../configuration/description/Formatter.java | 95 +++++++++++++++ .../description/HtmlFormatter.java | 62 ++++++++++ .../description/InlineElement.java | 26 ++++ .../description/LineBreakElement.java | 40 ++++++ .../description/LinkElement.java | 66 ++++++++++ .../description/ListElement.java | 59 +++++++++ .../description/TextElement.java | 73 +++++++++++ .../description/DescriptionHtmlTest.java | 115 ++++++++++++++++++ .../ConfigOptionsDocGenerator.java | 5 +- .../org/apache/flink/docs/util/Utils.java | 4 +- .../ConfigOptionsDocGeneratorTest.java | 7 +- .../ConfigOptionsDocsCompletenessITCase.java | 12 +- 19 files changed, 774 insertions(+), 28 deletions(-) create mode 100644 flink-core/src/main/java/org/apache/flink/configuration/description/BlockElement.java create mode 100644 flink-core/src/main/java/org/apache/flink/configuration/description/Description.java create mode 100644 flink-core/src/main/java/org/apache/flink/configuration/description/DescriptionElement.java create mode 100644 flink-core/src/main/java/org/apache/flink/configuration/description/Formatter.java create mode 100644 flink-core/src/main/java/org/apache/flink/configuration/description/HtmlFormatter.java create mode 100644 flink-core/src/main/java/org/apache/flink/configuration/description/InlineElement.java create mode 100644 flink-core/src/main/java/org/apache/flink/configuration/description/LineBreakElement.java create mode 100644 flink-core/src/main/java/org/apache/flink/configuration/description/LinkElement.java create mode 100644 flink-core/src/main/java/org/apache/flink/configuration/description/ListElement.java create mode 100644 flink-core/src/main/java/org/apache/flink/configuration/description/TextElement.java create mode 100644 flink-core/src/test/java/org/apache/flink/configuration/description/DescriptionHtmlTest.java diff --git a/docs/_includes/generated/akka_configuration.html b/docs/_includes/generated/akka_configuration.html index afaba6f0fa5c6..a2b49c74a44a4 100644 --- a/docs/_includes/generated/akka_configuration.html +++ b/docs/_includes/generated/akka_configuration.html @@ -80,17 +80,17 @@
akka.watch.heartbeat.interval
"10 s" - Heartbeat interval for Akka’s DeathWatch mechanism to detect dead TaskManagers. If TaskManagers are wrongly marked dead because of lost or delayed heartbeat messages, then you should decrease this value or increase akka.watch.heartbeat.pause. A thorough description of Akka’s DeathWatch can be found <a href="http://doc.akka.io/docs/akka/snapshot/scala/remoting.html#failure-detector">here</a>. + Heartbeat interval for Akka’s DeathWatch mechanism to detect dead TaskManagers. If TaskManagers are wrongly marked dead because of lost or delayed heartbeat messages, then you should decrease this value or increase akka.watch.heartbeat.pause. A thorough description of Akka’s DeathWatch can be found <a href="http://doc.akka.io/docs/akka/snapshot/scala/remoting.html#failure-detector">here</a>.
akka.watch.heartbeat.pause
"60 s" - Acceptable heartbeat pause for Akka’s DeathWatch mechanism. A low value does not allow an irregular heartbeat. If TaskManagers are wrongly marked dead because of lost or delayed heartbeat messages, then you should increase this value or decrease akka.watch.heartbeat.interval. Higher value increases the time to detect a dead TaskManager. A thorough description of Akka’s DeathWatch can be found <a href="http://doc.akka.io/docs/akka/snapshot/scala/remoting.html#failure-detector">here</a>. + Acceptable heartbeat pause for Akka’s DeathWatch mechanism. A low value does not allow an irregular heartbeat. If TaskManagers are wrongly marked dead because of lost or delayed heartbeat messages, then you should increase this value or decrease akka.watch.heartbeat.interval. Higher value increases the time to detect a dead TaskManager. A thorough description of Akka’s DeathWatch can be found <a href="http://doc.akka.io/docs/akka/snapshot/scala/remoting.html#failure-detector">here</a>.
akka.watch.threshold
12 - Threshold for the DeathWatch failure detector. A low value is prone to false positives whereas a high value increases the time to detect a dead TaskManager. A thorough description of Akka’s DeathWatch can be found <a href="http://doc.akka.io/docs/akka/snapshot/scala/remoting.html#failure-detector">here</a>. + Threshold for the DeathWatch failure detector. A low value is prone to false positives whereas a high value increases the time to detect a dead TaskManager. A thorough description of Akka’s DeathWatch can be found <a href="http://doc.akka.io/docs/akka/snapshot/scala/remoting.html#failure-detector">here</a>. diff --git a/docs/_includes/generated/metric_configuration.html b/docs/_includes/generated/metric_configuration.html index 868a9bd66e9b6..aef8fbb4f6035 100644 --- a/docs/_includes/generated/metric_configuration.html +++ b/docs/_includes/generated/metric_configuration.html @@ -13,19 +13,19 @@ Defines the number of measured latencies to maintain at each operator. -
metrics.reporter.<name>.<parameter>
+
metrics.reporter.<name>.<parameter>
(none) - Configures the parameter <parameter> for the reporter named <name>. + Configures the parameter <parameter> for the reporter named <name>. -
metrics.reporter.<name>.class
+
metrics.reporter.<name>.class
(none) - The reporter class to use for the reporter named <name>. + The reporter class to use for the reporter named <name>. -
metrics.reporter.<name>.interval
+
metrics.reporter.<name>.interval
(none) - The reporter interval to use for the reporter named <name>. + The reporter interval to use for the reporter named <name>.
metrics.reporters
@@ -39,32 +39,32 @@
metrics.scope.jm
- "<host>.jobmanager" + "<host>.jobmanager" Defines the scope format string that is applied to all metrics scoped to a JobManager.
metrics.scope.jm.job
- "<host>.jobmanager.<job_name>" + "<host>.jobmanager.<job_name>" Defines the scope format string that is applied to all metrics scoped to a job on a JobManager.
metrics.scope.operator
- "<host>.taskmanager.<tm_id>.<job_name>.<operator_name>.<subtask_index>" + "<host>.taskmanager.<tm_id>.<job_name>.<operator_name>.<subtask_index>" Defines the scope format string that is applied to all metrics scoped to an operator.
metrics.scope.task
- "<host>.taskmanager.<tm_id>.<job_name>.<task_name>.<subtask_index>" + "<host>.taskmanager.<tm_id>.<job_name>.<task_name>.<subtask_index>" Defines the scope format string that is applied to all metrics scoped to a task.
metrics.scope.tm
- "<host>.taskmanager.<tm_id>" + "<host>.taskmanager.<tm_id>" Defines the scope format string that is applied to all metrics scoped to a TaskManager.
metrics.scope.tm.job
- "<host>.taskmanager.<tm_id>.<job_name>" + "<host>.taskmanager.<tm_id>.<job_name>" Defines the scope format string that is applied to all metrics scoped to a job on a TaskManager. diff --git a/docs/_includes/generated/security_configuration.html b/docs/_includes/generated/security_configuration.html index 5042cf3df01c1..47846aed1819f 100644 --- a/docs/_includes/generated/security_configuration.html +++ b/docs/_includes/generated/security_configuration.html @@ -10,7 +10,7 @@
security.ssl.algorithms
"TLS_RSA_WITH_AES_128_CBC_SHA" - The comma separated list of standard SSL algorithms to be supported. Read more <a href="http://docs.oracle.com/javase/8/docs/technotes/guides/security/StandardNames.html#ciphersuites">here</a>. + The comma separated list of standard SSL algorithms to be supported. Read more <a href="http://docs.oracle.com/javase/8/docs/technotes/guides/security/StandardNames.html#ciphersuites">here</a>.
security.ssl.internal.enabled
diff --git a/flink-core/src/main/java/org/apache/flink/configuration/ConfigOption.java b/flink-core/src/main/java/org/apache/flink/configuration/ConfigOption.java index 8e7d79b230110..be242f54e25a9 100644 --- a/flink-core/src/main/java/org/apache/flink/configuration/ConfigOption.java +++ b/flink-core/src/main/java/org/apache/flink/configuration/ConfigOption.java @@ -19,6 +19,7 @@ package org.apache.flink.configuration; import org.apache.flink.annotation.PublicEvolving; +import org.apache.flink.configuration.description.Description; import java.util.Arrays; import java.util.Collections; @@ -52,7 +53,7 @@ public class ConfigOption { private final T defaultValue; /** The description for this option. */ - private final String description; + private final Description description; // ------------------------------------------------------------------------ @@ -64,7 +65,7 @@ public class ConfigOption { */ ConfigOption(String key, T defaultValue) { this.key = checkNotNull(key); - this.description = ""; + this.description = Description.builder().text("").build(); this.defaultValue = defaultValue; this.deprecatedKeys = EMPTY; } @@ -73,10 +74,28 @@ public class ConfigOption { * Creates a new config option with deprecated keys. * * @param key The current key for that config option + * @param description Description for that option * @param defaultValue The default value for this option * @param deprecatedKeys The list of deprecated keys, in the order to be checked + * @deprecated use version with {@link Description} instead */ + @Deprecated ConfigOption(String key, String description, T defaultValue, String... deprecatedKeys) { + this.key = checkNotNull(key); + this.description = Description.builder().text(description).build(); + this.defaultValue = defaultValue; + this.deprecatedKeys = deprecatedKeys == null || deprecatedKeys.length == 0 ? EMPTY : deprecatedKeys; + } + + /** + * Creates a new config option with deprecated keys. + * + * @param key The current key for that config option + * @param description Description for that option + * @param defaultValue The default value for this option + * @param deprecatedKeys The list of deprecated keys, in the order to be checked + */ + ConfigOption(String key, Description description, T defaultValue, String... deprecatedKeys) { this.key = checkNotNull(key); this.description = description; this.defaultValue = defaultValue; @@ -104,15 +123,26 @@ public ConfigOption withDeprecatedKeys(String... deprecatedKeys) { * Creates a new config option, using this option's key and default value, and * adding the given description. The given description is used when generation the configuration documention. * - *

NOTE: You can use html to format the output of the generated cell. - * * @param description The description for this option. * @return A new config option, with given description. + * @deprecated use version with {@link Description} */ + @Deprecated public ConfigOption withDescription(final String description) { return new ConfigOption<>(key, description, defaultValue, deprecatedKeys); } + /** + * Creates a new config option, using this option's key and default value, and + * adding the given description. The given description is used when generation the configuration documention. + * + * @param description The description for this option. + * @return A new config option, with given description. + */ + public ConfigOption withDescription(final Description description) { + return new ConfigOption<>(key, description, defaultValue, deprecatedKeys); + } + // ------------------------------------------------------------------------ /** @@ -159,7 +189,7 @@ public Iterable deprecatedKeys() { * Returns the description of this option. * @return The option's description. */ - public String description() { + public Description description() { return description; } diff --git a/flink-core/src/main/java/org/apache/flink/configuration/description/BlockElement.java b/flink-core/src/main/java/org/apache/flink/configuration/description/BlockElement.java new file mode 100644 index 0000000000000..c55b496bfb039 --- /dev/null +++ b/flink-core/src/main/java/org/apache/flink/configuration/description/BlockElement.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.configuration.description; + +/** + * Part of description that represents a block e.g. some text, linebreak or a list. + */ +public interface BlockElement extends DescriptionElement { + +} diff --git a/flink-core/src/main/java/org/apache/flink/configuration/description/Description.java b/flink-core/src/main/java/org/apache/flink/configuration/description/Description.java new file mode 100644 index 0000000000000..25a6a640b5820 --- /dev/null +++ b/flink-core/src/main/java/org/apache/flink/configuration/description/Description.java @@ -0,0 +1,109 @@ +/* + * 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.configuration.description; + +import java.util.ArrayList; +import java.util.List; + +/** + * Description for {@link org.apache.flink.configuration.ConfigOption}. Allows providing multiple rich formats. + */ +public class Description { + + private final List blocks; + + public static DescriptionBuilder builder() { + return new DescriptionBuilder(); + } + + public List getBlocks() { + return blocks; + } + + /** + * Builder for {@link Description}. Allows adding a rich formatting like lists, links, linebreaks etc. + * For example: + *

{@code
+	 * Description description = Description.builder()
+	 * 	.text("This is some list: ")
+	 * 	.list(
+	 * 		text("this is first element of list"),
+	 * 		text("this is second element of list with a %s", link("https://link")))
+	 * 	.build();
+	 * }
+ */ + public static class DescriptionBuilder { + + private final List blocks = new ArrayList<>(); + + /** + * Adds a block of text with placeholders ("%s") that will be replaced with proper string representation of + * given {@link InlineElement}. For example: + * + *

{@code text("This is a text with a link %s", link("https://somepage", "to here"))} + * + * @param format text with placeholders for elements + * @param elements elements to be put in the text + * @return description with added block of text + */ + public DescriptionBuilder text(String format, InlineElement... elements) { + blocks.add(TextElement.text(format, elements)); + return this; + } + + /** + * Creates a simple block of text. + * + * @param text a simple block of text + * @return block of text + */ + public DescriptionBuilder text(String text) { + blocks.add(TextElement.text(text)); + return this; + } + + /** + * Creates a line break in the description. + */ + public DescriptionBuilder linebreak() { + blocks.add(LineBreakElement.linebreak()); + return this; + } + + /** + * Adds a bulleted list to the description. + */ + public DescriptionBuilder list(InlineElement... elements) { + blocks.add(ListElement.list(elements)); + return this; + } + + /** + * Creates description representation. + */ + public Description build() { + return new Description(blocks); + } + + } + + private Description(List blocks) { + this.blocks = blocks; + } +} diff --git a/flink-core/src/main/java/org/apache/flink/configuration/description/DescriptionElement.java b/flink-core/src/main/java/org/apache/flink/configuration/description/DescriptionElement.java new file mode 100644 index 0000000000000..7a889f448bc06 --- /dev/null +++ b/flink-core/src/main/java/org/apache/flink/configuration/description/DescriptionElement.java @@ -0,0 +1,31 @@ +/* + * 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.configuration.description; + +/** + * Part of a {@link Description} that can be converted into String representation. + */ +interface DescriptionElement { + /** + * Transforms itself into String representation using given format. + * + * @param formatter formatter to use. + */ + void format(Formatter formatter); +} diff --git a/flink-core/src/main/java/org/apache/flink/configuration/description/Formatter.java b/flink-core/src/main/java/org/apache/flink/configuration/description/Formatter.java new file mode 100644 index 0000000000000..d3fcf403d5a0c --- /dev/null +++ b/flink-core/src/main/java/org/apache/flink/configuration/description/Formatter.java @@ -0,0 +1,95 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.configuration.description; + +/** + * Allows providing multiple formatters for the description. E.g. Html formatter, Markdown formatter etc. + */ +public abstract class Formatter { + + private final StringBuilder state = new StringBuilder(); + + /** + * Formats the description into a String using format specific tags. + * + * @param description description to be formatted + * @return string representation of the description + */ + public String format(Description description) { + for (BlockElement blockElement : description.getBlocks()) { + blockElement.format(this); + } + return finalizeFormatting(); + } + + public void format(LinkElement element) { + formatLink(state, element.getLink(), element.getText()); + } + + public void format(TextElement element) { + String[] inlineElements = element.getElements().stream().map(el -> { + Formatter formatter = newInstance(); + el.format(formatter); + return formatter.finalizeFormatting(); + } + ).toArray(String[]::new); + formatText(state, escapeFormatPlaceholder(element.getFormat()), inlineElements); + } + + public void format(LineBreakElement element) { + formatLineBreak(state); + } + + public void format(ListElement element) { + String[] inlineElements = element.getEntries().stream().map(el -> { + Formatter formatter = newInstance(); + el.format(formatter); + return formatter.finalizeFormatting(); + } + ).toArray(String[]::new); + formatList(state, inlineElements); + } + + private String finalizeFormatting() { + String result = state.toString(); + state.setLength(0); + return result.replaceAll("%%", "%"); + } + + protected abstract void formatLink(StringBuilder state, String link, String description); + + protected abstract void formatLineBreak(StringBuilder state); + + protected abstract void formatText(StringBuilder state, String format, String[] elements); + + protected abstract void formatList(StringBuilder state, String[] entries); + + protected abstract Formatter newInstance(); + + private static final String TEMPORARY_PLACEHOLDER = "randomPlaceholderForStringFormat"; + + private static String escapeFormatPlaceholder(String value) { + return value + .replaceAll("%s", TEMPORARY_PLACEHOLDER) + .replaceAll("%", "%%") + .replaceAll(TEMPORARY_PLACEHOLDER, "%s"); + } + +} + diff --git a/flink-core/src/main/java/org/apache/flink/configuration/description/HtmlFormatter.java b/flink-core/src/main/java/org/apache/flink/configuration/description/HtmlFormatter.java new file mode 100644 index 0000000000000..72531abc5f40f --- /dev/null +++ b/flink-core/src/main/java/org/apache/flink/configuration/description/HtmlFormatter.java @@ -0,0 +1,62 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.configuration.description; + +/** + * Formatter that transforms {@link Description} into Html representation. + */ +public class HtmlFormatter extends Formatter { + + @Override + protected void formatLink(StringBuilder state, String link, String description) { + state.append(String.format("%s", link, description)); + } + + @Override + protected void formatLineBreak(StringBuilder state) { + state.append("
"); + } + + @Override + protected void formatText(StringBuilder state, String format, String[] elements) { + String escapedFormat = escapeCharacters(format); + state.append(String.format(escapedFormat, elements)); + } + + @Override + protected void formatList(StringBuilder state, String[] entries) { + state.append("

    "); + for (String entry : entries) { + state.append(String.format("
  • %s
  • ", entry)); + } + state.append("
"); + } + + @Override + protected Formatter newInstance() { + return new HtmlFormatter(); + } + + private static String escapeCharacters(String value) { + return value + .replaceAll("<", "<") + .replaceAll(">", ">"); + } + +} diff --git a/flink-core/src/main/java/org/apache/flink/configuration/description/InlineElement.java b/flink-core/src/main/java/org/apache/flink/configuration/description/InlineElement.java new file mode 100644 index 0000000000000..2a218758545d1 --- /dev/null +++ b/flink-core/src/main/java/org/apache/flink/configuration/description/InlineElement.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.configuration.description; + +/** + * Part of description that represents an element inside a block e.g. a link. + */ +public interface InlineElement extends DescriptionElement { + +} diff --git a/flink-core/src/main/java/org/apache/flink/configuration/description/LineBreakElement.java b/flink-core/src/main/java/org/apache/flink/configuration/description/LineBreakElement.java new file mode 100644 index 0000000000000..4a3de9c3cb5f9 --- /dev/null +++ b/flink-core/src/main/java/org/apache/flink/configuration/description/LineBreakElement.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.configuration.description; + +/** + * Represents a line break in the {@link Description}. + */ +public class LineBreakElement implements BlockElement { + + /** + * Creates a line break in the description. + */ + public static LineBreakElement linebreak() { + return new LineBreakElement(); + } + + private LineBreakElement() { + } + + @Override + public void format(Formatter formatter) { + formatter.format(this); + } +} diff --git a/flink-core/src/main/java/org/apache/flink/configuration/description/LinkElement.java b/flink-core/src/main/java/org/apache/flink/configuration/description/LinkElement.java new file mode 100644 index 0000000000000..778844eb559df --- /dev/null +++ b/flink-core/src/main/java/org/apache/flink/configuration/description/LinkElement.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.configuration.description; + +/** + * Element that represents a link in the {@link Description}. + */ +public class LinkElement implements InlineElement { + private final String link; + private final String text; + + /** + * Creates a link with a given url and description. + * + * @param link address that this link should point to + * @param text a description for that link, that should be used in text + * @return link representation + */ + public static LinkElement link(String link, String text) { + return new LinkElement(link, text); + } + + /** + * Creates a link with a given url. This url will be used as a description for that link. + * + * @param link address that this link should point to + * @return link representation + */ + public static LinkElement link(String link) { + return new LinkElement(link, link); + } + + public String getLink() { + return link; + } + + public String getText() { + return text; + } + + private LinkElement(String link, String text) { + this.link = link; + this.text = text; + } + + @Override + public void format(Formatter formatter) { + formatter.format(this); + } +} diff --git a/flink-core/src/main/java/org/apache/flink/configuration/description/ListElement.java b/flink-core/src/main/java/org/apache/flink/configuration/description/ListElement.java new file mode 100644 index 0000000000000..1dea3ab49ecaf --- /dev/null +++ b/flink-core/src/main/java/org/apache/flink/configuration/description/ListElement.java @@ -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.configuration.description; + +import java.util.Arrays; +import java.util.List; + +/** + * Represents a list in the {@link Description}. + */ +public class ListElement implements BlockElement { + + private final List entries; + + /** + * Creates a list with blocks of text. For example: + *
{@code
+	 * .list(
+	 * 	text("this is first element of list"),
+	 * 	text("this is second element of list with a %s", link("https://link"))
+	 * )
+	 * }
+ * + * @param elements list of this list entries + * @return list representation + */ + public static ListElement list(InlineElement... elements) { + return new ListElement(Arrays.asList(elements)); + } + + public List getEntries() { + return entries; + } + + private ListElement(List entries) { + this.entries = entries; + } + + @Override + public void format(Formatter formatter) { + formatter.format(this); + } +} diff --git a/flink-core/src/main/java/org/apache/flink/configuration/description/TextElement.java b/flink-core/src/main/java/org/apache/flink/configuration/description/TextElement.java new file mode 100644 index 0000000000000..aeb1d7aeaee17 --- /dev/null +++ b/flink-core/src/main/java/org/apache/flink/configuration/description/TextElement.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.configuration.description; + +import java.util.Arrays; +import java.util.Collections; +import java.util.List; + +/** + * Represents a text block in the {@link Description}. + */ +public class TextElement implements BlockElement, InlineElement { + private final String format; + private final List elements; + + /** + * Creates a block of text with placeholders ("%s") that will be replaced with proper string representation of + * given {@link InlineElement}. For example: + * + *

{@code text("This is a text with a link %s", link("https://somepage", "to here"))} + * + * @param format text with placeholders for elements + * @param elements elements to be put in the text + * @return block of text + */ + public static TextElement text(String format, InlineElement... elements) { + return new TextElement(format, Arrays.asList(elements)); + } + + /** + * Creates a simple block of text. + * + * @param text a simple block of text + * @return block of text + */ + public static TextElement text(String text) { + return new TextElement(text, Collections.emptyList()); + } + + public String getFormat() { + return format; + } + + public List getElements() { + return elements; + } + + private TextElement(String format, List elements) { + this.format = format; + this.elements = elements; + } + + @Override + public void format(Formatter formatter) { + formatter.format(this); + } +} diff --git a/flink-core/src/test/java/org/apache/flink/configuration/description/DescriptionHtmlTest.java b/flink-core/src/test/java/org/apache/flink/configuration/description/DescriptionHtmlTest.java new file mode 100644 index 0000000000000..fcdb15583d647 --- /dev/null +++ b/flink-core/src/test/java/org/apache/flink/configuration/description/DescriptionHtmlTest.java @@ -0,0 +1,115 @@ +/* + * 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.configuration.description; + +import org.junit.Test; + +import static org.apache.flink.configuration.description.LinkElement.link; +import static org.apache.flink.configuration.description.TextElement.text; +import static org.junit.Assert.assertEquals; + +/** + * Tests for {@link Description} and formatting with {@link HtmlFormatter}. + */ +public class DescriptionHtmlTest { + @Test + public void testDescriptionWithLink() { + Description description = Description.builder() + .text("This is a text with a link %s", link("https://somepage", "to here")) + .build(); + + String formattedDescription = new HtmlFormatter().format(description); + + assertEquals("This is a text with a link " + + "to here", formattedDescription); + } + + @Test + public void testDescriptionWithPercents() { + Description description = Description.builder() + .text("This is a text that has some percentage value of 20%.") + .build(); + + String formattedDescription = new HtmlFormatter().format(description); + + assertEquals("This is a text that has some percentage value of 20%.", formattedDescription); + } + + @Test + public void testDescriptionWithMultipleLinks() { + Description description = Description.builder() + .text("This is a text with a link %s and another %s", link("https://somepage", "to here"), + link("https://link")) + .build(); + + String formattedDescription = new HtmlFormatter().format(description); + + assertEquals("This is a text with a link to here and another " + + "https://link", formattedDescription); + } + + @Test + public void testDescriptionWithList() { + Description description = Description.builder() + .text("This is some list: ") + .list( + link("http://first_link"), + text("this is second element of list with a %s", link("https://link"))) + .build(); + + String formattedDescription = new HtmlFormatter().format(description); + + assertEquals( + "This is some list:

", + formattedDescription); + } + + @Test + public void testDescriptionWithLineBreak() { + Description description = Description.builder() + .text("This is first line.") + .linebreak() + .text("This is second line.") + .build(); + + String formattedDescription = new HtmlFormatter().format(description); + + assertEquals( + "This is first line.
This is second line.", + formattedDescription); + } + + @Test + public void testDescriptionWithListAndEscaping() { + Description description = Description.builder() + .text("This is some list: ") + .list( + text("this is first element with illegal character '>' and '<'") + ) + .build(); + + String formattedDescription = new HtmlFormatter().format(description); + + assertEquals( + "This is some list:
  • this is first element with illegal character '>' and '<'
", + formattedDescription); + } +} diff --git a/flink-docs/src/main/java/org/apache/flink/docs/configuration/ConfigOptionsDocGenerator.java b/flink-docs/src/main/java/org/apache/flink/docs/configuration/ConfigOptionsDocGenerator.java index d333719883d41..d0f665a5051e4 100644 --- a/flink-docs/src/main/java/org/apache/flink/docs/configuration/ConfigOptionsDocGenerator.java +++ b/flink-docs/src/main/java/org/apache/flink/docs/configuration/ConfigOptionsDocGenerator.java @@ -24,6 +24,8 @@ import org.apache.flink.annotation.docs.Documentation; import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.configuration.ConfigOption; +import org.apache.flink.configuration.description.Formatter; +import org.apache.flink.configuration.description.HtmlFormatter; import org.apache.flink.util.function.ThrowingConsumer; import java.io.IOException; @@ -76,6 +78,7 @@ public class ConfigOptionsDocGenerator { private static final String CLASS_PREFIX_GROUP = "classPrefix"; private static final Pattern CLASS_NAME_PATTERN = Pattern.compile("(?<" + CLASS_NAME_GROUP + ">(?<" + CLASS_PREFIX_GROUP + ">[a-zA-Z]*)(?:Options|Config|Parameters))(?:\\.java)?"); + private static final Formatter formatter = new HtmlFormatter(); /** * This method generates html tables from set of classes containing {@link ConfigOption ConfigOptions}. * @@ -253,7 +256,7 @@ private static String toHtmlString(final OptionWithMetaInfo optionWithMetaInfo) " \n" + "
" + escapeCharacters(option.key()) + "
\n" + " " + escapeCharacters(addWordBreakOpportunities(defaultValue)) + "\n" + - " " + escapeCharacters(option.description()) + "\n" + + " " + formatter.format(option.description()) + "\n" + " \n"; } diff --git a/flink-docs/src/main/java/org/apache/flink/docs/util/Utils.java b/flink-docs/src/main/java/org/apache/flink/docs/util/Utils.java index 3a53046b698e1..89d6f63dd0140 100644 --- a/flink-docs/src/main/java/org/apache/flink/docs/util/Utils.java +++ b/flink-docs/src/main/java/org/apache/flink/docs/util/Utils.java @@ -33,8 +33,8 @@ public enum Utils { public static String escapeCharacters(String value) { return value .replaceAll("", TEMPORARY_PLACEHOLDER) - .replaceAll("<", "<") - .replaceAll(">", ">") + .replaceAll("<", "<") + .replaceAll(">", ">") .replaceAll(TEMPORARY_PLACEHOLDER, ""); } } diff --git a/flink-docs/src/test/java/org/apache/flink/docs/configuration/ConfigOptionsDocGeneratorTest.java b/flink-docs/src/test/java/org/apache/flink/docs/configuration/ConfigOptionsDocGeneratorTest.java index 26cba8001027c..a0eade69d455f 100644 --- a/flink-docs/src/test/java/org/apache/flink/docs/configuration/ConfigOptionsDocGeneratorTest.java +++ b/flink-docs/src/test/java/org/apache/flink/docs/configuration/ConfigOptionsDocGeneratorTest.java @@ -24,6 +24,8 @@ import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.configuration.ConfigOption; import org.apache.flink.configuration.ConfigOptions; +import org.apache.flink.configuration.description.Formatter; +import org.apache.flink.configuration.description.HtmlFormatter; import org.apache.flink.docs.configuration.data.TestCommonOptions; import org.apache.flink.util.FileUtils; @@ -235,6 +237,7 @@ public void testCommonOptions() throws IOException, ClassNotFoundException { }; ConfigOptionsDocGenerator.generateCommonSection(projectRootDir, outputDirectory, locations, "src/test/java"); + Formatter formatter = new HtmlFormatter(); String expected = "\n" + @@ -249,12 +252,12 @@ public void testCommonOptions() throws IOException, ClassNotFoundException { " \n" + " \n" + " \n" + - " \n" + + " \n" + " \n" + " \n" + " \n" + " \n" + - " \n" + + " \n" + " \n" + " \n" + "
" + TestCommonOptions.COMMON_POSITIONED_OPTION.key() + "
" + TestCommonOptions.COMMON_POSITIONED_OPTION.defaultValue() + "" + TestCommonOptions.COMMON_POSITIONED_OPTION.description() + "" + formatter.format(TestCommonOptions.COMMON_POSITIONED_OPTION.description()) + "
" + TestCommonOptions.COMMON_OPTION.key() + "
" + TestCommonOptions.COMMON_OPTION.defaultValue() + "" + TestCommonOptions.COMMON_OPTION.description() + "" + formatter.format(TestCommonOptions.COMMON_OPTION.description()) + "
\n"; diff --git a/flink-docs/src/test/java/org/apache/flink/docs/configuration/ConfigOptionsDocsCompletenessITCase.java b/flink-docs/src/test/java/org/apache/flink/docs/configuration/ConfigOptionsDocsCompletenessITCase.java index 2d20212892387..264d2edd19954 100644 --- a/flink-docs/src/test/java/org/apache/flink/docs/configuration/ConfigOptionsDocsCompletenessITCase.java +++ b/flink-docs/src/test/java/org/apache/flink/docs/configuration/ConfigOptionsDocsCompletenessITCase.java @@ -20,6 +20,8 @@ import org.apache.flink.annotation.docs.Documentation; import org.apache.flink.configuration.ConfigOption; +import org.apache.flink.configuration.description.Formatter; +import org.apache.flink.configuration.description.HtmlFormatter; import org.jsoup.Jsoup; import org.jsoup.nodes.Document; @@ -55,6 +57,8 @@ */ public class ConfigOptionsDocsCompletenessITCase { + private static final Formatter htmlFormatter = new HtmlFormatter(); + @Test public void testCommonSectionCompleteness() throws IOException, ClassNotFoundException { Map documentedOptions = parseDocumentedCommonOptions(); @@ -173,7 +177,11 @@ private static Collection parseDocumentedOptionsFromFile(Path .map(tableRow -> { String key = tableRow.child(0).text(); String defaultValue = tableRow.child(1).text(); - String description = tableRow.child(2).text(); + String description = tableRow.child(2) + .childNodes() + .stream() + .map(Object::toString) + .collect(Collectors.joining()); return new DocumentedOption(key, defaultValue, description, file.getName(file.getNameCount() - 1)); }) .collect(Collectors.toList()); @@ -189,7 +197,7 @@ private static Map findExistingOptions(Predicate Date: Wed, 18 Jul 2018 15:17:32 +0200 Subject: [PATCH 13/65] [FLINK-9866][hotfix] Pass program arguments last to standalone-job.sh --- flink-dist/src/main/flink-bin/bin/standalone-job.sh | 8 +++----- 1 file changed, 3 insertions(+), 5 deletions(-) diff --git a/flink-dist/src/main/flink-bin/bin/standalone-job.sh b/flink-dist/src/main/flink-bin/bin/standalone-job.sh index 586a8bacb00ae..889dab8983605 100644 --- a/flink-dist/src/main/flink-bin/bin/standalone-job.sh +++ b/flink-dist/src/main/flink-bin/bin/standalone-job.sh @@ -23,8 +23,6 @@ USAGE="Usage: standalone-job.sh ((start|start-foreground))|stop" STARTSTOP=$1 ENTRY_POINT_NAME="standalonejob" -ARGS=("${@:2}") - if [[ $STARTSTOP != "start" ]] && [[ $STARTSTOP != "start-foreground" ]] && [[ $STARTSTOP != "stop" ]] || [[ -z JOB_CLASSNAME ]]; then echo $USAGE exit 1 @@ -35,6 +33,9 @@ bin=`cd "$bin"; pwd` . "$bin"/config.sh +# Startup parameters +ARGS=("--configDir" "${FLINK_CONF_DIR}" "${@:2}") + if [[ $STARTSTOP == "start" ]] || [[ $STARTSTOP == "start-foreground" ]]; then if [ ! -z "${FLINK_JM_HEAP_MB}" ] && [ "${FLINK_JM_HEAP}" == 0 ]; then echo "used deprecated key \`${KEY_JOBM_MEM_MB}\`, please replace with key \`${KEY_JOBM_MEM_SIZE}\`" @@ -54,9 +55,6 @@ if [[ $STARTSTOP == "start" ]] || [[ $STARTSTOP == "start-foreground" ]]; then # Add cluster entry point specific JVM options export FLINK_ENV_JAVA_OPTS="${FLINK_ENV_JAVA_OPTS} ${FLINK_ENV_JAVA_OPTS_JM}" - - # Startup parameters - ARGS+=("--configDir" "${FLINK_CONF_DIR}") fi if [[ $STARTSTOP == "start-foreground" ]]; then From b9fe077d221bdb013ed57f2555405c9fe4a96aa1 Mon Sep 17 00:00:00 2001 From: Wosin Date: Wed, 4 Jul 2018 10:27:54 +0200 Subject: [PATCH 14/65] [FLINK-9575] Remove job-related BLOBS only if the job was removed sucessfully This closes #6322. --- .../flink/runtime/dispatcher/Dispatcher.java | 2 +- .../flink/runtime/jobmanager/JobManager.scala | 5 +- .../runtime/dispatcher/DispatcherTest.java | 46 +++++++++++++++++++ 3 files changed, 49 insertions(+), 4 deletions(-) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/Dispatcher.java b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/Dispatcher.java index 5306d6f0e65e3..0aa9dfc6af712 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 @@ -574,11 +574,11 @@ private CompletableFuture removeJob(JobID jobId, boolean cleanupHA) { return jobManagerRunnerTerminationFuture.thenRunAsync( () -> { jobManagerMetricGroup.removeJob(jobId); - blobServer.cleanupJob(jobId, cleanupHA); if (cleanupHA) { try { submittedJobGraphStore.removeJobGraph(jobId); + blobServer.cleanupJob(jobId, cleanupHA); } catch (Exception e) { log.warn("Could not properly remove job {} from submitted job graph store.", jobId); } 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 1c8174f48ce9b..94469a8b606a7 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 @@ -1734,6 +1734,8 @@ class JobManager( // and the ZooKeeper client is closed. Not removing the job immediately allow the // shutdown to release all resources. submittedJobGraphs.removeJobGraph(jobID) + val result = blobServer.cleanupJob(jobID, removeJobFromStateBackend) + } catch { case t: Throwable => log.warn(s"Could not remove submitted job graph $jobID.", t) } @@ -1759,10 +1761,7 @@ class JobManager( case None => None } - // remove all job-related BLOBs from local and HA store libraryCacheManager.unregisterJob(jobID) - blobServer.cleanupJob(jobID, removeJobFromStateBackend) - jobManagerMetricGroup.removeJob(jobID) futureOption 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 745e9cb664eb2..ac4f1a862c591 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 @@ -23,6 +23,7 @@ import org.apache.flink.configuration.BlobServerOptions; import org.apache.flink.configuration.Configuration; import org.apache.flink.runtime.blob.BlobServer; +import org.apache.flink.runtime.blob.PermanentBlobKey; import org.apache.flink.runtime.blob.VoidBlobStore; import org.apache.flink.runtime.checkpoint.Checkpoints; import org.apache.flink.runtime.checkpoint.StandaloneCheckpointRecoveryFactory; @@ -72,6 +73,7 @@ import org.junit.BeforeClass; import org.junit.Rule; import org.junit.Test; +import org.junit.rules.ExpectedException; import org.junit.rules.TemporaryFolder; import org.junit.rules.TestName; @@ -83,6 +85,7 @@ import java.net.URI; import java.net.URISyntaxException; import java.nio.file.Files; +import java.nio.file.NoSuchFileException; import java.nio.file.Path; import java.nio.file.Paths; import java.util.Collection; @@ -125,6 +128,9 @@ public class DispatcherTest extends TestLogger { @Rule public TestName name = new TestName(); + @Rule + public ExpectedException expectedException = ExpectedException.none(); + private JobGraph jobGraph; private TestingFatalErrorHandler fatalErrorHandler; @@ -294,6 +300,30 @@ public void testSubmittedJobGraphListener() throws Exception { assertThat(dispatcherGateway.listJobs(TIMEOUT).get(), hasSize(1)); } + @Test + public void testBlobsAreRemovedOnlyIfJobIsRemovedProperly() throws Exception { + dispatcherLeaderElectionService.isLeader(UUID.randomUUID()).get(); + PermanentBlobKey key = blobServer.putPermanent(TEST_JOB_ID, new byte[128]); + submittedJobGraphStore.setRemovalFailure(new Exception("Failed to Remove future")); + final DispatcherGateway dispatcherGateway = dispatcher.getSelfGateway(DispatcherGateway.class); + dispatcherGateway.submitJob(jobGraph, TIMEOUT).get(); + + ArchivedExecutionGraph executionGraph = new ArchivedExecutionGraphBuilder() + .setJobID(TEST_JOB_ID) + .setState(JobStatus.CANCELED) + .build(); + + dispatcher.completeJobExecution(executionGraph); + //Assert that blob was not removed, since exception was thrown while removing the job + assertThat(blobServer.getFile(TEST_JOB_ID, key), notNullValue(File.class)); + submittedJobGraphStore.setRemovalFailure(null); + dispatcher.completeJobExecution(executionGraph); + + //Job removing did not throw exception now, blob should be null + expectedException.expect(NoSuchFileException.class); + blobServer.getFile(TEST_JOB_ID, key); + } + @Test public void testOnAddedJobGraphRecoveryFailure() throws Exception { final FlinkException expectedFailure = new FlinkException("Expected failure"); @@ -619,10 +649,17 @@ private static final class FailableSubmittedJobGraphStore extends InMemorySubmit @Nullable private Exception recoveryFailure = null; + @Nullable + private Exception removalFailure = null; + void setRecoveryFailure(@Nullable Exception recoveryFailure) { this.recoveryFailure = recoveryFailure; } + void setRemovalFailure(@Nullable Exception removalFailure) { + this.removalFailure = removalFailure; + } + @Override public synchronized SubmittedJobGraph recoverJobGraph(JobID jobId) throws Exception { if (recoveryFailure != null) { @@ -631,5 +668,14 @@ public synchronized SubmittedJobGraph recoverJobGraph(JobID jobId) throws Except return super.recoverJobGraph(jobId); } } + + @Override + public synchronized void removeJobGraph(JobID jobId) throws Exception { + if (removalFailure != null) { + throw removalFailure; + } else { + super.removeJobGraph(jobId); + } + } } } From e2b4ffc016da822dda544b31fb3caf679f80a9d9 Mon Sep 17 00:00:00 2001 From: Till Rohrmann Date: Wed, 18 Jul 2018 13:49:09 +0200 Subject: [PATCH 15/65] [FLINK-9575][tests] Simplify DispatcherTest#testBlobsAreRemovedOnlyIfJobIsRemovedProperly Move DispatcherTest#testBlobsAreRemovedOnlyIfJobIsRemovedProperly into DispatcherResourceCleanupTest and split it up into a success and failure case. Moreover, this commit changes the logic of blob cleanup to also cleanup locally in case of a removal failure of a job from a SubmittedJobGraphStore. --- .../flink/runtime/dispatcher/Dispatcher.java | 11 ++- .../flink/runtime/jobmanager/JobManager.scala | 31 ++++--- .../DispatcherResourceCleanupTest.java | 89 +++++++++++-------- .../runtime/dispatcher/DispatcherTest.java | 70 +-------------- .../FaultySubmittedJobGraphStore.java | 64 +++++++++++++ 5 files changed, 142 insertions(+), 123 deletions(-) create mode 100644 flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/FaultySubmittedJobGraphStore.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 0aa9dfc6af712..c96acbd3192c9 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 @@ -575,20 +575,25 @@ private CompletableFuture removeJob(JobID jobId, boolean cleanupHA) { () -> { jobManagerMetricGroup.removeJob(jobId); + boolean cleanupHABlobs = false; if (cleanupHA) { try { submittedJobGraphStore.removeJobGraph(jobId); - blobServer.cleanupJob(jobId, cleanupHA); + + // only clean up the HA blobs if we could remove the job from HA storage + cleanupHABlobs = true; } catch (Exception e) { - log.warn("Could not properly remove job {} from submitted job graph store.", jobId); + log.warn("Could not properly remove job {} from submitted job graph store.", jobId, e); } try { runningJobsRegistry.clearJob(jobId); } catch (IOException e) { - log.warn("Could not properly remove job {} from the running jobs registry.", jobId); + log.warn("Could not properly remove job {} from the running jobs registry.", jobId, e); } } + + blobServer.cleanupJob(jobId, cleanupHABlobs); }, getRpcService().getExecutor()); } 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 94469a8b606a7..2a8f49267d927 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 @@ -1725,22 +1725,31 @@ class JobManager( */ private def removeJob(jobID: JobID, removeJobFromStateBackend: Boolean): Option[Future[Unit]] = { // Don't remove the job yet... - val futureOption = currentJobs.get(jobID) match { + val futureOption = currentJobs.remove(jobID) match { case Some((eg, _)) => - val result = if (removeJobFromStateBackend) { - val futureOption = Some(future { + val cleanUpFuture: Future[Unit] = Future { + val cleanupHABlobs = if (removeJobFromStateBackend) { try { // ...otherwise, we can have lingering resources when there is a concurrent shutdown // and the ZooKeeper client is closed. Not removing the job immediately allow the // shutdown to release all resources. submittedJobGraphs.removeJobGraph(jobID) - val result = blobServer.cleanupJob(jobID, removeJobFromStateBackend) - + true } catch { - case t: Throwable => log.warn(s"Could not remove submitted job graph $jobID.", t) + case t: Throwable => { + log.warn(s"Could not remove submitted job graph $jobID.", t) + false + } } - }(context.dispatcher)) + } else { + false + } + blobServer.cleanupJob(jobID, cleanupHABlobs) + () + }(context.dispatcher) + + if (removeJobFromStateBackend) { try { archive ! decorateMessage( ArchiveExecutionGraph( @@ -1749,15 +1758,9 @@ class JobManager( } catch { case t: Throwable => log.warn(s"Could not archive the execution graph $eg.", t) } - - futureOption - } else { - None } - currentJobs.remove(jobID) - - result + Option(cleanUpFuture) case None => None } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/DispatcherResourceCleanupTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/DispatcherResourceCleanupTest.java index e42b14aa29deb..d09ab8df728c1 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/DispatcherResourceCleanupTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/DispatcherResourceCleanupTest.java @@ -38,23 +38,19 @@ import org.apache.flink.runtime.jobgraph.JobGraph; import org.apache.flink.runtime.jobgraph.JobStatus; import org.apache.flink.runtime.jobgraph.JobVertex; -import org.apache.flink.runtime.jobmanager.SubmittedJobGraphStore; import org.apache.flink.runtime.jobmaster.JobManagerRunner; import org.apache.flink.runtime.jobmaster.JobManagerSharedServices; import org.apache.flink.runtime.jobmaster.JobNotFinishedException; import org.apache.flink.runtime.jobmaster.factories.JobManagerJobMetricGroupFactory; import org.apache.flink.runtime.leaderelection.TestingLeaderElectionService; import org.apache.flink.runtime.messages.Acknowledge; -import org.apache.flink.runtime.metrics.groups.JobManagerMetricGroup; import org.apache.flink.runtime.metrics.groups.UnregisteredMetricGroups; -import org.apache.flink.runtime.resourcemanager.ResourceManagerGateway; import org.apache.flink.runtime.resourcemanager.utils.TestingResourceManagerGateway; import org.apache.flink.runtime.rest.handler.legacy.utils.ArchivedExecutionGraphBuilder; import org.apache.flink.runtime.rpc.FatalErrorHandler; import org.apache.flink.runtime.rpc.RpcService; import org.apache.flink.runtime.rpc.TestingRpcService; import org.apache.flink.runtime.testtasks.NoOpInvokable; -import org.apache.flink.runtime.testutils.InMemorySubmittedJobGraphStore; import org.apache.flink.runtime.util.TestingFatalErrorHandler; import org.apache.flink.util.Preconditions; import org.apache.flink.util.TestLogger; @@ -64,11 +60,12 @@ import org.junit.Before; import org.junit.BeforeClass; import org.junit.ClassRule; +import org.junit.Rule; import org.junit.Test; +import org.junit.rules.ExpectedException; import org.junit.rules.TemporaryFolder; import javax.annotation.Nonnull; -import javax.annotation.Nullable; import java.io.File; import java.io.IOException; @@ -93,6 +90,9 @@ public class DispatcherResourceCleanupTest extends TestLogger { @ClassRule public static TemporaryFolder temporaryFolder = new TemporaryFolder(); + @Rule + public ExpectedException expectedException = ExpectedException.none(); + private static final Time timeout = Time.seconds(10L); private static TestingRpcService rpcService; @@ -123,11 +123,12 @@ public class DispatcherResourceCleanupTest extends TestLogger { private File blobFile; - private CompletableFuture storedBlobFuture; - private CompletableFuture deleteAllFuture; + private CompletableFuture storedHABlobFuture; + private CompletableFuture deleteAllHABlobsFuture; private CompletableFuture resultFuture; private CompletableFuture cleanupJobFuture; private CompletableFuture terminationFuture; + private FaultySubmittedJobGraphStore submittedJobGraphStore; @BeforeClass public static void setupClass() { @@ -151,15 +152,16 @@ public void setup() throws Exception { clearedJobLatch = new OneShotLatch(); runningJobsRegistry = new SingleRunningJobsRegistry(jobId, clearedJobLatch); highAvailabilityServices.setRunningJobsRegistry(runningJobsRegistry); - highAvailabilityServices.setSubmittedJobGraphStore(new InMemorySubmittedJobGraphStore()); + submittedJobGraphStore = new FaultySubmittedJobGraphStore(); + highAvailabilityServices.setSubmittedJobGraphStore(submittedJobGraphStore); - storedBlobFuture = new CompletableFuture<>(); - deleteAllFuture = new CompletableFuture<>(); + storedHABlobFuture = new CompletableFuture<>(); + deleteAllHABlobsFuture = new CompletableFuture<>(); final TestingBlobStore testingBlobStore = new TestingBlobStoreBuilder() .setPutFunction( - putArguments -> storedBlobFuture.complete(putArguments.f2)) - .setDeleteAllFunction(deleteAllFuture::complete) + putArguments -> storedHABlobFuture.complete(putArguments.f2)) + .setDeleteAllFunction(deleteAllHABlobsFuture::complete) .createTestingBlobStore(); cleanupJobFuture = new CompletableFuture<>(); @@ -180,7 +182,6 @@ public void setup() throws Exception { Dispatcher.DISPATCHER_NAME + UUID.randomUUID(), configuration, highAvailabilityServices, - highAvailabilityServices.getSubmittedJobGraphStore(), new TestingResourceManagerGateway(), blobServer, new HeartbeatServices(1000L, 1000L), @@ -199,7 +200,7 @@ public void setup() throws Exception { assertThat(blobFile.exists(), is(true)); // verify that we stored the blob also in the BlobStore - assertThat(storedBlobFuture.get(), equalTo(permanentBlobKey)); + assertThat(storedHABlobFuture.get(), equalTo(permanentBlobKey)); } @After @@ -232,7 +233,7 @@ public void testBlobServerCleanupWhenJobFinished() throws Exception { assertThat(cleanupJobFuture.get(), equalTo(jobId)); // verify that we also cleared the BlobStore - assertThat(deleteAllFuture.get(), equalTo(jobId)); + assertThat(deleteAllHABlobsFuture.get(), equalTo(jobId)); assertThat(blobFile.exists(), is(false)); } @@ -256,13 +257,13 @@ public void testBlobServerCleanupWhenJobNotFinished() throws Exception { // verify that we did not clear the BlobStore try { - deleteAllFuture.get(50L, TimeUnit.MILLISECONDS); + deleteAllHABlobsFuture.get(50L, TimeUnit.MILLISECONDS); fail("We should not delete the HA blobs."); } catch (TimeoutException ignored) { // expected } - assertThat(deleteAllFuture.isDone(), is(false)); + assertThat(deleteAllHABlobsFuture.isDone(), is(false)); } @Test @@ -279,13 +280,13 @@ public void testBlobServerCleanupWhenClosingDispatcher() throws Exception { // verify that we did not clear the BlobStore try { - deleteAllFuture.get(50L, TimeUnit.MILLISECONDS); + deleteAllHABlobsFuture.get(50L, TimeUnit.MILLISECONDS); fail("We should not delete the HA blobs."); } catch (TimeoutException ignored) { // expected } - assertThat(deleteAllFuture.isDone(), is(false)); + assertThat(deleteAllHABlobsFuture.isDone(), is(false)); } /** @@ -413,25 +414,37 @@ public void clearJob(JobID jobID) { } } - private static final class TestingDispatcher extends Dispatcher { - TestingDispatcher(RpcService rpcService, String endpointId, Configuration configuration, HighAvailabilityServices highAvailabilityServices, SubmittedJobGraphStore submittedJobGraphStore, ResourceManagerGateway resourceManagerGateway, BlobServer blobServer, HeartbeatServices heartbeatServices, JobManagerMetricGroup jobManagerMetricGroup, @Nullable String metricServiceQueryPath, ArchivedExecutionGraphStore archivedExecutionGraphStore, JobManagerRunnerFactory jobManagerRunnerFactory, FatalErrorHandler fatalErrorHandler) throws Exception { - super( - rpcService, - endpointId, - configuration, - highAvailabilityServices, - submittedJobGraphStore, - resourceManagerGateway, - blobServer, - heartbeatServices, - jobManagerMetricGroup, - metricServiceQueryPath, - archivedExecutionGraphStore, - jobManagerRunnerFactory, - fatalErrorHandler, - null, - VoidHistoryServerArchivist.INSTANCE); - } + @Test + public void testHABlobsAreNotRemovedIfHAJobGraphRemovalFails() throws Exception { + submittedJobGraphStore.setRemovalFailure(new Exception("Failed to Remove future")); + submitJob(); + + ArchivedExecutionGraph executionGraph = new ArchivedExecutionGraphBuilder() + .setJobID(jobId) + .setState(JobStatus.CANCELED) + .build(); + + resultFuture.complete(executionGraph); + terminationFuture.complete(null); + + assertThat(cleanupJobFuture.get(), equalTo(jobId)); + assertThat(deleteAllHABlobsFuture.isDone(), is(false)); + } + + @Test + public void testHABlobsAreRemovedIfHAJobGraphRemovalSucceeds() throws Exception { + submitJob(); + + ArchivedExecutionGraph executionGraph = new ArchivedExecutionGraphBuilder() + .setJobID(jobId) + .setState(JobStatus.CANCELED) + .build(); + + resultFuture.complete(executionGraph); + terminationFuture.complete(null); + + assertThat(cleanupJobFuture.get(), equalTo(jobId)); + assertThat(deleteAllHABlobsFuture.get(), equalTo(jobId)); } private static final class TestingJobManagerRunnerFactory implements Dispatcher.JobManagerRunnerFactory { 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 ac4f1a862c591..d405fcdcf447c 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 @@ -23,7 +23,6 @@ import org.apache.flink.configuration.BlobServerOptions; import org.apache.flink.configuration.Configuration; import org.apache.flink.runtime.blob.BlobServer; -import org.apache.flink.runtime.blob.PermanentBlobKey; import org.apache.flink.runtime.blob.VoidBlobStore; import org.apache.flink.runtime.checkpoint.Checkpoints; import org.apache.flink.runtime.checkpoint.StandaloneCheckpointRecoveryFactory; @@ -61,7 +60,6 @@ import org.apache.flink.runtime.state.CompletedCheckpointStorageLocation; import org.apache.flink.runtime.state.StateBackend; import org.apache.flink.runtime.testtasks.NoOpInvokable; -import org.apache.flink.runtime.testutils.InMemorySubmittedJobGraphStore; import org.apache.flink.runtime.util.TestingFatalErrorHandler; import org.apache.flink.util.ExceptionUtils; import org.apache.flink.util.FlinkException; @@ -73,19 +71,16 @@ import org.junit.BeforeClass; import org.junit.Rule; import org.junit.Test; -import org.junit.rules.ExpectedException; import org.junit.rules.TemporaryFolder; import org.junit.rules.TestName; import javax.annotation.Nonnull; -import javax.annotation.Nullable; import java.io.File; import java.io.IOException; import java.net.URI; import java.net.URISyntaxException; import java.nio.file.Files; -import java.nio.file.NoSuchFileException; import java.nio.file.Path; import java.nio.file.Paths; import java.util.Collection; @@ -128,14 +123,11 @@ public class DispatcherTest extends TestLogger { @Rule public TestName name = new TestName(); - @Rule - public ExpectedException expectedException = ExpectedException.none(); - private JobGraph jobGraph; private TestingFatalErrorHandler fatalErrorHandler; - private FailableSubmittedJobGraphStore submittedJobGraphStore; + private FaultySubmittedJobGraphStore submittedJobGraphStore; private TestingLeaderElectionService dispatcherLeaderElectionService; @@ -175,7 +167,7 @@ public void setUp() throws Exception { fatalErrorHandler = new TestingFatalErrorHandler(); final HeartbeatServices heartbeatServices = new HeartbeatServices(1000L, 10000L); - submittedJobGraphStore = new FailableSubmittedJobGraphStore(); + submittedJobGraphStore = new FaultySubmittedJobGraphStore(); dispatcherLeaderElectionService = new TestingLeaderElectionService(); jobMasterLeaderElectionService = new TestingLeaderElectionService(); @@ -300,30 +292,6 @@ public void testSubmittedJobGraphListener() throws Exception { assertThat(dispatcherGateway.listJobs(TIMEOUT).get(), hasSize(1)); } - @Test - public void testBlobsAreRemovedOnlyIfJobIsRemovedProperly() throws Exception { - dispatcherLeaderElectionService.isLeader(UUID.randomUUID()).get(); - PermanentBlobKey key = blobServer.putPermanent(TEST_JOB_ID, new byte[128]); - submittedJobGraphStore.setRemovalFailure(new Exception("Failed to Remove future")); - final DispatcherGateway dispatcherGateway = dispatcher.getSelfGateway(DispatcherGateway.class); - dispatcherGateway.submitJob(jobGraph, TIMEOUT).get(); - - ArchivedExecutionGraph executionGraph = new ArchivedExecutionGraphBuilder() - .setJobID(TEST_JOB_ID) - .setState(JobStatus.CANCELED) - .build(); - - dispatcher.completeJobExecution(executionGraph); - //Assert that blob was not removed, since exception was thrown while removing the job - assertThat(blobServer.getFile(TEST_JOB_ID, key), notNullValue(File.class)); - submittedJobGraphStore.setRemovalFailure(null); - dispatcher.completeJobExecution(executionGraph); - - //Job removing did not throw exception now, blob should be null - expectedException.expect(NoSuchFileException.class); - blobServer.getFile(TEST_JOB_ID, key); - } - @Test public void testOnAddedJobGraphRecoveryFailure() throws Exception { final FlinkException expectedFailure = new FlinkException("Expected failure"); @@ -644,38 +612,4 @@ public JobManagerRunner createJobManagerRunner( } } - private static final class FailableSubmittedJobGraphStore extends InMemorySubmittedJobGraphStore { - - @Nullable - private Exception recoveryFailure = null; - - @Nullable - private Exception removalFailure = null; - - void setRecoveryFailure(@Nullable Exception recoveryFailure) { - this.recoveryFailure = recoveryFailure; - } - - void setRemovalFailure(@Nullable Exception removalFailure) { - this.removalFailure = removalFailure; - } - - @Override - public synchronized SubmittedJobGraph recoverJobGraph(JobID jobId) throws Exception { - if (recoveryFailure != null) { - throw recoveryFailure; - } else { - return super.recoverJobGraph(jobId); - } - } - - @Override - public synchronized void removeJobGraph(JobID jobId) throws Exception { - if (removalFailure != null) { - throw removalFailure; - } else { - super.removeJobGraph(jobId); - } - } - } } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/FaultySubmittedJobGraphStore.java b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/FaultySubmittedJobGraphStore.java new file mode 100644 index 0000000000000..9238ec383c083 --- /dev/null +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/FaultySubmittedJobGraphStore.java @@ -0,0 +1,64 @@ +/* + * 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.api.common.JobID; +import org.apache.flink.runtime.jobmanager.SubmittedJobGraph; +import org.apache.flink.runtime.testutils.InMemorySubmittedJobGraphStore; + +import javax.annotation.Nullable; + +/** + * {@link InMemorySubmittedJobGraphStore} implementation which can throw artifical errors for + * testing purposes. + */ +final class FaultySubmittedJobGraphStore extends InMemorySubmittedJobGraphStore { + + @Nullable + private Exception recoveryFailure = null; + + @Nullable + private Exception removalFailure = null; + + void setRecoveryFailure(@Nullable Exception recoveryFailure) { + this.recoveryFailure = recoveryFailure; + } + + void setRemovalFailure(@Nullable Exception removalFailure) { + this.removalFailure = removalFailure; + } + + @Override + public synchronized SubmittedJobGraph recoverJobGraph(JobID jobId) throws Exception { + if (recoveryFailure != null) { + throw recoveryFailure; + } else { + return super.recoverJobGraph(jobId); + } + } + + @Override + public synchronized void removeJobGraph(JobID jobId) throws Exception { + if (removalFailure != null) { + throw removalFailure; + } else { + super.removeJobGraph(jobId); + } + } +} From e87532e01ce180ab8f6666c91edc57814bd25b92 Mon Sep 17 00:00:00 2001 From: zentol Date: Tue, 17 Jul 2018 11:34:29 +0200 Subject: [PATCH 16/65] [FLINK-9872][tests] Properly cancel test job This closes #6349. --- .../flink/test/checkpointing/SavepointITCase.java | 12 ++++++++++-- 1 file changed, 10 insertions(+), 2 deletions(-) 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 7eb5b8f346617..fa8fa34a6296a 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 @@ -592,7 +592,11 @@ public Integer map(Integer value) throws Exception { latch.await(); } savepointPath = client.triggerSavepoint(jobGraph.getJobID(), null).get(); - source.cancel(); + + client.cancel(jobGraph.getJobID()); + while (!client.getJobStatus(jobGraph.getJobID()).get().isGloballyTerminalState()) { + Thread.sleep(100); + } jobGraph = streamGraph.getJobGraph(); jobGraph.setSavepointRestoreSettings(SavepointRestoreSettings.forPath(savepointPath)); @@ -602,7 +606,11 @@ public Integer map(Integer value) throws Exception { for (OneShotLatch latch : iterTestRestoreWait) { latch.await(); } - source.cancel(); + + client.cancel(jobGraph.getJobID()); + while (!client.getJobStatus(jobGraph.getJobID()).get().isGloballyTerminalState()) { + Thread.sleep(100); + } } finally { if (null != savepointPath) { client.disposeSavepoint(savepointPath); From 9ae2c3bacc4c25b813a897bd419226c5aaa21bd3 Mon Sep 17 00:00:00 2001 From: zentol Date: Tue, 17 Jul 2018 13:29:16 +0200 Subject: [PATCH 17/65] [FLINK-9815][yarn][tests] Harden tests against slow job shutdowns This closes #6352. --- .../yarn/YARNHighAvailabilityITCase.java | 12 ++-- .../org/apache/flink/yarn/YarnTestBase.java | 63 ++++++++++++------- 2 files changed, 48 insertions(+), 27 deletions(-) diff --git a/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YARNHighAvailabilityITCase.java b/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YARNHighAvailabilityITCase.java index 18bcfebed329c..9a8f5033f3ff5 100644 --- a/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YARNHighAvailabilityITCase.java +++ b/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YARNHighAvailabilityITCase.java @@ -133,7 +133,7 @@ public void testMultipleAMKill() throws Exception { "@@" + CheckpointingOptions.CHECKPOINTS_DIRECTORY + "=" + fsStateHandlePath + "/checkpoints" + "@@" + HighAvailabilityOptions.HA_STORAGE_PATH.key() + "=" + fsStateHandlePath + "/recovery"); - ClusterClient yarnCluster = null; + ClusterClient yarnClusterClient = null; final FiniteDuration timeout = new FiniteDuration(2, TimeUnit.MINUTES); @@ -147,10 +147,10 @@ public void testMultipleAMKill() throws Exception { .createClusterSpecification(); try { - yarnCluster = flinkYarnClient.deploySessionCluster(clusterSpecification); + yarnClusterClient = flinkYarnClient.deploySessionCluster(clusterSpecification); highAvailabilityServices = HighAvailabilityServicesUtils.createHighAvailabilityServices( - yarnCluster.getFlinkConfiguration(), + yarnClusterClient.getFlinkConfiguration(), Executors.directExecutor(), HighAvailabilityServicesUtils.AddressResolution.TRY_ADDRESS_RESOLUTION); @@ -201,8 +201,10 @@ protected void run() { }}; } finally { - if (yarnCluster != null) { - yarnCluster.shutdown(); + if (yarnClusterClient != null) { + log.info("Shutting down the Flink Yarn application."); + yarnClusterClient.shutDownCluster(); + yarnClusterClient.shutdown(); } if (highAvailabilityServices != null) { diff --git a/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YarnTestBase.java b/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YarnTestBase.java index 514a3d55fff2f..1a0520f68fe9a 100644 --- a/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YarnTestBase.java +++ b/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YarnTestBase.java @@ -18,6 +18,7 @@ package org.apache.flink.yarn; +import org.apache.flink.api.common.time.Deadline; import org.apache.flink.client.cli.CliFrontend; import org.apache.flink.configuration.ConfigConstants; import org.apache.flink.configuration.CoreOptions; @@ -67,6 +68,7 @@ import java.io.PipedInputStream; import java.io.PipedOutputStream; import java.io.PrintStream; +import java.time.Duration; import java.util.ArrayList; import java.util.Arrays; import java.util.HashMap; @@ -79,6 +81,7 @@ import java.util.UUID; import java.util.concurrent.ConcurrentMap; import java.util.regex.Pattern; +import java.util.stream.Collectors; /** * This base class allows to use the MiniYARNCluster. @@ -186,39 +189,55 @@ public static void populateYarnSecureConfigurations(Configuration conf, String p conf.set("hadoop.security.auth_to_local", "RULE:[1:$1] RULE:[2:$1]"); } - /** - * Sleep a bit between the tests (we are re-using the YARN cluster for the tests). - */ - @After - public void sleep() { - try { - Thread.sleep(500); - } catch (InterruptedException e) { - Assert.fail("Should not happen"); - } - } - @Before - public void checkClusterEmpty() throws IOException, YarnException { + public void checkClusterEmpty() { if (yarnClient == null) { yarnClient = YarnClient.createYarnClient(); yarnClient.init(getYarnConfiguration()); yarnClient.start(); } - List apps = yarnClient.getApplications(); - for (ApplicationReport app : apps) { - if (app.getYarnApplicationState() != YarnApplicationState.FINISHED - && app.getYarnApplicationState() != YarnApplicationState.KILLED - && app.getYarnApplicationState() != YarnApplicationState.FAILED) { - Assert.fail("There is at least one application on the cluster is not finished." + - "App " + app.getApplicationId() + " is in state " + app.getYarnApplicationState()); + flinkConfiguration = new org.apache.flink.configuration.Configuration(globalConfiguration); + + isNewMode = Objects.equals(TestBaseUtils.CodebaseType.NEW, TestBaseUtils.getCodebaseType()); + } + + /** + * Sleep a bit between the tests (we are re-using the YARN cluster for the tests). + */ + @After + public void sleep() throws IOException, YarnException { + Deadline deadline = Deadline.now().plus(Duration.ofSeconds(10)); + + boolean isAnyJobRunning = yarnClient.getApplications().stream() + .anyMatch(YarnTestBase::isApplicationRunning); + + while (deadline.hasTimeLeft() && isAnyJobRunning) { + try { + Thread.sleep(500); + } catch (InterruptedException e) { + Assert.fail("Should not happen"); } + isAnyJobRunning = yarnClient.getApplications().stream() + .anyMatch(YarnTestBase::isApplicationRunning); } - flinkConfiguration = new org.apache.flink.configuration.Configuration(globalConfiguration); + if (isAnyJobRunning) { + final List runningApps = yarnClient.getApplications().stream() + .filter(YarnTestBase::isApplicationRunning) + .map(app -> "App " + app.getApplicationId() + " is in state " + app.getYarnApplicationState() + '.') + .collect(Collectors.toList()); + if (!runningApps.isEmpty()) { + Assert.fail("There is at least one application on the cluster that is not finished." + runningApps); + } + } + } - isNewMode = Objects.equals(TestBaseUtils.CodebaseType.NEW, TestBaseUtils.getCodebaseType()); + private static boolean isApplicationRunning(ApplicationReport app) { + final YarnApplicationState yarnApplicationState = app.getYarnApplicationState(); + return yarnApplicationState != YarnApplicationState.FINISHED + && app.getYarnApplicationState() != YarnApplicationState.KILLED + && app.getYarnApplicationState() != YarnApplicationState.FAILED; } @Nullable From 4cd682ded2a24a5a658eaab7b4bf9100b76a99f1 Mon Sep 17 00:00:00 2001 From: zentol Date: Fri, 13 Jul 2018 14:00:10 +0200 Subject: [PATCH 18/65] [FLINK-9499][rest] Support JSON request in JarHandlers This closes #6330. --- docs/_includes/generated/rest_dispatcher.html | 26 +- flink-runtime-web/pom.xml | 41 +++ .../webmonitor/handlers/JarRunHandler.java | 75 ++++- .../webmonitor/handlers/JarRunHeaders.java | 7 +- .../handlers/JarRunRequestBody.java | 109 ++++++ .../handlers/JarRunHandlerParameterTest.java | 314 ++++++++++++++++++ .../handlers/JarRunHandlerTest.java | 3 +- .../handlers/JarRunRequestBodyTest.java | 56 ++++ .../handlers/JarSubmissionITCase.java | 4 +- .../testutils/ParameterProgram.java | 36 ++ .../scripts/modules/submit/submit.ctrl.coffee | 9 +- .../scripts/modules/submit/submit.svc.coffee | 4 +- .../web-dashboard/web/js/hs/index.js | 4 +- .../web-dashboard/web/js/index.js | 2 +- 14 files changed, 662 insertions(+), 28 deletions(-) create mode 100644 flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/JarRunRequestBody.java create mode 100644 flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/JarRunHandlerParameterTest.java create mode 100644 flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/JarRunRequestBodyTest.java create mode 100644 flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/testutils/ParameterProgram.java diff --git a/docs/_includes/generated/rest_dispatcher.html b/docs/_includes/generated/rest_dispatcher.html index 6ed59becf2133..ab0f4e0a0a4f4 100644 --- a/docs/_includes/generated/rest_dispatcher.html +++ b/docs/_includes/generated/rest_dispatcher.html @@ -360,11 +360,31 @@ - -
+ +
             
-{}            
+{
+  "type" : "object",
+  "id" : "urn:jsonschema:org:apache:flink:runtime:webmonitor:handlers:JarRunRequestBody",
+  "properties" : {
+    "entryClass" : {
+      "type" : "string"
+    },
+    "programArgs" : {
+      "type" : "string"
+    },
+    "parallelism" : {
+      "type" : "integer"
+    },
+    "allowNonRestoredState" : {
+      "type" : "boolean"
+    },
+    "savepointPath" : {
+      "type" : "string"
+    }
+  }
+}            
           
diff --git a/flink-runtime-web/pom.xml b/flink-runtime-web/pom.xml index 943a737109256..058be81928bce 100644 --- a/flink-runtime-web/pom.xml +++ b/flink-runtime-web/pom.xml @@ -34,6 +34,11 @@ under the License. jar + + parameter-program + parameter-program-without-manifest + + test-program-jar process-test-classes @@ -153,6 +159,39 @@ under the License. test-program + + + test-parameter-program-jar + process-test-classes + + jar + + + + org/apache/flink/runtime/webmonitor/handlers/utils/TestProgram.java + + + + org.apache.flink.runtime.webmonitor.testutils.ParameterProgram + + + ${test.parameterProgram.name} + + + + + test-parameter-program-jar-without-manifest + process-test-classes + + jar + + + + org/apache/flink/runtime/webmonitor/handlers/utils/TestProgram.java + + ${test.ParameterProgramNoManifest.name} + + @@ -162,6 +201,8 @@ under the License. ${project.build.directory} + ${test.parameterProgram.name} + ${test.ParameterProgramNoManifest.name} 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 858a05ca00e5e..bae4ba86a0e33 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 @@ -33,13 +33,15 @@ import org.apache.flink.runtime.rest.handler.AbstractRestHandler; import org.apache.flink.runtime.rest.handler.HandlerRequest; import org.apache.flink.runtime.rest.handler.RestHandlerException; -import org.apache.flink.runtime.rest.messages.EmptyRequestBody; import org.apache.flink.runtime.rest.messages.MessageHeaders; import org.apache.flink.runtime.webmonitor.retriever.GatewayRetriever; import org.apache.flink.util.FlinkException; +import org.apache.flink.util.function.SupplierWithException; import org.apache.flink.shaded.netty4.io.netty.handler.codec.http.HttpResponseStatus; +import org.slf4j.Logger; + import javax.annotation.Nonnull; import javax.annotation.Nullable; @@ -61,7 +63,7 @@ * Handler to submit jobs uploaded via the Web UI. */ public class JarRunHandler extends - AbstractRestHandler { + AbstractRestHandler { private final Path jarDir; @@ -74,7 +76,7 @@ public JarRunHandler( final GatewayRetriever leaderRetriever, final Time timeout, final Map responseHeaders, - final MessageHeaders messageHeaders, + final MessageHeaders messageHeaders, final Path jarDir, final Configuration configuration, final Executor executor) { @@ -87,15 +89,33 @@ public JarRunHandler( @Override protected CompletableFuture handleRequest( - @Nonnull final HandlerRequest request, + @Nonnull final HandlerRequest request, @Nonnull final DispatcherGateway gateway) throws RestHandlerException { + final JarRunRequestBody requestBody = request.getRequestBody(); + final String pathParameter = request.getPathParameter(JarIdPathParameter.class); final Path jarFile = jarDir.resolve(pathParameter); - final String entryClass = emptyToNull(getQueryParameter(request, EntryClassQueryParameter.class)); - final List programArgs = tokenizeArguments(getQueryParameter(request, ProgramArgsQueryParameter.class)); - final int parallelism = getQueryParameter(request, ParallelismQueryParameter.class, ExecutionConfig.PARALLELISM_DEFAULT); + final String entryClass = fromRequestBodyOrQueryParameter( + emptyToNull(requestBody.getEntryClassName()), + () -> emptyToNull(getQueryParameter(request, EntryClassQueryParameter.class)), + null, + log); + + final List programArgs = tokenizeArguments( + fromRequestBodyOrQueryParameter( + emptyToNull(requestBody.getProgramArguments()), + () -> getQueryParameter(request, ProgramArgsQueryParameter.class), + null, + log)); + + final int parallelism = fromRequestBodyOrQueryParameter( + requestBody.getParallelism(), + () -> getQueryParameter(request, ParallelismQueryParameter.class), + ExecutionConfig.PARALLELISM_DEFAULT, + log); + final SavepointRestoreSettings savepointRestoreSettings = getSavepointRestoreSettings(request); final CompletableFuture jobGraphFuture = getJobGraphAsync( @@ -134,12 +154,22 @@ protected CompletableFuture handleRequest( }); } - private static SavepointRestoreSettings getSavepointRestoreSettings( - final @Nonnull HandlerRequest request) + private SavepointRestoreSettings getSavepointRestoreSettings( + final @Nonnull HandlerRequest request) throws RestHandlerException { - final boolean allowNonRestoredState = getQueryParameter(request, AllowNonRestoredStateQueryParameter.class, false); - final String savepointPath = getQueryParameter(request, SavepointPathQueryParameter.class); + final JarRunRequestBody requestBody = request.getRequestBody(); + + final boolean allowNonRestoredState = fromRequestBodyOrQueryParameter( + requestBody.getAllowNonRestoredState(), + () -> getQueryParameter(request, AllowNonRestoredStateQueryParameter.class), + false, + log); + final String savepointPath = fromRequestBodyOrQueryParameter( + emptyToNull(requestBody.getSavepointPath()), + () -> emptyToNull(getQueryParameter(request, SavepointPathQueryParameter.class)), + null, + log); final SavepointRestoreSettings savepointRestoreSettings; if (savepointPath != null) { savepointRestoreSettings = SavepointRestoreSettings.forPath( @@ -151,6 +181,29 @@ private static SavepointRestoreSettings getSavepointRestoreSettings( return savepointRestoreSettings; } + /** + * Returns {@code requestValue} if it is not null, otherwise returns the query parameter value + * if it is not null, otherwise returns the default value. + */ + private static T fromRequestBodyOrQueryParameter( + T requestValue, + SupplierWithException queryParameterExtractor, + T defaultValue, + Logger log) throws RestHandlerException { + if (requestValue != null) { + return requestValue; + } else { + T queryParameterValue = queryParameterExtractor.get(); + if (queryParameterValue != null) { + log.warn("Configuring the job submission via query parameters is deprecated." + + " Please migrate to submitting a JSON request instead."); + return queryParameterValue; + } else { + return defaultValue; + } + } + } + private CompletableFuture getJobGraphAsync( final Path jarFile, @Nullable final String entryClass, diff --git a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/JarRunHeaders.java b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/JarRunHeaders.java index a1ad9554ef482..fa062e906bc2a 100644 --- a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/JarRunHeaders.java +++ b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/JarRunHeaders.java @@ -19,7 +19,6 @@ package org.apache.flink.runtime.webmonitor.handlers; import org.apache.flink.runtime.rest.HttpMethodWrapper; -import org.apache.flink.runtime.rest.messages.EmptyRequestBody; import org.apache.flink.runtime.rest.messages.MessageHeaders; import org.apache.flink.shaded.netty4.io.netty.handler.codec.http.HttpResponseStatus; @@ -27,7 +26,7 @@ /** * {@link MessageHeaders} for {@link JarRunHandler}. */ -public class JarRunHeaders implements MessageHeaders { +public class JarRunHeaders implements MessageHeaders { private static final JarRunHeaders INSTANCE = new JarRunHeaders(); @@ -44,8 +43,8 @@ public HttpResponseStatus getResponseStatusCode() { } @Override - public Class getRequestClass() { - return EmptyRequestBody.class; + public Class getRequestClass() { + return JarRunRequestBody.class; } @Override diff --git a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/JarRunRequestBody.java b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/JarRunRequestBody.java new file mode 100644 index 0000000000000..b30ae6113ef8d --- /dev/null +++ b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/JarRunRequestBody.java @@ -0,0 +1,109 @@ +/* + * 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.webmonitor.handlers; + +import org.apache.flink.runtime.rest.messages.RequestBody; + +import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonCreator; +import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonIgnore; +import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonInclude; +import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonProperty; + +import javax.annotation.Nullable; + +/** + * {@link RequestBody} for running a jar. + */ +@JsonInclude(JsonInclude.Include.NON_NULL) +public class JarRunRequestBody implements RequestBody { + + private static final String FIELD_NAME_ENTRY_CLASS = "entryClass"; + private static final String FIELD_NAME_PROGRAM_ARGUMENTS = "programArgs"; + private static final String FIELD_NAME_PARALLELISM = "parallelism"; + private static final String FIELD_NAME_ALLOW_NON_RESTORED_STATE = "allowNonRestoredState"; + private static final String FIELD_NAME_SAVEPOINT_PATH = "savepointPath"; + + @JsonProperty(FIELD_NAME_ENTRY_CLASS) + @Nullable + private String entryClassName; + + @JsonProperty(FIELD_NAME_PROGRAM_ARGUMENTS) + @Nullable + private String programArguments; + + @JsonProperty(FIELD_NAME_PARALLELISM) + @Nullable + private Integer parallelism; + + @JsonProperty(FIELD_NAME_ALLOW_NON_RESTORED_STATE) + @Nullable + private Boolean allowNonRestoredState; + + @JsonProperty(FIELD_NAME_SAVEPOINT_PATH) + @Nullable + private String savepointPath; + + public JarRunRequestBody() { + this(null, null, null, null, null); + } + + @JsonCreator + public JarRunRequestBody( + @Nullable @JsonProperty(FIELD_NAME_ENTRY_CLASS) String entryClassName, + @Nullable @JsonProperty(FIELD_NAME_PROGRAM_ARGUMENTS) String programArguments, + @Nullable @JsonProperty(FIELD_NAME_PARALLELISM) Integer parallelism, + @Nullable @JsonProperty(FIELD_NAME_ALLOW_NON_RESTORED_STATE) Boolean allowNonRestoredState, + @Nullable @JsonProperty(FIELD_NAME_SAVEPOINT_PATH) String savepointPath) { + this.entryClassName = entryClassName; + this.programArguments = programArguments; + this.parallelism = parallelism; + this.allowNonRestoredState = allowNonRestoredState; + this.savepointPath = savepointPath; + } + + @Nullable + @JsonIgnore + public String getEntryClassName() { + return entryClassName; + } + + @Nullable + @JsonIgnore + public String getProgramArguments() { + return programArguments; + } + + @Nullable + @JsonIgnore + public Integer getParallelism() { + return parallelism; + } + + @Nullable + @JsonIgnore + public Boolean getAllowNonRestoredState() { + return allowNonRestoredState; + } + + @Nullable + @JsonIgnore + public String getSavepointPath() { + return savepointPath; + } +} diff --git a/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/JarRunHandlerParameterTest.java b/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/JarRunHandlerParameterTest.java new file mode 100644 index 0000000000000..8bb358a543c58 --- /dev/null +++ b/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/JarRunHandlerParameterTest.java @@ -0,0 +1,314 @@ +/* + * 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.webmonitor.handlers; + +import org.apache.flink.api.common.ExecutionConfig; +import org.apache.flink.api.common.time.Time; +import org.apache.flink.configuration.BlobServerOptions; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.runtime.dispatcher.DispatcherGateway; +import org.apache.flink.runtime.jobgraph.JobGraph; +import org.apache.flink.runtime.jobgraph.SavepointRestoreSettings; +import org.apache.flink.runtime.messages.Acknowledge; +import org.apache.flink.runtime.rest.handler.HandlerRequest; +import org.apache.flink.runtime.rest.handler.HandlerRequestException; +import org.apache.flink.runtime.rest.messages.MessageParameter; +import org.apache.flink.runtime.rest.messages.MessageQueryParameter; +import org.apache.flink.runtime.testingUtils.TestingUtils; +import org.apache.flink.runtime.util.BlobServerResource; +import org.apache.flink.runtime.webmonitor.TestingDispatcherGateway; +import org.apache.flink.runtime.webmonitor.retriever.GatewayRetriever; +import org.apache.flink.runtime.webmonitor.testutils.ParameterProgram; +import org.apache.flink.util.TestLogger; +import org.apache.flink.util.function.SupplierWithException; +import org.apache.flink.util.function.ThrowingConsumer; + +import org.junit.Assert; +import org.junit.Before; +import org.junit.BeforeClass; +import org.junit.ClassRule; +import org.junit.Test; +import org.junit.rules.TemporaryFolder; + +import java.nio.file.Files; +import java.nio.file.Path; +import java.nio.file.Paths; +import java.util.Collections; +import java.util.List; +import java.util.Map; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.Executor; +import java.util.concurrent.atomic.AtomicReference; +import java.util.stream.Collectors; + +/** + * Tests for the parameter handling of the {@link JarRunHandler}. + */ +public class JarRunHandlerParameterTest extends TestLogger { + + @ClassRule + public static final TemporaryFolder TMP = new TemporaryFolder(); + + @ClassRule + public static final BlobServerResource BLOB_SERVER_RESOURCE = new BlobServerResource(); + + private static final AtomicReference lastSubmittedJobGraphReference = new AtomicReference<>(); + private static JarRunHandler handler; + private static Path jarWithManifest; + private static Path jarWithoutManifest; + private static TestingDispatcherGateway restfulGateway; + + @BeforeClass + public static void setup() throws Exception { + Path jarDir = TMP.newFolder().toPath(); + + // properties are set property by surefire plugin + final String parameterProgramJarName = System.getProperty("parameterJarName") + ".jar"; + final String parameterProgramWithoutManifestJarName = System.getProperty("parameterJarWithoutManifestName") + ".jar"; + final Path jarLocation = Paths.get(System.getProperty("targetDir")); + + jarWithManifest = Files.copy( + jarLocation.resolve(parameterProgramJarName), + jarDir.resolve("program-with-manifest.jar")); + jarWithoutManifest = Files.copy( + jarLocation.resolve(parameterProgramWithoutManifestJarName), + jarDir.resolve("program-without-manifest.jar")); + + Configuration config = new Configuration(); + config.setString(BlobServerOptions.STORAGE_DIRECTORY, + TMP.newFolder().getAbsolutePath()); + + restfulGateway = new TestingDispatcherGateway.Builder() + .setBlobServerPort(BLOB_SERVER_RESOURCE.getBlobServerPort()) + .setSubmitFunction(jobGraph -> { + lastSubmittedJobGraphReference.set(jobGraph); + return CompletableFuture.completedFuture(Acknowledge.get()); + }) + .build(); + final GatewayRetriever gatewayRetriever = () -> CompletableFuture.completedFuture(restfulGateway); + final CompletableFuture localAddressFuture = CompletableFuture.completedFuture("shazam://localhost:12345"); + final Time timeout = Time.seconds(10); + final Map responseHeaders = Collections.emptyMap(); + final Executor executor = TestingUtils.defaultExecutor(); + + handler = new JarRunHandler( + localAddressFuture, + gatewayRetriever, + timeout, + responseHeaders, + JarRunHeaders.getInstance(), + jarDir, + new Configuration(), + executor); + } + + @Before + public void reset() { + ParameterProgram.actualArguments = null; + } + + @Test + public void testDefaultParameters() throws Exception { + // baseline, ensure that reasonable defaults are chosen + sendRequestAndValidateGraph( + handler, + restfulGateway, + () -> createRequest( + new JarRunRequestBody(), + JarRunHeaders.getInstance().getUnresolvedMessageParameters(), + jarWithManifest + ), + jobGraph -> { + Assert.assertEquals(0, ParameterProgram.actualArguments.length); + + Assert.assertEquals(ExecutionConfig.PARALLELISM_DEFAULT, getExecutionConfig(jobGraph).getParallelism()); + + final SavepointRestoreSettings savepointRestoreSettings = jobGraph.getSavepointRestoreSettings(); + Assert.assertFalse(savepointRestoreSettings.allowNonRestoredState()); + Assert.assertNull(savepointRestoreSettings.getRestorePath()); + } + ); + } + + @Test + public void testConfigurationViaQueryParameters() throws Exception { + // configure submission via query parameters + sendRequestAndValidateGraph( + handler, + restfulGateway, + () -> { + final JarRunMessageParameters parameters = JarRunHeaders.getInstance().getUnresolvedMessageParameters(); + parameters.allowNonRestoredStateQueryParameter.resolve(Collections.singletonList(true)); + parameters.savepointPathQueryParameter.resolve(Collections.singletonList("/foo/bar")); + parameters.entryClassQueryParameter.resolve(Collections.singletonList(ParameterProgram.class.getCanonicalName())); + parameters.parallelismQueryParameter.resolve(Collections.singletonList(4)); + parameters.programArgsQueryParameter.resolve(Collections.singletonList("--host localhost --port 1234")); + + return createRequest( + new JarRunRequestBody(), + parameters, + jarWithoutManifest + ); + }, + jobGraph -> { + Assert.assertEquals(4, ParameterProgram.actualArguments.length); + Assert.assertEquals("--host", ParameterProgram.actualArguments[0]); + Assert.assertEquals("localhost", ParameterProgram.actualArguments[1]); + Assert.assertEquals("--port", ParameterProgram.actualArguments[2]); + Assert.assertEquals("1234", ParameterProgram.actualArguments[3]); + + Assert.assertEquals(4, getExecutionConfig(jobGraph).getParallelism()); + + final SavepointRestoreSettings savepointRestoreSettings = jobGraph.getSavepointRestoreSettings(); + Assert.assertTrue(savepointRestoreSettings.allowNonRestoredState()); + Assert.assertEquals("/foo/bar", savepointRestoreSettings.getRestorePath()); + } + ); + } + + @Test + public void testConfigurationViaJsonRequest() throws Exception { + sendRequestAndValidateGraph( + handler, + restfulGateway, + () -> { + final JarRunRequestBody jsonRequest = new JarRunRequestBody( + ParameterProgram.class.getCanonicalName(), + "--host localhost --port 1234", + 4, + true, + "/foo/bar" + ); + + return createRequest( + jsonRequest, + JarRunHeaders.getInstance().getUnresolvedMessageParameters(), + jarWithoutManifest + ); + }, + jobGraph -> { + Assert.assertEquals(4, ParameterProgram.actualArguments.length); + Assert.assertEquals("--host", ParameterProgram.actualArguments[0]); + Assert.assertEquals("localhost", ParameterProgram.actualArguments[1]); + Assert.assertEquals("--port", ParameterProgram.actualArguments[2]); + Assert.assertEquals("1234", ParameterProgram.actualArguments[3]); + + Assert.assertEquals(4, getExecutionConfig(jobGraph).getParallelism()); + + final SavepointRestoreSettings savepointRestoreSettings = jobGraph.getSavepointRestoreSettings(); + Assert.assertTrue(savepointRestoreSettings.allowNonRestoredState()); + Assert.assertEquals("/foo/bar", savepointRestoreSettings.getRestorePath()); + } + ); + } + + @Test + public void testParameterPrioritization() throws Exception { + // configure submission via query parameters and JSON request, JSON should be prioritized + sendRequestAndValidateGraph( + handler, + restfulGateway, + () -> { + final JarRunRequestBody jsonRequest = new JarRunRequestBody( + ParameterProgram.class.getCanonicalName(), + "--host localhost --port 1234", + 4, + true, + "/foo/bar" + ); + + final JarRunMessageParameters parameters = JarRunHeaders.getInstance().getUnresolvedMessageParameters(); + parameters.allowNonRestoredStateQueryParameter.resolve(Collections.singletonList(false)); + parameters.savepointPathQueryParameter.resolve(Collections.singletonList("/no/uh")); + parameters.entryClassQueryParameter.resolve(Collections.singletonList("please.dont.run.me")); + parameters.parallelismQueryParameter.resolve(Collections.singletonList(64)); + parameters.programArgsQueryParameter.resolve(Collections.singletonList("--host wrong --port wrong")); + + return createRequest( + jsonRequest, + parameters, + jarWithoutManifest + ); + }, + jobGraph -> { + Assert.assertEquals(4, ParameterProgram.actualArguments.length); + Assert.assertEquals("--host", ParameterProgram.actualArguments[0]); + Assert.assertEquals("localhost", ParameterProgram.actualArguments[1]); + Assert.assertEquals("--port", ParameterProgram.actualArguments[2]); + Assert.assertEquals("1234", ParameterProgram.actualArguments[3]); + + Assert.assertEquals(4, getExecutionConfig(jobGraph).getParallelism()); + + final SavepointRestoreSettings savepointRestoreSettings = jobGraph.getSavepointRestoreSettings(); + Assert.assertTrue(savepointRestoreSettings.allowNonRestoredState()); + Assert.assertEquals("/foo/bar", savepointRestoreSettings.getRestorePath()); + } + ); + } + + private static HandlerRequest createRequest( + JarRunRequestBody requestBody, + JarRunMessageParameters parameters, + Path jar) throws HandlerRequestException { + + final Map> queryParameterAsMap = parameters.getQueryParameters().stream() + .filter(MessageParameter::isResolved) + .collect(Collectors.toMap( + MessageParameter::getKey, + JarRunHandlerParameterTest::getValuesAsString + )); + + return new HandlerRequest<>( + requestBody, + JarRunHeaders.getInstance().getUnresolvedMessageParameters(), + Collections.singletonMap(JarIdPathParameter.KEY, jar.getFileName().toString()), + queryParameterAsMap, + Collections.emptyList() + ); + } + + private static void sendRequestAndValidateGraph( + JarRunHandler handler, + DispatcherGateway dispatcherGateway, + SupplierWithException, HandlerRequestException> requestSupplier, + ThrowingConsumer validator) throws Exception { + + handler.handleRequest(requestSupplier.get(), dispatcherGateway) + .get(); + + JobGraph submittedJobGraph = lastSubmittedJobGraphReference.getAndSet(null); + + validator.accept(submittedJobGraph); + } + + private static ExecutionConfig getExecutionConfig(JobGraph jobGraph) { + ExecutionConfig executionConfig; + try { + executionConfig = jobGraph.getSerializedExecutionConfig().deserializeValue(ParameterProgram.class.getClassLoader()); + } catch (Exception e) { + throw new AssertionError("Exception while deserializing ExecutionConfig.", e); + } + return executionConfig; + } + + private static List getValuesAsString(MessageQueryParameter parameter) { + final List values = parameter.getValue(); + return values.stream().map(parameter::convertValueToString).collect(Collectors.toList()); + } +} 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 3e8e633b4a6e0..6427f4de58d31 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 @@ -24,7 +24,6 @@ import org.apache.flink.configuration.WebOptions; import org.apache.flink.runtime.rest.RestClient; import org.apache.flink.runtime.rest.RestClientConfiguration; -import org.apache.flink.runtime.rest.messages.EmptyRequestBody; import org.apache.flink.runtime.rest.util.RestClientException; import org.apache.flink.runtime.testingUtils.TestingUtils; import org.apache.flink.test.util.MiniClusterResource; @@ -85,7 +84,7 @@ public void testRunJar() throws Exception { int port = clientConfig.getInteger(RestOptions.PORT); try { - client.sendRequest(host, port, headers, parameters, EmptyRequestBody.getInstance()) + client.sendRequest(host, port, headers, parameters, new JarRunRequestBody()) .get(); } catch (Exception e) { Optional expected = ExceptionUtils.findThrowable(e, RestClientException.class); diff --git a/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/JarRunRequestBodyTest.java b/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/JarRunRequestBodyTest.java new file mode 100644 index 0000000000000..0706873c0fab3 --- /dev/null +++ b/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/JarRunRequestBodyTest.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.webmonitor.handlers; + +import org.apache.flink.runtime.rest.messages.RestRequestMarshallingTestBase; + +import static org.junit.Assert.assertEquals; + +/** + * Tests for {@link JarRunRequestBody}. + */ +public class JarRunRequestBodyTest extends RestRequestMarshallingTestBase { + + @Override + protected Class getTestRequestClass() { + return JarRunRequestBody.class; + } + + @Override + protected JarRunRequestBody getTestRequestInstance() throws Exception { + return new JarRunRequestBody( + "hello", + "world", + 4, + true, + "foo/bar" + ); + } + + @Override + protected void assertOriginalEqualsToUnmarshalled( + final JarRunRequestBody expected, + final JarRunRequestBody actual) { + assertEquals(expected.getEntryClassName(), actual.getEntryClassName()); + assertEquals(expected.getProgramArguments(), actual.getProgramArguments()); + assertEquals(expected.getParallelism(), actual.getParallelism()); + assertEquals(expected.getAllowNonRestoredState(), actual.getAllowNonRestoredState()); + assertEquals(expected.getSavepointPath(), actual.getSavepointPath()); + } +} diff --git a/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/JarSubmissionITCase.java b/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/JarSubmissionITCase.java index e47a38a55133e..e64c708dbe7bb 100644 --- a/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/JarSubmissionITCase.java +++ b/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/JarSubmissionITCase.java @@ -139,8 +139,8 @@ private static JobPlanInfo showPlan(JarPlanHandler handler, String jarName, Rest private static JarRunResponseBody runJar(JarRunHandler handler, String jarName, DispatcherGateway restfulGateway) throws Exception { final JarRunMessageParameters runParameters = JarRunHeaders.getInstance().getUnresolvedMessageParameters(); - HandlerRequest runRequest = new HandlerRequest<>( - EmptyRequestBody.getInstance(), + HandlerRequest runRequest = new HandlerRequest<>( + new JarRunRequestBody(), runParameters, Collections.singletonMap(runParameters.jarIdPathParameter.getKey(), jarName), Collections.emptyMap(), diff --git a/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/testutils/ParameterProgram.java b/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/testutils/ParameterProgram.java new file mode 100644 index 0000000000000..c96cce8a83887 --- /dev/null +++ b/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/testutils/ParameterProgram.java @@ -0,0 +1,36 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.runtime.webmonitor.testutils; + +import org.apache.flink.api.java.ExecutionEnvironment; + +/** + * Simple test program that exposes passed arguments. + */ +public class ParameterProgram { + + public static volatile String[] actualArguments = null; + + public static void main(String[] args) throws Exception { + actualArguments = args; + + ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); + env.fromElements("hello", "world").print(); + } +} diff --git a/flink-runtime-web/web-dashboard/app/scripts/modules/submit/submit.ctrl.coffee b/flink-runtime-web/web-dashboard/app/scripts/modules/submit/submit.ctrl.coffee index baf8396cff7b3..4f9e6d49c3082 100644 --- a/flink-runtime-web/web-dashboard/app/scripts/modules/submit/submit.ctrl.coffee +++ b/flink-runtime-web/web-dashboard/app/scripts/modules/submit/submit.ctrl.coffee @@ -118,25 +118,32 @@ angular.module('flinkApp') $scope.state['plan-button'] = "Show Plan" $scope.error = null + request = {} + # legacy compatibility queryParameters = {} if $scope.state['entry-class'] + request['entryClass'] = $scope.state['entry-class'] queryParameters['entry-class'] = $scope.state['entry-class'] if $scope.state.parallelism + request['parallelism'] = $scope.state['parallelism'] queryParameters['parallelism'] = $scope.state['parallelism'] if $scope.state['program-args'] + request['programArgs'] = $scope.state['program-args'] queryParameters['program-args'] = $scope.state['program-args'] if $scope.state['savepointPath'] + request['savepointPath'] = $scope.state['savepointPath'] queryParameters['savepointPath'] = $scope.state['savepointPath'] if $scope.state['allowNonRestoredState'] + request['allowNonRestoredState'] = $scope.state['allowNonRestoredState'] queryParameters['allowNonRestoredState'] = $scope.state['allowNonRestoredState'] JobSubmitService.runJob( - $scope.state.selected, queryParameters + $scope.state.selected, request, queryParameters ).then (data) -> if action == $scope.state['action-time'] $scope.state['submit-button'] = "Submit" diff --git a/flink-runtime-web/web-dashboard/app/scripts/modules/submit/submit.svc.coffee b/flink-runtime-web/web-dashboard/app/scripts/modules/submit/submit.svc.coffee index 989bdba7f0439..98cf51e261f52 100644 --- a/flink-runtime-web/web-dashboard/app/scripts/modules/submit/submit.svc.coffee +++ b/flink-runtime-web/web-dashboard/app/scripts/modules/submit/submit.svc.coffee @@ -52,10 +52,10 @@ angular.module('flinkApp') deferred.promise - @runJob = (id, args) -> + @runJob = (id, request, queryParameters) -> deferred = $q.defer() - $http.post(flinkConfig.jobServer + "jars/" + encodeURIComponent(id) + "/run", {}, {params: args}) + $http.post(flinkConfig.jobServer + "jars/" + encodeURIComponent(id) + "/run", request, {params: queryParameters}) .success (data, status, headers, config) -> deferred.resolve(data) .error (err) -> diff --git a/flink-runtime-web/web-dashboard/web/js/hs/index.js b/flink-runtime-web/web-dashboard/web/js/hs/index.js index 4c3a13856d4d2..424d33f2abf6f 100644 --- a/flink-runtime-web/web-dashboard/web/js/hs/index.js +++ b/flink-runtime-web/web-dashboard/web/js/hs/index.js @@ -1,2 +1,2 @@ -angular.module("flinkApp",["ui.router","angularMoment","dndLists"]).run(["$rootScope",function(e){return e.sidebarVisible=!1,e.showSidebar=function(){return e.sidebarVisible=!e.sidebarVisible,e.sidebarClass="force-show"}}]).value("flinkConfig",{jobServer:"","refresh-interval":1e4}).value("watermarksConfig",{noWatermark:-0x8000000000000000}).run(["JobsService","MainService","flinkConfig","$interval",function(e,t,r,n){return t.loadConfig().then(function(t){return angular.extend(r,t),e.listJobs(),n(function(){return e.listJobs()},r["refresh-interval"])})}]).config(["$uiViewScrollProvider",function(e){return e.useAnchorScroll()}]).run(["$rootScope","$state",function(e,t){return e.$on("$stateChangeStart",function(e,r,n,i){if(r.redirectTo)return e.preventDefault(),t.go(r.redirectTo,n)})}]).config(["$stateProvider","$urlRouterProvider",function(e,t){return e.state("completed-jobs",{url:"/completed-jobs",views:{main:{templateUrl:"partials/jobs/completed-jobs.html",controller:"CompletedJobsController"}}}).state("single-job",{url:"/jobs/{jobid}","abstract":!0,views:{main:{templateUrl:"partials/jobs/job.html",controller:"SingleJobController"}}}).state("single-job.plan",{url:"",redirectTo:"single-job.plan.subtasks",views:{details:{templateUrl:"partials/jobs/job.plan.html",controller:"JobPlanController"}}}).state("single-job.plan.subtasks",{url:"",views:{"node-details":{templateUrl:"partials/jobs/job.plan.node-list.subtasks.html",controller:"JobPlanSubtasksController"}}}).state("single-job.plan.metrics",{url:"/metrics",views:{"node-details":{templateUrl:"partials/jobs/job.plan.node-list.metrics.html",controller:"JobPlanMetricsController"}}}).state("single-job.plan.watermarks",{url:"/watermarks",views:{"node-details":{templateUrl:"partials/jobs/job.plan.node-list.watermarks.html"}}}).state("single-job.plan.taskmanagers",{url:"/taskmanagers",views:{"node-details":{templateUrl:"partials/jobs/job.plan.node-list.taskmanagers.html",controller:"JobPlanTaskManagersController"}}}).state("single-job.plan.accumulators",{url:"/accumulators",views:{"node-details":{templateUrl:"partials/jobs/job.plan.node-list.accumulators.html",controller:"JobPlanAccumulatorsController"}}}).state("single-job.plan.checkpoints",{url:"/checkpoints",redirectTo:"single-job.plan.checkpoints.overview",views:{"node-details":{templateUrl:"partials/jobs/job.plan.node-list.checkpoints.html",controller:"JobPlanCheckpointsController"}}}).state("single-job.plan.checkpoints.overview",{url:"/overview",views:{"checkpoints-view":{templateUrl:"partials/jobs/job.plan.node.checkpoints.overview.html",controller:"JobPlanCheckpointsController"}}}).state("single-job.plan.checkpoints.summary",{url:"/summary",views:{"checkpoints-view":{templateUrl:"partials/jobs/job.plan.node.checkpoints.summary.html",controller:"JobPlanCheckpointsController"}}}).state("single-job.plan.checkpoints.history",{url:"/history",views:{"checkpoints-view":{templateUrl:"partials/jobs/job.plan.node.checkpoints.history.html",controller:"JobPlanCheckpointsController"}}}).state("single-job.plan.checkpoints.config",{url:"/config",views:{"checkpoints-view":{templateUrl:"partials/jobs/job.plan.node.checkpoints.config.html",controller:"JobPlanCheckpointsController"}}}).state("single-job.plan.checkpoints.details",{url:"/details/{checkpointId}",views:{"checkpoints-view":{templateUrl:"partials/jobs/job.plan.node.checkpoints.details.html",controller:"JobPlanCheckpointDetailsController"}}}).state("single-job.plan.backpressure",{url:"/backpressure",views:{"node-details":{templateUrl:"partials/jobs/job.plan.node-list.backpressure.html",controller:"JobPlanBackPressureController"}}}).state("single-job.timeline",{url:"/timeline",views:{details:{templateUrl:"partials/jobs/job.timeline.html"}}}).state("single-job.timeline.vertex",{url:"/{vertexId}",views:{vertex:{templateUrl:"partials/jobs/job.timeline.vertex.html",controller:"JobTimelineVertexController"}}}).state("single-job.exceptions",{url:"/exceptions",views:{details:{templateUrl:"partials/jobs/job.exceptions.html",controller:"JobExceptionsController"}}}).state("single-job.config",{url:"/config",views:{details:{templateUrl:"partials/jobs/job.config.html"}}}),t.otherwise("/completed-jobs")}]),angular.module("flinkApp").directive("bsLabel",["JobsService",function(e){return{transclude:!0,replace:!0,scope:{getLabelClass:"&",status:"@"},template:"",link:function(t,r,n){return t.getLabelClass=function(){return"label label-"+e.translateLabelState(n.status)}}}}]).directive("bpLabel",["JobsService",function(e){return{transclude:!0,replace:!0,scope:{getBackPressureLabelClass:"&",status:"@"},template:"",link:function(t,r,n){return t.getBackPressureLabelClass=function(){return"label label-"+e.translateBackPressureLabelState(n.status)}}}}]).directive("indicatorPrimary",["JobsService",function(e){return{replace:!0,scope:{getLabelClass:"&",status:"@"},template:"",link:function(t,r,n){return t.getLabelClass=function(){return"fa fa-circle indicator indicator-"+e.translateLabelState(n.status)}}}}]).directive("tableProperty",function(){return{replace:!0,scope:{value:"="},template:"{{value || 'None'}}"}}),angular.module("flinkApp").filter("amDurationFormatExtended",["angularMomentConfig",function(e){var t;return t=function(e,t,r){return"undefined"==typeof e||null===e?"":moment.duration(e,t).format(r,{trim:!1})},t.$stateful=e.statefulFilters,t}]).filter("humanizeDuration",function(){return function(e,t){var r,n,i,o,s,a;return"undefined"==typeof e||null===e?"":(o=e%1e3,a=Math.floor(e/1e3),s=a%60,a=Math.floor(a/60),i=a%60,a=Math.floor(a/60),n=a%24,a=Math.floor(a/24),r=a,0===r?0===n?0===i?0===s?o+"ms":s+"s ":i+"m "+s+"s":t?n+"h "+i+"m":n+"h "+i+"m "+s+"s":t?r+"d "+n+"h":r+"d "+n+"h "+i+"m "+s+"s")}}).filter("limit",function(){return function(e){return e.length>73&&(e=e.substring(0,35)+"..."+e.substring(e.length-35,e.length)),e}}).filter("humanizeText",function(){return function(e){return e?e.replace(/>/g,">").replace(//g,""):""}}).filter("humanizeBytes",function(){return function(e){var t,r;return r=["B","KB","MB","GB","TB","PB","EB"],t=function(e,n){var i;return i=Math.pow(1024,n),e=r;n=0<=r?++e:--e)i.push(n+".currentInputWatermark");return i}(),i.getMetrics(o,t.id,s).then(function(e){var t,n,i,o,s,a,l;i=NaN,l={},o=e.values;for(t in o)a=o[t],s=t.replace(".currentInputWatermark",""),l[s]=a,(isNaN(i)||au.noWatermark?i:NaN,r.resolve({lowWatermark:n,watermarks:l})}),r.promise}}(this),r=l.defer(),s={},n=t.length,angular.forEach(t,function(e){return function(e,t){var i;return i=e.id,o(e).then(function(e){if(s[i]=e,t>=n-1)return r.resolve(s)})}}(this)),r.promise},e.hasWatermark=function(t){return e.watermarks[t]&&!isNaN(e.watermarks[t].lowWatermark)},e.$watch("plan",function(t){if(t)return c(t.nodes).then(function(t){return e.watermarks=t})}),e.$on("reload",function(){if(e.plan)return c(e.plan.nodes).then(function(t){return e.watermarks=t})})}]).controller("JobPlanController",["$scope","$state","$stateParams","$window","JobsService",function(e,t,r,n,i){return e.nodeid=null,e.nodeUnfolded=!1,e.stateList=i.stateList(),e.changeNode=function(t){return t!==e.nodeid?(e.nodeid=t,e.vertex=null,e.subtasks=null,e.accumulators=null,e.operatorCheckpointStats=null,e.$broadcast("reload"),e.$broadcast("node:change",e.nodeid)):(e.nodeid=null,e.nodeUnfolded=!1,e.vertex=null,e.subtasks=null,e.accumulators=null,e.operatorCheckpointStats=null)},e.deactivateNode=function(){return e.nodeid=null,e.nodeUnfolded=!1,e.vertex=null,e.subtasks=null,e.accumulators=null,e.operatorCheckpointStats=null},e.toggleFold=function(){return e.nodeUnfolded=!e.nodeUnfolded}}]).controller("JobPlanSubtasksController",["$scope","JobsService",function(e,t){var r;return e.aggregate=!1,r=function(){return e.aggregate?t.getTaskManagers(e.nodeid).then(function(t){return e.taskmanagers=t}):t.getSubtasks(e.nodeid).then(function(t){return e.subtasks=t})},!e.nodeid||e.vertex&&e.vertex.st||r(),e.$on("reload",function(t){if(e.nodeid)return r()})}]).controller("JobPlanAccumulatorsController",["$scope","JobsService",function(e,t){var r;return r=function(){return t.getAccumulators(e.nodeid).then(function(t){return e.accumulators=t.main,e.subtaskAccumulators=t.subtasks})},!e.nodeid||e.vertex&&e.vertex.accumulators||r(),e.$on("reload",function(t){if(e.nodeid)return r()})}]).controller("JobPlanCheckpointsController",["$scope","$state","$stateParams","JobsService",function(e,t,r,n){var i;return e.checkpointDetails={},e.checkpointDetails.id=-1,n.getCheckpointConfig().then(function(t){return e.checkpointConfig=t}),i=function(){return n.getCheckpointStats().then(function(t){if(null!==t)return e.checkpointStats=t})},i(),e.$on("reload",function(e){return i()})}]).controller("JobPlanCheckpointDetailsController",["$scope","$state","$stateParams","JobsService",function(e,t,r,n){var i,o;return e.subtaskDetails={},e.checkpointDetails.id=r.checkpointId,i=function(t){return n.getCheckpointDetails(t).then(function(t){return null!==t?e.checkpoint=t:e.unknown_checkpoint=!0})},o=function(t,r){return n.getCheckpointSubtaskDetails(t,r).then(function(t){if(null!==t)return e.subtaskDetails[r]=t})},i(r.checkpointId),e.nodeid&&o(r.checkpointId,e.nodeid),e.$on("reload",function(t){if(i(r.checkpointId),e.nodeid)return o(r.checkpointId,e.nodeid)}),e.$on("$destroy",function(){return e.checkpointDetails.id=-1})}]).controller("JobPlanBackPressureController",["$scope","JobsService",function(e,t){var r;return r=function(){if(e.now=Date.now(),e.nodeid)return t.getOperatorBackPressure(e.nodeid).then(function(t){return e.backPressureOperatorStats[e.nodeid]=t})},r(),e.$on("reload",function(e){return r()})}]).controller("JobTimelineVertexController",["$scope","$state","$stateParams","JobsService",function(e,t,r,n){var i;return i=function(){return n.getVertex(r.vertexId).then(function(t){return e.vertex=t})},i(),e.$on("reload",function(e){return i()})}]).controller("JobExceptionsController",["$scope","$state","$stateParams","JobsService",function(e,t,r,n){return n.loadExceptions().then(function(t){return e.exceptions=t})}]).controller("JobPropertiesController",["$scope","JobsService",function(e,t){return e.changeNode=function(r){return r!==e.nodeid?(e.nodeid=r,t.getNode(r).then(function(t){return e.node=t})):(e.nodeid=null,e.node=null)}}]).controller("JobPlanMetricsController",["$scope","JobsService","MetricsService",function(e,t,r){var n,i;if(e.dragging=!1,e.window=r.getWindow(),e.availableMetrics=null,e.$on("$destroy",function(){return r.unRegisterObserver()}),i=function(){return t.getVertex(e.nodeid).then(function(t){return e.vertex=t}),r.getAvailableMetrics(e.jobid,e.nodeid).then(function(t){return e.availableMetrics=t.sort(n),e.metrics=r.getMetricsSetup(e.jobid,e.nodeid).names,r.registerObserver(e.jobid,e.nodeid,function(t){return e.$broadcast("metrics:data:update",t.timestamp,t.values)})})},n=function(e,t){var r,n;return r=e.id.toLowerCase(),n=t.id.toLowerCase(),rn?1:0},e.dropped=function(t,n,o,s,a){return r.orderMetrics(e.jobid,e.nodeid,o,n),e.$broadcast("metrics:refresh",o),i(),!1},e.dragStart=function(){return e.dragging=!0},e.dragEnd=function(){return e.dragging=!1},e.addMetric=function(t){return r.addMetric(e.jobid,e.nodeid,t.id),i()},e.removeMetric=function(t){return r.removeMetric(e.jobid,e.nodeid,t),i()},e.setMetricSize=function(t,n){return r.setMetricSize(e.jobid,e.nodeid,t,n),i()},e.setMetricView=function(t,n){return r.setMetricView(e.jobid,e.nodeid,t,n),i()},e.getValues=function(t){return r.getValues(e.jobid,e.nodeid,t)},e.$on("node:change",function(t,r){if(!e.dragging)return i()}),e.nodeid)return i()}]),angular.module("flinkApp").directive("vertex",["$state",function(e){return{template:"",scope:{data:"="},link:function(e,t,r){var n,i,o;o=t.children()[0],i=t.width(),angular.element(o).attr("width",i),(n=function(e){var t,r,n;return d3.select(o).selectAll("*").remove(),n=[],angular.forEach(e.subtasks,function(e,t){var r;return r=[{label:"Scheduled",color:"#666",borderColor:"#555",starting_time:e.timestamps.SCHEDULED,ending_time:e.timestamps.DEPLOYING,type:"regular"},{label:"Deploying",color:"#aaa",borderColor:"#555",starting_time:e.timestamps.DEPLOYING,ending_time:e.timestamps.RUNNING,type:"regular"}],e.timestamps.FINISHED>0&&r.push({label:"Running",color:"#ddd",borderColor:"#555",starting_time:e.timestamps.RUNNING,ending_time:e.timestamps.FINISHED,type:"regular"}),n.push({label:"("+e.subtask+") "+e.host,times:r})}),t=d3.timeline().stack().tickFormat({format:d3.time.format("%L"),tickSize:1}).prefix("single").labelFormat(function(e){return e}).margin({left:100,right:0,top:0,bottom:0}).itemHeight(30).relativeTime(),r=d3.select(o).datum(n).call(t)})(e.data)}}}]).directive("timeline",["$state",function(e){return{template:"",scope:{vertices:"=",jobid:"="},link:function(t,r,n){var i,o,s,a;s=r.children()[0],o=r.width(),angular.element(s).attr("width",o),a=function(e){return e.replace(">",">")},i=function(r){var n,i,o;return d3.select(s).selectAll("*").remove(),o=[],angular.forEach(r,function(e){if(e["start-time"]>-1)return"scheduled"===e.type?o.push({times:[{label:a(e.name),color:"#cccccc",borderColor:"#555555",starting_time:e["start-time"],ending_time:e["end-time"],type:e.type}]}):o.push({times:[{label:a(e.name),color:"#d9f1f7",borderColor:"#62cdea",starting_time:e["start-time"],ending_time:e["end-time"],link:e.id,type:e.type}]})}),n=d3.timeline().stack().click(function(r,n,i){if(r.link)return e.go("single-job.timeline.vertex",{jobid:t.jobid,vertexId:r.link})}).tickFormat({format:d3.time.format("%L"),tickSize:1}).prefix("main").margin({left:0,right:0,top:0,bottom:0}).itemHeight(30).showBorderLine().showHourTimeline(),i=d3.select(s).datum(o).call(n)},t.$watch(n.vertices,function(e){if(e)return i(e)})}}}]).directive("split",function(){return{compile:function(e,t){return Split(e.children(),{sizes:[50,50],direction:"vertical"})}}}).directive("jobPlan",["$timeout",function(e){return{template:"
",scope:{plan:"=",watermarks:"=",setNode:"&"},link:function(e,t,r){var n,i,o,s,a,l,u,c,d,f,p,m,h,g,b,v,k,j,S,w,C,$,y,J,M;p=null,C=d3.behavior.zoom(),M=[],g=r.jobid,S=t.children()[0],j=t.children().children()[0],w=t.children()[1],l=d3.select(S),u=d3.select(j),c=d3.select(w),n=t.width(),angular.element(t.children()[0]).width(n),v=0,b=0,e.zoomIn=function(){var e,t,r;if(C.scale()<2.99)return e=C.translate(),t=e[0]*(C.scale()+.1/C.scale()),r=e[1]*(C.scale()+.1/C.scale()),C.scale(C.scale()+.1),C.translate([t,r]),u.attr("transform","translate("+t+","+r+") scale("+C.scale()+")"),v=C.scale(),b=C.translate()},e.zoomOut=function(){var e,t,r;if(C.scale()>.31)return C.scale(C.scale()-.1),e=C.translate(),t=e[0]*(C.scale()-.1/C.scale()),r=e[1]*(C.scale()-.1/C.scale()),C.translate([t,r]),u.attr("transform","translate("+t+","+r+") scale("+C.scale()+")"),v=C.scale(),b=C.translate()},o=function(e){var t;return t="",null==e.ship_strategy&&null==e.local_strategy||(t+="
",null!=e.ship_strategy&&(t+=e.ship_strategy),void 0!==e.temp_mode&&(t+=" ("+e.temp_mode+")"),void 0!==e.local_strategy&&(t+=",
"+e.local_strategy),t+="
"),t},h=function(e){return"partialSolution"===e||"nextPartialSolution"===e||"workset"===e||"nextWorkset"===e||"solutionSet"===e||"solutionDelta"===e},m=function(e,t){return"mirror"===t?"node-mirror":h(t)?"node-iteration":"node-normal"},s=function(e,t,r,n){var i,o;return i="
",i+="mirror"===t?"

Mirror of "+e.operator+"

":"

"+e.operator+"

",""===e.description?i+="":(o=e.description,o=J(o),i+="

"+o+"

"),null!=e.step_function?i+=f(e.id,r,n):(h(t)&&(i+="
"+t+" Node
"),""!==e.parallelism&&(i+="
Parallelism: "+e.parallelism+"
"),void 0!==e.lowWatermark&&(i+="
Low Watermark: "+e.lowWatermark+"
"),void 0!==e.operator&&e.operator_strategy&&(i+="
Operation: "+J(e.operator_strategy)+"
")),i+="
"},f=function(e,t,r){var n,i;return i="svg-"+e,n=""},J=function(e){var t;for("<"===e.charAt(0)&&(e=e.replace("<","<"),e=e.replace(">",">")),t="";e.length>30;)t=t+e.substring(0,30)+"
",e=e.substring(30,e.length);return t+=e},a=function(e,t,r,n,i,o){return null==n&&(n=!1),r.id===t.partial_solution?e.setNode(r.id,{label:s(r,"partialSolution",i,o),labelType:"html","class":m(r,"partialSolution")}):r.id===t.next_partial_solution?e.setNode(r.id,{label:s(r,"nextPartialSolution",i,o),labelType:"html","class":m(r,"nextPartialSolution")}):r.id===t.workset?e.setNode(r.id,{label:s(r,"workset",i,o),labelType:"html","class":m(r,"workset")}):r.id===t.next_workset?e.setNode(r.id,{label:s(r,"nextWorkset",i,o),labelType:"html","class":m(r,"nextWorkset")}):r.id===t.solution_set?e.setNode(r.id,{label:s(r,"solutionSet",i,o),labelType:"html","class":m(r,"solutionSet")}):r.id===t.solution_delta?e.setNode(r.id,{label:s(r,"solutionDelta",i,o),labelType:"html","class":m(r,"solutionDelta")}):e.setNode(r.id,{label:s(r,"",i,o),labelType:"html","class":m(r,"")})},i=function(e,t,r,n,i){return e.setEdge(i.id,r.id,{label:o(i),labelType:"html",arrowhead:"normal"})},k=function(e,t){var r,n,o,s,l,u,d,f,p,m,h,g,b,v;for(n=[],null!=t.nodes?v=t.nodes:(v=t.step_function,o=!0),s=0,u=v.length;s-1))return e["end-time"]=e["start-time"]+e.duration})},this.processVertices=function(e){return angular.forEach(e.vertices,function(e,t){return e.type="regular"}),e.vertices.unshift({name:"Scheduled","start-time":e.timestamps.CREATED,"end-time":e.timestamps.CREATED+1,type:"scheduled"})},this.listJobs=function(){var r;return r=i.defer(),e.get(t.jobServer+"jobs/overview").success(function(e){return function(t,n,i,o){return c.finished=[],c.running=[],_(t.jobs).groupBy(function(e){switch(e.state.toLowerCase()){case"finished":return"finished";case"failed":return"finished";case"canceled":return"finished";default:return"running"}}).forEach(function(t,r){switch(r){case"finished":return c.finished=e.setEndTimes(t);case"running":return c.running=e.setEndTimes(t)}}).value(),r.resolve(c),d()}}(this)),r.promise},this.getJobs=function(e){return c[e]},this.getAllJobs=function(){return c},this.loadJob=function(r){return s=null,l.job=i.defer(),e.get(t.jobServer+"jobs/"+r).success(function(n){return function(i,o,a,u){return n.setEndTimes(i.vertices),n.processVertices(i),e.get(t.jobServer+"jobs/"+r+"/config").success(function(e){return i=angular.extend(i,e),s=i,l.job.resolve(s)})}}(this)),l.job.promise},this.getNode=function(e){var t,r;return r=function(e,t){var n,i,o,s;for(n=0,i=t.length;n
{{metric.id}}
{{value | humanizeChartNumeric:metric}}
',replace:!0,scope:{metric:"=",window:"=",removeMetric:"&",setMetricSize:"=",setMetricView:"=",getValues:"&"},link:function(e,t,r){return e.btnClasses=["btn","btn-default","btn-xs"],e.value=null,e.data=[{values:e.getValues() -}],e.options={x:function(e,t){return e.x},y:function(e,t){return e.y},xTickFormat:function(e){return d3.time.format("%H:%M:%S")(new Date(e))},yTickFormat:function(e){var t,r,n,i;for(r=!1,n=0,i=1,t=Math.abs(e);!r&&n<50;)Math.pow(10,n)<=t&&t6?e/Math.pow(10,n)+"E"+n:""+e}},e.showChart=function(){return d3.select(t.find("svg")[0]).datum(e.data).transition().duration(250).call(e.chart)},e.chart=nv.models.lineChart().options(e.options).showLegend(!1).margin({top:15,left:60,bottom:30,right:30}),e.chart.yAxis.showMaxMin(!1),e.chart.tooltip.hideDelay(0),e.chart.tooltip.contentGenerator(function(e){return"

"+d3.time.format("%H:%M:%S")(new Date(e.point.x))+" | "+e.point.y+"

"}),nv.utils.windowResize(e.chart.update),e.setSize=function(t){return e.setMetricSize(e.metric,t)},e.setView=function(t){if(e.setMetricView(e.metric,t),"chart"===t)return e.showChart()},"chart"===e.metric.view&&e.showChart(),e.$on("metrics:data:update",function(t,r,n){return e.value=parseFloat(n[e.metric.id]),e.data[0].values.push({x:r,y:e.value}),e.data[0].values.length>e.window&&e.data[0].values.shift(),"chart"===e.metric.view&&e.showChart(),"chart"===e.metric.view&&e.chart.clearHighlights(),e.chart.tooltip.hidden(!0)}),t.find(".metric-title").qtip({content:{text:e.metric.id},position:{my:"bottom left",at:"top left"},style:{classes:"qtip-light qtip-timeline-bar"}})}}}),angular.module("flinkApp").service("MetricsService",["$http","$q","flinkConfig","$interval",function(e,t,r,n){return this.metrics={},this.values={},this.watched={},this.observer={jobid:null,nodeid:null,callback:null},this.refresh=n(function(e){return function(){return angular.forEach(e.metrics,function(t,r){return angular.forEach(t,function(t,n){var i;if(i=[],angular.forEach(t,function(e,t){return i.push(e.id)}),i.length>0)return e.getMetrics(r,n,i).then(function(t){if(r===e.observer.jobid&&n===e.observer.nodeid&&e.observer.callback)return e.observer.callback(t)})})})}}(this),r["refresh-interval"]),this.registerObserver=function(e,t,r){return this.observer.jobid=e,this.observer.nodeid=t,this.observer.callback=r},this.unRegisterObserver=function(){return this.observer={jobid:null,nodeid:null,callback:null}},this.setupMetrics=function(e,t){return this.setupLS(),this.watched[e]=[],angular.forEach(t,function(t){return function(r,n){if(r.id)return t.watched[e].push(r.id)}}(this))},this.getWindow=function(){return 100},this.setupLS=function(){return null==sessionStorage.flinkMetrics&&this.saveSetup(),this.metrics=JSON.parse(sessionStorage.flinkMetrics)},this.saveSetup=function(){return sessionStorage.flinkMetrics=JSON.stringify(this.metrics)},this.saveValue=function(e,t,r){if(null==this.values[e]&&(this.values[e]={}),null==this.values[e][t]&&(this.values[e][t]=[]),this.values[e][t].push(r),this.values[e][t].length>this.getWindow())return this.values[e][t].shift()},this.getValues=function(e,t,r){var n;return null==this.values[e]?[]:null==this.values[e][t]?[]:(n=[],angular.forEach(this.values[e][t],function(e){return function(e,t){if(null!=e.values[r])return n.push({x:e.timestamp,y:e.values[r]})}}(this)),n)},this.setupLSFor=function(e,t){if(null==this.metrics[e]&&(this.metrics[e]={}),null==this.metrics[e][t])return this.metrics[e][t]=[]},this.addMetric=function(e,t,r){return this.setupLSFor(e,t),this.metrics[e][t].push({id:r,size:"small",view:"chart"}),this.saveSetup()},this.removeMetric=function(e){return function(t,r,n){var i;if(null!=e.metrics[t][r])return i=e.metrics[t][r].indexOf(n),i===-1&&(i=_.findIndex(e.metrics[t][r],{id:n})),i!==-1&&e.metrics[t][r].splice(i,1),e.saveSetup()}}(this),this.setMetricSize=function(e){return function(t,r,n,i){var o;if(null!=e.metrics[t][r])return o=e.metrics[t][r].indexOf(n.id),o===-1&&(o=_.findIndex(e.metrics[t][r],{id:n.id})),o!==-1&&(e.metrics[t][r][o]={id:n.id,size:i,view:n.view}),e.saveSetup()}}(this),this.setMetricView=function(e){return function(t,r,n,i){var o;if(null!=e.metrics[t][r])return o=e.metrics[t][r].indexOf(n.id),o===-1&&(o=_.findIndex(e.metrics[t][r],{id:n.id})),o!==-1&&(e.metrics[t][r][o]={id:n.id,size:n.size,view:i}),e.saveSetup()}}(this),this.orderMetrics=function(e,t,r,n){return this.setupLSFor(e,t),angular.forEach(this.metrics[e][t],function(i){return function(o,s){if(o.id===r.id&&(i.metrics[e][t].splice(s,1),s",link:function(t,r,n){return t.getLabelClass=function(){return"label label-"+e.translateLabelState(n.status)}}}}]).directive("bpLabel",["JobsService",function(e){return{transclude:!0,replace:!0,scope:{getBackPressureLabelClass:"&",status:"@"},template:"",link:function(t,r,n){return t.getBackPressureLabelClass=function(){return"label label-"+e.translateBackPressureLabelState(n.status)}}}}]).directive("indicatorPrimary",["JobsService",function(e){return{replace:!0,scope:{getLabelClass:"&",status:"@"},template:"",link:function(t,r,n){return t.getLabelClass=function(){return"fa fa-circle indicator indicator-"+e.translateLabelState(n.status)}}}}]).directive("tableProperty",function(){return{replace:!0,scope:{value:"="},template:"{{value || 'None'}}"}}),angular.module("flinkApp").filter("amDurationFormatExtended",["angularMomentConfig",function(e){var t;return t=function(e,t,r){return"undefined"==typeof e||null===e?"":moment.duration(e,t).format(r,{trim:!1})},t.$stateful=e.statefulFilters,t}]).filter("humanizeDuration",function(){return function(e,t){var r,n,o,i,s,a;return"undefined"==typeof e||null===e?"":(i=e%1e3,a=Math.floor(e/1e3),s=a%60,a=Math.floor(a/60),o=a%60,a=Math.floor(a/60),n=a%24,a=Math.floor(a/24),r=a,0===r?0===n?0===o?0===s?i+"ms":s+"s ":o+"m "+s+"s":t?n+"h "+o+"m":n+"h "+o+"m "+s+"s":t?r+"d "+n+"h":r+"d "+n+"h "+o+"m "+s+"s")}}).filter("limit",function(){return function(e){return e.length>73&&(e=e.substring(0,35)+"..."+e.substring(e.length-35,e.length)),e}}).filter("humanizeText",function(){return function(e){return e?e.replace(/>/g,">").replace(//g,""):""}}).filter("humanizeBytes",function(){return function(e){var t,r;return r=["B","KB","MB","GB","TB","PB","EB"],t=function(e,n){var o;return o=Math.pow(1024,n),e=r;n=0<=r?++e:--e)o.push(n+".currentInputWatermark");return o}(),o.getMetrics(i,t.id,s).then(function(e){var t,n,o,i,s,a,l;o=NaN,l={},i=e.values;for(t in i)a=i[t],s=t.replace(".currentInputWatermark",""),l[s]=a,(isNaN(o)||au.noWatermark?o:NaN,r.resolve({lowWatermark:n,watermarks:l})}),r.promise}}(this),r=l.defer(),s={},n=t.length,angular.forEach(t,function(e){return function(e,t){var o;return o=e.id,i(e).then(function(e){if(s[o]=e,t>=n-1)return r.resolve(s)})}}(this)),r.promise},e.hasWatermark=function(t){return e.watermarks[t]&&!isNaN(e.watermarks[t].lowWatermark)},e.$watch("plan",function(t){if(t)return c(t.nodes).then(function(t){return e.watermarks=t})}),e.$on("reload",function(){if(e.plan)return c(e.plan.nodes).then(function(t){return e.watermarks=t})})}]).controller("JobPlanController",["$scope","$state","$stateParams","$window","JobsService",function(e,t,r,n,o){return e.nodeid=null,e.nodeUnfolded=!1,e.stateList=o.stateList(),e.changeNode=function(t){return t!==e.nodeid?(e.nodeid=t,e.vertex=null,e.subtasks=null,e.accumulators=null,e.operatorCheckpointStats=null,e.$broadcast("reload"),e.$broadcast("node:change",e.nodeid)):(e.nodeid=null,e.nodeUnfolded=!1,e.vertex=null,e.subtasks=null,e.accumulators=null,e.operatorCheckpointStats=null)},e.deactivateNode=function(){return e.nodeid=null,e.nodeUnfolded=!1,e.vertex=null,e.subtasks=null,e.accumulators=null,e.operatorCheckpointStats=null},e.toggleFold=function(){return e.nodeUnfolded=!e.nodeUnfolded}}]).controller("JobPlanSubtasksController",["$scope","JobsService",function(e,t){var r;return e.aggregate=!1,r=function(){return e.aggregate?t.getTaskManagers(e.nodeid).then(function(t){return e.taskmanagers=t}):t.getSubtasks(e.nodeid).then(function(t){return e.subtasks=t})},!e.nodeid||e.vertex&&e.vertex.st||r(),e.$on("reload",function(t){if(e.nodeid)return r()})}]).controller("JobPlanAccumulatorsController",["$scope","JobsService",function(e,t){var r;return r=function(){return t.getAccumulators(e.nodeid).then(function(t){return e.accumulators=t.main,e.subtaskAccumulators=t.subtasks})},!e.nodeid||e.vertex&&e.vertex.accumulators||r(),e.$on("reload",function(t){if(e.nodeid)return r()})}]).controller("JobPlanCheckpointsController",["$scope","$state","$stateParams","JobsService",function(e,t,r,n){var o;return e.checkpointDetails={},e.checkpointDetails.id=-1,n.getCheckpointConfig().then(function(t){return e.checkpointConfig=t}),o=function(){return n.getCheckpointStats().then(function(t){if(null!==t)return e.checkpointStats=t})},o(),e.$on("reload",function(e){return o()})}]).controller("JobPlanCheckpointDetailsController",["$scope","$state","$stateParams","JobsService",function(e,t,r,n){var o,i;return e.subtaskDetails={},e.checkpointDetails.id=r.checkpointId,o=function(t){return n.getCheckpointDetails(t).then(function(t){return null!==t?e.checkpoint=t:e.unknown_checkpoint=!0})},i=function(t,r){return n.getCheckpointSubtaskDetails(t,r).then(function(t){if(null!==t)return e.subtaskDetails[r]=t})},o(r.checkpointId),e.nodeid&&i(r.checkpointId,e.nodeid),e.$on("reload",function(t){if(o(r.checkpointId),e.nodeid)return i(r.checkpointId,e.nodeid)}),e.$on("$destroy",function(){return e.checkpointDetails.id=-1})}]).controller("JobPlanBackPressureController",["$scope","JobsService",function(e,t){var r;return r=function(){if(e.now=Date.now(),e.nodeid)return t.getOperatorBackPressure(e.nodeid).then(function(t){return e.backPressureOperatorStats[e.nodeid]=t})},r(),e.$on("reload",function(e){return r()})}]).controller("JobTimelineVertexController",["$scope","$state","$stateParams","JobsService",function(e,t,r,n){var o;return o=function(){return n.getVertex(r.vertexId).then(function(t){return e.vertex=t})},o(),e.$on("reload",function(e){return o()})}]).controller("JobExceptionsController",["$scope","$state","$stateParams","JobsService",function(e,t,r,n){return n.loadExceptions().then(function(t){return e.exceptions=t})}]).controller("JobPropertiesController",["$scope","JobsService",function(e,t){return e.changeNode=function(r){return r!==e.nodeid?(e.nodeid=r,t.getNode(r).then(function(t){return e.node=t})):(e.nodeid=null,e.node=null)}}]).controller("JobPlanMetricsController",["$scope","JobsService","MetricsService",function(e,t,r){var n,o;if(e.dragging=!1,e.window=r.getWindow(),e.availableMetrics=null,e.$on("$destroy",function(){return r.unRegisterObserver()}),o=function(){return t.getVertex(e.nodeid).then(function(t){return e.vertex=t}),r.getAvailableMetrics(e.jobid,e.nodeid).then(function(t){return e.availableMetrics=t.sort(n),e.metrics=r.getMetricsSetup(e.jobid,e.nodeid).names,r.registerObserver(e.jobid,e.nodeid,function(t){return e.$broadcast("metrics:data:update",t.timestamp,t.values)})})},n=function(e,t){var r,n;return r=e.id.toLowerCase(),n=t.id.toLowerCase(),rn?1:0},e.dropped=function(t,n,i,s,a){return r.orderMetrics(e.jobid,e.nodeid,i,n),e.$broadcast("metrics:refresh",i),o(),!1},e.dragStart=function(){return e.dragging=!0},e.dragEnd=function(){return e.dragging=!1},e.addMetric=function(t){return r.addMetric(e.jobid,e.nodeid,t.id),o()},e.removeMetric=function(t){return r.removeMetric(e.jobid,e.nodeid,t),o()},e.setMetricSize=function(t,n){return r.setMetricSize(e.jobid,e.nodeid,t,n),o()},e.setMetricView=function(t,n){return r.setMetricView(e.jobid,e.nodeid,t,n),o()},e.getValues=function(t){return r.getValues(e.jobid,e.nodeid,t)},e.$on("node:change",function(t,r){if(!e.dragging)return o()}),e.nodeid)return o()}]),angular.module("flinkApp").directive("vertex",["$state",function(e){return{template:"",scope:{data:"="},link:function(e,t,r){var n,o,i;i=t.children()[0],o=t.width(),angular.element(i).attr("width",o),(n=function(e){var t,r,n;return d3.select(i).selectAll("*").remove(),n=[],angular.forEach(e.subtasks,function(e,t){var r;return r=[{label:"Scheduled",color:"#666",borderColor:"#555",starting_time:e.timestamps.SCHEDULED,ending_time:e.timestamps.DEPLOYING,type:"regular"},{label:"Deploying",color:"#aaa",borderColor:"#555",starting_time:e.timestamps.DEPLOYING,ending_time:e.timestamps.RUNNING,type:"regular"}],e.timestamps.FINISHED>0&&r.push({label:"Running",color:"#ddd",borderColor:"#555",starting_time:e.timestamps.RUNNING,ending_time:e.timestamps.FINISHED,type:"regular"}),n.push({label:"("+e.subtask+") "+e.host,times:r})}),t=d3.timeline().stack().tickFormat({format:d3.time.format("%L"),tickSize:1}).prefix("single").labelFormat(function(e){return e}).margin({left:100,right:0,top:0,bottom:0}).itemHeight(30).relativeTime(),r=d3.select(i).datum(n).call(t)})(e.data)}}}]).directive("timeline",["$state",function(e){return{template:"",scope:{vertices:"=",jobid:"="},link:function(t,r,n){var o,i,s,a;s=r.children()[0],i=r.width(),angular.element(s).attr("width",i),a=function(e){return e.replace(">",">")},o=function(r){var n,o,i;return d3.select(s).selectAll("*").remove(),i=[],angular.forEach(r,function(e){if(e["start-time"]>-1)return"scheduled"===e.type?i.push({times:[{label:a(e.name),color:"#cccccc",borderColor:"#555555",starting_time:e["start-time"],ending_time:e["end-time"],type:e.type}]}):i.push({times:[{label:a(e.name),color:"#d9f1f7",borderColor:"#62cdea",starting_time:e["start-time"],ending_time:e["end-time"],link:e.id,type:e.type}]})}),n=d3.timeline().stack().click(function(r,n,o){if(r.link)return e.go("single-job.timeline.vertex",{jobid:t.jobid,vertexId:r.link})}).tickFormat({format:d3.time.format("%L"),tickSize:1}).prefix("main").margin({left:0,right:0,top:0,bottom:0}).itemHeight(30).showBorderLine().showHourTimeline(),o=d3.select(s).datum(i).call(n)},t.$watch(n.vertices,function(e){if(e)return o(e)})}}}]).directive("split",function(){return{compile:function(e,t){return Split(e.children(),{sizes:[50,50],direction:"vertical"})}}}).directive("jobPlan",["$timeout",function(e){return{template:"
",scope:{plan:"=",watermarks:"=",setNode:"&"},link:function(e,t,r){var n,o,i,s,a,l,u,c,d,f,p,m,h,g,b,v,k,j,S,w,C,$,y,J,M;p=null,C=d3.behavior.zoom(),M=[],g=r.jobid,S=t.children()[0],j=t.children().children()[0],w=t.children()[1],l=d3.select(S),u=d3.select(j),c=d3.select(w),n=t.width(),angular.element(t.children()[0]).width(n),v=0,b=0,e.zoomIn=function(){var e,t,r;if(C.scale()<2.99)return e=C.translate(),t=e[0]*(C.scale()+.1/C.scale()),r=e[1]*(C.scale()+.1/C.scale()),C.scale(C.scale()+.1),C.translate([t,r]),u.attr("transform","translate("+t+","+r+") scale("+C.scale()+")"),v=C.scale(),b=C.translate()},e.zoomOut=function(){var e,t,r;if(C.scale()>.31)return C.scale(C.scale()-.1),e=C.translate(),t=e[0]*(C.scale()-.1/C.scale()),r=e[1]*(C.scale()-.1/C.scale()),C.translate([t,r]),u.attr("transform","translate("+t+","+r+") scale("+C.scale()+")"),v=C.scale(),b=C.translate()},i=function(e){var t;return t="",null==e.ship_strategy&&null==e.local_strategy||(t+="
",null!=e.ship_strategy&&(t+=e.ship_strategy),void 0!==e.temp_mode&&(t+=" ("+e.temp_mode+")"),void 0!==e.local_strategy&&(t+=",
"+e.local_strategy),t+="
"),t},h=function(e){return"partialSolution"===e||"nextPartialSolution"===e||"workset"===e||"nextWorkset"===e||"solutionSet"===e||"solutionDelta"===e},m=function(e,t){return"mirror"===t?"node-mirror":h(t)?"node-iteration":"node-normal"},s=function(e,t,r,n){var o,i;return o="
",o+="mirror"===t?"

Mirror of "+e.operator+"

":"

"+e.operator+"

",""===e.description?o+="":(i=e.description,i=J(i),o+="

"+i+"

"),null!=e.step_function?o+=f(e.id,r,n):(h(t)&&(o+="
"+t+" Node
"),""!==e.parallelism&&(o+="
Parallelism: "+e.parallelism+"
"),void 0!==e.lowWatermark&&(o+="
Low Watermark: "+e.lowWatermark+"
"),void 0!==e.operator&&e.operator_strategy&&(o+="
Operation: "+J(e.operator_strategy)+"
")),o+="
"},f=function(e,t,r){var n,o;return o="svg-"+e,n=""},J=function(e){var t;for("<"===e.charAt(0)&&(e=e.replace("<","<"),e=e.replace(">",">")),t="";e.length>30;)t=t+e.substring(0,30)+"
",e=e.substring(30,e.length);return t+=e},a=function(e,t,r,n,o,i){return null==n&&(n=!1),r.id===t.partial_solution?e.setNode(r.id,{label:s(r,"partialSolution",o,i),labelType:"html","class":m(r,"partialSolution")}):r.id===t.next_partial_solution?e.setNode(r.id,{label:s(r,"nextPartialSolution",o,i),labelType:"html","class":m(r,"nextPartialSolution")}):r.id===t.workset?e.setNode(r.id,{label:s(r,"workset",o,i),labelType:"html","class":m(r,"workset")}):r.id===t.next_workset?e.setNode(r.id,{label:s(r,"nextWorkset",o,i),labelType:"html","class":m(r,"nextWorkset")}):r.id===t.solution_set?e.setNode(r.id,{label:s(r,"solutionSet",o,i),labelType:"html","class":m(r,"solutionSet")}):r.id===t.solution_delta?e.setNode(r.id,{label:s(r,"solutionDelta",o,i),labelType:"html","class":m(r,"solutionDelta")}):e.setNode(r.id,{label:s(r,"",o,i),labelType:"html","class":m(r,"")})},o=function(e,t,r,n,o){return e.setEdge(o.id,r.id,{label:i(o),labelType:"html",arrowhead:"normal"})},k=function(e,t){var r,n,i,s,l,u,d,f,p,m,h,g,b,v;for(n=[],null!=t.nodes?v=t.nodes:(v=t.step_function,i=!0),s=0,u=v.length;s-1))return e["end-time"]=e["start-time"]+e.duration})},this.processVertices=function(e){return angular.forEach(e.vertices,function(e,t){return e.type="regular"}),e.vertices.unshift({name:"Scheduled","start-time":e.timestamps.CREATED,"end-time":e.timestamps.CREATED+1,type:"scheduled"})},this.listJobs=function(){var r;return r=o.defer(),e.get(t.jobServer+"jobs/overview").success(function(e){return function(t,n,o,i){return c.finished=[],c.running=[],_(t.jobs).groupBy(function(e){switch(e.state.toLowerCase()){case"finished":return"finished";case"failed":return"finished";case"canceled":return"finished";default:return"running"}}).forEach(function(t,r){switch(r){case"finished":return c.finished=e.setEndTimes(t);case"running":return c.running=e.setEndTimes(t)}}).value(),r.resolve(c),d()}}(this)),r.promise},this.getJobs=function(e){return c[e]},this.getAllJobs=function(){return c},this.loadJob=function(r){return s=null,l.job=o.defer(),e.get(t.jobServer+"jobs/"+r).success(function(n){return function(o,i,a,u){return n.setEndTimes(o.vertices),n.processVertices(o),e.get(t.jobServer+"jobs/"+r+"/config").success(function(e){return o=angular.extend(o,e),s=o,l.job.resolve(s)})}}(this)),l.job.promise},this.getNode=function(e){var t,r;return r=function(e,t){var n,o,i,s;for(n=0,o=t.length;n
{{metric.id}}
{{value | humanizeChartNumeric:metric}}
',replace:!0,scope:{metric:"=",window:"=",removeMetric:"&",setMetricSize:"=",setMetricView:"=",getValues:"&"},link:function(e,t,r){return e.btnClasses=["btn","btn-default","btn-xs"],e.value=null,e.data=[{values:e.getValues() +}],e.options={x:function(e,t){return e.x},y:function(e,t){return e.y},xTickFormat:function(e){return d3.time.format("%H:%M:%S")(new Date(e))},yTickFormat:function(e){var t,r,n,o;for(r=!1,n=0,o=1,t=Math.abs(e);!r&&n<50;)Math.pow(10,n)<=t&&t6?e/Math.pow(10,n)+"E"+n:""+e}},e.showChart=function(){return d3.select(t.find("svg")[0]).datum(e.data).transition().duration(250).call(e.chart)},e.chart=nv.models.lineChart().options(e.options).showLegend(!1).margin({top:15,left:60,bottom:30,right:30}),e.chart.yAxis.showMaxMin(!1),e.chart.tooltip.hideDelay(0),e.chart.tooltip.contentGenerator(function(e){return"

"+d3.time.format("%H:%M:%S")(new Date(e.point.x))+" | "+e.point.y+"

"}),nv.utils.windowResize(e.chart.update),e.setSize=function(t){return e.setMetricSize(e.metric,t)},e.setView=function(t){if(e.setMetricView(e.metric,t),"chart"===t)return e.showChart()},"chart"===e.metric.view&&e.showChart(),e.$on("metrics:data:update",function(t,r,n){return e.value=parseFloat(n[e.metric.id]),e.data[0].values.push({x:r,y:e.value}),e.data[0].values.length>e.window&&e.data[0].values.shift(),"chart"===e.metric.view&&e.showChart(),"chart"===e.metric.view&&e.chart.clearHighlights(),e.chart.tooltip.hidden(!0)}),t.find(".metric-title").qtip({content:{text:e.metric.id},position:{my:"bottom left",at:"top left"},style:{classes:"qtip-light qtip-timeline-bar"}})}}}),angular.module("flinkApp").service("MetricsService",["$http","$q","flinkConfig","$interval",function(e,t,r,n){return this.metrics={},this.values={},this.watched={},this.observer={jobid:null,nodeid:null,callback:null},this.refresh=n(function(e){return function(){return angular.forEach(e.metrics,function(t,r){return angular.forEach(t,function(t,n){var o;if(o=[],angular.forEach(t,function(e,t){return o.push(e.id)}),o.length>0)return e.getMetrics(r,n,o).then(function(t){if(r===e.observer.jobid&&n===e.observer.nodeid&&e.observer.callback)return e.observer.callback(t)})})})}}(this),r["refresh-interval"]),this.registerObserver=function(e,t,r){return this.observer.jobid=e,this.observer.nodeid=t,this.observer.callback=r},this.unRegisterObserver=function(){return this.observer={jobid:null,nodeid:null,callback:null}},this.setupMetrics=function(e,t){return this.setupLS(),this.watched[e]=[],angular.forEach(t,function(t){return function(r,n){if(r.id)return t.watched[e].push(r.id)}}(this))},this.getWindow=function(){return 100},this.setupLS=function(){return null==sessionStorage.flinkMetrics&&this.saveSetup(),this.metrics=JSON.parse(sessionStorage.flinkMetrics)},this.saveSetup=function(){return sessionStorage.flinkMetrics=JSON.stringify(this.metrics)},this.saveValue=function(e,t,r){if(null==this.values[e]&&(this.values[e]={}),null==this.values[e][t]&&(this.values[e][t]=[]),this.values[e][t].push(r),this.values[e][t].length>this.getWindow())return this.values[e][t].shift()},this.getValues=function(e,t,r){var n;return null==this.values[e]?[]:null==this.values[e][t]?[]:(n=[],angular.forEach(this.values[e][t],function(e){return function(e,t){if(null!=e.values[r])return n.push({x:e.timestamp,y:e.values[r]})}}(this)),n)},this.setupLSFor=function(e,t){if(null==this.metrics[e]&&(this.metrics[e]={}),null==this.metrics[e][t])return this.metrics[e][t]=[]},this.addMetric=function(e,t,r){return this.setupLSFor(e,t),this.metrics[e][t].push({id:r,size:"small",view:"chart"}),this.saveSetup()},this.removeMetric=function(e){return function(t,r,n){var o;if(null!=e.metrics[t][r])return o=e.metrics[t][r].indexOf(n),o===-1&&(o=_.findIndex(e.metrics[t][r],{id:n})),o!==-1&&e.metrics[t][r].splice(o,1),e.saveSetup()}}(this),this.setMetricSize=function(e){return function(t,r,n,o){var i;if(null!=e.metrics[t][r])return i=e.metrics[t][r].indexOf(n.id),i===-1&&(i=_.findIndex(e.metrics[t][r],{id:n.id})),i!==-1&&(e.metrics[t][r][i]={id:n.id,size:o,view:n.view}),e.saveSetup()}}(this),this.setMetricView=function(e){return function(t,r,n,o){var i;if(null!=e.metrics[t][r])return i=e.metrics[t][r].indexOf(n.id),i===-1&&(i=_.findIndex(e.metrics[t][r],{id:n.id})),i!==-1&&(e.metrics[t][r][i]={id:n.id,size:n.size,view:o}),e.saveSetup()}}(this),this.orderMetrics=function(e,t,r,n){return this.setupLSFor(e,t),angular.forEach(this.metrics[e][t],function(o){return function(i,s){if(i.id===r.id&&(o.metrics[e][t].splice(s,1),s",link:function(t,r,n){return t.getLabelClass=function(){return"label label-"+e.translateLabelState(n.status)}}}}]).directive("bpLabel",["JobsService",function(e){return{transclude:!0,replace:!0,scope:{getBackPressureLabelClass:"&",status:"@"},template:"",link:function(t,r,n){return t.getBackPressureLabelClass=function(){return"label label-"+e.translateBackPressureLabelState(n.status)}}}}]).directive("indicatorPrimary",["JobsService",function(e){return{replace:!0,scope:{getLabelClass:"&",status:"@"},template:"",link:function(t,r,n){return t.getLabelClass=function(){return"fa fa-circle indicator indicator-"+e.translateLabelState(n.status)}}}}]).directive("tableProperty",function(){return{replace:!0,scope:{value:"="},template:"{{value || 'None'}}"}}),angular.module("flinkApp").filter("amDurationFormatExtended",["angularMomentConfig",function(e){var t;return t=function(e,t,r){return"undefined"==typeof e||null===e?"":moment.duration(e,t).format(r,{trim:!1})},t.$stateful=e.statefulFilters,t}]).filter("humanizeDuration",function(){return function(e,t){var r,n,o,i,a,s;return"undefined"==typeof e||null===e?"":(i=e%1e3,s=Math.floor(e/1e3),a=s%60,s=Math.floor(s/60),o=s%60,s=Math.floor(s/60),n=s%24,s=Math.floor(s/24),r=s,0===r?0===n?0===o?0===a?i+"ms":a+"s ":o+"m "+a+"s":t?n+"h "+o+"m":n+"h "+o+"m "+a+"s":t?r+"d "+n+"h":r+"d "+n+"h "+o+"m "+a+"s")}}).filter("limit",function(){return function(e){return e.length>73&&(e=e.substring(0,35)+"..."+e.substring(e.length-35,e.length)),e}}).filter("humanizeText",function(){return function(e){return e?e.replace(/>/g,">").replace(//g,""):""}}).filter("humanizeBytes",function(){return function(e){var t,r;return r=["B","KB","MB","GB","TB","PB","EB"],t=function(e,n){var o;return o=Math.pow(1024,n),e=r;n=0<=r?++e:--e)o.push(n+".currentInputWatermark");return o}(),o.getMetrics(i,t.id,a).then(function(e){var t,n,o,i,a,s,l;o=NaN,l={},i=e.values;for(t in i)s=i[t],a=t.replace(".currentInputWatermark",""),l[a]=s,(isNaN(o)||su.noWatermark?o:NaN,r.resolve({lowWatermark:n,watermarks:l})}),r.promise}}(this),r=l.defer(),a={},n=t.length,angular.forEach(t,function(e){return function(e,t){var o;return o=e.id,i(e).then(function(e){if(a[o]=e,t>=n-1)return r.resolve(a)})}}(this)),r.promise},e.hasWatermark=function(t){return e.watermarks[t]&&!isNaN(e.watermarks[t].lowWatermark)},e.$watch("plan",function(t){if(t)return c(t.nodes).then(function(t){return e.watermarks=t})}),e.$on("reload",function(){if(e.plan)return c(e.plan.nodes).then(function(t){return e.watermarks=t})})}]).controller("JobPlanController",["$scope","$state","$stateParams","$window","JobsService",function(e,t,r,n,o){return e.nodeid=null,e.nodeUnfolded=!1,e.stateList=o.stateList(),e.changeNode=function(t){return t!==e.nodeid?(e.nodeid=t,e.vertex=null,e.subtasks=null,e.accumulators=null,e.operatorCheckpointStats=null,e.$broadcast("reload"),e.$broadcast("node:change",e.nodeid)):(e.nodeid=null,e.nodeUnfolded=!1,e.vertex=null,e.subtasks=null,e.accumulators=null,e.operatorCheckpointStats=null)},e.deactivateNode=function(){return e.nodeid=null,e.nodeUnfolded=!1,e.vertex=null,e.subtasks=null,e.accumulators=null,e.operatorCheckpointStats=null},e.toggleFold=function(){return e.nodeUnfolded=!e.nodeUnfolded}}]).controller("JobPlanSubtasksController",["$scope","JobsService",function(e,t){var r;return e.aggregate=!1,r=function(){return e.aggregate?t.getTaskManagers(e.nodeid).then(function(t){return e.taskmanagers=t}):t.getSubtasks(e.nodeid).then(function(t){return e.subtasks=t})},!e.nodeid||e.vertex&&e.vertex.st||r(),e.$on("reload",function(t){if(e.nodeid)return r()})}]).controller("JobPlanAccumulatorsController",["$scope","JobsService",function(e,t){var r;return r=function(){return t.getAccumulators(e.nodeid).then(function(t){return e.accumulators=t.main,e.subtaskAccumulators=t.subtasks})},!e.nodeid||e.vertex&&e.vertex.accumulators||r(),e.$on("reload",function(t){if(e.nodeid)return r()})}]).controller("JobPlanCheckpointsController",["$scope","$state","$stateParams","JobsService",function(e,t,r,n){var o;return e.checkpointDetails={},e.checkpointDetails.id=-1,n.getCheckpointConfig().then(function(t){return e.checkpointConfig=t}),o=function(){return n.getCheckpointStats().then(function(t){if(null!==t)return e.checkpointStats=t})},o(),e.$on("reload",function(e){return o()})}]).controller("JobPlanCheckpointDetailsController",["$scope","$state","$stateParams","JobsService",function(e,t,r,n){var o,i;return e.subtaskDetails={},e.checkpointDetails.id=r.checkpointId,o=function(t){return n.getCheckpointDetails(t).then(function(t){return null!==t?e.checkpoint=t:e.unknown_checkpoint=!0})},i=function(t,r){return n.getCheckpointSubtaskDetails(t,r).then(function(t){if(null!==t)return e.subtaskDetails[r]=t})},o(r.checkpointId),e.nodeid&&i(r.checkpointId,e.nodeid),e.$on("reload",function(t){if(o(r.checkpointId),e.nodeid)return i(r.checkpointId,e.nodeid)}),e.$on("$destroy",function(){return e.checkpointDetails.id=-1})}]).controller("JobPlanBackPressureController",["$scope","JobsService",function(e,t){var r;return r=function(){if(e.now=Date.now(),e.nodeid)return t.getOperatorBackPressure(e.nodeid).then(function(t){return e.backPressureOperatorStats[e.nodeid]=t})},r(),e.$on("reload",function(e){return r()})}]).controller("JobTimelineVertexController",["$scope","$state","$stateParams","JobsService",function(e,t,r,n){var o;return o=function(){return n.getVertex(r.vertexId).then(function(t){return e.vertex=t})},o(),e.$on("reload",function(e){return o()})}]).controller("JobExceptionsController",["$scope","$state","$stateParams","JobsService",function(e,t,r,n){return n.loadExceptions().then(function(t){return e.exceptions=t})}]).controller("JobPropertiesController",["$scope","JobsService",function(e,t){return e.changeNode=function(r){return r!==e.nodeid?(e.nodeid=r,t.getNode(r).then(function(t){return e.node=t})):(e.nodeid=null,e.node=null)}}]).controller("JobPlanMetricsController",["$scope","JobsService","MetricsService",function(e,t,r){var n,o;if(e.dragging=!1,e.window=r.getWindow(),e.availableMetrics=null,e.$on("$destroy",function(){return r.unRegisterObserver()}),o=function(){return t.getVertex(e.nodeid).then(function(t){return e.vertex=t}),r.getAvailableMetrics(e.jobid,e.nodeid).then(function(t){return e.availableMetrics=t.sort(n),e.metrics=r.getMetricsSetup(e.jobid,e.nodeid).names,r.registerObserver(e.jobid,e.nodeid,function(t){return e.$broadcast("metrics:data:update",t.timestamp,t.values)})})},n=function(e,t){var r,n;return r=e.id.toLowerCase(),n=t.id.toLowerCase(),rn?1:0},e.dropped=function(t,n,i,a,s){return r.orderMetrics(e.jobid,e.nodeid,i,n),e.$broadcast("metrics:refresh",i),o(),!1},e.dragStart=function(){return e.dragging=!0},e.dragEnd=function(){return e.dragging=!1},e.addMetric=function(t){return r.addMetric(e.jobid,e.nodeid,t.id),o()},e.removeMetric=function(t){return r.removeMetric(e.jobid,e.nodeid,t),o()},e.setMetricSize=function(t,n){return r.setMetricSize(e.jobid,e.nodeid,t,n),o()},e.setMetricView=function(t,n){return r.setMetricView(e.jobid,e.nodeid,t,n),o()},e.getValues=function(t){return r.getValues(e.jobid,e.nodeid,t)},e.$on("node:change",function(t,r){if(!e.dragging)return o()}),e.nodeid)return o()}]),angular.module("flinkApp").directive("vertex",["$state",function(e){return{template:"",scope:{data:"="},link:function(e,t,r){var n,o,i;i=t.children()[0],o=t.width(),angular.element(i).attr("width",o),(n=function(e){var t,r,n;return d3.select(i).selectAll("*").remove(),n=[],angular.forEach(e.subtasks,function(e,t){var r;return r=[{label:"Scheduled",color:"#666",borderColor:"#555",starting_time:e.timestamps.SCHEDULED,ending_time:e.timestamps.DEPLOYING,type:"regular"},{label:"Deploying",color:"#aaa",borderColor:"#555",starting_time:e.timestamps.DEPLOYING,ending_time:e.timestamps.RUNNING,type:"regular"}],e.timestamps.FINISHED>0&&r.push({label:"Running",color:"#ddd",borderColor:"#555",starting_time:e.timestamps.RUNNING,ending_time:e.timestamps.FINISHED,type:"regular"}),n.push({label:"("+e.subtask+") "+e.host,times:r})}),t=d3.timeline().stack().tickFormat({format:d3.time.format("%L"),tickSize:1}).prefix("single").labelFormat(function(e){return e}).margin({left:100,right:0,top:0,bottom:0}).itemHeight(30).relativeTime(),r=d3.select(i).datum(n).call(t)})(e.data)}}}]).directive("timeline",["$state",function(e){return{template:"",scope:{vertices:"=",jobid:"="},link:function(t,r,n){var o,i,a,s;a=r.children()[0],i=r.width(),angular.element(a).attr("width",i),s=function(e){return e.replace(">",">")},o=function(r){var n,o,i;return d3.select(a).selectAll("*").remove(),i=[],angular.forEach(r,function(e){if(e["start-time"]>-1)return"scheduled"===e.type?i.push({times:[{label:s(e.name),color:"#cccccc",borderColor:"#555555",starting_time:e["start-time"],ending_time:e["end-time"],type:e.type}]}):i.push({times:[{label:s(e.name),color:"#d9f1f7",borderColor:"#62cdea",starting_time:e["start-time"],ending_time:e["end-time"],link:e.id,type:e.type}]})}),n=d3.timeline().stack().click(function(r,n,o){if(r.link)return e.go("single-job.timeline.vertex",{jobid:t.jobid,vertexId:r.link})}).tickFormat({format:d3.time.format("%L"),tickSize:1}).prefix("main").margin({left:0,right:0,top:0,bottom:0}).itemHeight(30).showBorderLine().showHourTimeline(),o=d3.select(a).datum(i).call(n)},t.$watch(n.vertices,function(e){if(e)return o(e)})}}}]).directive("split",function(){return{compile:function(e,t){return Split(e.children(),{sizes:[50,50],direction:"vertical"})}}}).directive("jobPlan",["$timeout",function(e){return{template:"
",scope:{plan:"=",watermarks:"=",setNode:"&"},link:function(e,t,r){var n,o,i,a,s,l,u,c,d,f,p,m,g,h,b,v,k,j,S,w,C,$,y,M,J;p=null,C=d3.behavior.zoom(),J=[],h=r.jobid,S=t.children()[0],j=t.children().children()[0],w=t.children()[1],l=d3.select(S),u=d3.select(j),c=d3.select(w),n=t.width(),angular.element(t.children()[0]).width(n),v=0,b=0,e.zoomIn=function(){var e,t,r;if(C.scale()<2.99)return e=C.translate(),t=e[0]*(C.scale()+.1/C.scale()),r=e[1]*(C.scale()+.1/C.scale()),C.scale(C.scale()+.1),C.translate([t,r]),u.attr("transform","translate("+t+","+r+") scale("+C.scale()+")"),v=C.scale(),b=C.translate()},e.zoomOut=function(){var e,t,r;if(C.scale()>.31)return C.scale(C.scale()-.1),e=C.translate(),t=e[0]*(C.scale()-.1/C.scale()),r=e[1]*(C.scale()-.1/C.scale()),C.translate([t,r]),u.attr("transform","translate("+t+","+r+") scale("+C.scale()+")"),v=C.scale(),b=C.translate()},i=function(e){var t;return t="",null==e.ship_strategy&&null==e.local_strategy||(t+="
",null!=e.ship_strategy&&(t+=e.ship_strategy),void 0!==e.temp_mode&&(t+=" ("+e.temp_mode+")"),void 0!==e.local_strategy&&(t+=",
"+e.local_strategy),t+="
"),t},g=function(e){return"partialSolution"===e||"nextPartialSolution"===e||"workset"===e||"nextWorkset"===e||"solutionSet"===e||"solutionDelta"===e},m=function(e,t){return"mirror"===t?"node-mirror":g(t)?"node-iteration":"node-normal"},a=function(e,t,r,n){var o,i;return o="
",o+="mirror"===t?"

Mirror of "+e.operator+"

":"

"+e.operator+"

",""===e.description?o+="":(i=e.description,i=M(i),o+="

"+i+"

"),null!=e.step_function?o+=f(e.id,r,n):(g(t)&&(o+="
"+t+" Node
"),""!==e.parallelism&&(o+="
Parallelism: "+e.parallelism+"
"),void 0!==e.lowWatermark&&(o+="
Low Watermark: "+e.lowWatermark+"
"),void 0!==e.operator&&e.operator_strategy&&(o+="
Operation: "+M(e.operator_strategy)+"
")),o+="
"},f=function(e,t,r){var n,o;return o="svg-"+e,n=""},M=function(e){var t;for("<"===e.charAt(0)&&(e=e.replace("<","<"),e=e.replace(">",">")),t="";e.length>30;)t=t+e.substring(0,30)+"
",e=e.substring(30,e.length);return t+=e},s=function(e,t,r,n,o,i){return null==n&&(n=!1),r.id===t.partial_solution?e.setNode(r.id,{label:a(r,"partialSolution",o,i),labelType:"html","class":m(r,"partialSolution")}):r.id===t.next_partial_solution?e.setNode(r.id,{label:a(r,"nextPartialSolution",o,i),labelType:"html","class":m(r,"nextPartialSolution")}):r.id===t.workset?e.setNode(r.id,{label:a(r,"workset",o,i),labelType:"html","class":m(r,"workset")}):r.id===t.next_workset?e.setNode(r.id,{label:a(r,"nextWorkset",o,i),labelType:"html","class":m(r,"nextWorkset")}):r.id===t.solution_set?e.setNode(r.id,{label:a(r,"solutionSet",o,i),labelType:"html","class":m(r,"solutionSet")}):r.id===t.solution_delta?e.setNode(r.id,{label:a(r,"solutionDelta",o,i),labelType:"html","class":m(r,"solutionDelta")}):e.setNode(r.id,{label:a(r,"",o,i),labelType:"html","class":m(r,"")})},o=function(e,t,r,n,o){return e.setEdge(o.id,r.id,{label:i(o),labelType:"html",arrowhead:"normal"})},k=function(e,t){var r,n,i,a,l,u,d,f,p,m,g,h,b,v;for(n=[],null!=t.nodes?v=t.nodes:(v=t.step_function,i=!0),a=0,u=v.length;a-1))return e["end-time"]=e["start-time"]+e.duration})},this.processVertices=function(e){return angular.forEach(e.vertices,function(e,t){return e.type="regular"}),e.vertices.unshift({name:"Scheduled","start-time":e.timestamps.CREATED,"end-time":e.timestamps.CREATED+1,type:"scheduled"})},this.listJobs=function(){var r;return r=o.defer(),e.get(t.jobServer+"jobs/overview").success(function(e){return function(t,n,o,i){return c.finished=[],c.running=[],_(t.jobs).groupBy(function(e){switch(e.state.toLowerCase()){case"finished":return"finished";case"failed":return"finished";case"canceled":return"finished";default:return"running"}}).forEach(function(t,r){switch(r){case"finished":return c.finished=e.setEndTimes(t);case"running":return c.running=e.setEndTimes(t)}}).value(),r.resolve(c),d()}}(this)),r.promise},this.getJobs=function(e){return c[e]},this.getAllJobs=function(){return c},this.loadJob=function(r){return a=null,l.job=o.defer(),e.get(t.jobServer+"jobs/"+r).success(function(n){return function(o,i,s,u){return n.setEndTimes(o.vertices),n.processVertices(o),e.get(t.jobServer+"jobs/"+r+"/config").success(function(e){return o=angular.extend(o,e),a=o,l.job.resolve(a)})}}(this)),l.job.promise},this.getNode=function(e){var t,r;return r=function(e,t){var n,o,i,a;for(n=0,o=t.length;n
{{metric.id}}
{{value | humanizeChartNumeric:metric}}
',replace:!0,scope:{metric:"=",window:"=",removeMetric:"&",setMetricSize:"=",setMetricView:"=",getValues:"&"},link:function(e,t,r){return e.btnClasses=["btn","btn-default","btn-xs"],e.value=null,e.data=[{values:e.getValues()}],e.options={x:function(e,t){return e.x},y:function(e,t){return e.y},xTickFormat:function(e){return d3.time.format("%H:%M:%S")(new Date(e))},yTickFormat:function(e){var t,r,n,o;for(r=!1,n=0,o=1,t=Math.abs(e);!r&&n<50;)Math.pow(10,n)<=t&&t6?e/Math.pow(10,n)+"E"+n:""+e}},e.showChart=function(){return d3.select(t.find("svg")[0]).datum(e.data).transition().duration(250).call(e.chart)},e.chart=nv.models.lineChart().options(e.options).showLegend(!1).margin({top:15,left:60,bottom:30,right:30}),e.chart.yAxis.showMaxMin(!1),e.chart.tooltip.hideDelay(0),e.chart.tooltip.contentGenerator(function(e){return"

"+d3.time.format("%H:%M:%S")(new Date(e.point.x))+" | "+e.point.y+"

"}),nv.utils.windowResize(e.chart.update),e.setSize=function(t){return e.setMetricSize(e.metric,t)},e.setView=function(t){if(e.setMetricView(e.metric,t),"chart"===t)return e.showChart()},"chart"===e.metric.view&&e.showChart(),e.$on("metrics:data:update",function(t,r,n){return e.value=parseFloat(n[e.metric.id]),e.data[0].values.push({x:r,y:e.value}),e.data[0].values.length>e.window&&e.data[0].values.shift(),"chart"===e.metric.view&&e.showChart(),"chart"===e.metric.view&&e.chart.clearHighlights(),e.chart.tooltip.hidden(!0)}),t.find(".metric-title").qtip({content:{text:e.metric.id},position:{my:"bottom left",at:"top left"},style:{classes:"qtip-light qtip-timeline-bar"}})}}}),angular.module("flinkApp").service("MetricsService",["$http","$q","flinkConfig","$interval",function(e,t,r,n){return this.metrics={},this.values={},this.watched={},this.observer={jobid:null,nodeid:null,callback:null},this.refresh=n(function(e){return function(){return angular.forEach(e.metrics,function(t,r){return angular.forEach(t,function(t,n){var o;if(o=[],angular.forEach(t,function(e,t){return o.push(e.id)}),o.length>0)return e.getMetrics(r,n,o).then(function(t){if(r===e.observer.jobid&&n===e.observer.nodeid&&e.observer.callback)return e.observer.callback(t)})})})}}(this),r["refresh-interval"]),this.registerObserver=function(e,t,r){return this.observer.jobid=e,this.observer.nodeid=t,this.observer.callback=r},this.unRegisterObserver=function(){return this.observer={jobid:null,nodeid:null,callback:null}},this.setupMetrics=function(e,t){return this.setupLS(),this.watched[e]=[],angular.forEach(t,function(t){return function(r,n){if(r.id)return t.watched[e].push(r.id)}}(this))},this.getWindow=function(){return 100},this.setupLS=function(){return null==sessionStorage.flinkMetrics&&this.saveSetup(),this.metrics=JSON.parse(sessionStorage.flinkMetrics)},this.saveSetup=function(){return sessionStorage.flinkMetrics=JSON.stringify(this.metrics)},this.saveValue=function(e,t,r){if(null==this.values[e]&&(this.values[e]={}),null==this.values[e][t]&&(this.values[e][t]=[]),this.values[e][t].push(r),this.values[e][t].length>this.getWindow())return this.values[e][t].shift()},this.getValues=function(e,t,r){var n;return null==this.values[e]?[]:null==this.values[e][t]?[]:(n=[],angular.forEach(this.values[e][t],function(e){return function(e,t){if(null!=e.values[r])return n.push({x:e.timestamp,y:e.values[r]})}}(this)),n)},this.setupLSFor=function(e,t){if(null==this.metrics[e]&&(this.metrics[e]={}),null==this.metrics[e][t])return this.metrics[e][t]=[]},this.addMetric=function(e,t,r){return this.setupLSFor(e,t),this.metrics[e][t].push({id:r,size:"small",view:"chart"}),this.saveSetup()},this.removeMetric=function(e){return function(t,r,n){var o;if(null!=e.metrics[t][r])return o=e.metrics[t][r].indexOf(n),o===-1&&(o=_.findIndex(e.metrics[t][r],{id:n})),o!==-1&&e.metrics[t][r].splice(o,1),e.saveSetup()}}(this),this.setMetricSize=function(e){return function(t,r,n,o){var i;if(null!=e.metrics[t][r])return i=e.metrics[t][r].indexOf(n.id),i===-1&&(i=_.findIndex(e.metrics[t][r],{id:n.id})),i!==-1&&(e.metrics[t][r][i]={id:n.id,size:o,view:n.view}),e.saveSetup()}}(this),this.setMetricView=function(e){return function(t,r,n,o){var i;if(null!=e.metrics[t][r])return i=e.metrics[t][r].indexOf(n.id),i===-1&&(i=_.findIndex(e.metrics[t][r],{id:n.id})),i!==-1&&(e.metrics[t][r][i]={id:n.id,size:n.size,view:o}),e.saveSetup()}}(this),this.orderMetrics=function(e,t,r,n){return this.setupLSFor(e,t),angular.forEach(this.metrics[e][t],function(o){return function(i,a){if(i.id===r.id&&(o.metrics[e][t].splice(a,1),a
{{metric.id}}
{{value | humanizeChartNumeric:metric}}
',replace:!0,scope:{metric:"=",window:"=",removeMetric:"&",setMetricSize:"=",setMetricView:"=",getValues:"&"},link:function(e,t,r){return e.btnClasses=["btn","btn-default","btn-xs"],e.value=null,e.data=[{values:e.getValues()}],e.options={x:function(e,t){return e.x},y:function(e,t){return e.y},xTickFormat:function(e){return d3.time.format("%H:%M:%S")(new Date(e))},yTickFormat:function(e){var t,r,n,o;for(r=!1,n=0,o=1,t=Math.abs(e);!r&&n<50;)Math.pow(10,n)<=t&&t6?e/Math.pow(10,n)+"E"+n:""+e}},e.showChart=function(){return d3.select(t.find("svg")[0]).datum(e.data).transition().duration(250).call(e.chart)},e.chart=nv.models.lineChart().options(e.options).showLegend(!1).margin({top:15,left:60,bottom:30,right:30}),e.chart.yAxis.showMaxMin(!1),e.chart.tooltip.hideDelay(0),e.chart.tooltip.contentGenerator(function(e){return"

"+d3.time.format("%H:%M:%S")(new Date(e.point.x))+" | "+e.point.y+"

"}),nv.utils.windowResize(e.chart.update),e.setSize=function(t){return e.setMetricSize(e.metric,t)},e.setView=function(t){if(e.setMetricView(e.metric,t),"chart"===t)return e.showChart()},"chart"===e.metric.view&&e.showChart(),e.$on("metrics:data:update",function(t,r,n){return e.value=parseFloat(n[e.metric.id]),e.data[0].values.push({x:r,y:e.value}),e.data[0].values.length>e.window&&e.data[0].values.shift(),"chart"===e.metric.view&&e.showChart(),"chart"===e.metric.view&&e.chart.clearHighlights(),e.chart.tooltip.hidden(!0)}),t.find(".metric-title").qtip({content:{text:e.metric.id},position:{my:"bottom left",at:"top left"},style:{classes:"qtip-light qtip-timeline-bar"}})}}}),angular.module("flinkApp").service("MetricsService",["$http","$q","flinkConfig","$interval",function(e,t,r,n){return this.metrics={},this.values={},this.watched={},this.observer={jobid:null,nodeid:null,callback:null},this.refresh=n(function(e){return function(){return angular.forEach(e.metrics,function(t,r){return angular.forEach(t,function(t,n){var o;if(o=[],angular.forEach(t,function(e,t){return o.push(e.id)}),o.length>0)return e.getMetrics(r,n,o).then(function(t){if(r===e.observer.jobid&&n===e.observer.nodeid&&e.observer.callback)return e.observer.callback(t)})})})}}(this),r["refresh-interval"]),this.registerObserver=function(e,t,r){return this.observer.jobid=e,this.observer.nodeid=t,this.observer.callback=r},this.unRegisterObserver=function(){return this.observer={jobid:null,nodeid:null,callback:null}},this.setupMetrics=function(e,t){return this.setupLS(),this.watched[e]=[],angular.forEach(t,function(t){return function(r,n){if(r.id)return t.watched[e].push(r.id)}}(this))},this.getWindow=function(){return 100},this.setupLS=function(){return null==sessionStorage.flinkMetrics&&this.saveSetup(),this.metrics=JSON.parse(sessionStorage.flinkMetrics)},this.saveSetup=function(){return sessionStorage.flinkMetrics=JSON.stringify(this.metrics)},this.saveValue=function(e,t,r){if(null==this.values[e]&&(this.values[e]={}),null==this.values[e][t]&&(this.values[e][t]=[]),this.values[e][t].push(r),this.values[e][t].length>this.getWindow())return this.values[e][t].shift()},this.getValues=function(e,t,r){var n;return null==this.values[e]?[]:null==this.values[e][t]?[]:(n=[],angular.forEach(this.values[e][t],function(e){return function(e,t){if(null!=e.values[r])return n.push({x:e.timestamp,y:e.values[r]})}}(this)),n)},this.setupLSFor=function(e,t){if(null==this.metrics[e]&&(this.metrics[e]={}),null==this.metrics[e][t])return this.metrics[e][t]=[]},this.addMetric=function(e,t,r){return this.setupLSFor(e,t),this.metrics[e][t].push({id:r,size:"small",view:"chart"}),this.saveSetup()},this.removeMetric=function(e){return function(t,r,n){var o;if(null!=e.metrics[t][r])return o=e.metrics[t][r].indexOf(n),o===-1&&(o=_.findIndex(e.metrics[t][r],{id:n})),o!==-1&&e.metrics[t][r].splice(o,1),e.saveSetup()}}(this),this.setMetricSize=function(e){return function(t,r,n,o){var i;if(null!=e.metrics[t][r])return i=e.metrics[t][r].indexOf(n.id),i===-1&&(i=_.findIndex(e.metrics[t][r],{id:n.id})),i!==-1&&(e.metrics[t][r][i]={id:n.id,size:o,view:n.view}),e.saveSetup()}}(this),this.setMetricView=function(e){return function(t,r,n,o){var i;if(null!=e.metrics[t][r])return i=e.metrics[t][r].indexOf(n.id),i===-1&&(i=_.findIndex(e.metrics[t][r],{id:n.id})),i!==-1&&(e.metrics[t][r][i]={id:n.id,size:n.size,view:o}),e.saveSetup()}}(this),this.orderMetrics=function(e,t,r,n){return this.setupLSFor(e,t),angular.forEach(this.metrics[e][t],function(o){return function(i,a){if(i.id===r.id&&(o.metrics[e][t].splice(a,1),a Date: Wed, 18 Jul 2018 21:39:57 +0800 Subject: [PATCH 19/65] [hotfix] [sql-client] Fix typo in SqlExecutionException This closes #6364. --- .../flink/table/client/gateway/local/ExecutionContext.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/flink-libraries/flink-sql-client/src/main/java/org/apache/flink/table/client/gateway/local/ExecutionContext.java b/flink-libraries/flink-sql-client/src/main/java/org/apache/flink/table/client/gateway/local/ExecutionContext.java index 926bdb01a1664..9152908645a49 100644 --- a/flink-libraries/flink-sql-client/src/main/java/org/apache/flink/table/client/gateway/local/ExecutionContext.java +++ b/flink-libraries/flink-sql-client/src/main/java/org/apache/flink/table/client/gateway/local/ExecutionContext.java @@ -236,7 +236,7 @@ private static TableSink createTableSink(Execution execution, Map Date: Thu, 19 Jul 2018 08:50:09 +0200 Subject: [PATCH 20/65] [hotfix] [docs] Update docs about streaming joins --- docs/dev/table/tableApi.md | 10 ++++++++-- 1 file changed, 8 insertions(+), 2 deletions(-) diff --git a/docs/dev/table/tableApi.md b/docs/dev/table/tableApi.md index 44436c4bb6438..2f651454903c1 100644 --- a/docs/dev/table/tableApi.md +++ b/docs/dev/table/tableApi.md @@ -506,7 +506,7 @@ Table result = left.join(right).where("a = d").select("a, b, e"); - Outer Joins
+ Outer Join
Batch Streaming Result Updating @@ -521,6 +521,7 @@ Table leftOuterResult = left.leftOuterJoin(right, "a = d").select("a, b, e"); Table rightOuterResult = left.rightOuterJoin(right, "a = d").select("a, b, e"); Table fullOuterResult = left.fullOuterJoin(right, "a = d").select("a, b, e"); {% endhighlight %} +

Note: For streaming queries the required state to compute the query result might grow infinitely depending on the number of distinct input rows. Please provide a query configuration with valid retention interval to prevent excessive state size. See Streaming Concepts for details.

@@ -612,6 +613,7 @@ Table result = orders Inner Join
Batch + Streaming

Similar to a SQL JOIN clause. Joins two tables. Both tables must have distinct field names and at least one equality join predicate must be defined through join operator or using a where or filter operator.

@@ -620,12 +622,15 @@ val left = ds1.toTable(tableEnv, 'a, 'b, 'c) val right = ds2.toTable(tableEnv, 'd, 'e, 'f) val result = left.join(right).where('a === 'd).select('a, 'b, 'e) {% endhighlight %} +

Note: For streaming queries the required state to compute the query result might grow infinitely depending on the number of distinct input rows. Please provide a query configuration with valid retention interval to prevent excessive state size. See Streaming Concepts for details.

- Outer Joins
+ Outer Join
Batch + Streaming + Result Updating

Similar to SQL LEFT/RIGHT/FULL OUTER JOIN clauses. Joins two tables. Both tables must have distinct field names and at least one equality join predicate must be defined.

@@ -637,6 +642,7 @@ val leftOuterResult = left.leftOuterJoin(right, 'a === 'd).select('a, 'b, 'e) val rightOuterResult = left.rightOuterJoin(right, 'a === 'd).select('a, 'b, 'e) val fullOuterResult = left.fullOuterJoin(right, 'a === 'd).select('a, 'b, 'e) {% endhighlight %} +

Note: For streaming queries the required state to compute the query result might grow infinitely depending on the number of distinct input rows. Please provide a query configuration with valid retention interval to prevent excessive state size. See Streaming Concepts for details.

From 4b7eff7712ce009ef6641f94cf06848d83a2f2ec Mon Sep 17 00:00:00 2001 From: Ashwin Sinha Date: Wed, 18 Jul 2018 02:19:20 +0530 Subject: [PATCH 21/65] [FLINK-9881] [table] Fix a typo in table.scala This closes #6354. --- .../scala/org/apache/flink/table/api/table.scala | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/table.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/table.scala index 071cc69b7a240..a44bbaaccd5e1 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/table.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/table.scala @@ -66,7 +66,7 @@ class Table( // Check if the plan has an unbounded TableFunctionCall as child node. // A TableFunctionCall is tolerated as root node because the Table holds the initial call. - if (containsUnboudedUDTFCall(logicalPlan) && + if (containsUnboundedUDTFCall(logicalPlan) && !logicalPlan.isInstanceOf[LogicalTableFunctionCall]) { throw new ValidationException("TableFunction can only be used in join and leftOuterJoin.") } @@ -87,7 +87,7 @@ class Table( def relBuilder: FlinkRelBuilder = tableEnv.getRelBuilder - def getRelNode: RelNode = if (containsUnboudedUDTFCall(logicalPlan)) { + def getRelNode: RelNode = if (containsUnboundedUDTFCall(logicalPlan)) { throw new ValidationException("Cannot translate a query with an unbounded table function call.") } else { logicalPlan.toRelNode(relBuilder) @@ -504,7 +504,7 @@ class Table( private def join(right: Table, joinPredicate: Option[Expression], joinType: JoinType): Table = { // check if we join with a table or a table function - if (!containsUnboudedUDTFCall(right.logicalPlan)) { + if (!containsUnboundedUDTFCall(right.logicalPlan)) { // regular table-table join // check that the TableEnvironment of right table is not null @@ -971,11 +971,11 @@ class Table( * @param n the node to check * @return true if the plan contains an unbounded UDTF call, false otherwise. */ - private def containsUnboudedUDTFCall(n: LogicalNode): Boolean = { + private def containsUnboundedUDTFCall(n: LogicalNode): Boolean = { n match { case functionCall: LogicalTableFunctionCall if functionCall.child == null => true - case u: UnaryNode => containsUnboudedUDTFCall(u.child) - case b: BinaryNode => containsUnboudedUDTFCall(b.left) || containsUnboudedUDTFCall(b.right) + case u: UnaryNode => containsUnboundedUDTFCall(u.child) + case b: BinaryNode => containsUnboundedUDTFCall(b.left) || containsUnboundedUDTFCall(b.right) case _: LeafNode => false } } From 134bdd336334337e6b3e33cd60a5b080ba811777 Mon Sep 17 00:00:00 2001 From: zentol Date: Mon, 16 Jul 2018 15:16:19 +0200 Subject: [PATCH 22/65] [FLINK-9748][release] Use dedicated directory for release artifacts This closes #6342. --- .gitignore | 1 + pom.xml | 2 ++ tools/releasing/create_binary_release.sh | 15 ++++++++++----- tools/releasing/create_source_release.sh | 21 +++++++++++++-------- 4 files changed, 26 insertions(+), 13 deletions(-) diff --git a/.gitignore b/.gitignore index 8fc9fce6fb2d7..1fde2d9e4a14f 100644 --- a/.gitignore +++ b/.gitignore @@ -34,3 +34,4 @@ out/ *.iws tools/flink tools/flink-* +tools/releasing/release diff --git a/pom.xml b/pom.xml index 1f35cd44892ce..874f72a78c908 100644 --- a/pom.xml +++ b/pom.xml @@ -1150,6 +1150,8 @@ under the License. tools/artifacts/** tools/flink*/** + + tools/releasing/release/** apache-maven-3.2.5/** diff --git a/tools/releasing/create_binary_release.sh b/tools/releasing/create_binary_release.sh index 374c785a22580..0bdff60425b36 100755 --- a/tools/releasing/create_binary_release.sh +++ b/tools/releasing/create_binary_release.sh @@ -44,6 +44,12 @@ else SHASUM="sha512sum" fi +cd .. + +FLINK_DIR=`pwd` +RELEASE_DIR=${FLINK_DIR}/tools/releasing/release +mkdir -p ${RELEASE_DIR} + ########################### # build maven package, create Flink distribution, generate signature @@ -65,18 +71,17 @@ make_binary_release() { cd flink-dist/target/flink-*-bin/ tar czf "${dir_name}.tgz" flink-* - cp flink-*.tgz ../../../ - cd ../../../ + cp flink-*.tgz ${RELEASE_DIR} + cd ${RELEASE_DIR} # Sign sha the tgz if [ "$SKIP_GPG" == "false" ] ; then gpg --armor --detach-sig "${dir_name}.tgz" fi $SHASUM "${dir_name}.tgz" > "${dir_name}.tgz.sha512" -} - -cd .. + cd ${FLINK_DIR} +} if [ "$SCALA_VERSION" == "none" ] && [ "$HADOOP_VERSION" == "none" ]; then make_binary_release "" "-DwithoutHadoop" "2.11" diff --git a/tools/releasing/create_source_release.sh b/tools/releasing/create_source_release.sh index dca72819cb84f..0cc32feed130b 100755 --- a/tools/releasing/create_source_release.sh +++ b/tools/releasing/create_source_release.sh @@ -45,11 +45,17 @@ fi cd .. +FLINK_DIR=`pwd` +RELEASE_DIR=${FLINK_DIR}/tools/releasing/release +CLONE_DIR=${RELEASE_DIR}/flink-tmp-clone + echo "Creating source package" +mkdir -p ${RELEASE_DIR} + # create a temporary git clone to ensure that we have a pristine source release -git clone . flink-tmp-clone -cd flink-tmp-clone +git clone ${FLINK_DIR} ${CLONE_DIR} +cd ${CLONE_DIR} rsync -a \ --exclude ".git" --exclude ".gitignore" --exclude ".gitattributes" --exclude ".travis.yml" \ @@ -58,10 +64,9 @@ rsync -a \ --exclude "docs/content" --exclude ".rubydeps" \ . flink-$RELEASE_VERSION -tar czf flink-${RELEASE_VERSION}-src.tgz flink-$RELEASE_VERSION -gpg --armor --detach-sig flink-$RELEASE_VERSION-src.tgz -$SHASUM flink-$RELEASE_VERSION-src.tgz > flink-$RELEASE_VERSION-src.tgz.sha512 +tar czf ${RELEASE_DIR}/flink-${RELEASE_VERSION}-src.tgz flink-$RELEASE_VERSION +gpg --armor --detach-sig ${RELEASE_DIR}/flink-$RELEASE_VERSION-src.tgz +$SHASUM ${RELEASE_DIR}/flink-$RELEASE_VERSION-src.tgz > ${RELEASE_DIR}/flink-$RELEASE_VERSION-src.tgz.sha512 -mv flink-$RELEASE_VERSION-src.* ../ -cd .. -rm -rf flink-tmp-clone +cd ${CURR_DIR} +rm -rf ${CLONE_DIR} From 877cd7ef6e8a876a2a3579d0761bc2d160a4daf4 Mon Sep 17 00:00:00 2001 From: Oleksandr Nitavskyi Date: Mon, 2 Jul 2018 09:42:17 +0200 Subject: [PATCH 23/65] [FLINK-9762][yarn] Use local default tmp directories on Yarn and Mesos This closes #6284. --- .../generated/core_configuration.html | 7 ++++- .../flink/configuration/Configuration.java | 26 +++++++++++++++++ .../flink/configuration/CoreOptions.java | 9 +++++- .../DelegatingConfiguration.java | 5 ++++ .../configuration/ConfigurationTest.java | 29 +++++++++++++++++++ .../entrypoint/MesosEntrypointUtils.java | 12 ++------ .../clusterframework/BootstrapTools.java | 24 ++++++++++++++- .../clusterframework/BootstrapToolsTest.java | 19 ++++++++++++ .../yarn/YarnApplicationMasterRunner.java | 13 ++------- .../flink/yarn/YarnResourceManager.java | 12 ++++++-- .../flink/yarn/YarnTaskExecutorRunner.java | 12 ++------ .../yarn/YarnTaskManagerRunnerFactory.java | 12 ++------ .../yarn/entrypoint/YarnEntrypointUtils.java | 13 ++------- 13 files changed, 135 insertions(+), 58 deletions(-) diff --git a/docs/_includes/generated/core_configuration.html b/docs/_includes/generated/core_configuration.html index 91fa1a56e0695..8281adcd3671e 100644 --- a/docs/_includes/generated/core_configuration.html +++ b/docs/_includes/generated/core_configuration.html @@ -22,9 +22,14 @@ "child-first" Defines the class resolution strategy when loading classes from user code, meaning whether to first check the user code jar ("child-first") or the application classpath ("parent-first"). The default settings indicate to load classes first from the user code jar, which means that user code jars can include and load different dependencies than Flink uses (transitively). + +
internal.io.tmp.dirs.use-local-default
+ true + key, which says if default value is used for `io.tmp.dirs` config variable. +
io.tmp.dirs
- System.getProperty("java.io.tmpdir") + 'LOCAL_DIRS' on Yarn. '_FLINK_TMP_DIR' on Mesos. System.getProperty("java.io.tmpdir") in standalone. 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 7d99fbb7696da..00c4c38e82009 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 @@ -729,6 +729,32 @@ public Map toMap() { } } + /** + * Removes given config option from the configuration. + * + * @param configOption config option to remove + * @param Type of the config option + * @return true is config has been removed, false otherwise + */ + public boolean removeConfig(ConfigOption configOption){ + synchronized (this.confData){ + // try the current key + Object oldValue = this.confData.remove(configOption.key()); + if (oldValue == null){ + for (String deprecatedKey : configOption.deprecatedKeys()){ + oldValue = this.confData.remove(deprecatedKey); + if (oldValue != null){ + LOG.warn("Config uses deprecated configuration key '{}' instead of proper key '{}'", + deprecatedKey, configOption.key()); + return true; + } + } + return false; + } + return true; + } + } + // -------------------------------------------------------------------------------------------- diff --git a/flink-core/src/main/java/org/apache/flink/configuration/CoreOptions.java b/flink-core/src/main/java/org/apache/flink/configuration/CoreOptions.java index 656943f59bbbe..ea27e28eda4df 100644 --- a/flink-core/src/main/java/org/apache/flink/configuration/CoreOptions.java +++ b/flink-core/src/main/java/org/apache/flink/configuration/CoreOptions.java @@ -181,12 +181,19 @@ public static String[] getParentFirstLoaderPatterns(Configuration config) { * The config parameter defining the directories for temporary files, separated by * ",", "|", or the system's {@link java.io.File#pathSeparator}. */ - @Documentation.OverrideDefault("System.getProperty(\"java.io.tmpdir\")") + @Documentation.OverrideDefault("'LOCAL_DIRS' on Yarn. '_FLINK_TMP_DIR' on Mesos. System.getProperty(\"java.io.tmpdir\") in standalone.") public static final ConfigOption TMP_DIRS = key("io.tmp.dirs") .defaultValue(System.getProperty("java.io.tmpdir")) .withDeprecatedKeys("taskmanager.tmp.dirs"); + /** + * String key, which says if default value is used for `io.tmp.dirs` config variable. + */ + public static final ConfigOption USE_LOCAL_DEFAULT_TMP_DIRS = key("internal.io.tmp.dirs.use-local-default") + .defaultValue(true) + .withDescription("key, which says if default value is used for `io.tmp.dirs` config variable."); + // ------------------------------------------------------------------------ // program // ------------------------------------------------------------------------ diff --git a/flink-core/src/main/java/org/apache/flink/configuration/DelegatingConfiguration.java b/flink-core/src/main/java/org/apache/flink/configuration/DelegatingConfiguration.java index 7b75c7a6999e3..1a637f65626ce 100644 --- a/flink-core/src/main/java/org/apache/flink/configuration/DelegatingConfiguration.java +++ b/flink-core/src/main/java/org/apache/flink/configuration/DelegatingConfiguration.java @@ -310,6 +310,11 @@ public Map toMap() { return prefixed; } + @Override + public boolean removeConfig(ConfigOption configOption){ + return backingConfig.removeConfig(configOption); + } + @Override public boolean containsKey(String key) { return backingConfig.containsKey(prefix + key); diff --git a/flink-core/src/test/java/org/apache/flink/configuration/ConfigurationTest.java b/flink-core/src/test/java/org/apache/flink/configuration/ConfigurationTest.java index 232c82931a8fe..3b98a4490891f 100644 --- a/flink-core/src/test/java/org/apache/flink/configuration/ConfigurationTest.java +++ b/flink-core/src/test/java/org/apache/flink/configuration/ConfigurationTest.java @@ -25,6 +25,7 @@ import static org.junit.Assert.assertArrayEquals; import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertNull; import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; @@ -303,4 +304,32 @@ public void testDeprecatedKeys() { assertEquals(13, cfg.getInteger(matchesThird)); assertEquals(-1, cfg.getInteger(notContained)); } + + @Test + public void testRemove(){ + Configuration cfg = new Configuration(); + cfg.setInteger("a", 1); + cfg.setInteger("b", 2); + + ConfigOption validOption = ConfigOptions + .key("a") + .defaultValue(-1); + + ConfigOption deprecatedOption = ConfigOptions + .key("c") + .defaultValue(-1) + .withDeprecatedKeys("d", "b"); + + ConfigOption unexistedOption = ConfigOptions + .key("e") + .defaultValue(-1) + .withDeprecatedKeys("f", "g", "j"); + + assertEquals("Wrong expectation about size", cfg.keySet().size(), 2); + assertTrue("Expected 'validOption' is removed", cfg.removeConfig(validOption)); + assertEquals("Wrong expectation about size", cfg.keySet().size(), 1); + assertTrue("Expected 'existedOption' is removed", cfg.removeConfig(deprecatedOption)); + assertEquals("Wrong expectation about size", cfg.keySet().size(), 0); + assertFalse("Expected 'unexistedOption' is not removed", cfg.removeConfig(unexistedOption)); + } } 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 498f4354bd4c8..2059c8ecc4cb5 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,13 +19,13 @@ package org.apache.flink.mesos.entrypoint; import org.apache.flink.configuration.Configuration; -import org.apache.flink.configuration.CoreOptions; 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.MesosConfigKeys; 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; @@ -173,15 +173,7 @@ public static Configuration loadConfiguration(Configuration dynamicProperties, L final Map envs = System.getenv(); final String tmpDirs = envs.get(MesosConfigKeys.ENV_FLINK_TMP_DIR); - // configure local directory - if (configuration.contains(CoreOptions.TMP_DIRS)) { - log.info("Overriding Mesos' temporary file directories with those " + - "specified in the Flink config: " + configuration.getValue(CoreOptions.TMP_DIRS)); - } - else if (tmpDirs != null) { - log.info("Setting directories for temporary files to: {}", tmpDirs); - configuration.setString(CoreOptions.TMP_DIRS, tmpDirs); - } + BootstrapTools.updateTmpDirectoriesInConfiguration(configuration, tmpDirs); return configuration; } 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 7a8403a151cfd..3b606f7fde2d7 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 @@ -250,7 +250,11 @@ public static Configuration generateTaskManagerConfiguration( cfg.setInteger(TaskManagerOptions.NUM_TASK_SLOTS, numSlots); } - return cfg; + if (baseConfig.getBoolean(CoreOptions.USE_LOCAL_DEFAULT_TMP_DIRS)){ + cfg.removeConfig(CoreOptions.TMP_DIRS); + } + + return cfg; } /** @@ -467,4 +471,22 @@ public static String getStartCommand(String template, } return template; } + + /** + * Set temporary configuration directories if necessary + * + * @param configuration flink config to patch + * @param defaultDirs in case no tmp directories is set, next directories will be applied + */ + public static void updateTmpDirectoriesInConfiguration(Configuration configuration, String defaultDirs){ + if (configuration.contains(CoreOptions.TMP_DIRS)) { + LOG.info("Overriding Fink's temporary file directories with those " + + "specified in the Flink config: {}", configuration.getValue(CoreOptions.TMP_DIRS)); + configuration.setBoolean(CoreOptions.USE_LOCAL_DEFAULT_TMP_DIRS, false); + } + else { + LOG.info("Setting directories for temporary files to: {}", defaultDirs); + configuration.setString(CoreOptions.TMP_DIRS, defaultDirs); + } + } } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/clusterframework/BootstrapToolsTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/clusterframework/BootstrapToolsTest.java index cf38fea27b785..7e31c8e0f6bfc 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/clusterframework/BootstrapToolsTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/clusterframework/BootstrapToolsTest.java @@ -276,4 +276,23 @@ public void testGetTaskManagerShellCommand() { true, true, true, this.getClass())); } + + @Test + public void testUpdateTmpDirectoriesInConfiguration(){ + Configuration config = new Configuration(); + + // test that default value is taken + BootstrapTools.updateTmpDirectoriesInConfiguration(config, "default/directory/path"); + assertEquals(config.getString(CoreOptions.TMP_DIRS), "default/directory/path"); + + // test that we ignore default value is value is set before + BootstrapTools.updateTmpDirectoriesInConfiguration(config, "not/default/directory/path"); + assertEquals(config.getString(CoreOptions.TMP_DIRS), "default/directory/path"); + + //test that empty value is not a magic string + config.setString(CoreOptions.TMP_DIRS, ""); + BootstrapTools.updateTmpDirectoriesInConfiguration(config, "some/new/path"); + assertEquals(config.getString(CoreOptions.TMP_DIRS), ""); + } + } 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 eb977bfaa9352..497ac87c4a3d3 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 @@ -21,7 +21,6 @@ import org.apache.flink.api.common.time.Time; import org.apache.flink.configuration.ConfigConstants; import org.apache.flink.configuration.Configuration; -import org.apache.flink.configuration.CoreOptions; import org.apache.flink.configuration.GlobalConfiguration; import org.apache.flink.configuration.HighAvailabilityOptions; import org.apache.flink.configuration.JobManagerOptions; @@ -523,16 +522,8 @@ private static Configuration createConfiguration(String baseDirectory, Map Date: Wed, 18 Jul 2018 15:45:15 +0200 Subject: [PATCH 24/65] [FLINK-9762] Consolidate configuration cloning in BootstrapTools --- .../generated/core_configuration.html | 5 --- .../flink/configuration/CoreOptions.java | 7 ---- .../UnmodifiableConfiguration.java | 6 ++++ .../clusterframework/BootstrapTools.java | 34 +++++++++++++++---- .../flink/yarn/YarnResourceManager.java | 9 ++--- 5 files changed, 37 insertions(+), 24 deletions(-) diff --git a/docs/_includes/generated/core_configuration.html b/docs/_includes/generated/core_configuration.html index 8281adcd3671e..98cca9125a0fb 100644 --- a/docs/_includes/generated/core_configuration.html +++ b/docs/_includes/generated/core_configuration.html @@ -22,11 +22,6 @@ "child-first" Defines the class resolution strategy when loading classes from user code, meaning whether to first check the user code jar ("child-first") or the application classpath ("parent-first"). The default settings indicate to load classes first from the user code jar, which means that user code jars can include and load different dependencies than Flink uses (transitively). - -
internal.io.tmp.dirs.use-local-default
- true - key, which says if default value is used for `io.tmp.dirs` config variable. -
io.tmp.dirs
'LOCAL_DIRS' on Yarn. '_FLINK_TMP_DIR' on Mesos. System.getProperty("java.io.tmpdir") in standalone. diff --git a/flink-core/src/main/java/org/apache/flink/configuration/CoreOptions.java b/flink-core/src/main/java/org/apache/flink/configuration/CoreOptions.java index ea27e28eda4df..16c9b545eae12 100644 --- a/flink-core/src/main/java/org/apache/flink/configuration/CoreOptions.java +++ b/flink-core/src/main/java/org/apache/flink/configuration/CoreOptions.java @@ -187,13 +187,6 @@ public static String[] getParentFirstLoaderPatterns(Configuration config) { .defaultValue(System.getProperty("java.io.tmpdir")) .withDeprecatedKeys("taskmanager.tmp.dirs"); - /** - * String key, which says if default value is used for `io.tmp.dirs` config variable. - */ - public static final ConfigOption USE_LOCAL_DEFAULT_TMP_DIRS = key("internal.io.tmp.dirs.use-local-default") - .defaultValue(true) - .withDescription("key, which says if default value is used for `io.tmp.dirs` config variable."); - // ------------------------------------------------------------------------ // program // ------------------------------------------------------------------------ diff --git a/flink-core/src/main/java/org/apache/flink/configuration/UnmodifiableConfiguration.java b/flink-core/src/main/java/org/apache/flink/configuration/UnmodifiableConfiguration.java index f92de1c05ebb4..0a1bcc4285679 100644 --- a/flink-core/src/main/java/org/apache/flink/configuration/UnmodifiableConfiguration.java +++ b/flink-core/src/main/java/org/apache/flink/configuration/UnmodifiableConfiguration.java @@ -65,6 +65,12 @@ final void setValueInternal(String key, T value){ error(); } + @Override + public boolean removeConfig(ConfigOption configOption) { + error(); + return false; + } + private void error(){ throw new UnsupportedOperationException("The configuration is unmodifiable; its contents cannot be changed."); } 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 3b606f7fde2d7..ebe714087a393 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 @@ -20,6 +20,7 @@ import org.apache.flink.api.common.time.Time; import org.apache.flink.configuration.ConfigConstants; +import org.apache.flink.configuration.ConfigOption; import org.apache.flink.configuration.Configuration; import org.apache.flink.configuration.CoreOptions; import org.apache.flink.configuration.JobManagerOptions; @@ -60,11 +61,19 @@ import scala.Tuple2; import scala.concurrent.duration.FiniteDuration; +import static org.apache.flink.configuration.ConfigOptions.key; + /** * Tools for starting JobManager and TaskManager processes, including the * Actor Systems used to run the JobManager and TaskManager actors. */ public class BootstrapTools { + /** + * Internal option which says if default value is used for {@link CoreOptions#TMP_DIRS}. + */ + private static final ConfigOption USE_LOCAL_DEFAULT_TMP_DIRS = key("internal.io.tmpdirs.use-local-default") + .defaultValue(false); + private static final Logger LOG = LoggerFactory.getLogger(BootstrapTools.class); /** @@ -235,7 +244,7 @@ public static Configuration generateTaskManagerConfiguration( int numSlots, FiniteDuration registrationTimeout) { - Configuration cfg = baseConfig.clone(); + Configuration cfg = cloneConfiguration(baseConfig); if (jobManagerHostname != null && !jobManagerHostname.isEmpty()) { cfg.setString(JobManagerOptions.ADDRESS, jobManagerHostname); @@ -250,10 +259,6 @@ public static Configuration generateTaskManagerConfiguration( cfg.setInteger(TaskManagerOptions.NUM_TASK_SLOTS, numSlots); } - if (baseConfig.getBoolean(CoreOptions.USE_LOCAL_DEFAULT_TMP_DIRS)){ - cfg.removeConfig(CoreOptions.TMP_DIRS); - } - return cfg; } @@ -482,11 +487,28 @@ public static void updateTmpDirectoriesInConfiguration(Configuration configurati if (configuration.contains(CoreOptions.TMP_DIRS)) { LOG.info("Overriding Fink's temporary file directories with those " + "specified in the Flink config: {}", configuration.getValue(CoreOptions.TMP_DIRS)); - configuration.setBoolean(CoreOptions.USE_LOCAL_DEFAULT_TMP_DIRS, false); } else { LOG.info("Setting directories for temporary files to: {}", defaultDirs); configuration.setString(CoreOptions.TMP_DIRS, defaultDirs); + configuration.setBoolean(USE_LOCAL_DEFAULT_TMP_DIRS, true); + } + } + + /** + * Clones the given configuration and resets instance specific config options. + * + * @param configuration to clone + * @return Cloned configuration with reset instance specific config options + */ + public static Configuration cloneConfiguration(Configuration configuration) { + final Configuration clonedConfiguration = new Configuration(configuration); + + if (clonedConfiguration.getBoolean(USE_LOCAL_DEFAULT_TMP_DIRS)){ + clonedConfiguration.removeConfig(CoreOptions.TMP_DIRS); + clonedConfiguration.removeConfig(USE_LOCAL_DEFAULT_TMP_DIRS); } + + return clonedConfiguration; } } 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 b08993aedd2c9..49385e564c915 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 @@ -21,9 +21,9 @@ import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.configuration.Configuration; import org.apache.flink.configuration.ConfigurationUtils; -import org.apache.flink.configuration.CoreOptions; import org.apache.flink.configuration.TaskManagerOptions; import org.apache.flink.runtime.clusterframework.ApplicationStatus; +import org.apache.flink.runtime.clusterframework.BootstrapTools; import org.apache.flink.runtime.clusterframework.ContaineredTaskManagerParameters; import org.apache.flink.runtime.clusterframework.types.ResourceID; import org.apache.flink.runtime.clusterframework.types.ResourceProfile; @@ -473,15 +473,12 @@ private ContainerLaunchContext createTaskExecutorLaunchContext(Resource resource taskManagerParameters.taskManagerHeapSizeMB(), taskManagerParameters.taskManagerDirectMemoryLimitMB()); - Configuration taskManagerConfig = flinkConfig.clone(); - if (flinkConfig.getBoolean(CoreOptions.USE_LOCAL_DEFAULT_TMP_DIRS)){ - taskManagerConfig.removeConfig(CoreOptions.TMP_DIRS); - } + Configuration taskManagerConfig = BootstrapTools.cloneConfiguration(flinkConfig); log.debug("TaskManager configuration: {}", taskManagerConfig); ContainerLaunchContext taskExecutorLaunchContext = Utils.createTaskExecutorContext( - taskManagerConfig, + flinkConfig, yarnConfig, env, taskManagerParameters, From 1d03720cfd1d4e28ce5ec1afebe38ff81ca82f1d Mon Sep 17 00:00:00 2001 From: Till Rohrmann Date: Wed, 18 Jul 2018 15:49:55 +0200 Subject: [PATCH 25/65] [hotfix] Fix checkstyle violations in BootstrapTool --- .../clusterframework/BootstrapTools.java | 23 ++++++++----------- 1 file changed, 10 insertions(+), 13 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 ebe714087a393..b43946ca5bb23 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 @@ -270,8 +270,7 @@ public static Configuration generateTaskManagerConfiguration( */ public static void writeConfiguration(Configuration cfg, File file) throws IOException { try (FileWriter fwrt = new FileWriter(file); - PrintWriter out = new PrintWriter(fwrt)) - { + PrintWriter out = new PrintWriter(fwrt)) { for (String key : cfg.keySet()) { String value = cfg.getString(key, null); out.print(key); @@ -331,7 +330,7 @@ public static void substituteDeprecatedConfigPrefix( /** * Get an instance of the dynamic properties option. * - * Dynamic properties allow the user to specify additional configuration values with -D, such as + *

Dynamic properties allow the user to specify additional configuration values with -D, such as * -Dfs.overwrite-files=true -Dtaskmanager.network.memory.min=536346624 */ public static Option newDynamicPropertiesOption() { @@ -345,13 +344,13 @@ public static Configuration parseDynamicProperties(CommandLine cmd) { final Configuration config = new Configuration(); String[] values = cmd.getOptionValues(DYNAMIC_PROPERTIES_OPT); - if(values != null) { - for(String value : values) { + if (values != null) { + for (String value : values) { String[] pair = value.split("=", 2); - if(pair.length == 1) { + if (pair.length == 1) { config.setString(pair[0], Boolean.TRUE.toString()); } - else if(pair.length == 2) { + else if (pair.length == 2) { config.setString(pair[0], pair[1]); } } @@ -401,7 +400,7 @@ public static String getTaskManagerShellCommand( } //applicable only for YarnMiniCluster secure test run //krb5.conf file will be available as local resource in JM/TM container - if(hasKrb5) { + if (hasKrb5) { javaOpts += " -Djava.security.krb5.conf=krb5.conf"; } startCommandValues.put("jvmopts", javaOpts); @@ -439,12 +438,11 @@ public static String getTaskManagerShellCommand( // ------------------------------------------------------------------------ - /** Private constructor to prevent instantiation */ + /** Private constructor to prevent instantiation. */ private BootstrapTools() {} /** - * Replaces placeholders in the template start command with values from - * startCommandValues. + * Replaces placeholders in the template start command with values from startCommandValues. * *

If the default template {@link ConfigConstants#DEFAULT_YARN_CONTAINER_START_COMMAND_TEMPLATE} * is used, the following keys must be present in the map or the resulting @@ -458,7 +456,6 @@ private BootstrapTools() {} *

  • args = arguments for the main class
  • *
  • redirects = output redirects
  • * - *

    * * @param template * a template start command with placeholders @@ -478,7 +475,7 @@ public static String getStartCommand(String template, } /** - * Set temporary configuration directories if necessary + * Set temporary configuration directories if necessary. * * @param configuration flink config to patch * @param defaultDirs in case no tmp directories is set, next directories will be applied From 49c6f385df97958cda77933edfce93cd71d46600 Mon Sep 17 00:00:00 2001 From: Nico Kruber Date: Wed, 18 Jul 2018 00:51:38 +0200 Subject: [PATCH 26/65] [FLINK-9860][REST] fix buffer leak in FileUploadHandler Change the parent's instance of SimpleChannelInboundHandler to auto-release objects and make sure we retain them only when needed. -> This should be the safer approach to not leaking. Enable Netty resource leak detection for FileUploadHandlerTest and have the test fail on detection. This closes #6363. --- .../flink/runtime/rest/FileUploadHandler.java | 17 ++- .../netty/NettyLeakDetectionResource.java | 103 ++++++++++++++++++ .../runtime/rest/FileUploadHandlerTest.java | 4 + .../src/test/resources/log4j-test.properties | 2 + 4 files changed, 120 insertions(+), 6 deletions(-) create mode 100644 flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/NettyLeakDetectionResource.java diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/FileUploadHandler.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/FileUploadHandler.java index a957b636da734..d62875076977a 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/FileUploadHandler.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/FileUploadHandler.java @@ -41,6 +41,7 @@ import org.apache.flink.shaded.netty4.io.netty.handler.codec.http.multipart.HttpPostRequestDecoder; import org.apache.flink.shaded.netty4.io.netty.handler.codec.http.multipart.InterfaceHttpData; import org.apache.flink.shaded.netty4.io.netty.util.AttributeKey; +import org.apache.flink.shaded.netty4.io.netty.util.ReferenceCountUtil; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -80,7 +81,7 @@ public class FileUploadHandler extends SimpleChannelInboundHandler { private Path currentUploadDir; public FileUploadHandler(final Path uploadDir) { - super(false); + super(true); DiskFileUpload.baseDirectory = uploadDir.normalize().toAbsolutePath().toString(); this.uploadDir = requireNonNull(uploadDir); } @@ -93,14 +94,17 @@ protected void channelRead0(final ChannelHandlerContext ctx, final HttpObject ms LOG.trace("Received request. URL:{} Method:{}", httpRequest.getUri(), httpRequest.getMethod()); if (httpRequest.getMethod().equals(HttpMethod.POST)) { if (HttpPostRequestDecoder.isMultipart(httpRequest)) { + checkState(currentHttpPostRequestDecoder == null); + checkState(currentHttpRequest == null); + checkState(currentUploadDir == null); currentHttpPostRequestDecoder = new HttpPostRequestDecoder(DATA_FACTORY, httpRequest); - currentHttpRequest = httpRequest; + currentHttpRequest = ReferenceCountUtil.retain(httpRequest); currentUploadDir = Files.createDirectory(uploadDir.resolve(UUID.randomUUID().toString())); } else { - ctx.fireChannelRead(msg); + ctx.fireChannelRead(ReferenceCountUtil.retain(msg)); } } else { - ctx.fireChannelRead(msg); + ctx.fireChannelRead(ReferenceCountUtil.retain(msg)); } } else if (msg instanceof HttpContent && currentHttpPostRequestDecoder != null) { // make sure that we still have a upload dir in case that it got deleted in the meanwhile @@ -135,12 +139,12 @@ protected void channelRead0(final ChannelHandlerContext ctx, final HttpObject ms if (currentJsonPayload != null) { ctx.fireChannelRead(httpContent.replace(Unpooled.wrappedBuffer(currentJsonPayload))); } else { - ctx.fireChannelRead(httpContent); + ctx.fireChannelRead(ReferenceCountUtil.retain(httpContent)); } reset(); } } else { - ctx.fireChannelRead(msg); + ctx.fireChannelRead(ReferenceCountUtil.retain(msg)); } } catch (Exception e) { handleError(ctx, "File upload failed.", HttpResponseStatus.INTERNAL_SERVER_ERROR, e); @@ -159,6 +163,7 @@ private void handleError(ChannelHandlerContext ctx, String errorMessage, HttpRes responseStatus, Collections.emptyMap() ); + ReferenceCountUtil.release(tmpRequest); } private void deleteUploadedFiles() { diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/NettyLeakDetectionResource.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/NettyLeakDetectionResource.java new file mode 100644 index 0000000000000..027f454d2fe54 --- /dev/null +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/NettyLeakDetectionResource.java @@ -0,0 +1,103 @@ +/* + * 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.io.network.netty; + +import org.apache.flink.shaded.netty4.io.netty.util.ResourceLeakDetector; +import org.apache.flink.shaded.netty4.io.netty.util.ResourceLeakDetectorFactory; + +import org.junit.rules.ExternalResource; + +import javax.annotation.concurrent.GuardedBy; + +import static org.junit.Assert.fail; + +/** + * JUnit resource to fail with an assertion when Netty detects a resource leak (only with + * ERROR logging enabled for + * org.apache.flink.shaded.netty4.io.netty.util.ResourceLeakDetector). + * + *

    This should be used in a class rule: + *

    {@code
    + * @literal @ClassRule
    + *  public static final NettyLeakDetectionResource LEAK_DETECTION = new NettyLeakDetectionResource();
    + * }
    + */ +public class NettyLeakDetectionResource extends ExternalResource { + @GuardedBy("refCountLock") + private static ResourceLeakDetectorFactory previousLeakDetector; + + @GuardedBy("refCountLock") + private static ResourceLeakDetector.Level previousLeakDetectorLevel; + + private static final Object refCountLock = new Object(); + private static int refCount = 0; + + @Override + protected void before() { + synchronized (refCountLock) { + if (refCount == 0) { + previousLeakDetector = ResourceLeakDetectorFactory.instance(); + previousLeakDetectorLevel = ResourceLeakDetector.getLevel(); + + ++refCount; + ResourceLeakDetector.setLevel(ResourceLeakDetector.Level.PARANOID); + ResourceLeakDetectorFactory + .setResourceLeakDetectorFactory(new FailingResourceLeakDetectorFactory()); + } + } + } + + @Override + protected synchronized void after() { + synchronized (refCountLock) { + --refCount; + if (refCount == 0) { + ResourceLeakDetectorFactory.setResourceLeakDetectorFactory(previousLeakDetector); + ResourceLeakDetector.setLevel(previousLeakDetectorLevel); + } + } + } + + private static class FailingResourceLeakDetectorFactory extends ResourceLeakDetectorFactory { + public ResourceLeakDetector newResourceLeakDetector( + Class resource, int samplingInterval, long maxActive) { + return new FailingResourceLeakDetector(resource, samplingInterval, maxActive); + } + } + + private static class FailingResourceLeakDetector extends ResourceLeakDetector { + FailingResourceLeakDetector(Class resourceType, int samplingInterval, long maxActive) { + super(resourceType, samplingInterval, maxActive); + } + + @Override + protected void reportTracedLeak(String resourceType, String records) { + super.reportTracedLeak(resourceType, records); + fail(String.format("LEAK: %s.release() was not called before it's garbage-collected.%s", + resourceType, records)); + } + + @Override + protected void reportUntracedLeak(String resourceType) { + super.reportUntracedLeak(resourceType); + fail(String.format("LEAK: %s.release() was not called before it's garbage-collected.", + resourceType)); + } + } +} diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/rest/FileUploadHandlerTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/rest/FileUploadHandlerTest.java index 6818406b7088c..858c6620b3c54 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/rest/FileUploadHandlerTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/rest/FileUploadHandlerTest.java @@ -18,6 +18,7 @@ package org.apache.flink.runtime.rest; +import org.apache.flink.runtime.io.network.netty.NettyLeakDetectionResource; import org.apache.flink.runtime.rest.util.RestMapperUtils; import org.apache.flink.util.TestLogger; @@ -50,6 +51,9 @@ public class FileUploadHandlerTest extends TestLogger { private static final ObjectMapper OBJECT_MAPPER = RestMapperUtils.getStrictObjectMapper(); + @ClassRule + public static final NettyLeakDetectionResource LEAK_DETECTION = new NettyLeakDetectionResource(); + @After public void reset() { MULTIPART_UPLOAD_RESOURCE.resetState(); diff --git a/flink-runtime/src/test/resources/log4j-test.properties b/flink-runtime/src/test/resources/log4j-test.properties index d7ef9d46e5c1f..2778b7fd9c53a 100644 --- a/flink-runtime/src/test/resources/log4j-test.properties +++ b/flink-runtime/src/test/resources/log4j-test.properties @@ -28,3 +28,5 @@ log4j.appender.testlogger.layout.ConversionPattern=%-4r [%t] %-5p %c %x - %m%n # Suppress the irrelevant (wrong) warnings from the Netty channel handler log4j.logger.org.apache.flink.shaded.akka.org.jboss.netty.channel.DefaultChannelPipeline=ERROR, testlogger +# Resource leak detector only works with logging enabled at error level +log4j.logger.org.apache.flink.shaded.netty4.io.netty.util.ResourceLeakDetector=ERROR, testlogger From 41d4d8d00ee0a9a73a7674a3b3143a5452cd436d Mon Sep 17 00:00:00 2001 From: Dawid Wysakowicz Date: Tue, 17 Jul 2018 11:25:42 +0200 Subject: [PATCH 27/65] [FLINK-9871] Use Description class for ConfigOptions with rich formatting This closes #6371 --- .../generated/akka_configuration.html | 6 ++-- .../generated/security_configuration.html | 2 +- .../flink/configuration/AkkaOptions.java | 34 ++++++++++++------- .../flink/configuration/SecurityOptions.java | 10 ++++-- 4 files changed, 34 insertions(+), 18 deletions(-) diff --git a/docs/_includes/generated/akka_configuration.html b/docs/_includes/generated/akka_configuration.html index a2b49c74a44a4..352c656488eb1 100644 --- a/docs/_includes/generated/akka_configuration.html +++ b/docs/_includes/generated/akka_configuration.html @@ -80,17 +80,17 @@
    akka.watch.heartbeat.interval
    "10 s" - Heartbeat interval for Akka’s DeathWatch mechanism to detect dead TaskManagers. If TaskManagers are wrongly marked dead because of lost or delayed heartbeat messages, then you should decrease this value or increase akka.watch.heartbeat.pause. A thorough description of Akka’s DeathWatch can be found <a href="http://doc.akka.io/docs/akka/snapshot/scala/remoting.html#failure-detector">here</a>. + Heartbeat interval for Akka’s DeathWatch mechanism to detect dead TaskManagers. If TaskManagers are wrongly marked dead because of lost or delayed heartbeat messages, then you should decrease this value or increase akka.watch.heartbeat.pause. A thorough description of Akka’s DeathWatch can be found here
    akka.watch.heartbeat.pause
    "60 s" - Acceptable heartbeat pause for Akka’s DeathWatch mechanism. A low value does not allow an irregular heartbeat. If TaskManagers are wrongly marked dead because of lost or delayed heartbeat messages, then you should increase this value or decrease akka.watch.heartbeat.interval. Higher value increases the time to detect a dead TaskManager. A thorough description of Akka’s DeathWatch can be found <a href="http://doc.akka.io/docs/akka/snapshot/scala/remoting.html#failure-detector">here</a>. + Acceptable heartbeat pause for Akka’s DeathWatch mechanism. A low value does not allow an irregular heartbeat. If TaskManagers are wrongly marked dead because of lost or delayed heartbeat messages, then you should increase this value or decrease akka.watch.heartbeat.interval. Higher value increases the time to detect a dead TaskManager. A thorough description of Akka’s DeathWatch can be found here
    akka.watch.threshold
    12 - Threshold for the DeathWatch failure detector. A low value is prone to false positives whereas a high value increases the time to detect a dead TaskManager. A thorough description of Akka’s DeathWatch can be found <a href="http://doc.akka.io/docs/akka/snapshot/scala/remoting.html#failure-detector">here</a>. + Threshold for the DeathWatch failure detector. A low value is prone to false positives whereas a high value increases the time to detect a dead TaskManager. A thorough description of Akka’s DeathWatch can be found here diff --git a/docs/_includes/generated/security_configuration.html b/docs/_includes/generated/security_configuration.html index 47846aed1819f..19d0287df9996 100644 --- a/docs/_includes/generated/security_configuration.html +++ b/docs/_includes/generated/security_configuration.html @@ -10,7 +10,7 @@
    security.ssl.algorithms
    "TLS_RSA_WITH_AES_128_CBC_SHA" - The comma separated list of standard SSL algorithms to be supported. Read more <a href="http://docs.oracle.com/javase/8/docs/technotes/guides/security/StandardNames.html#ciphersuites">here</a>. + The comma separated list of standard SSL algorithms to be supported. Read more here
    security.ssl.internal.enabled
    diff --git a/flink-core/src/main/java/org/apache/flink/configuration/AkkaOptions.java b/flink-core/src/main/java/org/apache/flink/configuration/AkkaOptions.java index 360ba860f6b5e..43c7876fbd823 100644 --- a/flink-core/src/main/java/org/apache/flink/configuration/AkkaOptions.java +++ b/flink-core/src/main/java/org/apache/flink/configuration/AkkaOptions.java @@ -19,6 +19,9 @@ package org.apache.flink.configuration; import org.apache.flink.annotation.PublicEvolving; +import org.apache.flink.configuration.description.Description; + +import static org.apache.flink.configuration.description.LinkElement.link; /** * Akka configuration options. @@ -42,10 +45,12 @@ public class AkkaOptions { public static final ConfigOption WATCH_HEARTBEAT_INTERVAL = ConfigOptions .key("akka.watch.heartbeat.interval") .defaultValue(ASK_TIMEOUT.defaultValue()) - .withDescription("Heartbeat interval for Akka’s DeathWatch mechanism to detect dead TaskManagers. If" + - " TaskManagers are wrongly marked dead because of lost or delayed heartbeat messages, then you should" + - " decrease this value or increase akka.watch.heartbeat.pause. A thorough description of Akka’s DeathWatch" + - " can be found here."); + .withDescription(Description.builder() + .text("Heartbeat interval for Akka’s DeathWatch mechanism to detect dead TaskManagers. If" + + " TaskManagers are wrongly marked dead because of lost or delayed heartbeat messages, then you" + + " should decrease this value or increase akka.watch.heartbeat.pause. A thorough description of" + + " Akka’s DeathWatch can be found %s", + link("http://doc.akka.io/docs/akka/snapshot/scala/remoting.html#failure-detector", "here")).build()); /** * The maximum acceptable Akka death watch heartbeat pause. @@ -53,11 +58,14 @@ public class AkkaOptions { public static final ConfigOption WATCH_HEARTBEAT_PAUSE = ConfigOptions .key("akka.watch.heartbeat.pause") .defaultValue("60 s") - .withDescription("Acceptable heartbeat pause for Akka’s DeathWatch mechanism. A low value does not allow an" + - " irregular heartbeat. If TaskManagers are wrongly marked dead because of lost or delayed heartbeat messages," + - " then you should increase this value or decrease akka.watch.heartbeat.interval. Higher value increases the" + - " time to detect a dead TaskManager. A thorough description of Akka’s DeathWatch can be found" + - " here."); + .withDescription(Description.builder() + .text("Acceptable heartbeat pause for Akka’s DeathWatch mechanism. A low value does not allow an" + + " irregular heartbeat. If TaskManagers are wrongly marked dead because of lost or delayed" + + " heartbeat messages, then you should increase this value or decrease akka.watch.heartbeat.interval." + + " Higher value increases the time to detect a dead TaskManager. A thorough description of Akka’s" + + " DeathWatch can be found %s", + link("http://doc.akka.io/docs/akka/snapshot/scala/remoting.html#failure-detector", "here") + ).build()); /** * The Akka tcp connection timeout. */ @@ -112,9 +120,11 @@ public class AkkaOptions { public static final ConfigOption WATCH_THRESHOLD = ConfigOptions .key("akka.watch.threshold") .defaultValue(12) - .withDescription("Threshold for the DeathWatch failure detector. A low value is prone to false positives whereas" + - " a high value increases the time to detect a dead TaskManager. A thorough description of Akka’s DeathWatch" + - " can be found here."); + .withDescription(Description.builder() + .text("Threshold for the DeathWatch failure detector. A low value is prone to false positives whereas" + + " a high value increases the time to detect a dead TaskManager. A thorough description of Akka’s" + + " DeathWatch can be found %s", + link("http://doc.akka.io/docs/akka/snapshot/scala/remoting.html#failure-detector", "here")).build()); /** * Override SSL support for the Akka transport. diff --git a/flink-core/src/main/java/org/apache/flink/configuration/SecurityOptions.java b/flink-core/src/main/java/org/apache/flink/configuration/SecurityOptions.java index 03ee4f2db6033..750170c80bbb2 100644 --- a/flink-core/src/main/java/org/apache/flink/configuration/SecurityOptions.java +++ b/flink-core/src/main/java/org/apache/flink/configuration/SecurityOptions.java @@ -22,8 +22,10 @@ import org.apache.flink.annotation.docs.ConfigGroup; import org.apache.flink.annotation.docs.ConfigGroups; import org.apache.flink.annotation.docs.Documentation; +import org.apache.flink.configuration.description.Description; import static org.apache.flink.configuration.ConfigOptions.key; +import static org.apache.flink.configuration.description.LinkElement.link; /** * The set of configuration options relating to security. @@ -274,8 +276,12 @@ public class SecurityOptions { public static final ConfigOption SSL_ALGORITHMS = key("security.ssl.algorithms") .defaultValue("TLS_RSA_WITH_AES_128_CBC_SHA") - .withDescription("The comma separated list of standard SSL algorithms to be supported. Read more" + - " here."); + .withDescription(Description.builder() + .text("The comma separated list of standard SSL algorithms to be supported. Read more %s", + link( + "http://docs.oracle.com/javase/8/docs/technotes/guides/security/StandardNames.html#ciphersuites", + "here")) + .build()); /** * Flag to enable/disable hostname verification for the ssl connections. From 0fec75c03bba0fa85a14e3f73baeb01998c83be0 Mon Sep 17 00:00:00 2001 From: Nico Kruber Date: Thu, 5 Jul 2018 00:48:33 +0200 Subject: [PATCH 28/65] [FLINK-9755][network] forward exceptions in RemoteInputChannel#notifyBufferAvailable() to the responsible thread This mainly involves state checks but previously these have only been swallowed without re-registration or any other logging/handling. This may have lead to some thread stalling while waiting for the notification that never came. This closes #6272. --- .../io/network/buffer/BufferListener.java | 9 ++ .../io/network/buffer/LocalBufferPool.java | 36 ++------ .../consumer/RemoteInputChannel.java | 52 ++++++----- .../consumer/RemoteInputChannelTest.java | 90 +++++++++++++++++-- 4 files changed, 131 insertions(+), 56 deletions(-) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/buffer/BufferListener.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/buffer/BufferListener.java index 05b415625fc28..4cc32c0a66147 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/buffer/BufferListener.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/buffer/BufferListener.java @@ -27,6 +27,15 @@ public interface BufferListener { /** * Notification callback if a buffer is recycled and becomes available in buffer pool. * + *

    Note: responsibility on recycling the given buffer is transferred to this implementation, + * including any errors that lead to exceptions being thrown! + * + *

    BEWARE: since this may be called from outside the thread that relies on + * the listener's logic, any exception that occurs with this handler should be forwarded to the + * responsible thread for handling and otherwise ignored in the processing of this method. The + * buffer pool forwards any {@link Throwable} from here upwards to a potentially unrelated call + * stack! + * * @param buffer buffer that becomes available in buffer pool. * @return true if the listener wants to be notified next time. */ diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/buffer/LocalBufferPool.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/buffer/LocalBufferPool.java index e874723755d9e..1596fded6f3f3 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/buffer/LocalBufferPool.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/buffer/LocalBufferPool.java @@ -262,8 +262,7 @@ public void recycle(MemorySegment segment) { if (isDestroyed || numberOfRequestedMemorySegments > currentPoolSize) { returnMemorySegment(segment); return; - } - else { + } else { listener = registeredListeners.poll(); if (listener == null) { @@ -277,37 +276,18 @@ public void recycle(MemorySegment segment) { // We do not know which locks have been acquired before the recycle() or are needed in the // notification and which other threads also access them. // -> call notifyBufferAvailable() outside of the synchronized block to avoid a deadlock (FLINK-9676) - boolean success = false; - boolean needMoreBuffers = false; - try { - needMoreBuffers = listener.notifyBufferAvailable(new NetworkBuffer(segment, this)); - success = true; - } catch (Throwable ignored) { - // handled below, under the lock - } + // Note that in case of any exceptions notifyBufferAvailable() should recycle the buffer + // (either directly or later during error handling) and therefore eventually end up in this + // method again. + boolean needMoreBuffers = listener.notifyBufferAvailable(new NetworkBuffer(segment, this)); - if (!success || needMoreBuffers) { + if (needMoreBuffers) { synchronized (availableMemorySegments) { if (isDestroyed) { // cleanup tasks how they would have been done if we only had one synchronized block - if (needMoreBuffers) { - listener.notifyBufferDestroyed(); - } - if (!success) { - returnMemorySegment(segment); - } + listener.notifyBufferDestroyed(); } else { - if (needMoreBuffers) { - registeredListeners.add(listener); - } - if (!success) { - if (numberOfRequestedMemorySegments > currentPoolSize) { - returnMemorySegment(segment); - } else { - availableMemorySegments.add(segment); - availableMemorySegments.notify(); - } - } + registeredListeners.add(listener); } } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/RemoteInputChannel.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/RemoteInputChannel.java index 0f70d448020a3..b94f48a3a79f8 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/RemoteInputChannel.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/RemoteInputChannel.java @@ -360,32 +360,44 @@ public boolean notifyBufferAvailable(Buffer buffer) { return false; } - boolean needMoreBuffers = false; - synchronized (bufferQueue) { - checkState(isWaitingForFloatingBuffers, "This channel should be waiting for floating buffers."); + boolean recycleBuffer = true; + try { + boolean needMoreBuffers = false; + synchronized (bufferQueue) { + checkState(isWaitingForFloatingBuffers, + "This channel should be waiting for floating buffers."); + + // Important: double check the isReleased state inside synchronized block, so there is no + // race condition when notifyBufferAvailable and releaseAllResources running in parallel. + if (isReleased.get() || bufferQueue.getAvailableBufferSize() >= numRequiredBuffers) { + isWaitingForFloatingBuffers = false; + recycleBuffer = false; // just in case + buffer.recycleBuffer(); + return false; + } - // Important: double check the isReleased state inside synchronized block, so there is no - // race condition when notifyBufferAvailable and releaseAllResources running in parallel. - if (isReleased.get() || bufferQueue.getAvailableBufferSize() >= numRequiredBuffers) { - isWaitingForFloatingBuffers = false; - buffer.recycleBuffer(); - return false; - } + recycleBuffer = false; + bufferQueue.addFloatingBuffer(buffer); - bufferQueue.addFloatingBuffer(buffer); + if (bufferQueue.getAvailableBufferSize() == numRequiredBuffers) { + isWaitingForFloatingBuffers = false; + } else { + needMoreBuffers = true; + } - if (bufferQueue.getAvailableBufferSize() == numRequiredBuffers) { - isWaitingForFloatingBuffers = false; - } else { - needMoreBuffers = true; + if (unannouncedCredit.getAndAdd(1) == 0) { + notifyCreditAvailable(); + } } - } - if (unannouncedCredit.getAndAdd(1) == 0) { - notifyCreditAvailable(); + return needMoreBuffers; + } catch (Throwable t) { + if (recycleBuffer) { + buffer.recycleBuffer(); + } + setError(t); + return false; } - - return needMoreBuffers; } @Override 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 6c6fd9680fc16..63054923498ce 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 @@ -52,9 +52,13 @@ import scala.Tuple2; +import static org.hamcrest.MatcherAssert.assertThat; +import static org.hamcrest.Matchers.hasProperty; +import static org.hamcrest.Matchers.isA; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.assertNull; import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; import static org.mockito.Matchers.any; @@ -452,7 +456,7 @@ public void testAvailableBuffersLessThanRequiredBuffers() throws Exception { } catch (Throwable t) { thrown = t; } finally { - cleanup(networkBufferPool, null, thrown, inputChannel); + cleanup(networkBufferPool, null, null, thrown, inputChannel); } } @@ -528,7 +532,7 @@ public void testAvailableBuffersEqualToRequiredBuffers() throws Exception { } catch (Throwable t) { thrown = t; } finally { - cleanup(networkBufferPool, null, thrown, inputChannel); + cleanup(networkBufferPool, null, null, thrown, inputChannel); } } @@ -618,7 +622,7 @@ public void testAvailableBuffersMoreThanRequiredBuffers() throws Exception { } catch (Throwable t) { thrown = t; } finally { - cleanup(networkBufferPool, null, thrown, inputChannel); + cleanup(networkBufferPool, null, null, thrown, inputChannel); } } @@ -687,7 +691,72 @@ public void testFairDistributionFloatingBuffers() throws Exception { } catch (Throwable t) { thrown = t; } finally { - cleanup(networkBufferPool, null, thrown, channel1, channel2, channel3); + cleanup(networkBufferPool, null, null, thrown, channel1, channel2, channel3); + } + } + + /** + * Tests that failures are propagated correctly if + * {@link RemoteInputChannel#notifyBufferAvailable(Buffer)} throws an exception. Also tests that + * a second listener will be notified in this case. + */ + @Test + public void testFailureInNotifyBufferAvailable() throws Exception { + // Setup + final int numExclusiveBuffers = 0; + final int numFloatingBuffers = 1; + final int numTotalBuffers = numExclusiveBuffers + numFloatingBuffers; + final NetworkBufferPool networkBufferPool = new NetworkBufferPool( + numTotalBuffers, 32); + + final SingleInputGate inputGate = createSingleInputGate(); + final RemoteInputChannel successfulRemoteIC = createRemoteInputChannel(inputGate); + inputGate.setInputChannel(successfulRemoteIC.partitionId.getPartitionId(), successfulRemoteIC); + + successfulRemoteIC.requestSubpartition(0); + + // late creation -> no exclusive buffers, also no requested subpartition in successfulRemoteIC + // (to trigger a failure in RemoteInputChannel#notifyBufferAvailable()) + final RemoteInputChannel failingRemoteIC = createRemoteInputChannel(inputGate); + inputGate.setInputChannel(failingRemoteIC.partitionId.getPartitionId(), failingRemoteIC); + + Buffer buffer = null; + Throwable thrown = null; + try { + final BufferPool bufferPool = + networkBufferPool.createBufferPool(numFloatingBuffers, numFloatingBuffers); + inputGate.setBufferPool(bufferPool); + + buffer = bufferPool.requestBufferBlocking(); + + // trigger subscription to buffer pool + failingRemoteIC.onSenderBacklog(1); + successfulRemoteIC.onSenderBacklog(numExclusiveBuffers + 1); + // recycling will call RemoteInputChannel#notifyBufferAvailable() which will fail and + // this exception will be swallowed and set as an error in failingRemoteIC + buffer.recycleBuffer(); + buffer = null; + try { + failingRemoteIC.checkError(); + fail("The input channel should have an error based on the failure in RemoteInputChannel#notifyBufferAvailable()"); + } catch (IOException e) { + assertThat(e, hasProperty("cause", isA(IllegalStateException.class))); + } + // currently, the buffer is still enqueued in the bufferQueue of failingRemoteIC + assertEquals(0, bufferPool.getNumberOfAvailableMemorySegments()); + buffer = successfulRemoteIC.requestBuffer(); + assertNull("buffer should still remain in failingRemoteIC", buffer); + + // releasing resources in failingRemoteIC should free the buffer again and immediately + // recycle it into successfulRemoteIC + failingRemoteIC.releaseAllResources(); + assertEquals(0, bufferPool.getNumberOfAvailableMemorySegments()); + buffer = successfulRemoteIC.requestBuffer(); + assertNotNull("no buffer given to successfulRemoteIC", buffer); + } catch (Throwable t) { + thrown = t; + } finally { + cleanup(networkBufferPool, null, buffer, thrown, failingRemoteIC, successfulRemoteIC); } } @@ -749,7 +818,7 @@ public Void call() throws Exception { } catch (Throwable t) { thrown = t; } finally { - cleanup(networkBufferPool, executor, thrown, inputChannel); + cleanup(networkBufferPool, executor, null, thrown, inputChannel); } } @@ -802,7 +871,7 @@ public Void call() throws Exception { } catch (Throwable t) { thrown = t; } finally { - cleanup(networkBufferPool, executor, thrown, inputChannel); + cleanup(networkBufferPool, executor, null, thrown, inputChannel); } } @@ -854,7 +923,7 @@ public Void call() throws Exception { } catch (Throwable t) { thrown = t; } finally { - cleanup(networkBufferPool, executor, thrown, inputChannel); + cleanup(networkBufferPool, executor, null, thrown, inputChannel); } } @@ -936,7 +1005,7 @@ public void testConcurrentRecycleAndRelease2() throws Exception { } catch (Throwable t) { thrown = t; } finally { - cleanup(networkBufferPool, executor, thrown, inputChannel); + cleanup(networkBufferPool, executor, null, thrown, inputChannel); } } @@ -1064,6 +1133,7 @@ private void submitTasksAndWaitForResults(ExecutorService executor, Callable[] t private void cleanup( NetworkBufferPool networkBufferPool, @Nullable ExecutorService executor, + @Nullable Buffer buffer, @Nullable Throwable throwable, InputChannel... inputChannels) throws Exception { for (InputChannel inputChannel : inputChannels) { @@ -1074,6 +1144,10 @@ private void cleanup( } } + if (buffer != null && !buffer.isRecycled()) { + buffer.recycleBuffer(); + } + try { networkBufferPool.destroyAllBufferPools(); } catch (Throwable tInner) { From 402745ebad3eaf01622ea85524f7ff029fa8df8b Mon Sep 17 00:00:00 2001 From: Nico Kruber Date: Fri, 25 May 2018 00:09:37 +0200 Subject: [PATCH 29/65] [FLINK-9435][java] optimise ComparableKeySelector and ArrayKeySelector for more efficient Tuple creation MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Benchmark results (2 runs) by running the benchmarks from https://github.com/dataArtisans/flink-benchmarks/pull/5: Benchmark Mode Cnt Score Error Units ================= old ================= KeyByBenchmarks.arrayKeyBy thrpt 30 1151.305 ± 21.096 ops/ms KeyByBenchmarks.arrayKeyBy thrpt 30 1117.486 ± 43.508 ops/ms KeyByBenchmarks.tupleKeyBy thrpt 30 1659.634 ± 28.627 ops/ms KeyByBenchmarks.tupleKeyBy thrpt 30 1554.265 ± 82.604 ops/ms ================= new ================= KeyByBenchmarks.arrayKeyBy thrpt 30 1150.552 ± 51.185 ops/ms KeyByBenchmarks.arrayKeyBy thrpt 30 1195.777 ± 10.621 ops/ms KeyByBenchmarks.tupleKeyBy thrpt 30 1743.633 ± 27.109 ops/ms KeyByBenchmarks.tupleKeyBy thrpt 30 1697.885 ± 22.101 ops/ms This closes #6115. --- .../apache/flink/api/java/tuple/Tuple.java | 32 +++++++++++++++++++ .../flink/api/java/tuple/TupleGenerator.java | 11 +++++++ .../streaming/util/keys/KeySelectorUtil.java | 10 +++--- 3 files changed, 47 insertions(+), 6 deletions(-) diff --git a/flink-core/src/main/java/org/apache/flink/api/java/tuple/Tuple.java b/flink-core/src/main/java/org/apache/flink/api/java/tuple/Tuple.java index c282c59eac7ba..7ce38f8c14a6d 100644 --- a/flink-core/src/main/java/org/apache/flink/api/java/tuple/Tuple.java +++ b/flink-core/src/main/java/org/apache/flink/api/java/tuple/Tuple.java @@ -113,6 +113,38 @@ public static Class getTupleClass(int arity) { // BEGIN_OF_TUPLE_DEPENDENT_CODE // GENERATED FROM org.apache.flink.api.java.tuple.TupleGenerator. + public static Tuple newInstance(int arity) { + switch (arity) { + case 0: return Tuple0.INSTANCE; + case 1: return new Tuple1(); + case 2: return new Tuple2(); + case 3: return new Tuple3(); + case 4: return new Tuple4(); + case 5: return new Tuple5(); + case 6: return new Tuple6(); + case 7: return new Tuple7(); + case 8: return new Tuple8(); + case 9: return new Tuple9(); + case 10: return new Tuple10(); + case 11: return new Tuple11(); + case 12: return new Tuple12(); + case 13: return new Tuple13(); + case 14: return new Tuple14(); + case 15: return new Tuple15(); + case 16: return new Tuple16(); + case 17: return new Tuple17(); + case 18: return new Tuple18(); + case 19: return new Tuple19(); + case 20: return new Tuple20(); + case 21: return new Tuple21(); + case 22: return new Tuple22(); + case 23: return new Tuple23(); + case 24: return new Tuple24(); + case 25: return new Tuple25(); + default: throw new IllegalArgumentException("The tuple arity must be in [0, " + MAX_ARITY + "]."); + } + } + private static final Class[] CLASSES = new Class[] { Tuple0.class, Tuple1.class, Tuple2.class, Tuple3.class, Tuple4.class, Tuple5.class, Tuple6.class, Tuple7.class, Tuple8.class, Tuple9.class, Tuple10.class, Tuple11.class, Tuple12.class, Tuple13.class, Tuple14.class, Tuple15.class, Tuple16.class, Tuple17.class, Tuple18.class, Tuple19.class, Tuple20.class, Tuple21.class, Tuple22.class, Tuple23.class, Tuple24.class, Tuple25.class }; diff --git a/flink-core/src/test/java/org/apache/flink/api/java/tuple/TupleGenerator.java b/flink-core/src/test/java/org/apache/flink/api/java/tuple/TupleGenerator.java index bd5598ad9837a..d68496764666d 100644 --- a/flink-core/src/test/java/org/apache/flink/api/java/tuple/TupleGenerator.java +++ b/flink-core/src/test/java/org/apache/flink/api/java/tuple/TupleGenerator.java @@ -133,6 +133,17 @@ private static void insertCodeIntoFile(String code, File file) throws IOExceptio private static void modifyTupleType(File root) throws IOException { // generate code StringBuilder sb = new StringBuilder(); + sb.append("\tpublic static Tuple newInstance(int arity) {\n"); + sb.append("\t\tswitch (arity) {\n"); + // special case for Tuple0: + sb.append("\t\t\tcase 0: return Tuple0.INSTANCE;\n"); + for (int i = FIRST; i <= LAST; i++) { + sb.append("\t\t\tcase ").append(i).append(": return new Tuple").append(i).append("();\n"); + } + sb.append("\t\t\tdefault: throw new IllegalArgumentException(\"The tuple arity must be in [0, \" + MAX_ARITY + \"].\");\n"); + sb.append("\t\t}\n"); + sb.append("\t}\n\n"); + sb.append("\tprivate static final Class[] CLASSES = new Class[] {\n\t\tTuple0.class"); for (int i = FIRST; i <= LAST; i++) { sb.append(", Tuple").append(i).append(".class"); diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/util/keys/KeySelectorUtil.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/util/keys/KeySelectorUtil.java index 27ce5733fc755..ab608ef1a5dc6 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/util/keys/KeySelectorUtil.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/util/keys/KeySelectorUtil.java @@ -180,8 +180,8 @@ public static final class ComparableKeySelector implements KeySelector implements KeySelector private static final long serialVersionUID = 1L; private final int[] fields; - private final Class tupleClass; private transient TupleTypeInfo returnType; ArrayKeySelector(int[] fields, TupleTypeInfo returnType) { this.fields = requireNonNull(fields); this.returnType = requireNonNull(returnType); - this.tupleClass = Tuple.getTupleClass(fields.length); } @Override - public Tuple getKey(IN value) throws Exception { - Tuple key = tupleClass.newInstance(); + public Tuple getKey(IN value) { + Tuple key = Tuple.newInstance(fields.length); for (int i = 0; i < fields.length; i++) { key.setField(Array.get(value, fields[i]), i); } From 4862101ddcc5ede17fcd0a304583e91e2f266044 Mon Sep 17 00:00:00 2001 From: Timo Walther Date: Mon, 4 Jun 2018 12:49:43 +0200 Subject: [PATCH 30/65] [FLINK-7251] [types] Remove the flink-java8 module and improve lambda type extraction This commit removes the flink-java8 module and merges some tests into flink-core/flink-runtime. It ensures to have the possibility for passing explicit type information in DataStream API as a fallback. Since the tycho compiler approach was very hacky and seems not to work anymore, this commit also removes all references in the docs and quickstarts. This closes #6120. --- docs/dev/java8.md | 198 ------------ docs/dev/java_lambdas.md | 138 +++++++++ .../java/typeutils/TypeExtractionUtils.java | 23 ++ .../api/java/typeutils/TypeExtractor.java | 290 +++++++----------- .../java/typeutils}/LambdaExtractionTest.java | 239 ++++++--------- .../examples/java/relational/TPCHQuery10.java | 28 +- .../examples/java/wordcount/WordCount.java | 3 +- .../examples/wordcount/WordCount.java | 14 +- flink-java8/pom.xml | 225 -------------- .../java8/relational/TPCHQuery10.java | 212 ------------- .../examples/java8/wordcount/WordCount.java | 124 -------- .../examples/java8/wordcount/WordCount.java | 124 -------- .../org/apache/flink/cep/CEPLambdaTest.java | 104 ------- .../util/JarFileCreatorLambdaTest.java | 113 ------- .../util/jartestprogram/FilterLambda3.java | 39 --- .../lambdas/AllGroupReduceITCase.java | 59 ---- .../java/operators/lambdas/CoGroupITCase.java | 74 ----- .../java/operators/lambdas/CrossITCase.java | 73 ----- .../java/operators/lambdas/FilterITCase.java | 91 ------ .../operators/lambdas/FlatJoinITCase.java | 68 ---- .../java/operators/lambdas/FlatMapITCase.java | 56 ---- .../operators/lambdas/GroupReduceITCase.java | 69 ----- .../java/operators/lambdas/JoinITCase.java | 69 ----- .../api/java/operators/lambdas/MapITCase.java | 74 ----- .../java/operators/lambdas/ReduceITCase.java | 109 ------- .../src/test/resources/log4j-test.properties | 19 -- .../org/apache/flink/cep/PatternStream.java | 16 +- .../java/org/apache/flink/cep/CEPITCase.java | 35 +-- .../flink/graph/asm/translate/Translate.java | 4 - .../resources/archetype-resources/pom.xml | 19 -- .../runtime/util/JarFileCreatorTest.java | 91 ++++-- .../jartestprogram/FilterWithIndirection.java | 6 +- .../util/jartestprogram/FilterWithLambda.java | 7 +- .../FilterWithMethodReference.java | 6 +- .../util/jartestprogram/UtilFunction.java | 4 +- .../jartestprogram/UtilFunctionWrapper.java | 4 +- .../util/jartestprogram/WordFilter.java | 3 +- .../api/datastream/AllWindowedStream.java | 2 - .../api/datastream/AsyncDataStream.java | 1 - .../datastream/BroadcastConnectedStream.java | 4 - .../api/datastream/CoGroupedStreams.java | 40 ++- .../api/datastream/ConnectedStreams.java | 28 +- .../streaming/api/datastream/DataStream.java | 16 +- .../api/datastream/IterativeStream.java | 4 + .../api/datastream/JoinedStreams.java | 43 ++- .../streaming/api/datastream/KeyedStream.java | 65 ++-- .../api/datastream/WindowedStream.java | 2 - .../flink/streaming/api/TypeFillTest.java | 65 +++- .../flink/test/operators/CoGroupITCase.java | 33 ++ .../flink/test/operators/MapITCase.java | 34 ++ .../flink/test/operators/ReduceITCase.java | 6 +- pom.xml | 1 - 52 files changed, 782 insertions(+), 2392 deletions(-) delete mode 100644 docs/dev/java8.md create mode 100644 docs/dev/java_lambdas.md rename {flink-java8/src/test/java/org/apache/flink/api/java/type/lambdas => flink-core/src/test/java/org/apache/flink/api/java/typeutils}/LambdaExtractionTest.java (54%) delete mode 100644 flink-java8/pom.xml delete mode 100644 flink-java8/src/main/java/org/apache/flink/examples/java8/relational/TPCHQuery10.java delete mode 100644 flink-java8/src/main/java/org/apache/flink/examples/java8/wordcount/WordCount.java delete mode 100644 flink-java8/src/main/java/org/apache/flink/streaming/examples/java8/wordcount/WordCount.java delete mode 100644 flink-java8/src/test/java/org/apache/flink/cep/CEPLambdaTest.java delete mode 100644 flink-java8/src/test/java/org/apache/flink/runtime/util/JarFileCreatorLambdaTest.java delete mode 100644 flink-java8/src/test/java/org/apache/flink/runtime/util/jartestprogram/FilterLambda3.java delete mode 100644 flink-java8/src/test/java/org/apache/flink/test/api/java/operators/lambdas/AllGroupReduceITCase.java delete mode 100644 flink-java8/src/test/java/org/apache/flink/test/api/java/operators/lambdas/CoGroupITCase.java delete mode 100644 flink-java8/src/test/java/org/apache/flink/test/api/java/operators/lambdas/CrossITCase.java delete mode 100644 flink-java8/src/test/java/org/apache/flink/test/api/java/operators/lambdas/FilterITCase.java delete mode 100644 flink-java8/src/test/java/org/apache/flink/test/api/java/operators/lambdas/FlatJoinITCase.java delete mode 100644 flink-java8/src/test/java/org/apache/flink/test/api/java/operators/lambdas/FlatMapITCase.java delete mode 100644 flink-java8/src/test/java/org/apache/flink/test/api/java/operators/lambdas/GroupReduceITCase.java delete mode 100644 flink-java8/src/test/java/org/apache/flink/test/api/java/operators/lambdas/JoinITCase.java delete mode 100644 flink-java8/src/test/java/org/apache/flink/test/api/java/operators/lambdas/MapITCase.java delete mode 100644 flink-java8/src/test/java/org/apache/flink/test/api/java/operators/lambdas/ReduceITCase.java delete mode 100644 flink-java8/src/test/resources/log4j-test.properties rename flink-java8/src/test/java/org/apache/flink/runtime/util/jartestprogram/FilterLambda4.java => flink-runtime/src/test/java/org/apache/flink/runtime/util/jartestprogram/FilterWithIndirection.java (93%) rename flink-java8/src/test/java/org/apache/flink/runtime/util/jartestprogram/FilterLambda2.java => flink-runtime/src/test/java/org/apache/flink/runtime/util/jartestprogram/FilterWithLambda.java (89%) rename flink-java8/src/test/java/org/apache/flink/runtime/util/jartestprogram/FilterLambda1.java => flink-runtime/src/test/java/org/apache/flink/runtime/util/jartestprogram/FilterWithMethodReference.java (93%) rename {flink-java8 => flink-runtime}/src/test/java/org/apache/flink/runtime/util/jartestprogram/UtilFunction.java (96%) rename {flink-java8 => flink-runtime}/src/test/java/org/apache/flink/runtime/util/jartestprogram/UtilFunctionWrapper.java (96%) rename {flink-java8 => flink-runtime}/src/test/java/org/apache/flink/runtime/util/jartestprogram/WordFilter.java (99%) diff --git a/docs/dev/java8.md b/docs/dev/java8.md deleted file mode 100644 index 8e7e6435e6b6e..0000000000000 --- a/docs/dev/java8.md +++ /dev/null @@ -1,198 +0,0 @@ ---- -title: "Java 8" -nav-parent_id: api-concepts -nav-pos: 20 ---- - - -Java 8 introduces several new language features designed for faster and clearer coding. With the most important feature, -the so-called "Lambda Expressions", Java 8 opens the door to functional programming. Lambda Expressions allow for implementing and -passing functions in a straightforward way without having to declare additional (anonymous) classes. - -The newest version of Flink supports the usage of Lambda Expressions for all operators of the Java API. -This document shows how to use Lambda Expressions and describes current limitations. For a general introduction to the -Flink API, please refer to the [Programming Guide]({{ site.baseurl }}/dev/api_concepts.html) - -* TOC -{:toc} - -### Examples - -The following example illustrates how to implement a simple, inline `map()` function that squares its input using a Lambda Expression. -The types of input `i` and output parameters of the `map()` function need not to be declared as they are inferred by the Java 8 compiler. - -{% highlight java %} -env.fromElements(1, 2, 3) -// returns the squared i -.map(i -> i*i) -.print(); -{% endhighlight %} - -The next two examples show different implementations of a function that uses a `Collector` for output. -Functions, such as `flatMap()`, require an output type (in this case `String`) to be defined for the `Collector` in order to be type-safe. -If the `Collector` type can not be inferred from the surrounding context, it needs to be declared in the Lambda Expression's parameter list manually. -Otherwise the output will be treated as type `Object` which can lead to undesired behaviour. - -{% highlight java %} -DataSet input = env.fromElements(1, 2, 3); - -// collector type must be declared -input.flatMap((Integer number, Collector out) -> { - StringBuilder builder = new StringBuilder(); - for(int i = 0; i < number; i++) { - builder.append("a"); - out.collect(builder.toString()); - } -}) -// returns (on separate lines) "a", "a", "aa", "a", "aa", "aaa" -.print(); -{% endhighlight %} - -{% highlight java %} -DataSet input = env.fromElements(1, 2, 3); - -// collector type must not be declared, it is inferred from the type of the dataset -DataSet manyALetters = input.flatMap((number, out) -> { - StringBuilder builder = new StringBuilder(); - for(int i = 0; i < number; i++) { - builder.append("a"); - out.collect(builder.toString()); - } -}); - -// returns (on separate lines) "a", "a", "aa", "a", "aa", "aaa" -manyALetters.print(); -{% endhighlight %} - -The following code demonstrates a word count which makes extensive use of Lambda Expressions. - -{% highlight java %} -DataSet input = env.fromElements("Please count", "the words", "but not this"); - -// filter out strings that contain "not" -input.filter(line -> !line.contains("not")) -// split each line by space -.map(line -> line.split(" ")) -// emit a pair for each array element -.flatMap((String[] wordArray, Collector> out) - -> Arrays.stream(wordArray).forEach(t -> out.collect(new Tuple2<>(t, 1))) - ) -// group and sum up -.groupBy(0).sum(1) -// print -.print(); -{% endhighlight %} - -### Compiler Limitations -Currently, Flink only supports jobs containing Lambda Expressions completely if they are **compiled with the Eclipse JDT compiler contained in Eclipse Luna 4.4.2 (and above)**. - -Only the Eclipse JDT compiler preserves the generic type information necessary to use the entire Lambda Expressions feature type-safely. -Other compilers such as the OpenJDK's and Oracle JDK's `javac` throw away all generic parameters related to Lambda Expressions. This means that types such as `Tuple2` or `Collector` declared as a Lambda function input or output parameter will be pruned to `Tuple2` or `Collector` in the compiled `.class` files, which is too little information for the Flink compiler. - -How to compile a Flink job that contains Lambda Expressions with the JDT compiler will be covered in the next section. - -However, it is possible to implement functions such as `map()` or `filter()` with Lambda Expressions in Java 8 compilers other than the Eclipse JDT compiler as long as the function has no `Collector`s or `Iterable`s *and* only if the function handles unparameterized types such as `Integer`, `Long`, `String`, `MyOwnClass` (types without Generics!). - -#### Compile Flink jobs with the Eclipse JDT compiler and Maven - -If you are using the Eclipse IDE, you can run and debug your Flink code within the IDE without any problems after some configuration steps. The Eclipse IDE by default compiles its Java sources with the Eclipse JDT compiler. The next section describes how to configure the Eclipse IDE. - -If you are using a different IDE such as IntelliJ IDEA or you want to package your Jar-File with Maven to run your job on a cluster, you need to modify your project's `pom.xml` file and build your program with Maven. The [quickstart]({{site.baseurl}}/quickstart/setup_quickstart.html) contains preconfigured Maven projects which can be used for new projects or as a reference. Uncomment the mentioned lines in your generated quickstart `pom.xml` file if you want to use Java 8 with Lambda Expressions. - -Alternatively, you can manually insert the following lines to your Maven `pom.xml` file. Maven will then use the Eclipse JDT compiler for compilation. - -{% highlight xml %} - - - - - maven-compiler-plugin - - 1.8 - 1.8 - jdt - - - - - org.eclipse.tycho - tycho-compiler-jdt - 0.21.0 - - - -{% endhighlight %} - -If you are using Eclipse for development, the m2e plugin might complain about the inserted lines above and marks your `pom.xml` as invalid. If so, insert the following lines to your `pom.xml`. - -{% highlight xml %} - - - - - org.apache.maven.plugins - maven-compiler-plugin - [3.1,) - - testCompile - compile - - - - - - -{% endhighlight %} - -#### Run and debug Flink jobs within the Eclipse IDE - -First of all, make sure you are running a current version of Eclipse IDE (4.4.2 or later). Also make sure that you have a Java 8 Runtime Environment (JRE) installed in Eclipse IDE (`Window` -> `Preferences` -> `Java` -> `Installed JREs`). - -Create/Import your Eclipse project. - -If you are using Maven, you also need to change the Java version in your `pom.xml` for the `maven-compiler-plugin`. Otherwise right click the `JRE System Library` section of your project and open the `Properties` window in order to switch to a Java 8 JRE (or above) that supports Lambda Expressions. - -The Eclipse JDT compiler needs a special compiler flag in order to store type information in `.class` files. Open the JDT configuration file at `{project directory}/.settings/org.eclipse.jdt.core.prefs` with your favorite text editor and add the following line: - -{% highlight plain %} -org.eclipse.jdt.core.compiler.codegen.lambda.genericSignature=generate -{% endhighlight %} - -If not already done, also modify the Java versions of the following properties to `1.8` (or above): - -{% highlight plain %} -org.eclipse.jdt.core.compiler.codegen.targetPlatform=1.8 -org.eclipse.jdt.core.compiler.compliance=1.8 -org.eclipse.jdt.core.compiler.source=1.8 -{% endhighlight %} - -After you have saved the file, perform a complete project refresh in Eclipse IDE. - -If you are using Maven, right click your Eclipse project and select `Maven` -> `Update Project...`. - -You have configured everything correctly, if the following Flink program runs without exceptions: - -{% highlight java %} -final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); -env.fromElements(1, 2, 3).map((in) -> new Tuple1(" " + in)).print(); -env.execute(); -{% endhighlight %} - -{% top %} diff --git a/docs/dev/java_lambdas.md b/docs/dev/java_lambdas.md new file mode 100644 index 0000000000000..4b306ac455d02 --- /dev/null +++ b/docs/dev/java_lambdas.md @@ -0,0 +1,138 @@ +--- +title: "Java Lambda Expressions" +nav-parent_id: api-concepts +nav-pos: 20 +--- + + +Java 8 introduced several new language features designed for faster and clearer coding. With the most important feature, +the so-called "Lambda Expressions", it opened the door to functional programming. Lambda expressions allow for implementing and +passing functions in a straightforward way without having to declare additional (anonymous) classes. + +Attention Flink supports the usage of lambda expressions for all operators of the Java API, however, whenever a lambda expression uses Java generics you need to declare type information *explicitly*. + +This document shows how to use lambda expressions and describes current limitations. For a general introduction to the +Flink API, please refer to the [Programming Guide]({{ site.baseurl }}/dev/api_concepts.html) + +### Examples and Limitations + +The following example illustrates how to implement a simple, inline `map()` function that squares its input using a lambda expression. +The types of input `i` and output parameters of the `map()` function need not to be declared as they are inferred by the Java compiler. + +{% highlight java %} +env.fromElements(1, 2, 3) +// returns the squared i +.map(i -> i*i) +.print(); +{% endhighlight %} + +Flink can automatically extract the result type information from the implementation of the method signature `OUT map(IN value)` because `OUT` is not generic but `Integer`. + +Unfortunately, functions such as `flatMap()` with a signature `void flatMap(IN value, Collector out)` are compiled into `void flatMap(IN value, Collector out)` by the Java compiler. This makes it impossible for Flink to infer the type information for the output type automatically. + +Flink will most likely throw an exception similar to the following: + +{% highlight plain%} +org.apache.flink.api.common.functions.InvalidTypesException: The generic type parameters of 'Collector' are missing. + In many cases lambda methods don't provide enough information for automatic type extraction when Java generics are involved. + An easy workaround is to use an (anonymous) class instead that implements the 'org.apache.flink.api.common.functions.FlatMapFunction' interface. + Otherwise the type has to be specified explicitly using type information. +{% endhighlight %} + +In this case, the type information needs to be *specified explicitly*, otherwise the output will be treated as type `Object` which leads to unefficient serialization. + +{% highlight java %} +import org.apache.flink.api.common.typeinfo.Types; +import org.apache.flink.api.java.DataSet; +import org.apache.flink.util.Collector; + +DataSet input = env.fromElements(1, 2, 3); + +// collector type must be declared +input.flatMap((Integer number, Collector out) -> { + StringBuilder builder = new StringBuilder(); + for(int i = 0; i < number; i++) { + builder.append("a"); + out.collect(builder.toString()); + } +}) +// provide type information explicitly +.returns(Types.STRING) +// prints "a", "a", "aa", "a", "aa", "aaa" +.print(); +{% endhighlight %} + +Similar problems occur when using a `map()` function with a generic return type. A method signature `Tuple2 map(Integer value)` is erasured to `Tuple2 map(Integer value)` in the example below. + +{% highlight java %} +import org.apache.flink.api.common.functions.MapFunction; +import org.apache.flink.api.java.tuple.Tuple2; + +env.fromElements(1, 2, 3) + .map(i -> Tuple2.of(i, i)) // no information about fields of Tuple2 + .print(); +{% endhighlight %} + +In general, those problems can be solved in multiple ways: + +{% highlight java %} +import org.apache.flink.api.common.typeinfo.Types; +import org.apache.flink.api.java.tuple.Tuple2; + +// use the explicit ".returns(...)" +env.fromElements(1, 2, 3) + .map(i -> Tuple2.of(i, i)) + .returns(Types.TUPLE(Types.INT, Types.INT)) + .print(); + +// use a class instead +env.fromElements(1, 2, 3) + .map(new MyTuple2Mapper()) + .print(); + +public static class MyTuple2Mapper extends MapFunction { + @Override + public Tuple2 map(Integer i) { + return Tuple2.of(i, i); + } +} + +// use an anonymous class instead +env.fromElements(1, 2, 3) + .map(new MapFunction> { + @Override + public Tuple2 map(Integer i) { + return Tuple2.of(i, i); + } + }) + .print(); + +// or in this example use a tuple subclass instead +env.fromElements(1, 2, 3) + .map(i -> new DoubleTuple(i, i)) + .print(); + +public static class DoubleTuple extends Tuple2 { + public DoubleTuple(int f0, int f1) { + this.f0 = f0; + this.f1 = f1; + } +} +{% endhighlight %} \ No newline at end of file diff --git a/flink-core/src/main/java/org/apache/flink/api/java/typeutils/TypeExtractionUtils.java b/flink-core/src/main/java/org/apache/flink/api/java/typeutils/TypeExtractionUtils.java index f005ed9ee7576..07f1e1e2ca404 100644 --- a/flink-core/src/main/java/org/apache/flink/api/java/typeutils/TypeExtractionUtils.java +++ b/flink-core/src/main/java/org/apache/flink/api/java/typeutils/TypeExtractionUtils.java @@ -158,6 +158,7 @@ public static LambdaExecutable checkAndExtractLambda(Function function) throws T /** * Extracts type from given index from lambda. It supports nested types. * + * @param baseClass SAM function that the lambda implements * @param exec lambda function to extract the type from * @param lambdaTypeArgumentIndices position of type to extract in type hierarchy * @param paramLen count of total parameters of the lambda (including closure parameters) @@ -165,14 +166,17 @@ public static LambdaExecutable checkAndExtractLambda(Function function) throws T * @return extracted type */ public static Type extractTypeFromLambda( + Class baseClass, LambdaExecutable exec, int[] lambdaTypeArgumentIndices, int paramLen, int baseParametersLen) { Type output = exec.getParameterTypes()[paramLen - baseParametersLen + lambdaTypeArgumentIndices[0]]; for (int i = 1; i < lambdaTypeArgumentIndices.length; i++) { + validateLambdaType(baseClass, output); output = extractTypeArgument(output, lambdaTypeArgumentIndices[i]); } + validateLambdaType(baseClass, output); return output; } @@ -328,4 +332,23 @@ public static Class getRawClass(Type t) { } return Object.class; } + + /** + * Checks whether the given type has the generic parameters declared in the class definition. + * + * @param t type to be validated + */ + public static void validateLambdaType(Class baseClass, Type t) { + if (!(t instanceof Class)) { + return; + } + final Class clazz = (Class) t; + + if (clazz.getTypeParameters().length > 0) { + throw new InvalidTypesException("The generic type parameters of '" + clazz.getSimpleName() + "' are missing. " + + "In many cases lambda methods don't provide enough information for automatic type extraction when Java generics are involved. " + + "An easy workaround is to use an (anonymous) class instead that implements the '" + baseClass.getName() + "' interface. " + + "Otherwise the type has to be specified explicitly using type information."); + } + } } diff --git a/flink-core/src/main/java/org/apache/flink/api/java/typeutils/TypeExtractor.java b/flink-core/src/main/java/org/apache/flink/api/java/typeutils/TypeExtractor.java index f514384232f44..07b6cfebbfa71 100644 --- a/flink-core/src/main/java/org/apache/flink/api/java/typeutils/TypeExtractor.java +++ b/flink-core/src/main/java/org/apache/flink/api/java/typeutils/TypeExtractor.java @@ -84,6 +84,12 @@ /** * A utility for reflection analysis on classes, to determine the return type of implementations of transformation * functions. + * + *

    NOTES FOR USERS OF THIS CLASS: + * Automatic type extraction is a hacky business that depends on a lot of variables such as generics, + * compiler, interfaces, etc. The type extraction fails regularly with either {@link MissingTypeInfo} or + * hard exceptions. Whenever you use methods of this class, make sure to provide a way to pass custom + * type information as a fallback. */ @Public public class TypeExtractor { @@ -171,7 +177,6 @@ public static TypeInformation getMapReturnTypes(MapFunction TypeInformation getFlatMapReturnTypes(FlatMapFuncti FlatMapFunction.class, 0, 1, - new int[]{0}, new int[]{1, 0}, inType, functionName, @@ -222,7 +226,6 @@ public static TypeInformation getFoldReturnTypes(FoldFunction TypeInformation getAggregateFunctionAccumulatorType AggregateFunction.class, 0, 1, - new int[]{0}, NO_INDEX, inType, functionName, @@ -261,7 +263,6 @@ public static TypeInformation getAggregateFunctionReturnType( 0, 2, NO_INDEX, - NO_INDEX, inType, functionName, allowMissing); @@ -281,7 +282,6 @@ public static TypeInformation getMapPartitionReturnTypes(MapParti MapPartitionFunction.class, 0, 1, - new int[]{0, 0}, new int[]{1, 0}, inType, functionName, @@ -302,7 +302,6 @@ public static TypeInformation getGroupReduceReturnTypes(GroupRedu GroupReduceFunction.class, 0, 1, - new int[]{0, 0}, new int[]{1, 0}, inType, functionName, @@ -323,7 +322,6 @@ public static TypeInformation getGroupCombineReturnTypes(GroupCom GroupCombineFunction.class, 0, 1, - new int[]{0, 0}, new int[]{1, 0}, inType, functionName, @@ -347,8 +345,6 @@ public static TypeInformation getFlatJoinReturnTypes(FlatJo 0, 1, 2, - new int[]{0}, - new int[]{1}, new int[]{2, 0}, in1Type, in2Type, @@ -373,8 +369,6 @@ public static TypeInformation getJoinReturnTypes(JoinFuncti 0, 1, 2, - new int[]{0}, - new int[]{1}, NO_INDEX, in1Type, in2Type, @@ -399,8 +393,6 @@ public static TypeInformation getCoGroupReturnTypes(CoGroup 0, 1, 2, - new int[]{0, 0}, - new int[]{1, 0}, new int[]{2, 0}, in1Type, in2Type, @@ -425,8 +417,6 @@ public static TypeInformation getCrossReturnTypes(CrossFunc 0, 1, 2, - new int[]{0}, - new int[]{1}, NO_INDEX, in1Type, in2Type, @@ -448,7 +438,6 @@ public static TypeInformation getKeySelectorTypes(KeySelector TypeInformation getPartitionerTypes( Partitioner partitioner, String functionName, boolean allowMissing) { - try { - final LambdaExecutable exec; - try { - exec = checkAndExtractLambda(partitioner); - } catch (TypeExtractionException e) { - throw new InvalidTypesException("Internal error occurred.", e); - } - if (exec != null) { - // check for lambda type erasure - validateLambdaGenericParameters(exec); - - // parameters must be accessed from behind, since JVM can add additional parameters e.g. when using local variables inside lambda function - // paramLen is the total number of parameters of the provided lambda, it includes parameters added through closure - final int paramLen = exec.getParameterTypes().length; - final Method sam = TypeExtractionUtils.getSingleAbstractMethod(Partitioner.class); - // number of parameters the SAM of implemented interface has; the parameter indexing applies to this range - final int baseParametersLen = sam.getParameterTypes().length; - - final Type keyType = TypeExtractionUtils.extractTypeFromLambda( - exec, - new int[]{0}, - paramLen, - baseParametersLen); - return new TypeExtractor().privateCreateTypeInfo(keyType, null, null); - } else { - return new TypeExtractor().privateCreateTypeInfo( - Partitioner.class, - partitioner.getClass(), - 0, - null, - null); - } - } catch (InvalidTypesException e) { - if (allowMissing) { - return (TypeInformation) new MissingTypeInfo(functionName != null ? functionName : partitioner.toString(), e); - } else { - throw e; - } - } + return getUnaryOperatorReturnType( + partitioner, + Partitioner.class, + -1, + 0, + new int[]{0}, + null, + functionName, + allowMissing); } @@ -524,24 +483,43 @@ public static TypeInformation getInputFormatTypes(InputFormat in /** * Returns the unary operator's return type. * - *

    NOTE: lambda type indices allow extraction of Type from lambdas. To extract input type IN - * from the function given below one should pass {@code new int[] {0,1,0}} as lambdaInputTypeArgumentIndices. + *

    This method can extract a type in 4 different ways: + * + *

    1. By using the generics of the base class like MyFunction. + * This is what outputTypeArgumentIndex (in this example "4") is good for. + * + *

    2. By using input type inference SubMyFunction. + * This is what inputTypeArgumentIndex (in this example "0") and inType is good for. + * + *

    3. By using the static method that a compiler generates for Java lambdas. + * This is what lambdaOutputTypeArgumentIndices is good for. Given that MyFunction has + * the following single abstract method: * *

     	 * 
    -	 * OUT apply(Map> value)
    +	 * void apply(IN value, Collector value)
     	 * 
     	 * 
    * + *

    Lambda type indices allow the extraction of a type from lambdas. To extract the + * output type OUT from the function one should pass {@code new int[] {1, 0}}. + * "1" for selecting the parameter and 0 for the first generic in this type. + * Use {@code TypeExtractor.NO_INDEX} for selecting the return type of the lambda for + * extraction or if the class cannot be a lambda because it is not a single abstract + * method interface. + * + *

    4. By using interfaces such as {@link TypeInfoFactory} or {@link ResultTypeQueryable}. + * + *

    See also comments in the header of this class. + * * @param function Function to extract the return type from * @param baseClass Base class of the function - * @param inputTypeArgumentIndex Index of input type in the class specification - * @param outputTypeArgumentIndex Index of output type in the class specification - * @param lambdaInputTypeArgumentIndices Table of indices of the type argument specifying the input type. See example. + * @param inputTypeArgumentIndex Index of input generic type in the base class specification (ignored if inType is null) + * @param outputTypeArgumentIndex Index of output generic type in the base class specification * @param lambdaOutputTypeArgumentIndices Table of indices of the type argument specifying the input type. See example. - * @param inType Type of the input elements (In case of an iterable, it is the element type) + * @param inType Type of the input elements (In case of an iterable, it is the element type) or null * @param functionName Function name - * @param allowMissing Can the type information be missing + * @param allowMissing Can the type information be missing (this generates a MissingTypeInfo for postponing an exception) * @param Input type * @param Output type * @return TypeInformation of the return type of the function @@ -553,11 +531,23 @@ public static TypeInformation getUnaryOperatorReturnType( Class baseClass, int inputTypeArgumentIndex, int outputTypeArgumentIndex, - int[] lambdaInputTypeArgumentIndices, int[] lambdaOutputTypeArgumentIndices, TypeInformation inType, String functionName, boolean allowMissing) { + + Preconditions.checkArgument(inType == null || inputTypeArgumentIndex >= 0, "Input type argument index was not provided"); + Preconditions.checkArgument(outputTypeArgumentIndex >= 0, "Output type argument index was not provided"); + Preconditions.checkArgument( + lambdaOutputTypeArgumentIndices != null, + "Indices for output type arguments within lambda not provided"); + + // explicit result type has highest precedence + if (function instanceof ResultTypeQueryable) { + return ((ResultTypeQueryable) function).getProducedType(); + } + + // perform extraction try { final LambdaExecutable exec; try { @@ -566,14 +556,6 @@ public static TypeInformation getUnaryOperatorReturnType( throw new InvalidTypesException("Internal error occurred.", e); } if (exec != null) { - Preconditions.checkArgument( - lambdaInputTypeArgumentIndices != null && lambdaInputTypeArgumentIndices.length >= 1, - "Indices for input type arguments within lambda not provided"); - Preconditions.checkArgument( - lambdaOutputTypeArgumentIndices != null, - "Indices for output type arguments within lambda not provided"); - // check for lambda type erasure - validateLambdaGenericParameters(exec); // parameters must be accessed from behind, since JVM can add additional parameters e.g. when using local variables inside lambda function // paramLen is the total number of parameters of the provided lambda, it includes parameters added through closure @@ -584,43 +566,23 @@ public static TypeInformation getUnaryOperatorReturnType( // number of parameters the SAM of implemented interface has; the parameter indexing applies to this range final int baseParametersLen = sam.getParameterTypes().length; - // executable references "this" implicitly - if (paramLen <= 0) { - // executable declaring class can also be a super class of the input type - // we only validate if the executable exists in input type - validateInputContainsExecutable(exec, inType); - } - else { - final Type input = TypeExtractionUtils.extractTypeFromLambda( - exec, - lambdaInputTypeArgumentIndices, - paramLen, - baseParametersLen); - validateInputType(input, inType); - } - - if (function instanceof ResultTypeQueryable) { - return ((ResultTypeQueryable) function).getProducedType(); - } - final Type output; if (lambdaOutputTypeArgumentIndices.length > 0) { output = TypeExtractionUtils.extractTypeFromLambda( + baseClass, exec, lambdaOutputTypeArgumentIndices, paramLen, baseParametersLen); } else { output = exec.getReturnType(); + TypeExtractionUtils.validateLambdaType(baseClass, output); } return new TypeExtractor().privateCreateTypeInfo(output, inType, null); } else { - Preconditions.checkArgument(inputTypeArgumentIndex >= 0, "Input type argument index was not provided"); - Preconditions.checkArgument(outputTypeArgumentIndex >= 0, "Output type argument index was not provided"); - validateInputType(baseClass, function.getClass(), inputTypeArgumentIndex, inType); - if(function instanceof ResultTypeQueryable) { - return ((ResultTypeQueryable) function).getProducedType(); + if (inType != null) { + validateInputType(baseClass, function.getClass(), inputTypeArgumentIndex, inType); } return new TypeExtractor().privateCreateTypeInfo(baseClass, function.getClass(), outputTypeArgumentIndex, inType, null); } @@ -637,27 +599,45 @@ public static TypeInformation getUnaryOperatorReturnType( /** * Returns the binary operator's return type. * - *

    NOTE: lambda type indices allows extraction of Type from lambdas. To extract input type IN1 - * from the function given below one should pass {@code new int[] {0,1,0}} as lambdaInput1TypeArgumentIndices. + *

    This method can extract a type in 4 different ways: + * + *

    1. By using the generics of the base class like MyFunction. + * This is what outputTypeArgumentIndex (in this example "4") is good for. + * + *

    2. By using input type inference SubMyFunction. + * This is what inputTypeArgumentIndex (in this example "0") and inType is good for. + * + *

    3. By using the static method that a compiler generates for Java lambdas. + * This is what lambdaOutputTypeArgumentIndices is good for. Given that MyFunction has + * the following single abstract method: * *

     	 * 
    -	 * OUT apply(Map> value1, List value2)
    +	 * void apply(IN value, Collector value)
     	 * 
     	 * 
    * + *

    Lambda type indices allow the extraction of a type from lambdas. To extract the + * output type OUT from the function one should pass {@code new int[] {1, 0}}. + * "1" for selecting the parameter and 0 for the first generic in this type. + * Use {@code TypeExtractor.NO_INDEX} for selecting the return type of the lambda for + * extraction or if the class cannot be a lambda because it is not a single abstract + * method interface. + * + *

    4. By using interfaces such as {@link TypeInfoFactory} or {@link ResultTypeQueryable}. + * + *

    See also comments in the header of this class. + * * @param function Function to extract the return type from * @param baseClass Base class of the function - * @param input1TypeArgumentIndex Index of first input type in the class specification - * @param input2TypeArgumentIndex Index of second input type in the class specification - * @param outputTypeArgumentIndex Index of output type in the class specification - * @param lambdaInput1TypeArgumentIndices Table of indices of the type argument specifying the first input type. See example. - * @param lambdaInput2TypeArgumentIndices Table of indices of the type argument specifying the second input type. See example. + * @param input1TypeArgumentIndex Index of first input generic type in the class specification (ignored if in1Type is null) + * @param input2TypeArgumentIndex Index of second input generic type in the class specification (ignored if in2Type is null) + * @param outputTypeArgumentIndex Index of output generic type in the class specification * @param lambdaOutputTypeArgumentIndices Table of indices of the type argument specifying the output type. See example. * @param in1Type Type of the left side input elements (In case of an iterable, it is the element type) * @param in2Type Type of the right side input elements (In case of an iterable, it is the element type) * @param functionName Function name - * @param allowMissing Can the type information be missing + * @param allowMissing Can the type information be missing (this generates a MissingTypeInfo for postponing an exception) * @param Left side input type * @param Right side input type * @param Output type @@ -671,13 +651,25 @@ public static TypeInformation getBinaryOperatorReturnType( int input1TypeArgumentIndex, int input2TypeArgumentIndex, int outputTypeArgumentIndex, - int[] lambdaInput1TypeArgumentIndices, - int[] lambdaInput2TypeArgumentIndices, int[] lambdaOutputTypeArgumentIndices, TypeInformation in1Type, TypeInformation in2Type, String functionName, boolean allowMissing) { + + Preconditions.checkArgument(in1Type == null || input1TypeArgumentIndex >= 0, "Input 1 type argument index was not provided"); + Preconditions.checkArgument(in2Type == null || input2TypeArgumentIndex >= 0, "Input 2 type argument index was not provided"); + Preconditions.checkArgument(outputTypeArgumentIndex >= 0, "Output type argument index was not provided"); + Preconditions.checkArgument( + lambdaOutputTypeArgumentIndices != null, + "Indices for output type arguments within lambda not provided"); + + // explicit result type has highest precedence + if (function instanceof ResultTypeQueryable) { + return ((ResultTypeQueryable) function).getProducedType(); + } + + // perform extraction try { final LambdaExecutable exec; try { @@ -686,17 +678,6 @@ public static TypeInformation getBinaryOperatorReturnType( throw new InvalidTypesException("Internal error occurred.", e); } if (exec != null) { - Preconditions.checkArgument( - lambdaInput1TypeArgumentIndices != null && lambdaInput1TypeArgumentIndices.length >= 1, - "Indices for first input type arguments within lambda not provided"); - Preconditions.checkArgument( - lambdaInput2TypeArgumentIndices != null && lambdaInput2TypeArgumentIndices.length >= 1, - "Indices for second input type arguments within lambda not provided"); - Preconditions.checkArgument( - lambdaOutputTypeArgumentIndices != null, - "Indices for output type arguments within lambda not provided"); - // check for lambda type erasure - validateLambdaGenericParameters(exec); final Method sam = TypeExtractionUtils.getSingleAbstractMethod(baseClass); final int baseParametersLen = sam.getParameterTypes().length; @@ -704,32 +685,17 @@ public static TypeInformation getBinaryOperatorReturnType( // parameters must be accessed from behind, since JVM can add additional parameters e.g. when using local variables inside lambda function final int paramLen = exec.getParameterTypes().length; - final Type input1 = TypeExtractionUtils.extractTypeFromLambda( - exec, - lambdaInput1TypeArgumentIndices, - paramLen, - baseParametersLen); - final Type input2 = TypeExtractionUtils.extractTypeFromLambda( - exec, - lambdaInput2TypeArgumentIndices, - paramLen, - baseParametersLen); - - validateInputType(input1, in1Type); - validateInputType(input2, in2Type); - if(function instanceof ResultTypeQueryable) { - return ((ResultTypeQueryable) function).getProducedType(); - } - final Type output; if (lambdaOutputTypeArgumentIndices.length > 0) { output = TypeExtractionUtils.extractTypeFromLambda( + baseClass, exec, lambdaOutputTypeArgumentIndices, paramLen, baseParametersLen); } else { output = exec.getReturnType(); + TypeExtractionUtils.validateLambdaType(baseClass, output); } return new TypeExtractor().privateCreateTypeInfo( @@ -738,13 +704,11 @@ public static TypeInformation getBinaryOperatorReturnType( in2Type); } else { - Preconditions.checkArgument(input1TypeArgumentIndex >= 0, "Input 1 type argument index was not provided"); - Preconditions.checkArgument(input2TypeArgumentIndex >= 0, "Input 2 type argument index was not provided"); - Preconditions.checkArgument(outputTypeArgumentIndex >= 0, "Output type argument index was not provided"); - validateInputType(baseClass, function.getClass(), input1TypeArgumentIndex, in1Type); - validateInputType(baseClass, function.getClass(), input2TypeArgumentIndex, in2Type); - if(function instanceof ResultTypeQueryable) { - return ((ResultTypeQueryable) function).getProducedType(); + if (in1Type != null) { + validateInputType(baseClass, function.getClass(), input1TypeArgumentIndex, in1Type); + } + if (in2Type != null) { + validateInputType(baseClass, function.getClass(), input2TypeArgumentIndex, in2Type); } return new TypeExtractor().privateCreateTypeInfo(baseClass, function.getClass(), outputTypeArgumentIndex, in1Type, in2Type); } @@ -915,9 +879,10 @@ else if (t instanceof TypeVariable) { return typeInfo; } else { throw new InvalidTypesException("Type of TypeVariable '" + ((TypeVariable) t).getName() + "' in '" - + ((TypeVariable) t).getGenericDeclaration() + "' could not be determined. This is most likely a type erasure problem. " - + "The type extraction currently supports types with generic variables only in cases where " - + "all variables in the return type can be deduced from the input type(s)."); + + ((TypeVariable) t).getGenericDeclaration() + "' could not be determined. This is most likely a type erasure problem. " + + "The type extraction currently supports types with generic variables only in cases where " + + "all variables in the return type can be deduced from the input type(s). " + + "Otherwise the type has to be specified explicitly using type information."); } } } @@ -1165,10 +1130,11 @@ private TypeInformation[] createSubTypesInfo(Type originalType, Pa // variable could not be determined if (subTypesInfo[i] == null && !lenient) { throw new InvalidTypesException("Type of TypeVariable '" + ((TypeVariable) subtypes[i]).getName() + "' in '" - + ((TypeVariable) subtypes[i]).getGenericDeclaration() - + "' could not be determined. This is most likely a type erasure problem. " - + "The type extraction currently supports types with generic variables only in cases where " - + "all variables in the return type can be deduced from the input type(s)."); + + ((TypeVariable) subtypes[i]).getGenericDeclaration() + + "' could not be determined. This is most likely a type erasure problem. " + + "The type extraction currently supports types with generic variables only in cases where " + + "all variables in the return type can be deduced from the input type(s). " + + "Otherwise the type has to be specified explicitly using type information."); } } else { // create the type information of the subtype or null/exception @@ -1618,30 +1584,6 @@ private int countFieldsInClass(Class clazz) { return fieldCount; } - private static void validateLambdaGenericParameters(LambdaExecutable exec) { - // check the arguments - for (Type t : exec.getParameterTypes()) { - validateLambdaGenericParameter(t); - } - - // check the return type - validateLambdaGenericParameter(exec.getReturnType()); - } - - private static void validateLambdaGenericParameter(Type t) { - if(!(t instanceof Class)) { - return; - } - final Class clazz = (Class) t; - - if(clazz.getTypeParameters().length > 0) { - throw new InvalidTypesException("The generic type parameters of '" + clazz.getSimpleName() + "' are missing. \n" - + "It seems that your compiler has not stored them into the .class file. \n" - + "Currently, only the Eclipse JDT compiler preserves the type information necessary to use the lambdas feature type-safely. \n" - + "See the documentation for more information about how to compile jobs containing lambda expressions."); - } - } - /** * Tries to find a concrete value (Class, ParameterizedType etc. ) for a TypeVariable by traversing the type hierarchy downwards. * If a value could not be found it will return the most bottom type variable in the hierarchy. diff --git a/flink-java8/src/test/java/org/apache/flink/api/java/type/lambdas/LambdaExtractionTest.java b/flink-core/src/test/java/org/apache/flink/api/java/typeutils/LambdaExtractionTest.java similarity index 54% rename from flink-java8/src/test/java/org/apache/flink/api/java/type/lambdas/LambdaExtractionTest.java rename to flink-core/src/test/java/org/apache/flink/api/java/typeutils/LambdaExtractionTest.java index de1f395ac1be1..1d5cf22a5c1aa 100644 --- a/flink-java8/src/test/java/org/apache/flink/api/java/type/lambdas/LambdaExtractionTest.java +++ b/flink-core/src/test/java/org/apache/flink/api/java/typeutils/LambdaExtractionTest.java @@ -16,13 +16,10 @@ * limitations under the License. */ -package org.apache.flink.api.java.type.lambdas; +package org.apache.flink.api.java.typeutils; import org.apache.flink.api.common.functions.CoGroupFunction; -import org.apache.flink.api.common.functions.CrossFunction; -import org.apache.flink.api.common.functions.FlatJoinFunction; import org.apache.flink.api.common.functions.FlatMapFunction; -import org.apache.flink.api.common.functions.GroupReduceFunction; import org.apache.flink.api.common.functions.InvalidTypesException; import org.apache.flink.api.common.functions.JoinFunction; import org.apache.flink.api.common.functions.MapFunction; @@ -36,12 +33,7 @@ import org.apache.flink.api.java.functions.KeySelector; import org.apache.flink.api.java.tuple.Tuple1; import org.apache.flink.api.java.tuple.Tuple2; -import org.apache.flink.api.java.typeutils.MissingTypeInfo; -import org.apache.flink.api.java.typeutils.TupleTypeInfo; -import org.apache.flink.api.java.typeutils.TypeExtractionUtils; -import org.apache.flink.api.java.typeutils.TypeExtractor; -import org.junit.Assert; import org.junit.Test; import java.lang.reflect.Method; @@ -50,12 +42,12 @@ import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertNotNull; import static org.junit.Assert.assertNull; -import static org.junit.Assert.fail; +import static org.junit.Assert.assertTrue; /** - * Tests the type extractor for lambda functions. + * Tests the type extractor for lambda functions. Many tests only work if the compiler supports + * lambdas properly otherwise a MissingTypeInfo is returned. */ -@SuppressWarnings("serial") public class LambdaExtractionTest { private static final TypeInformation, Boolean>> NESTED_TUPLE_BOOLEAN_TYPE = @@ -65,48 +57,43 @@ public class LambdaExtractionTest { new TypeHint, Double>>(){}.getTypeInfo(); @Test - public void testIdentifyLambdas() { - try { - MapFunction anonymousFromInterface = new MapFunction() { - @Override - public Integer map(String value) { - return Integer.parseInt(value); - } - }; - - MapFunction anonymousFromClass = new RichMapFunction() { - @Override - public Integer map(String value) { - return Integer.parseInt(value); - } - }; - - MapFunction fromProperClass = new StaticMapper(); - - MapFunction fromDerived = new ToTuple() { - @Override - public Tuple2 map(Integer value) { - return new Tuple2<>(value, 1L); - } - }; - - MapFunction staticLambda = Integer::parseInt; - MapFunction instanceLambda = Object::toString; - MapFunction constructorLambda = Integer::new; - - assertNull(checkAndExtractLambda(anonymousFromInterface)); - assertNull(checkAndExtractLambda(anonymousFromClass)); - assertNull(checkAndExtractLambda(fromProperClass)); - assertNull(checkAndExtractLambda(fromDerived)); - assertNotNull(checkAndExtractLambda(staticLambda)); - assertNotNull(checkAndExtractLambda(instanceLambda)); - assertNotNull(checkAndExtractLambda(constructorLambda)); - assertNotNull(checkAndExtractLambda(STATIC_LAMBDA)); - } - catch (Exception e) { - e.printStackTrace(); - fail(e.getMessage()); - } + @SuppressWarnings({"Convert2Lambda", "Anonymous2MethodRef"}) + public void testIdentifyLambdas() throws TypeExtractionException { + MapFunction anonymousFromInterface = new MapFunction() { + @Override + public Integer map(String value) { + return Integer.parseInt(value); + } + }; + + MapFunction anonymousFromClass = new RichMapFunction() { + @Override + public Integer map(String value) { + return Integer.parseInt(value); + } + }; + + MapFunction fromProperClass = new StaticMapper(); + + MapFunction fromDerived = new ToTuple() { + @Override + public Tuple2 map(Integer value) { + return new Tuple2<>(value, 1L); + } + }; + + MapFunction staticLambda = Integer::parseInt; + MapFunction instanceLambda = Object::toString; + MapFunction constructorLambda = Integer::new; + + assertNull(checkAndExtractLambda(anonymousFromInterface)); + assertNull(checkAndExtractLambda(anonymousFromClass)); + assertNull(checkAndExtractLambda(fromProperClass)); + assertNull(checkAndExtractLambda(fromDerived)); + assertNotNull(checkAndExtractLambda(staticLambda)); + assertNotNull(checkAndExtractLambda(instanceLambda)); + assertNotNull(checkAndExtractLambda(constructorLambda)); + assertNotNull(checkAndExtractLambda(STATIC_LAMBDA)); } private static class StaticMapper implements MapFunction { @@ -134,7 +121,7 @@ public MapFunction getMapFunction() { @Test public void testLambdaWithMemberVariable() { TypeInformation ti = TypeExtractor.getMapReturnTypes(new MyClass().getMapFunction(), Types.INT); - Assert.assertEquals(ti, BasicTypeInfo.STRING_TYPE_INFO); + assertEquals(ti, BasicTypeInfo.STRING_TYPE_INFO); } @Test @@ -146,32 +133,41 @@ public void testLambdaWithLocalVariable() { MapFunction f = (i) -> s + k + j; TypeInformation ti = TypeExtractor.getMapReturnTypes(f, Types.INT); - Assert.assertEquals(ti, BasicTypeInfo.STRING_TYPE_INFO); + assertEquals(ti, BasicTypeInfo.STRING_TYPE_INFO); + } + + @Test + public void testLambdaWithNonGenericResultType() { + MapFunction, Boolean>, Boolean> f = (i) -> null; + + TypeInformation ti = TypeExtractor.getMapReturnTypes(f, NESTED_TUPLE_BOOLEAN_TYPE, null, true); + assertTrue(ti instanceof BasicTypeInfo); + assertEquals(BasicTypeInfo.BOOLEAN_TYPE_INFO, ti); } @Test public void testMapLambda() { MapFunction, Boolean>, Tuple2, String>> f = (i) -> null; - TypeInformation ti = TypeExtractor.getMapReturnTypes(f, NESTED_TUPLE_BOOLEAN_TYPE); + TypeInformation ti = TypeExtractor.getMapReturnTypes(f, NESTED_TUPLE_BOOLEAN_TYPE, null, true); if (!(ti instanceof MissingTypeInfo)) { - Assert.assertTrue(ti.isTupleType()); - Assert.assertEquals(2, ti.getArity()); - Assert.assertTrue(((TupleTypeInfo) ti).getTypeAt(0).isTupleType()); - Assert.assertEquals(((TupleTypeInfo) ti).getTypeAt(1), BasicTypeInfo.STRING_TYPE_INFO); + assertTrue(ti.isTupleType()); + assertEquals(2, ti.getArity()); + assertTrue(((TupleTypeInfo) ti).getTypeAt(0).isTupleType()); + assertEquals(((TupleTypeInfo) ti).getTypeAt(1), BasicTypeInfo.STRING_TYPE_INFO); } } @Test public void testFlatMapLambda() { - FlatMapFunction, Boolean>, Tuple2, String>> f = (i, o) -> {}; + FlatMapFunction, Boolean>, Tuple2, String>> f = (i, out) -> out.collect(null); - TypeInformation ti = TypeExtractor.getFlatMapReturnTypes(f, NESTED_TUPLE_BOOLEAN_TYPE); + TypeInformation ti = TypeExtractor.getFlatMapReturnTypes(f, NESTED_TUPLE_BOOLEAN_TYPE, null, true); if (!(ti instanceof MissingTypeInfo)) { - Assert.assertTrue(ti.isTupleType()); - Assert.assertEquals(2, ti.getArity()); - Assert.assertTrue(((TupleTypeInfo) ti).getTypeAt(0).isTupleType()); - Assert.assertEquals(((TupleTypeInfo) ti).getTypeAt(1), BasicTypeInfo.STRING_TYPE_INFO); + assertTrue(ti.isTupleType()); + assertEquals(2, ti.getArity()); + assertTrue(((TupleTypeInfo) ti).getTypeAt(0).isTupleType()); + assertEquals(((TupleTypeInfo) ti).getTypeAt(1), BasicTypeInfo.STRING_TYPE_INFO); } } @@ -179,38 +175,12 @@ public void testFlatMapLambda() { public void testMapPartitionLambda() { MapPartitionFunction, Boolean>, Tuple2, String>> f = (i, o) -> {}; - TypeInformation ti = TypeExtractor.getMapPartitionReturnTypes(f, NESTED_TUPLE_BOOLEAN_TYPE); - if (!(ti instanceof MissingTypeInfo)) { - Assert.assertTrue(ti.isTupleType()); - Assert.assertEquals(2, ti.getArity()); - Assert.assertTrue(((TupleTypeInfo) ti).getTypeAt(0).isTupleType()); - Assert.assertEquals(((TupleTypeInfo) ti).getTypeAt(1), BasicTypeInfo.STRING_TYPE_INFO); - } - } - - @Test - public void testGroupReduceLambda() { - GroupReduceFunction, Boolean>, Tuple2, String>> f = (i, o) -> {}; - - TypeInformation ti = TypeExtractor.getGroupReduceReturnTypes(f, NESTED_TUPLE_BOOLEAN_TYPE); - if (!(ti instanceof MissingTypeInfo)) { - Assert.assertTrue(ti.isTupleType()); - Assert.assertEquals(2, ti.getArity()); - Assert.assertTrue(((TupleTypeInfo) ti).getTypeAt(0).isTupleType()); - Assert.assertEquals(((TupleTypeInfo) ti).getTypeAt(1), BasicTypeInfo.STRING_TYPE_INFO); - } - } - - @Test - public void testFlatJoinLambda() { - FlatJoinFunction, Boolean>, Tuple2, Double>, Tuple2, String>> f = (i1, i2, o) -> {}; - - TypeInformation ti = TypeExtractor.getFlatJoinReturnTypes(f, NESTED_TUPLE_BOOLEAN_TYPE, NESTED_TUPLE_DOUBLE_TYPE); + TypeInformation ti = TypeExtractor.getMapPartitionReturnTypes(f, NESTED_TUPLE_BOOLEAN_TYPE, null, true); if (!(ti instanceof MissingTypeInfo)) { - Assert.assertTrue(ti.isTupleType()); - Assert.assertEquals(2, ti.getArity()); - Assert.assertTrue(((TupleTypeInfo) ti).getTypeAt(0).isTupleType()); - Assert.assertEquals(((TupleTypeInfo) ti).getTypeAt(1), BasicTypeInfo.STRING_TYPE_INFO); + assertTrue(ti.isTupleType()); + assertEquals(2, ti.getArity()); + assertTrue(((TupleTypeInfo) ti).getTypeAt(0).isTupleType()); + assertEquals(((TupleTypeInfo) ti).getTypeAt(1), BasicTypeInfo.STRING_TYPE_INFO); } } @@ -218,12 +188,12 @@ public void testFlatJoinLambda() { public void testJoinLambda() { JoinFunction, Boolean>, Tuple2, Double>, Tuple2, String>> f = (i1, i2) -> null; - TypeInformation ti = TypeExtractor.getJoinReturnTypes(f, NESTED_TUPLE_BOOLEAN_TYPE, NESTED_TUPLE_DOUBLE_TYPE); + TypeInformation ti = TypeExtractor.getJoinReturnTypes(f, NESTED_TUPLE_BOOLEAN_TYPE, NESTED_TUPLE_DOUBLE_TYPE, null, true); if (!(ti instanceof MissingTypeInfo)) { - Assert.assertTrue(ti.isTupleType()); - Assert.assertEquals(2, ti.getArity()); - Assert.assertTrue(((TupleTypeInfo) ti).getTypeAt(0).isTupleType()); - Assert.assertEquals(((TupleTypeInfo) ti).getTypeAt(1), BasicTypeInfo.STRING_TYPE_INFO); + assertTrue(ti.isTupleType()); + assertEquals(2, ti.getArity()); + assertTrue(((TupleTypeInfo) ti).getTypeAt(0).isTupleType()); + assertEquals(((TupleTypeInfo) ti).getTypeAt(1), BasicTypeInfo.STRING_TYPE_INFO); } } @@ -231,25 +201,12 @@ public void testJoinLambda() { public void testCoGroupLambda() { CoGroupFunction, Boolean>, Tuple2, Double>, Tuple2, String>> f = (i1, i2, o) -> {}; - TypeInformation ti = TypeExtractor.getCoGroupReturnTypes(f, NESTED_TUPLE_BOOLEAN_TYPE, NESTED_TUPLE_DOUBLE_TYPE); - if (!(ti instanceof MissingTypeInfo)) { - Assert.assertTrue(ti.isTupleType()); - Assert.assertEquals(2, ti.getArity()); - Assert.assertTrue(((TupleTypeInfo) ti).getTypeAt(0).isTupleType()); - Assert.assertEquals(((TupleTypeInfo) ti).getTypeAt(1), BasicTypeInfo.STRING_TYPE_INFO); - } - } - - @Test - public void testCrossLambda() { - CrossFunction, Boolean>, Tuple2, Double>, Tuple2, String>> f = (i1, i2) -> null; - - TypeInformation ti = TypeExtractor.getCrossReturnTypes(f, NESTED_TUPLE_BOOLEAN_TYPE, NESTED_TUPLE_DOUBLE_TYPE); + TypeInformation ti = TypeExtractor.getCoGroupReturnTypes(f, NESTED_TUPLE_BOOLEAN_TYPE, NESTED_TUPLE_DOUBLE_TYPE, null, true); if (!(ti instanceof MissingTypeInfo)) { - Assert.assertTrue(ti.isTupleType()); - Assert.assertEquals(2, ti.getArity()); - Assert.assertTrue(((TupleTypeInfo) ti).getTypeAt(0).isTupleType()); - Assert.assertEquals(((TupleTypeInfo) ti).getTypeAt(1), BasicTypeInfo.STRING_TYPE_INFO); + assertTrue(ti.isTupleType()); + assertEquals(2, ti.getArity()); + assertTrue(((TupleTypeInfo) ti).getTypeAt(0).isTupleType()); + assertEquals(((TupleTypeInfo) ti).getTypeAt(1), BasicTypeInfo.STRING_TYPE_INFO); } } @@ -257,12 +214,12 @@ public void testCrossLambda() { public void testKeySelectorLambda() { KeySelector, Boolean>, Tuple2, String>> f = (i) -> null; - TypeInformation ti = TypeExtractor.getKeySelectorTypes(f, NESTED_TUPLE_BOOLEAN_TYPE); + TypeInformation ti = TypeExtractor.getKeySelectorTypes(f, NESTED_TUPLE_BOOLEAN_TYPE, null, true); if (!(ti instanceof MissingTypeInfo)) { - Assert.assertTrue(ti.isTupleType()); - Assert.assertEquals(2, ti.getArity()); - Assert.assertTrue(((TupleTypeInfo) ti).getTypeAt(0).isTupleType()); - Assert.assertEquals(((TupleTypeInfo) ti).getTypeAt(1), BasicTypeInfo.STRING_TYPE_INFO); + assertTrue(ti.isTupleType()); + assertEquals(2, ti.getArity()); + assertTrue(((TupleTypeInfo) ti).getTypeAt(0).isTupleType()); + assertEquals(((TupleTypeInfo) ti).getTypeAt(1), BasicTypeInfo.STRING_TYPE_INFO); } } @@ -271,19 +228,20 @@ public void testKeySelectorLambda() { public void testLambdaTypeErasure() { MapFunction, Tuple1> f = (i) -> null; TypeInformation ti = TypeExtractor.getMapReturnTypes(f, new TypeHint>(){}.getTypeInfo(), null, true); - Assert.assertTrue(ti instanceof MissingTypeInfo); + assertTrue(ti instanceof MissingTypeInfo); } @Test public void testPartitionerLambda() { Partitioner> partitioner = (key, numPartitions) -> key.f1.length() % numPartitions; - final TypeInformation ti = TypeExtractor.getPartitionerTypes(partitioner); - - Assert.assertTrue(ti.isTupleType()); - Assert.assertEquals(2, ti.getArity()); - Assert.assertEquals(((TupleTypeInfo) ti).getTypeAt(0), BasicTypeInfo.INT_TYPE_INFO); - Assert.assertEquals(((TupleTypeInfo) ti).getTypeAt(1), BasicTypeInfo.STRING_TYPE_INFO); + final TypeInformation ti = TypeExtractor.getPartitionerTypes(partitioner, null, true); + if (!(ti instanceof MissingTypeInfo)) { + assertTrue(ti.isTupleType()); + assertEquals(2, ti.getArity()); + assertEquals(((TupleTypeInfo) ti).getTypeAt(0), BasicTypeInfo.INT_TYPE_INFO); + assertEquals(((TupleTypeInfo) ti).getTypeAt(1), BasicTypeInfo.STRING_TYPE_INFO); + } } private static class MyType { @@ -306,14 +264,14 @@ protected int getKey2() { public void testInstanceMethodRefSameType() { MapFunction f = MyType::getKey; TypeInformation ti = TypeExtractor.getMapReturnTypes(f, TypeExtractor.createTypeInfo(MyType.class)); - Assert.assertEquals(BasicTypeInfo.INT_TYPE_INFO, ti); + assertEquals(BasicTypeInfo.INT_TYPE_INFO, ti); } @Test public void testInstanceMethodRefSuperType() { MapFunction f = Object::toString; TypeInformation ti = TypeExtractor.getMapReturnTypes(f, BasicTypeInfo.INT_TYPE_INFO); - Assert.assertEquals(BasicTypeInfo.STRING_TYPE_INFO, ti); + assertEquals(BasicTypeInfo.STRING_TYPE_INFO, ti); } private static class MySubtype extends MyType { @@ -324,14 +282,14 @@ private static class MySubtype extends MyType { public void testInstanceMethodRefSuperTypeProtected() { MapFunction f = MyType::getKey2; TypeInformation ti = TypeExtractor.getMapReturnTypes(f, TypeExtractor.createTypeInfo(MySubtype.class)); - Assert.assertEquals(BasicTypeInfo.INT_TYPE_INFO, ti); + assertEquals(BasicTypeInfo.INT_TYPE_INFO, ti); } @Test public void testConstructorMethodRef() { MapFunction f = Integer::new; TypeInformation ti = TypeExtractor.getMapReturnTypes(f, BasicTypeInfo.STRING_TYPE_INFO); - Assert.assertEquals(BasicTypeInfo.INT_TYPE_INFO, ti); + assertEquals(BasicTypeInfo.INT_TYPE_INFO, ti); } private interface InterfaceWithDefaultMethod { @@ -356,7 +314,7 @@ private interface InterfaceWithMultipleMethods { } @Test(expected = InvalidTypesException.class) - public void getSingleAbstractMethodMultipleMethods() throws Exception { + public void getSingleAbstractMethodMultipleMethods() { TypeExtractionUtils.getSingleAbstractMethod(InterfaceWithMultipleMethods.class); } @@ -367,7 +325,7 @@ default void defaultMethod() { } @Test(expected = InvalidTypesException.class) - public void getSingleAbstractMethodNoAbstractMethods() throws Exception { + public void testSingleAbstractMethodNoAbstractMethods() { TypeExtractionUtils.getSingleAbstractMethod(InterfaceWithoutAbstractMethod.class); } @@ -376,8 +334,7 @@ private abstract class AbstractClassWithSingleAbstractMethod { } @Test(expected = InvalidTypesException.class) - public void getSingleAbstractMethodNotAnInterface() throws Exception { + public void testSingleAbstractMethodNotAnInterface() { TypeExtractionUtils.getSingleAbstractMethod(AbstractClassWithSingleAbstractMethod.class); } - } diff --git a/flink-examples/flink-examples-batch/src/main/java/org/apache/flink/examples/java/relational/TPCHQuery10.java b/flink-examples/flink-examples-batch/src/main/java/org/apache/flink/examples/java/relational/TPCHQuery10.java index c585e82b4ba1b..0874999b1c9cc 100644 --- a/flink-examples/flink-examples-batch/src/main/java/org/apache/flink/examples/java/relational/TPCHQuery10.java +++ b/flink-examples/flink-examples-batch/src/main/java/org/apache/flink/examples/java/relational/TPCHQuery10.java @@ -18,8 +18,7 @@ package org.apache.flink.examples.java.relational; -import org.apache.flink.api.common.functions.FilterFunction; -import org.apache.flink.api.common.functions.MapFunction; +import org.apache.flink.api.common.typeinfo.Types; import org.apache.flink.api.java.DataSet; import org.apache.flink.api.java.ExecutionEnvironment; import org.apache.flink.api.java.aggregation.Aggregations; @@ -118,33 +117,18 @@ public static void main(String[] args) throws Exception { // orders filtered by year: (orderkey, custkey) DataSet> ordersFilteredByYear = // filter by year - orders.filter( - new FilterFunction>() { - @Override - public boolean filter(Tuple3 o) { - return Integer.parseInt(o.f2.substring(0, 4)) > 1990; - } - }) + orders.filter(order -> Integer.parseInt(order.f2.substring(0, 4)) > 1990) // project fields out that are no longer required .project(0, 1); // lineitems filtered by flag: (orderkey, revenue) DataSet> lineitemsFilteredByFlag = // filter by flag - lineitems.filter(new FilterFunction>() { - @Override - public boolean filter(Tuple4 l) { - return l.f3.equals("R"); - } - }) + lineitems.filter(lineitem -> lineitem.f3.equals("R")) // compute revenue and project out return flag - .map(new MapFunction, Tuple2>() { - @Override - public Tuple2 map(Tuple4 l) { - // revenue per item = l_extendedprice * (1 - l_discount) - return new Tuple2(l.f0, l.f1 * (1 - l.f2)); - } - }); + // revenue per item = l_extendedprice * (1 - l_discount) + .map(lineitem -> new Tuple2<>(lineitem.f0, lineitem.f1 * (1 - lineitem.f2))) + .returns(Types.TUPLE(Types.INT, Types.DOUBLE)); // for lambda with generics // join orders with lineitems: (custkey, revenue) DataSet> revenueByCustomer = diff --git a/flink-examples/flink-examples-batch/src/main/java/org/apache/flink/examples/java/wordcount/WordCount.java b/flink-examples/flink-examples-batch/src/main/java/org/apache/flink/examples/java/wordcount/WordCount.java index c494c6f3e68be..2882fc7e908d4 100644 --- a/flink-examples/flink-examples-batch/src/main/java/org/apache/flink/examples/java/wordcount/WordCount.java +++ b/flink-examples/flink-examples-batch/src/main/java/org/apache/flink/examples/java/wordcount/WordCount.java @@ -43,7 +43,6 @@ * * */ -@SuppressWarnings("serial") public class WordCount { // ************************************************************************* @@ -110,7 +109,7 @@ public void flatMap(String value, Collector> out) { // emit the pairs for (String token : tokens) { if (token.length() > 0) { - out.collect(new Tuple2(token, 1)); + out.collect(new Tuple2<>(token, 1)); } } } diff --git a/flink-examples/flink-examples-streaming/src/main/java/org/apache/flink/streaming/examples/wordcount/WordCount.java b/flink-examples/flink-examples-streaming/src/main/java/org/apache/flink/streaming/examples/wordcount/WordCount.java index 9c7c435c51f28..ad34d71380333 100644 --- a/flink-examples/flink-examples-streaming/src/main/java/org/apache/flink/streaming/examples/wordcount/WordCount.java +++ b/flink-examples/flink-examples-streaming/src/main/java/org/apache/flink/streaming/examples/wordcount/WordCount.java @@ -72,10 +72,10 @@ public static void main(String[] args) throws Exception { } DataStream> counts = - // split up the lines in pairs (2-tuples) containing: (word,1) - text.flatMap(new Tokenizer()) - // group by the tuple field "0" and sum up tuple field "1" - .keyBy(0).sum(1); + // split up the lines in pairs (2-tuples) containing: (word,1) + text.flatMap(new Tokenizer()) + // group by the tuple field "0" and sum up tuple field "1" + .keyBy(0).sum(1); // emit result if (params.has("output")) { @@ -100,18 +100,16 @@ public static void main(String[] args) throws Exception { * Integer>}). */ public static final class Tokenizer implements FlatMapFunction> { - private static final long serialVersionUID = 1L; @Override - public void flatMap(String value, Collector> out) - throws Exception { + public void flatMap(String value, Collector> out) { // normalize and split the line String[] tokens = value.toLowerCase().split("\\W+"); // emit the pairs for (String token : tokens) { if (token.length() > 0) { - out.collect(new Tuple2(token, 1)); + out.collect(new Tuple2<>(token, 1)); } } } diff --git a/flink-java8/pom.xml b/flink-java8/pom.xml deleted file mode 100644 index a8750288f99c7..0000000000000 --- a/flink-java8/pom.xml +++ /dev/null @@ -1,225 +0,0 @@ - - - - - 4.0.0 - - - org.apache.flink - flink-parent - 1.6-SNAPSHOT - .. - - - flink-java8_${scala.binary.version} - flink-java8 - - jar - - - - - - - org.apache.flink - flink-java - ${project.version} - - - - org.apache.flink - flink-streaming-java_${scala.binary.version} - ${project.version} - - - - org.apache.flink - flink-examples-batch_${scala.binary.version} - ${project.version} - - - - - - org.apache.flink - flink-test-utils_${scala.binary.version} - ${project.version} - test - - - - org.apache.flink - flink-cep_${scala.binary.version} - ${project.version} - test - - - - - - - - org.apache.maven.plugins - maven-compiler-plugin - 3.1 - - 1.8 - 1.8 - - - - org.apache.maven.plugins - maven-surefire-plugin - - - WARN - - - - - - - org.apache.maven.plugins - maven-dependency-plugin - 2.9 - - - unpack - prepare-package - - unpack - - - - - org.apache.flink - flink-examples-batch_${scala.binary.version} - ${project.version} - jar - false - ${project.build.directory}/classes - org/apache/flink/examples/java/wordcount/util/WordCountData.class - - - - - - - - - org.apache.maven.plugins - maven-jar-plugin - - - - - test-jar - - - - - - - - - - - maven-compiler-plugin - - 1.8 - 1.8 - jdt - - - - - org.eclipse.tycho - tycho-compiler-jdt - 0.21.0 - - - - - - org.apache.maven.plugins - maven-deploy-plugin - 2.4 - - true - - - - - org.eclipse.m2e - lifecycle-mapping - 1.0.0 - - - - - - org.apache.maven.plugins - maven-assembly-plugin - [2.4,) - - single - - - - - - - - - org.apache.maven.plugins - maven-compiler-plugin - [3.1,) - - testCompile - compile - - - - - - - - - org.apache.maven.plugins - maven-dependency-plugin - [2.9,) - - unpack - - - - - - - - - - - - - - diff --git a/flink-java8/src/main/java/org/apache/flink/examples/java8/relational/TPCHQuery10.java b/flink-java8/src/main/java/org/apache/flink/examples/java8/relational/TPCHQuery10.java deleted file mode 100644 index c0fce4d084882..0000000000000 --- a/flink-java8/src/main/java/org/apache/flink/examples/java8/relational/TPCHQuery10.java +++ /dev/null @@ -1,212 +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.examples.java8.relational; - -import org.apache.flink.api.java.DataSet; -import org.apache.flink.api.java.ExecutionEnvironment; -import org.apache.flink.api.java.tuple.Tuple2; -import org.apache.flink.api.java.tuple.Tuple3; -import org.apache.flink.api.java.tuple.Tuple4; -import org.apache.flink.api.java.tuple.Tuple5; -import org.apache.flink.api.java.tuple.Tuple6; - -/** - * This program implements a modified version of the TPC-H query 10. - * The original query can be found at - * http://www.tpc.org/tpch/spec/tpch2.16.0.pdf (page 45). - * - *

    This program implements the following SQL equivalent: - * - *

    {@code
    - * SELECT
    - *        c_custkey,
    - *        c_name,
    - *        c_address,
    - *        n_name,
    - *        c_acctbal
    - *        SUM(l_extendedprice * (1 - l_discount)) AS revenue,
    - * FROM
    - *        customer,
    - *        orders,
    - *        lineitem,
    - *        nation
    - * WHERE
    - *        c_custkey = o_custkey
    - *        AND l_orderkey = o_orderkey
    - *        AND YEAR(o_orderdate) > '1990'
    - *        AND l_returnflag = 'R'
    - *        AND c_nationkey = n_nationkey
    - * GROUP BY
    - *        c_custkey,
    - *        c_name,
    - *        c_acctbal,
    - *        n_name,
    - *        c_address
    - * }
    - * - *

    Compared to the original TPC-H query this version does not print - * c_phone and c_comment, only filters by years greater than 1990 instead of - * a period of 3 months, and does not sort the result by revenue. - * - *

    Input files are plain text CSV files using the pipe character ('|') as field separator - * as generated by the TPC-H data generator which is available at http://www.tpc.org/tpch/. - * - *

    Usage: TPCHQuery10 <customer-csv path> <orders-csv path> <lineitem-csv path> <nation-csv path> <result path>
    - * - *

    This example shows how to use: - *

      - *
    • inline-defined functions using Java 8 Lambda Expressions - *
    - */ -public class TPCHQuery10 { - - // ************************************************************************* - // PROGRAM - // ************************************************************************* - - public static void main(String[] args) throws Exception { - - if (!parseParameters(args)) { - return; - } - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - // get customer data set: (custkey, name, address, nationkey, acctbal) - DataSet> customers = getCustomerDataSet(env); - - // get orders data set: (orderkey, custkey, orderdate) - DataSet> orders = getOrdersDataSet(env); - - // get lineitem data set: (orderkey, extendedprice, discount, returnflag) - DataSet> lineitems = getLineitemDataSet(env); - - // get nation data set: (nationkey, name) - DataSet> nations = getNationsDataSet(env); - - // orders filtered by year: (orderkey, custkey) - DataSet> ordersFilteredByYear = - // filter by year - orders.filter(order -> Integer.parseInt(order.f2.substring(0, 4)) > 1990) - // project fields out that are no longer required - .project(0, 1); - - // lineitems filtered by flag: (orderkey, extendedprice, discount) - DataSet> lineitemsFilteredByFlag = - // filter by flag - lineitems.filter(lineitem -> lineitem.f3.equals("R")) - // project fields out that are no longer required - .project(0, 1, 2); - - // join orders with lineitems: (custkey, extendedprice, discount) - DataSet> lineitemsOfCustomerKey = - ordersFilteredByYear.joinWithHuge(lineitemsFilteredByFlag) - .where(0).equalTo(0) - .projectFirst(1).projectSecond(1, 2); - - // aggregate for revenue: (custkey, revenue) - DataSet> revenueOfCustomerKey = lineitemsOfCustomerKey - // calculate the revenue for each item - // revenue per item = l_extendedprice * (1 - l_discount) - .map(i -> new Tuple2<>(i.f0, i.f1 * (1 - i.f2))) - // aggregate the revenues per item to revenue per customer - .groupBy(0).sum(1); - - // join customer with nation (custkey, name, address, nationname, acctbal) - DataSet> customerWithNation = customers - .joinWithTiny(nations) - .where(3).equalTo(0) - .projectFirst(0, 1, 2).projectSecond(1).projectFirst(4); - - // join customer (with nation) with revenue (custkey, name, address, nationname, acctbal, revenue) - DataSet> customerWithRevenue = - customerWithNation.join(revenueOfCustomerKey) - .where(0).equalTo(0) - .projectFirst(0, 1, 2, 3, 4).projectSecond(1); - - // emit result - customerWithRevenue.writeAsCsv(outputPath); - - // execute program - env.execute("TPCH Query 10 Example"); - - } - - // ************************************************************************* - // UTIL METHODS - // ************************************************************************* - - private static String customerPath; - private static String ordersPath; - private static String lineitemPath; - private static String nationPath; - private static String outputPath; - - private static boolean parseParameters(String[] programArguments) { - - if (programArguments.length > 0) { - if (programArguments.length == 5) { - customerPath = programArguments[0]; - ordersPath = programArguments[1]; - lineitemPath = programArguments[2]; - nationPath = programArguments[3]; - outputPath = programArguments[4]; - } else { - System.err.println("Usage: TPCHQuery10 "); - return false; - } - } else { - System.err.println("This program expects data from the TPC-H benchmark as input data.\n" + - " Due to legal restrictions, we can not ship generated data.\n" + - " You can find the TPC-H data generator at http://www.tpc.org/tpch/.\n" + - " Usage: TPCHQuery10 "); - return false; - } - return true; - } - - private static DataSet> getCustomerDataSet(ExecutionEnvironment env) { - return env.readCsvFile(customerPath) - .fieldDelimiter("|") - .includeFields("11110100") - .types(Integer.class, String.class, String.class, Integer.class, Double.class); - } - - private static DataSet> getOrdersDataSet(ExecutionEnvironment env) { - return env.readCsvFile(ordersPath) - .fieldDelimiter("|") - .includeFields("110010000") - .types(Integer.class, Integer.class, String.class); - } - - private static DataSet> getLineitemDataSet(ExecutionEnvironment env) { - return env.readCsvFile(lineitemPath) - .fieldDelimiter("|") - .includeFields("1000011010000000") - .types(Integer.class, Double.class, Double.class, String.class); - } - - private static DataSet> getNationsDataSet(ExecutionEnvironment env) { - return env.readCsvFile(nationPath) - .fieldDelimiter("|") - .includeFields("1100") - .types(Integer.class, String.class); - } - -} diff --git a/flink-java8/src/main/java/org/apache/flink/examples/java8/wordcount/WordCount.java b/flink-java8/src/main/java/org/apache/flink/examples/java8/wordcount/WordCount.java deleted file mode 100644 index 8f36f662e51e5..0000000000000 --- a/flink-java8/src/main/java/org/apache/flink/examples/java8/wordcount/WordCount.java +++ /dev/null @@ -1,124 +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.examples.java8.wordcount; - -import org.apache.flink.api.java.DataSet; -import org.apache.flink.api.java.ExecutionEnvironment; -import org.apache.flink.api.java.tuple.Tuple2; -import org.apache.flink.examples.java.wordcount.util.WordCountData; -import org.apache.flink.util.Collector; - -import java.util.Arrays; - -/** - * Implements the "WordCount" program that computes a simple word occurrence histogram - * over text files. - * - *

    The input is a plain text file with lines separated by newline characters. - * - *

    Usage: WordCount <text path> <result path>
    - * If no parameters are provided, the program is run with default data from {@link WordCountData}. - * - *

    This example shows how to: - *

      - *
    • write a compact Flink program with Java 8 Lambda Expressions. - *
    - * - */ -public class WordCount { - - // ************************************************************************* - // PROGRAM - // ************************************************************************* - - public static void main(String[] args) throws Exception { - - if (!parseParameters(args)) { - return; - } - - // set up the execution environment - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - // get input data - DataSet text = getTextDataSet(env); - - DataSet> counts = - // normalize and split each line - text.map(line -> line.toLowerCase().split("\\W+")) - // convert split line in pairs (2-tuples) containing: (word,1) - .flatMap((String[] tokens, Collector> out) -> { - // emit the pairs with non-zero-length words - Arrays.stream(tokens) - .filter(t -> t.length() > 0) - .forEach(t -> out.collect(new Tuple2<>(t, 1))); - }) - // group by the tuple field "0" and sum up tuple field "1" - .groupBy(0) - .sum(1); - - // emit result - if (fileOutput) { - counts.writeAsCsv(outputPath, "\n", " "); - } else { - counts.print(); - } - - // execute program - env.execute("WordCount Example"); - } - - // ************************************************************************* - // UTIL METHODS - // ************************************************************************* - - private static boolean fileOutput = false; - private static String textPath; - private static String outputPath; - - private static boolean parseParameters(String[] args) { - - if (args.length > 0) { - // parse input arguments - fileOutput = true; - if (args.length == 2) { - textPath = args[0]; - outputPath = args[1]; - } else { - System.err.println("Usage: WordCount "); - return false; - } - } else { - System.out.println("Executing WordCount example with built-in default data."); - System.out.println(" Provide parameters to read input data from a file."); - System.out.println(" Usage: WordCount "); - } - return true; - } - - private static DataSet getTextDataSet(ExecutionEnvironment env) { - if (fileOutput) { - // read the text file from given input path - return env.readTextFile(textPath); - } else { - // get default test text data - return WordCountData.getDefaultTextLineDataSet(env); - } - } -} diff --git a/flink-java8/src/main/java/org/apache/flink/streaming/examples/java8/wordcount/WordCount.java b/flink-java8/src/main/java/org/apache/flink/streaming/examples/java8/wordcount/WordCount.java deleted file mode 100644 index b9dba77f245e3..0000000000000 --- a/flink-java8/src/main/java/org/apache/flink/streaming/examples/java8/wordcount/WordCount.java +++ /dev/null @@ -1,124 +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.examples.java8.wordcount; - -import org.apache.flink.api.java.tuple.Tuple2; -import org.apache.flink.examples.java.wordcount.util.WordCountData; -import org.apache.flink.streaming.api.datastream.DataStream; -import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; -import org.apache.flink.util.Collector; - -import java.util.Arrays; - -/** - * Implements the streaming "WordCount" program that computes a simple word occurrences - * over text files. - * - *

    The input is a plain text file with lines separated by newline characters. - * - *

    Usage: WordCount <text path> <result path>
    - * If no parameters are provided, the program is run with default data from {@link WordCountData}. - * - *

    This example shows how to: - *

      - *
    • write a compact Flink Streaming program with Java 8 Lambda Expressions. - *
    - * - */ -public class WordCount { - - // ************************************************************************* - // PROGRAM - // ************************************************************************* - - public static void main(String[] args) throws Exception { - - if (!parseParameters(args)) { - return; - } - - // set up the execution environment - final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); - - // get input data - DataStream text = getTextDataStream(env); - - DataStream> counts = - // normalize and split each line - text.map(line -> line.toLowerCase().split("\\W+")) - // convert split line in pairs (2-tuples) containing: (word,1) - .flatMap((String[] tokens, Collector> out) -> { - // emit the pairs with non-zero-length words - Arrays.stream(tokens) - .filter(t -> t.length() > 0) - .forEach(t -> out.collect(new Tuple2<>(t, 1))); - }) - // group by the tuple field "0" and sum up tuple field "1" - .keyBy(0) - .sum(1); - - // emit result - if (fileOutput) { - counts.writeAsCsv(outputPath); - } else { - counts.print(); - } - - // execute program - env.execute("Streaming WordCount Example"); - } - - // ************************************************************************* - // UTIL METHODS - // ************************************************************************* - - private static boolean fileOutput = false; - private static String textPath; - private static String outputPath; - - private static boolean parseParameters(String[] args) { - - if (args.length > 0) { - // parse input arguments - fileOutput = true; - if (args.length == 2) { - textPath = args[0]; - outputPath = args[1]; - } else { - System.err.println("Usage: WordCount "); - return false; - } - } else { - System.out.println("Executing WordCount example with built-in default data."); - System.out.println(" Provide parameters to read input data from a file."); - System.out.println(" Usage: WordCount "); - } - return true; - } - - private static DataStream getTextDataStream(StreamExecutionEnvironment env) { - if (fileOutput) { - // read the text file from given input path - return env.readTextFile(textPath); - } else { - // get default test text data - return env.fromElements(WordCountData.WORDS); - } - } -} diff --git a/flink-java8/src/test/java/org/apache/flink/cep/CEPLambdaTest.java b/flink-java8/src/test/java/org/apache/flink/cep/CEPLambdaTest.java deleted file mode 100644 index 7cbdf6a8a0657..0000000000000 --- a/flink-java8/src/test/java/org/apache/flink/cep/CEPLambdaTest.java +++ /dev/null @@ -1,104 +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.cep; - -import org.apache.flink.api.common.typeinfo.TypeInformation; -import org.apache.flink.api.java.typeutils.TypeExtractor; -import org.apache.flink.cep.pattern.Pattern; -import org.apache.flink.streaming.api.datastream.DataStream; -import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; -import org.apache.flink.streaming.api.transformations.SourceTransformation; -import org.apache.flink.util.Collector; -import org.apache.flink.util.TestLogger; - -import org.junit.Test; - -import java.util.List; -import java.util.Map; - -import static org.junit.Assert.assertEquals; - -/** - * Tests for lambda support in CEP. - */ -public class CEPLambdaTest extends TestLogger { - /** - * Test event class. - */ - public static class EventA {} - - /** - * Test event class. - */ - public static class EventB {} - - /** - * Tests that a Java8 lambda can be passed as a CEP select function. - */ - @Test - public void testLambdaSelectFunction() { - TypeInformation eventTypeInformation = TypeExtractor.getForClass(EventA.class); - TypeInformation outputTypeInformation = TypeExtractor.getForClass(EventB.class); - - DataStream inputStream = new DataStream<>( - StreamExecutionEnvironment.getExecutionEnvironment(), - new SourceTransformation<>( - "source", - null, - eventTypeInformation, - 1)); - - Pattern dummyPattern = Pattern.begin("start"); - - PatternStream patternStream = new PatternStream<>(inputStream, dummyPattern); - - DataStream result = patternStream.select( - (Map> map) -> new EventB() - ); - - assertEquals(outputTypeInformation, result.getType()); - } - - /** - * Tests that a Java8 lambda can be passed as a CEP flat select function. - */ - @Test - public void testLambdaFlatSelectFunction() { - TypeInformation eventTypeInformation = TypeExtractor.getForClass(EventA.class); - TypeInformation outputTypeInformation = TypeExtractor.getForClass(EventB.class); - - DataStream inputStream = new DataStream<>( - StreamExecutionEnvironment.getExecutionEnvironment(), - new SourceTransformation<>( - "source", - null, - eventTypeInformation, - 1)); - - Pattern dummyPattern = Pattern.begin("start"); - - PatternStream patternStream = new PatternStream<>(inputStream, dummyPattern); - - DataStream result = patternStream.flatSelect( - (Map> map, Collector collector) -> collector.collect(new EventB()) - ); - - assertEquals(outputTypeInformation, result.getType()); - } -} diff --git a/flink-java8/src/test/java/org/apache/flink/runtime/util/JarFileCreatorLambdaTest.java b/flink-java8/src/test/java/org/apache/flink/runtime/util/JarFileCreatorLambdaTest.java deleted file mode 100644 index ca11275c5988a..0000000000000 --- a/flink-java8/src/test/java/org/apache/flink/runtime/util/JarFileCreatorLambdaTest.java +++ /dev/null @@ -1,113 +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.util; - -import org.apache.flink.runtime.util.jartestprogram.FilterLambda1; -import org.apache.flink.runtime.util.jartestprogram.FilterLambda2; -import org.apache.flink.runtime.util.jartestprogram.FilterLambda3; -import org.apache.flink.runtime.util.jartestprogram.FilterLambda4; - -import org.junit.Assert; -import org.junit.Test; - -import java.io.File; -import java.io.FileInputStream; -import java.util.HashSet; -import java.util.Set; -import java.util.jar.JarInputStream; -import java.util.zip.ZipEntry; - -/** - * Tests for the {@link JarFileCreator}. - */ -public class JarFileCreatorLambdaTest { - @Test - public void testFilterFunctionOnLambda1() throws Exception { - File out = new File(System.getProperty("java.io.tmpdir"), "jarcreatortest.jar"); - JarFileCreator jfc = new JarFileCreator(out); - jfc.addClass(FilterLambda1.class) - .createJarFile(); - - Set ans = new HashSet(); - ans.add("org/apache/flink/runtime/util/jartestprogram/FilterLambda1.class"); - ans.add("org/apache/flink/runtime/util/jartestprogram/WordFilter.class"); - - Assert.assertTrue("Jar file for java 8 lambda is not correct", validate(ans, out)); - out.delete(); - } - - @Test - public void testFilterFunctionOnLambda2() throws Exception{ - File out = new File(System.getProperty("java.io.tmpdir"), "jarcreatortest.jar"); - JarFileCreator jfc = new JarFileCreator(out); - jfc.addClass(FilterLambda2.class) - .createJarFile(); - - Set ans = new HashSet(); - ans.add("org/apache/flink/runtime/util/jartestprogram/FilterLambda2.class"); - ans.add("org/apache/flink/runtime/util/jartestprogram/WordFilter.class"); - - Assert.assertTrue("Jar file for java 8 lambda is not correct", validate(ans, out)); - out.delete(); - } - - @Test - public void testFilterFunctionOnLambda3() throws Exception { - File out = new File(System.getProperty("java.io.tmpdir"), "jarcreatortest.jar"); - JarFileCreator jfc = new JarFileCreator(out); - jfc.addClass(FilterLambda3.class) - .createJarFile(); - - Set ans = new HashSet(); - ans.add("org/apache/flink/runtime/util/jartestprogram/FilterLambda3.class"); - ans.add("org/apache/flink/runtime/util/jartestprogram/WordFilter.class"); - ans.add("org/apache/flink/runtime/util/jartestprogram/UtilFunction.class"); - - Assert.assertTrue("Jar file for java 8 lambda is not correct", validate(ans, out)); - out.delete(); - } - - @Test - public void testFilterFunctionOnLambda4() throws Exception { - File out = new File(System.getProperty("java.io.tmpdir"), "jarcreatortest.jar"); - JarFileCreator jfc = new JarFileCreator(out); - jfc.addClass(FilterLambda4.class) - .createJarFile(); - - Set ans = new HashSet(); - ans.add("org/apache/flink/runtime/util/jartestprogram/FilterLambda4.class"); - ans.add("org/apache/flink/runtime/util/jartestprogram/WordFilter.class"); - ans.add("org/apache/flink/runtime/util/jartestprogram/UtilFunctionWrapper$UtilFunction.class"); - - Assert.assertTrue("Jar file for java 8 lambda is not correct", validate(ans, out)); - out.delete(); - } - - public boolean validate(Set expected, File out) throws Exception { - int count = expected.size(); - try (JarInputStream jis = new JarInputStream(new FileInputStream(out))) { - ZipEntry ze; - while ((ze = jis.getNextEntry()) != null) { - count--; - expected.remove(ze.getName()); - } - } - return count == 0 && expected.size() == 0; - } -} diff --git a/flink-java8/src/test/java/org/apache/flink/runtime/util/jartestprogram/FilterLambda3.java b/flink-java8/src/test/java/org/apache/flink/runtime/util/jartestprogram/FilterLambda3.java deleted file mode 100644 index b4937225af8d1..0000000000000 --- a/flink-java8/src/test/java/org/apache/flink/runtime/util/jartestprogram/FilterLambda3.java +++ /dev/null @@ -1,39 +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.util.jartestprogram; - -import org.apache.flink.api.java.DataSet; -import org.apache.flink.api.java.ExecutionEnvironment; - -/** - * Similar to {@link FilterLambda2}, but uses a getter to retrieve a lambda filter instance. - */ -public class FilterLambda3 { - - public static void main(String[] args) throws Exception { - ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - DataSet input = env.fromElements("Please filter", "the words", "but not this"); - - DataSet output = input.filter(UtilFunction.getWordFilter()); - output.print(); - - env.execute(); - } - -} diff --git a/flink-java8/src/test/java/org/apache/flink/test/api/java/operators/lambdas/AllGroupReduceITCase.java b/flink-java8/src/test/java/org/apache/flink/test/api/java/operators/lambdas/AllGroupReduceITCase.java deleted file mode 100644 index cee34af9a74ce..0000000000000 --- a/flink-java8/src/test/java/org/apache/flink/test/api/java/operators/lambdas/AllGroupReduceITCase.java +++ /dev/null @@ -1,59 +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.test.api.java.operators.lambdas; - -import org.apache.flink.api.java.DataSet; -import org.apache.flink.api.java.ExecutionEnvironment; -import org.apache.flink.test.util.JavaProgramTestBase; - -/** - * IT cases for lambda allreduce functions. - */ -public class AllGroupReduceITCase extends JavaProgramTestBase { - - private static final String EXPECTED_RESULT = "aaabacad\n"; - - private String resultPath; - - @Override - protected void preSubmit() throws Exception { - resultPath = getTempDirPath("result"); - } - - @Override - protected void testProgram() throws Exception { - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - DataSet stringDs = env.fromElements("aa", "ab", "ac", "ad"); - DataSet concatDs = stringDs.reduceGroup((values, out) -> { - String conc = ""; - for (String s : values) { - conc = conc.concat(s); - } - out.collect(conc); - }); - concatDs.writeAsText(resultPath); - env.execute(); - } - - @Override - protected void postSubmit() throws Exception { - compareResultsByLinesInMemory(EXPECTED_RESULT, resultPath); - } -} diff --git a/flink-java8/src/test/java/org/apache/flink/test/api/java/operators/lambdas/CoGroupITCase.java b/flink-java8/src/test/java/org/apache/flink/test/api/java/operators/lambdas/CoGroupITCase.java deleted file mode 100644 index a70f37a5dcd46..0000000000000 --- a/flink-java8/src/test/java/org/apache/flink/test/api/java/operators/lambdas/CoGroupITCase.java +++ /dev/null @@ -1,74 +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.test.api.java.operators.lambdas; - -import org.apache.flink.api.java.DataSet; -import org.apache.flink.api.java.ExecutionEnvironment; -import org.apache.flink.api.java.tuple.Tuple2; -import org.apache.flink.test.util.JavaProgramTestBase; - -/** - * IT cases for lambda cogroup functions. - */ -public class CoGroupITCase extends JavaProgramTestBase { - - private static final String EXPECTED_RESULT = "6\n3\n"; - - private String resultPath; - - @Override - protected void preSubmit() throws Exception { - resultPath = getTempDirPath("result"); - } - - @SuppressWarnings("unchecked") - @Override - protected void testProgram() throws Exception { - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - DataSet> left = env.fromElements( - new Tuple2(1, "hello"), - new Tuple2(2, "what's"), - new Tuple2(2, "up") - ); - DataSet> right = env.fromElements( - new Tuple2(1, "not"), - new Tuple2(1, "much"), - new Tuple2(2, "really") - ); - DataSet joined = left.coGroup(right).where(0).equalTo(0) - .with((values1, values2, out) -> { - int sum = 0; - for (Tuple2 next : values1) { - sum += next.f0; - } - for (Tuple2 next : values2) { - sum += next.f0; - } - out.collect(sum); - }); - joined.writeAsText(resultPath); - env.execute(); - } - - @Override - protected void postSubmit() throws Exception { - compareResultsByLinesInMemory(EXPECTED_RESULT, resultPath); - } -} diff --git a/flink-java8/src/test/java/org/apache/flink/test/api/java/operators/lambdas/CrossITCase.java b/flink-java8/src/test/java/org/apache/flink/test/api/java/operators/lambdas/CrossITCase.java deleted file mode 100644 index 32cd910614343..0000000000000 --- a/flink-java8/src/test/java/org/apache/flink/test/api/java/operators/lambdas/CrossITCase.java +++ /dev/null @@ -1,73 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.flink.test.api.java.operators.lambdas; - -import org.apache.flink.api.java.DataSet; -import org.apache.flink.api.java.ExecutionEnvironment; -import org.apache.flink.api.java.tuple.Tuple2; -import org.apache.flink.test.util.JavaProgramTestBase; - -/** - * IT cases for lambda cross functions. - */ -public class CrossITCase extends JavaProgramTestBase { - - private static final String EXPECTED_RESULT = "2,hello not\n" + - "3,what's not\n" + - "3,up not\n" + - "2,hello much\n" + - "3,what's much\n" + - "3,up much\n" + - "3,hello really\n" + - "4,what's really\n" + - "4,up really"; - - private String resultPath; - - @Override - protected void preSubmit() throws Exception { - resultPath = getTempDirPath("result"); - } - - @SuppressWarnings("unchecked") - @Override - protected void testProgram() throws Exception { - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - DataSet> left = env.fromElements( - new Tuple2(1, "hello"), - new Tuple2(2, "what's"), - new Tuple2(2, "up") - ); - DataSet> right = env.fromElements( - new Tuple2(1, "not"), - new Tuple2(1, "much"), - new Tuple2(2, "really") - ); - DataSet> joined = left.cross(right) - .with((t, s) -> new Tuple2<> (t.f0 + s.f0, t.f1 + " " + s.f1)); - joined.writeAsCsv(resultPath); - env.execute(); - } - - @Override - protected void postSubmit() throws Exception { - compareResultsByLinesInMemory(EXPECTED_RESULT, resultPath); - } -} diff --git a/flink-java8/src/test/java/org/apache/flink/test/api/java/operators/lambdas/FilterITCase.java b/flink-java8/src/test/java/org/apache/flink/test/api/java/operators/lambdas/FilterITCase.java deleted file mode 100644 index 6ad1058545715..0000000000000 --- a/flink-java8/src/test/java/org/apache/flink/test/api/java/operators/lambdas/FilterITCase.java +++ /dev/null @@ -1,91 +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.test.api.java.operators.lambdas; - -import org.apache.flink.api.java.DataSet; -import org.apache.flink.api.java.ExecutionEnvironment; -import org.apache.flink.api.java.tuple.Tuple3; -import org.apache.flink.test.util.JavaProgramTestBase; - -import java.util.ArrayList; -import java.util.Collections; -import java.util.List; - -/** - * IT cases for lambda filter functions. - */ -public class FilterITCase extends JavaProgramTestBase { - - private static final String EXPECTED_RESULT = "3,2,Hello world\n" + - "4,3,Hello world, how are you?\n"; - - public static DataSet> get3TupleDataSet(ExecutionEnvironment env) { - - List> data = new ArrayList>(); - data.add(new Tuple3<>(1, 1L, "Hi")); - data.add(new Tuple3<>(2, 2L, "Hello")); - data.add(new Tuple3<>(3, 2L, "Hello world")); - data.add(new Tuple3<>(4, 3L, "Hello world, how are you?")); - data.add(new Tuple3<>(5, 3L, "I am fine.")); - data.add(new Tuple3<>(6, 3L, "Luke Skywalker")); - data.add(new Tuple3<>(7, 4L, "Comment#1")); - data.add(new Tuple3<>(8, 4L, "Comment#2")); - data.add(new Tuple3<>(9, 4L, "Comment#3")); - data.add(new Tuple3<>(10, 4L, "Comment#4")); - data.add(new Tuple3<>(11, 5L, "Comment#5")); - data.add(new Tuple3<>(12, 5L, "Comment#6")); - data.add(new Tuple3<>(13, 5L, "Comment#7")); - data.add(new Tuple3<>(14, 5L, "Comment#8")); - data.add(new Tuple3<>(15, 5L, "Comment#9")); - data.add(new Tuple3<>(16, 6L, "Comment#10")); - data.add(new Tuple3<>(17, 6L, "Comment#11")); - data.add(new Tuple3<>(18, 6L, "Comment#12")); - data.add(new Tuple3<>(19, 6L, "Comment#13")); - data.add(new Tuple3<>(20, 6L, "Comment#14")); - data.add(new Tuple3<>(21, 6L, "Comment#15")); - - Collections.shuffle(data); - - return env.fromCollection(data); - } - - private String resultPath; - - @Override - protected void preSubmit() throws Exception { - resultPath = getTempDirPath("result"); - } - - @Override - protected void testProgram() throws Exception { - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - DataSet> ds = get3TupleDataSet(env); - DataSet> filterDs = ds. - filter(value -> value.f2.contains("world")); - filterDs.writeAsCsv(resultPath); - env.execute(); - } - - @Override - protected void postSubmit() throws Exception { - compareResultsByLinesInMemory(EXPECTED_RESULT, resultPath); - } -} - diff --git a/flink-java8/src/test/java/org/apache/flink/test/api/java/operators/lambdas/FlatJoinITCase.java b/flink-java8/src/test/java/org/apache/flink/test/api/java/operators/lambdas/FlatJoinITCase.java deleted file mode 100644 index f7934507781e3..0000000000000 --- a/flink-java8/src/test/java/org/apache/flink/test/api/java/operators/lambdas/FlatJoinITCase.java +++ /dev/null @@ -1,68 +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.test.api.java.operators.lambdas; - -import org.apache.flink.api.java.DataSet; -import org.apache.flink.api.java.ExecutionEnvironment; -import org.apache.flink.api.java.tuple.Tuple2; -import org.apache.flink.test.util.JavaProgramTestBase; - -/** - * IT cases for lambda join functions. - */ -public class FlatJoinITCase extends JavaProgramTestBase { - - private static final String EXPECTED_RESULT = "2,what's really\n" + - "2,up really\n" + - "1,hello not\n" + - "1,hello much\n"; - - private String resultPath; - - @Override - protected void preSubmit() throws Exception { - resultPath = getTempDirPath("result"); - } - - @SuppressWarnings("unchecked") - @Override - protected void testProgram() throws Exception { - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - DataSet> left = env.fromElements( - new Tuple2(1, "hello"), - new Tuple2(2, "what's"), - new Tuple2(2, "up") - ); - DataSet> right = env.fromElements( - new Tuple2(1, "not"), - new Tuple2(1, "much"), - new Tuple2(2, "really") - ); - DataSet> joined = left.join(right).where(0).equalTo(0) - .with((t, s, out) -> out.collect(new Tuple2(t.f0, t.f1 + " " + s.f1))); - joined.writeAsCsv(resultPath); - env.execute(); - } - - @Override - protected void postSubmit() throws Exception { - compareResultsByLinesInMemory(EXPECTED_RESULT, resultPath); - } -} diff --git a/flink-java8/src/test/java/org/apache/flink/test/api/java/operators/lambdas/FlatMapITCase.java b/flink-java8/src/test/java/org/apache/flink/test/api/java/operators/lambdas/FlatMapITCase.java deleted file mode 100644 index d395d7da4b412..0000000000000 --- a/flink-java8/src/test/java/org/apache/flink/test/api/java/operators/lambdas/FlatMapITCase.java +++ /dev/null @@ -1,56 +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.test.api.java.operators.lambdas; - -import org.apache.flink.api.java.DataSet; -import org.apache.flink.api.java.ExecutionEnvironment; -import org.apache.flink.test.util.JavaProgramTestBase; - -/** - * IT cases for lambda flatmap functions. - */ -public class FlatMapITCase extends JavaProgramTestBase { - - private static final String EXPECTED_RESULT = "bb\n" + - "bb\n" + - "bc\n" + - "bd\n"; - - private String resultPath; - - @Override - protected void preSubmit() throws Exception { - resultPath = getTempDirPath("result"); - } - - @Override - protected void testProgram() throws Exception { - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - DataSet stringDs = env.fromElements("aa", "ab", "ac", "ad"); - DataSet flatMappedDs = stringDs.flatMap((s, out) -> out.collect(s.replace("a", "b"))); - flatMappedDs.writeAsText(resultPath); - env.execute(); - } - - @Override - protected void postSubmit() throws Exception { - compareResultsByLinesInMemory(EXPECTED_RESULT, resultPath); - } -} diff --git a/flink-java8/src/test/java/org/apache/flink/test/api/java/operators/lambdas/GroupReduceITCase.java b/flink-java8/src/test/java/org/apache/flink/test/api/java/operators/lambdas/GroupReduceITCase.java deleted file mode 100644 index 53db5415bd63e..0000000000000 --- a/flink-java8/src/test/java/org/apache/flink/test/api/java/operators/lambdas/GroupReduceITCase.java +++ /dev/null @@ -1,69 +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.test.api.java.operators.lambdas; - -import org.apache.flink.api.java.DataSet; -import org.apache.flink.api.java.ExecutionEnvironment; -import org.apache.flink.api.java.tuple.Tuple2; -import org.apache.flink.test.util.JavaProgramTestBase; - -/** - * IT cases for lambda groupreduce functions. - */ -public class GroupReduceITCase extends JavaProgramTestBase { - - private static final String EXPECTED_RESULT = "abad\n" + - "aaac\n"; - - private String resultPath; - - @Override - protected void preSubmit() throws Exception { - resultPath = getTempDirPath("result"); - } - - @SuppressWarnings("unchecked") - @Override - protected void testProgram() throws Exception { - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - DataSet> stringDs = env.fromElements( - new Tuple2<>(1, "aa"), - new Tuple2<>(2, "ab"), - new Tuple2<>(1, "ac"), - new Tuple2<>(2, "ad") - ); - DataSet concatDs = stringDs - .groupBy(0) - .reduceGroup((values, out) -> { - String conc = ""; - for (Tuple2 next : values) { - conc = conc.concat(next.f1); - } - out.collect(conc); - }); - concatDs.writeAsText(resultPath); - env.execute(); - } - - @Override - protected void postSubmit() throws Exception { - compareResultsByLinesInMemory(EXPECTED_RESULT, resultPath); - } -} diff --git a/flink-java8/src/test/java/org/apache/flink/test/api/java/operators/lambdas/JoinITCase.java b/flink-java8/src/test/java/org/apache/flink/test/api/java/operators/lambdas/JoinITCase.java deleted file mode 100644 index d86ea4969607d..0000000000000 --- a/flink-java8/src/test/java/org/apache/flink/test/api/java/operators/lambdas/JoinITCase.java +++ /dev/null @@ -1,69 +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.test.api.java.operators.lambdas; - -import org.apache.flink.api.java.DataSet; -import org.apache.flink.api.java.ExecutionEnvironment; -import org.apache.flink.api.java.tuple.Tuple2; -import org.apache.flink.test.util.JavaProgramTestBase; - -/** - * IT cases for lambda join functions. - */ -public class JoinITCase extends JavaProgramTestBase { - - private static final String EXPECTED_RESULT = "2,what's really\n" + - "2,up really\n" + - "1,hello not\n" + - "1,hello much\n"; - - private String resultPath; - - @Override - protected void preSubmit() throws Exception { - resultPath = getTempDirPath("result"); - } - - @SuppressWarnings("unchecked") - @Override - protected void testProgram() throws Exception { - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - DataSet> left = env.fromElements( - new Tuple2(1, "hello"), - new Tuple2(2, "what's"), - new Tuple2(2, "up") - ); - DataSet> right = env.fromElements( - new Tuple2(1, "not"), - new Tuple2(1, "much"), - new Tuple2(2, "really") - ); - DataSet> joined = left.join(right).where(0).equalTo(0) - .with((t, s) -> new Tuple2<>(t.f0, t.f1 + " " + s.f1)); - joined.writeAsCsv(resultPath); - env.execute(); - } - - @Override - protected void postSubmit() throws Exception { - compareResultsByLinesInMemory(EXPECTED_RESULT, resultPath); - } -} - diff --git a/flink-java8/src/test/java/org/apache/flink/test/api/java/operators/lambdas/MapITCase.java b/flink-java8/src/test/java/org/apache/flink/test/api/java/operators/lambdas/MapITCase.java deleted file mode 100644 index 15a9b9d91df57..0000000000000 --- a/flink-java8/src/test/java/org/apache/flink/test/api/java/operators/lambdas/MapITCase.java +++ /dev/null @@ -1,74 +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.test.api.java.operators.lambdas; - -import org.apache.flink.api.java.DataSet; -import org.apache.flink.api.java.ExecutionEnvironment; -import org.apache.flink.test.util.JavaProgramTestBase; - -/** - * IT cases for lambda map functions. - */ -public class MapITCase extends JavaProgramTestBase { - - private static class Trade { - - public String v; - - public Trade(String v) { - this.v = v; - } - - @Override - public String toString() { - return v; - } - } - - private static final String EXPECTED_RESULT = "22\n" + - "22\n" + - "23\n" + - "24\n"; - - private String resultPath; - - @Override - protected void preSubmit() throws Exception { - resultPath = getTempDirPath("result"); - } - - @Override - protected void testProgram() throws Exception { - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - DataSet stringDs = env.fromElements(11, 12, 13, 14); - DataSet mappedDs = stringDs - .map(Object::toString) - .map (s -> s.replace("1", "2")) - .map(Trade::new) - .map(Trade::toString); - mappedDs.writeAsText(resultPath); - env.execute(); - } - - @Override - protected void postSubmit() throws Exception { - compareResultsByLinesInMemory(EXPECTED_RESULT, resultPath); - } -} diff --git a/flink-java8/src/test/java/org/apache/flink/test/api/java/operators/lambdas/ReduceITCase.java b/flink-java8/src/test/java/org/apache/flink/test/api/java/operators/lambdas/ReduceITCase.java deleted file mode 100644 index 712132cad87e1..0000000000000 --- a/flink-java8/src/test/java/org/apache/flink/test/api/java/operators/lambdas/ReduceITCase.java +++ /dev/null @@ -1,109 +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.test.api.java.operators.lambdas; - -import org.apache.flink.api.common.typeinfo.BasicTypeInfo; -import org.apache.flink.api.java.DataSet; -import org.apache.flink.api.java.ExecutionEnvironment; -import org.apache.flink.api.java.tuple.Tuple5; -import org.apache.flink.api.java.typeutils.TupleTypeInfo; -import org.apache.flink.test.util.JavaProgramTestBase; - -import java.util.ArrayList; -import java.util.Collections; -import java.util.List; - -/** - * IT cases for lambda reduce functions. - */ -public class ReduceITCase extends JavaProgramTestBase { - - private static final String EXPECTED_RESULT = "1,1,0,Hallo,1\n" + - "2,3,2,Hallo Welt wie,1\n" + - "2,2,1,Hallo Welt,2\n" + - "3,9,0,P-),2\n" + - "3,6,5,BCD,3\n" + - "4,17,0,P-),1\n" + - "4,17,0,P-),2\n" + - "5,11,10,GHI,1\n" + - "5,29,0,P-),2\n" + - "5,25,0,P-),3\n"; - - public static DataSet> get5TupleDataSet(ExecutionEnvironment env) { - - List> data = new ArrayList>(); - data.add(new Tuple5<>(1, 1L, 0, "Hallo", 1L)); - data.add(new Tuple5<>(2, 2L, 1, "Hallo Welt", 2L)); - data.add(new Tuple5<>(2, 3L, 2, "Hallo Welt wie", 1L)); - data.add(new Tuple5<>(3, 4L, 3, "Hallo Welt wie gehts?", 2L)); - data.add(new Tuple5<>(3, 5L, 4, "ABC", 2L)); - data.add(new Tuple5<>(3, 6L, 5, "BCD", 3L)); - data.add(new Tuple5<>(4, 7L, 6, "CDE", 2L)); - data.add(new Tuple5<>(4, 8L, 7, "DEF", 1L)); - data.add(new Tuple5<>(4, 9L, 8, "EFG", 1L)); - data.add(new Tuple5<>(4, 10L, 9, "FGH", 2L)); - data.add(new Tuple5<>(5, 11L, 10, "GHI", 1L)); - data.add(new Tuple5<>(5, 12L, 11, "HIJ", 3L)); - data.add(new Tuple5<>(5, 13L, 12, "IJK", 3L)); - data.add(new Tuple5<>(5, 14L, 13, "JKL", 2L)); - data.add(new Tuple5<>(5, 15L, 14, "KLM", 2L)); - - Collections.shuffle(data); - - TupleTypeInfo> type = new - TupleTypeInfo>( - BasicTypeInfo.INT_TYPE_INFO, - BasicTypeInfo.LONG_TYPE_INFO, - BasicTypeInfo.INT_TYPE_INFO, - BasicTypeInfo.STRING_TYPE_INFO, - BasicTypeInfo.LONG_TYPE_INFO - ); - - return env.fromCollection(data, type); - } - - private String resultPath; - - @Override - protected void preSubmit() throws Exception { - resultPath = getTempDirPath("result"); - } - - @Override - protected void testProgram() throws Exception { - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - DataSet> ds = get5TupleDataSet(env); - DataSet> reduceDs = ds - .groupBy(4, 0) - .reduce((in1, in2) -> { - Tuple5 out = new Tuple5(); - out.setFields(in1.f0, in1.f1 + in2.f1, 0, "P-)", in1.f4); - return out; - }); - - reduceDs.writeAsCsv(resultPath); - env.execute(); - } - - @Override - protected void postSubmit() throws Exception { - compareResultsByLinesInMemory(EXPECTED_RESULT, resultPath); - } -} diff --git a/flink-java8/src/test/resources/log4j-test.properties b/flink-java8/src/test/resources/log4j-test.properties deleted file mode 100644 index c977d4c3ae39f..0000000000000 --- a/flink-java8/src/test/resources/log4j-test.properties +++ /dev/null @@ -1,19 +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. -################################################################################ - -log4j.rootLogger=OFF diff --git a/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/PatternStream.java b/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/PatternStream.java index 0e6c2febe0979..521665f9a631f 100644 --- a/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/PatternStream.java +++ b/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/PatternStream.java @@ -104,8 +104,7 @@ public SingleOutputStreamOperator select(final PatternSelectFunction SingleOutputStreamOperator select( PatternSelectFunction.class, 0, 1, - new int[]{0, 1, 0}, - new int[]{}, + TypeExtractor.NO_INDEX, inputStream.getType(), null, false); @@ -259,8 +257,7 @@ public SingleOutputStreamOperator> select( PatternSelectFunction.class, 0, 1, - new int[]{0, 1, 0}, - new int[]{}, + TypeExtractor.NO_INDEX, inputStream.getType(), null, false); @@ -270,8 +267,7 @@ public SingleOutputStreamOperator> select( PatternTimeoutFunction.class, 0, 1, - new int[]{0, 1, 0}, - new int[]{}, + TypeExtractor.NO_INDEX, inputStream.getType(), null, false); @@ -314,7 +310,6 @@ public SingleOutputStreamOperator flatSelect(final PatternFlatSelectFunct PatternFlatSelectFunction.class, 0, 1, - new int[] {0, 1, 0}, new int[] {1, 0}, inputStream.getType(), null, @@ -381,7 +376,6 @@ public SingleOutputStreamOperator flatSelect( PatternFlatSelectFunction.class, 0, 1, - new int[]{0, 1, 0}, new int[]{1, 0}, inputStream.getType(), null, @@ -465,7 +459,6 @@ public SingleOutputStreamOperator> flatSelect( PatternFlatTimeoutFunction.class, 0, 1, - new int[]{0, 1, 0}, new int[]{2, 0}, inputStream.getType(), null, @@ -476,7 +469,6 @@ public SingleOutputStreamOperator> flatSelect( PatternFlatSelectFunction.class, 0, 1, - new int[]{0, 1, 0}, new int[]{1, 0}, inputStream.getType(), null, diff --git a/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/CEPITCase.java b/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/CEPITCase.java index 6d1013ca810b6..e397d318241b0 100644 --- a/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/CEPITCase.java +++ b/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/CEPITCase.java @@ -19,6 +19,7 @@ package org.apache.flink.cep; import org.apache.flink.api.common.functions.MapFunction; +import org.apache.flink.api.common.typeinfo.Types; import org.apache.flink.api.java.functions.KeySelector; import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.cep.nfa.NFA; @@ -96,19 +97,15 @@ public boolean filter(Event value) throws Exception { } }); - DataStream result = CEP.pattern(input, pattern).select(new PatternSelectFunction() { + DataStream result = CEP.pattern(input, pattern).flatSelect((p, o) -> { + StringBuilder builder = new StringBuilder(); - @Override - public String select(Map> pattern) { - StringBuilder builder = new StringBuilder(); + builder.append(p.get("start").get(0).getId()).append(",") + .append(p.get("middle").get(0).getId()).append(",") + .append(p.get("end").get(0).getId()); - builder.append(pattern.get("start").get(0).getId()).append(",") - .append(pattern.get("middle").get(0).getId()).append(",") - .append(pattern.get("end").get(0).getId()); - - return builder.toString(); - } - }); + o.collect(builder.toString()); + }, Types.STRING); List resultList = new ArrayList<>(); @@ -170,18 +167,14 @@ public boolean filter(Event value) throws Exception { } }); - DataStream result = CEP.pattern(input, pattern).select(new PatternSelectFunction() { - - @Override - public String select(Map> pattern) { - StringBuilder builder = new StringBuilder(); + DataStream result = CEP.pattern(input, pattern).select(p -> { + StringBuilder builder = new StringBuilder(); - builder.append(pattern.get("start").get(0).getId()).append(",") - .append(pattern.get("middle").get(0).getId()).append(",") - .append(pattern.get("end").get(0).getId()); + builder.append(p.get("start").get(0).getId()).append(",") + .append(p.get("middle").get(0).getId()).append(",") + .append(p.get("end").get(0).getId()); - return builder.toString(); - } + return builder.toString(); }); List resultList = new ArrayList<>(); diff --git a/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/asm/translate/Translate.java b/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/asm/translate/Translate.java index 6dcf766cce397..95310b4568a74 100644 --- a/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/asm/translate/Translate.java +++ b/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/asm/translate/Translate.java @@ -78,7 +78,6 @@ public static DataSet> translateVertexIds(DataSet TranslateFunction.class, 0, 1, - new int[]{0}, new int[]{1}, oldType, null, @@ -162,7 +161,6 @@ public static DataSet> translateEdgeIds(DataSet DataSet> translateVertexValues(DataSe TranslateFunction.class, 0, 1, - new int[]{0}, new int[]{1}, oldType, null, @@ -332,7 +329,6 @@ public static DataSet> translateEdgeValues(DataSet - - - org.eclipse.m2e diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/util/JarFileCreatorTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/util/JarFileCreatorTest.java index 8f8016e14c6fa..f5d07de0492c8 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/util/JarFileCreatorTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/util/JarFileCreatorTest.java @@ -1,4 +1,4 @@ -/** +/* * 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 @@ -16,9 +16,11 @@ * limitations under the License. */ - package org.apache.flink.runtime.util; +import org.apache.flink.runtime.util.jartestprogram.FilterWithIndirection; +import org.apache.flink.runtime.util.jartestprogram.FilterWithLambda; +import org.apache.flink.runtime.util.jartestprogram.FilterWithMethodReference; import org.apache.flink.runtime.util.jartestprogram.WordCountWithAnonymousClass; import org.apache.flink.runtime.util.jartestprogram.WordCountWithExternalClass; import org.apache.flink.runtime.util.jartestprogram.WordCountWithExternalClass2; @@ -28,6 +30,7 @@ import org.apache.flink.runtime.util.jartestprogram.AnonymousInNonStaticMethod2; import org.apache.flink.runtime.util.jartestprogram.NestedAnonymousInnerClass; import org.junit.Assert; +import org.junit.Ignore; import org.junit.Test; import java.io.File; import java.io.FileInputStream; @@ -37,7 +40,6 @@ import java.util.jar.JarInputStream; import java.util.zip.ZipEntry; - public class JarFileCreatorTest { //anonymous inner class in static method accessing a local variable in its closure. @@ -48,14 +50,14 @@ public void TestAnonymousInnerClassTrick1() throws Exception { jfc.addClass(AnonymousInStaticMethod.class) .createJarFile(); - Set ans = new HashSet(); + Set ans = new HashSet<>(); ans.add("org/apache/flink/runtime/util/jartestprogram/AnonymousInStaticMethod$1.class"); ans.add("org/apache/flink/runtime/util/jartestprogram/AnonymousInStaticMethod$A.class"); ans.add("org/apache/flink/runtime/util/jartestprogram/AnonymousInStaticMethod.class"); Assert.assertTrue("Jar file for Anonymous Inner Class is not correct", validate(ans, out)); - out.delete(); + Assert.assertTrue(out.delete()); } //anonymous inner class in non static method accessing a local variable in its closure. @@ -66,14 +68,14 @@ public void TestAnonymousInnerClassTrick2() throws Exception { jfc.addClass(AnonymousInNonStaticMethod.class) .createJarFile(); - Set ans = new HashSet(); + Set ans = new HashSet<>(); ans.add("org/apache/flink/runtime/util/jartestprogram/AnonymousInNonStaticMethod$1.class"); ans.add("org/apache/flink/runtime/util/jartestprogram/AnonymousInNonStaticMethod$A.class"); ans.add("org/apache/flink/runtime/util/jartestprogram/AnonymousInNonStaticMethod.class"); Assert.assertTrue("Jar file for Anonymous Inner Class is not correct", validate(ans, out)); - out.delete(); + Assert.assertTrue(out.delete()); } //anonymous inner class in non static method accessing a field of its enclosing class. @@ -84,14 +86,14 @@ public void TestAnonymousInnerClassTrick3() throws Exception { jfc.addClass(AnonymousInNonStaticMethod2.class) .createJarFile(); - Set ans = new HashSet(); + Set ans = new HashSet<>(); ans.add("org/apache/flink/runtime/util/jartestprogram/AnonymousInNonStaticMethod2$1.class"); ans.add("org/apache/flink/runtime/util/jartestprogram/AnonymousInNonStaticMethod2$A.class"); ans.add("org/apache/flink/runtime/util/jartestprogram/AnonymousInNonStaticMethod2.class"); Assert.assertTrue("Jar file for Anonymous Inner Class is not correct", validate(ans, out)); - out.delete(); + Assert.assertTrue(out.delete()); } //anonymous inner class in an anonymous inner class accessing a field of the outermost enclosing class. @@ -102,7 +104,7 @@ public void TestAnonymousInnerClassTrick4() throws Exception { jfc.addClass(NestedAnonymousInnerClass.class) .createJarFile(); - Set ans = new HashSet(); + Set ans = new HashSet<>(); ans.add("org/apache/flink/runtime/util/jartestprogram/NestedAnonymousInnerClass.class"); ans.add("org/apache/flink/runtime/util/jartestprogram/NestedAnonymousInnerClass$1$1.class"); ans.add("org/apache/flink/runtime/util/jartestprogram/NestedAnonymousInnerClass$1.class"); @@ -110,7 +112,54 @@ public void TestAnonymousInnerClassTrick4() throws Exception { Assert.assertTrue("Jar file for Anonymous Inner Class is not correct", validate(ans, out)); - out.delete(); + Assert.assertTrue(out.delete()); + } + + @Ignore // this is currently not supported (see FLINK-9520) + @Test + public void testFilterWithMethodReference() throws Exception { + File out = new File(System.getProperty("java.io.tmpdir"), "jarcreatortest.jar"); + JarFileCreator jfc = new JarFileCreator(out); + jfc.addClass(FilterWithMethodReference.class) + .createJarFile(); + + Set ans = new HashSet<>(); + ans.add("org/apache/flink/runtime/util/jartestprogram/FilterWithMethodReference.class"); + ans.add("org/apache/flink/runtime/util/jartestprogram/WordFilter.class"); + + Assert.assertTrue("Jar file for Java 8 method reference is not correct", validate(ans, out)); + Assert.assertTrue(out.delete()); + } + + @Test + public void testFilterWithLambda() throws Exception{ + File out = new File(System.getProperty("java.io.tmpdir"), "jarcreatortest.jar"); + JarFileCreator jfc = new JarFileCreator(out); + jfc.addClass(FilterWithLambda.class) + .createJarFile(); + + Set ans = new HashSet<>(); + ans.add("org/apache/flink/runtime/util/jartestprogram/FilterWithLambda.class"); + ans.add("org/apache/flink/runtime/util/jartestprogram/WordFilter.class"); + + Assert.assertTrue("Jar file for Java 8 lambda is not correct", validate(ans, out)); + Assert.assertTrue(out.delete()); + } + + @Test + public void testFilterWithIndirection() throws Exception { + File out = new File(System.getProperty("java.io.tmpdir"), "jarcreatortest.jar"); + JarFileCreator jfc = new JarFileCreator(out); + jfc.addClass(FilterWithIndirection.class) + .createJarFile(); + + Set ans = new HashSet<>(); + ans.add("org/apache/flink/runtime/util/jartestprogram/FilterWithIndirection.class"); + ans.add("org/apache/flink/runtime/util/jartestprogram/WordFilter.class"); + ans.add("org/apache/flink/runtime/util/jartestprogram/UtilFunctionWrapper$UtilFunction.class"); + + Assert.assertTrue("Jar file for java 8 lambda is not correct", validate(ans, out)); + Assert.assertTrue(out.delete()); } //---------------------------------------------------------------------------------------------- @@ -123,14 +172,14 @@ public void TestExternalClass() throws IOException { jfc.addClass(WordCountWithExternalClass.class) .createJarFile(); - Set ans = new HashSet(); + Set ans = new HashSet<>(); ans.add("org/apache/flink/runtime/util/jartestprogram/StaticData.class"); ans.add("org/apache/flink/runtime/util/jartestprogram/WordCountWithExternalClass.class"); ans.add("org/apache/flink/runtime/util/jartestprogram/ExternalTokenizer.class"); Assert.assertTrue("Jar file for External Class is not correct", validate(ans, out)); - out.delete(); + Assert.assertTrue(out.delete()); } @Test @@ -140,14 +189,14 @@ public void TestInnerClass() throws IOException { jfc.addClass(WordCountWithInnerClass.class) .createJarFile(); - Set ans = new HashSet(); + Set ans = new HashSet<>(); ans.add("org/apache/flink/runtime/util/jartestprogram/StaticData.class"); ans.add("org/apache/flink/runtime/util/jartestprogram/WordCountWithInnerClass.class"); ans.add("org/apache/flink/runtime/util/jartestprogram/WordCountWithInnerClass$Tokenizer.class"); Assert.assertTrue("Jar file for Inner Class is not correct", validate(ans, out)); - out.delete(); + Assert.assertTrue(out.delete()); } @Test @@ -157,14 +206,14 @@ public void TestAnonymousClass() throws IOException { jfc.addClass(WordCountWithAnonymousClass.class) .createJarFile(); - Set ans = new HashSet(); + Set ans = new HashSet<>(); ans.add("org/apache/flink/runtime/util/jartestprogram/StaticData.class"); ans.add("org/apache/flink/runtime/util/jartestprogram/WordCountWithAnonymousClass.class"); ans.add("org/apache/flink/runtime/util/jartestprogram/WordCountWithAnonymousClass$1.class"); Assert.assertTrue("Jar file for Anonymous Class is not correct", validate(ans, out)); - out.delete(); + Assert.assertTrue(out.delete()); } @Test @@ -174,7 +223,7 @@ public void TestExtendIdentifier() throws IOException { jfc.addClass(WordCountWithExternalClass2.class) .createJarFile(); - Set ans = new HashSet(); + Set ans = new HashSet<>(); ans.add("org/apache/flink/runtime/util/jartestprogram/StaticData.class"); ans.add("org/apache/flink/runtime/util/jartestprogram/WordCountWithExternalClass2.class"); ans.add("org/apache/flink/runtime/util/jartestprogram/ExternalTokenizer2.class"); @@ -182,7 +231,7 @@ public void TestExtendIdentifier() throws IOException { Assert.assertTrue("Jar file for Extend Identifier is not correct", validate(ans, out)); - out.delete(); + Assert.assertTrue(out.delete()); } @Test @@ -193,7 +242,7 @@ public void TestUDFPackage() throws IOException { .addPackage("org.apache.flink.util") .createJarFile(); - Set ans = new HashSet(); + Set ans = new HashSet<>(); ans.add("org/apache/flink/runtime/util/jartestprogram/StaticData.class"); ans.add("org/apache/flink/runtime/util/jartestprogram/WordCountWithInnerClass.class"); ans.add("org/apache/flink/runtime/util/jartestprogram/WordCountWithInnerClass$Tokenizer.class"); @@ -201,7 +250,7 @@ public void TestUDFPackage() throws IOException { Assert.assertTrue("Jar file for UDF package is not correct", validate(ans, out)); - out.delete(); + Assert.assertTrue(out.delete()); } private boolean validate(Set expected, File out) throws IOException { diff --git a/flink-java8/src/test/java/org/apache/flink/runtime/util/jartestprogram/FilterLambda4.java b/flink-runtime/src/test/java/org/apache/flink/runtime/util/jartestprogram/FilterWithIndirection.java similarity index 93% rename from flink-java8/src/test/java/org/apache/flink/runtime/util/jartestprogram/FilterLambda4.java rename to flink-runtime/src/test/java/org/apache/flink/runtime/util/jartestprogram/FilterWithIndirection.java index 606ef5eb61335..12026e9c4b410 100644 --- a/flink-java8/src/test/java/org/apache/flink/runtime/util/jartestprogram/FilterLambda4.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/util/jartestprogram/FilterWithIndirection.java @@ -1,4 +1,4 @@ -/** +/* * 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 @@ -22,9 +22,9 @@ import org.apache.flink.api.java.ExecutionEnvironment; /** - * Similar to {@link FilterLambda3} with additional indirection. + * Filter with additional indirections. */ -public class FilterLambda4 { +public class FilterWithIndirection { public static void main(String[] args) throws Exception { ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); diff --git a/flink-java8/src/test/java/org/apache/flink/runtime/util/jartestprogram/FilterLambda2.java b/flink-runtime/src/test/java/org/apache/flink/runtime/util/jartestprogram/FilterWithLambda.java similarity index 89% rename from flink-java8/src/test/java/org/apache/flink/runtime/util/jartestprogram/FilterLambda2.java rename to flink-runtime/src/test/java/org/apache/flink/runtime/util/jartestprogram/FilterWithLambda.java index 955560745ca1f..ffa57562e1fe8 100644 --- a/flink-java8/src/test/java/org/apache/flink/runtime/util/jartestprogram/FilterLambda2.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/util/jartestprogram/FilterWithLambda.java @@ -1,4 +1,4 @@ -/** +/* * 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 @@ -23,10 +23,11 @@ import org.apache.flink.api.java.ExecutionEnvironment; /** - * Similar to {@link FilterLambda1}, but the filter lambda is directly passed to {@link DataSet#filter(FilterFunction)}. + * Filter with lambda that is directly passed to {@link DataSet#filter(FilterFunction)}. */ -public class FilterLambda2 { +public class FilterWithLambda { + @SuppressWarnings("Convert2MethodRef") public static void main(String[] args) throws Exception { ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); DataSet input = env.fromElements("Please filter", "the words", "but not this"); diff --git a/flink-java8/src/test/java/org/apache/flink/runtime/util/jartestprogram/FilterLambda1.java b/flink-runtime/src/test/java/org/apache/flink/runtime/util/jartestprogram/FilterWithMethodReference.java similarity index 93% rename from flink-java8/src/test/java/org/apache/flink/runtime/util/jartestprogram/FilterLambda1.java rename to flink-runtime/src/test/java/org/apache/flink/runtime/util/jartestprogram/FilterWithMethodReference.java index 12abff9a12e18..ddb76b4480a40 100644 --- a/flink-java8/src/test/java/org/apache/flink/runtime/util/jartestprogram/FilterLambda1.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/util/jartestprogram/FilterWithMethodReference.java @@ -1,4 +1,4 @@ -/** +/* * 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 @@ -25,13 +25,13 @@ /** * A lambda filter using a static method. */ -public class FilterLambda1 { +public class FilterWithMethodReference { public static void main(String[] args) throws Exception { ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); DataSet input = env.fromElements("Please filter", "the words", "but not this"); - FilterFunction filter = (v) -> WordFilter.filter(v); + FilterFunction filter = WordFilter::filter; DataSet output = input.filter(filter); output.print(); diff --git a/flink-java8/src/test/java/org/apache/flink/runtime/util/jartestprogram/UtilFunction.java b/flink-runtime/src/test/java/org/apache/flink/runtime/util/jartestprogram/UtilFunction.java similarity index 96% rename from flink-java8/src/test/java/org/apache/flink/runtime/util/jartestprogram/UtilFunction.java rename to flink-runtime/src/test/java/org/apache/flink/runtime/util/jartestprogram/UtilFunction.java index 1d5394a7f7c3d..89fca0d8ea1c4 100644 --- a/flink-java8/src/test/java/org/apache/flink/runtime/util/jartestprogram/UtilFunction.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/util/jartestprogram/UtilFunction.java @@ -1,4 +1,4 @@ -/** +/* * 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 @@ -24,6 +24,8 @@ * Static factory for a lambda filter function. */ public class UtilFunction { + + @SuppressWarnings("Convert2MethodRef") public static FilterFunction getWordFilter() { return (v) -> WordFilter.filter(v); } diff --git a/flink-java8/src/test/java/org/apache/flink/runtime/util/jartestprogram/UtilFunctionWrapper.java b/flink-runtime/src/test/java/org/apache/flink/runtime/util/jartestprogram/UtilFunctionWrapper.java similarity index 96% rename from flink-java8/src/test/java/org/apache/flink/runtime/util/jartestprogram/UtilFunctionWrapper.java rename to flink-runtime/src/test/java/org/apache/flink/runtime/util/jartestprogram/UtilFunctionWrapper.java index de8f68a11e711..498c4cf017943 100644 --- a/flink-java8/src/test/java/org/apache/flink/runtime/util/jartestprogram/UtilFunctionWrapper.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/util/jartestprogram/UtilFunctionWrapper.java @@ -1,4 +1,4 @@ -/** +/* * 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 @@ -24,10 +24,12 @@ * A wrapper around {@link WordFilter} to introduce additional indirection. */ public class UtilFunctionWrapper { + /** * Static factory for a lambda filter function. */ public static class UtilFunction { + @SuppressWarnings("Convert2MethodRef") public static FilterFunction getWordFilter() { return (v) -> WordFilter.filter(v); } diff --git a/flink-java8/src/test/java/org/apache/flink/runtime/util/jartestprogram/WordFilter.java b/flink-runtime/src/test/java/org/apache/flink/runtime/util/jartestprogram/WordFilter.java similarity index 99% rename from flink-java8/src/test/java/org/apache/flink/runtime/util/jartestprogram/WordFilter.java rename to flink-runtime/src/test/java/org/apache/flink/runtime/util/jartestprogram/WordFilter.java index 4a5b16f6191d7..2d072cbb83b7f 100644 --- a/flink-java8/src/test/java/org/apache/flink/runtime/util/jartestprogram/WordFilter.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/util/jartestprogram/WordFilter.java @@ -1,4 +1,4 @@ -/** +/* * 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 @@ -22,6 +22,7 @@ * Static filter method for lambda tests. */ public class WordFilter { + public static boolean filter(String value) { return !value.contains("not"); } diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/datastream/AllWindowedStream.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/datastream/AllWindowedStream.java index 7fb3822992a29..3f935e34e5685 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/datastream/AllWindowedStream.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/datastream/AllWindowedStream.java @@ -521,7 +521,6 @@ private static TypeInformation getAllWindowFunctionReturnType( AllWindowFunction.class, 0, 1, - new int[]{1, 0}, new int[]{2, 0}, inType, null, @@ -537,7 +536,6 @@ private static TypeInformation getProcessAllWindowFunctionReturnT 0, 1, TypeExtractor.NO_INDEX, - TypeExtractor.NO_INDEX, inType, null, false); diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/datastream/AsyncDataStream.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/datastream/AsyncDataStream.java index cb18a3f6e4db3..a3e28abd95e8f 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/datastream/AsyncDataStream.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/datastream/AsyncDataStream.java @@ -72,7 +72,6 @@ private static SingleOutputStreamOperator addOperator( AsyncFunction.class, 0, 1, - new int[]{0}, new int[]{1, 0}, in.getType(), Utils.getCallLocationName(), diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/datastream/BroadcastConnectedStream.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/datastream/BroadcastConnectedStream.java index cb7d8c95d548e..30047cbf87fe6 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/datastream/BroadcastConnectedStream.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/datastream/BroadcastConnectedStream.java @@ -131,8 +131,6 @@ public SingleOutputStreamOperator process(final KeyedBroadcastPro 2, 3, TypeExtractor.NO_INDEX, - TypeExtractor.NO_INDEX, - TypeExtractor.NO_INDEX, getType1(), getType2(), Utils.getCallLocationName(), @@ -183,8 +181,6 @@ public SingleOutputStreamOperator process(final BroadcastProcessFunct 1, 2, TypeExtractor.NO_INDEX, - TypeExtractor.NO_INDEX, - TypeExtractor.NO_INDEX, getType1(), getType2(), Utils.getCallLocationName(), diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/datastream/CoGroupedStreams.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/datastream/CoGroupedStreams.java index c2ebdf483ed11..55009e1b4cb2d 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/datastream/CoGroupedStreams.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/datastream/CoGroupedStreams.java @@ -43,6 +43,7 @@ import org.apache.flink.streaming.api.windowing.triggers.Trigger; import org.apache.flink.streaming.api.windowing.windows.Window; import org.apache.flink.util.Collector; +import org.apache.flink.util.Preconditions; import java.io.IOException; import java.util.ArrayList; @@ -95,9 +96,24 @@ public CoGroupedStreams(DataStream input1, DataStream input2) { /** * Specifies a {@link KeySelector} for elements from the first input. + * + * @param keySelector The KeySelector to be used for extracting the first input's key for partitioning. */ public Where where(KeySelector keySelector) { - TypeInformation keyType = TypeExtractor.getKeySelectorTypes(keySelector, input1.getType()); + Preconditions.checkNotNull(keySelector); + final TypeInformation keyType = TypeExtractor.getKeySelectorTypes(keySelector, input1.getType()); + return where(keySelector, keyType); + } + + /** + * Specifies a {@link KeySelector} for elements from the first input with explicit type information. + * + * @param keySelector The KeySelector to be used for extracting the first input's key for partitioning. + * @param keyType The type information describing the key type. + */ + public Where where(KeySelector keySelector, TypeInformation keyType) { + Preconditions.checkNotNull(keySelector); + Preconditions.checkNotNull(keyType); return new Where<>(input1.clean(keySelector), keyType); } @@ -121,12 +137,28 @@ public class Where { /** * Specifies a {@link KeySelector} for elements from the second input. + * + * @param keySelector The KeySelector to be used for extracting the second input's key for partitioning. */ public EqualTo equalTo(KeySelector keySelector) { - TypeInformation otherKey = TypeExtractor.getKeySelectorTypes(keySelector, input2.getType()); - if (!otherKey.equals(this.keyType)) { + Preconditions.checkNotNull(keySelector); + final TypeInformation otherKey = TypeExtractor.getKeySelectorTypes(keySelector, input2.getType()); + return equalTo(keySelector, otherKey); + } + + /** + * Specifies a {@link KeySelector} for elements from the second input with explicit type information for the key type. + * + * @param keySelector The KeySelector to be used for extracting the key for partitioning. + * @param keyType The type information describing the key type. + */ + public EqualTo equalTo(KeySelector keySelector, TypeInformation keyType) { + Preconditions.checkNotNull(keySelector); + Preconditions.checkNotNull(keyType); + + if (!keyType.equals(this.keyType)) { throw new IllegalArgumentException("The keys for the two inputs are not equal: " + - "first key = " + this.keyType + " , second key = " + otherKey); + "first key = " + this.keyType + " , second key = " + keyType); } return new EqualTo(input2.clean(keySelector)); diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/datastream/ConnectedStreams.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/datastream/ConnectedStreams.java index e244bd26bb0c1..0ada54a8f4cc9 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/datastream/ConnectedStreams.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/datastream/ConnectedStreams.java @@ -191,6 +191,28 @@ public ConnectedStreams keyBy(KeySelector keySelector1, KeySel inputStream2.keyBy(keySelector2)); } + /** + * KeyBy operation for connected data stream. Assigns keys to the elements of + * input1 and input2 using keySelector1 and keySelector2 with explicit type information + * for the common key type. + * + * @param keySelector1 + * The {@link KeySelector} used for grouping the first input + * @param keySelector2 + * The {@link KeySelector} used for grouping the second input + * @param keyType The type information of the common key type. + * @return The partitioned {@link ConnectedStreams} + */ + public ConnectedStreams keyBy( + KeySelector keySelector1, + KeySelector keySelector2, + TypeInformation keyType) { + return new ConnectedStreams<>( + environment, + inputStream1.keyBy(keySelector1, keyType), + inputStream2.keyBy(keySelector2, keyType)); + } + /** * Applies a CoMap transformation on a {@link ConnectedStreams} and maps * the output to a common type. The transformation calls a @@ -210,8 +232,6 @@ public SingleOutputStreamOperator map(CoMapFunction coMapper 1, 2, TypeExtractor.NO_INDEX, - TypeExtractor.NO_INDEX, - TypeExtractor.NO_INDEX, getType1(), getType2(), Utils.getCallLocationName(), @@ -244,8 +264,6 @@ public SingleOutputStreamOperator flatMap( 1, 2, TypeExtractor.NO_INDEX, - TypeExtractor.NO_INDEX, - TypeExtractor.NO_INDEX, getType1(), getType2(), Utils.getCallLocationName(), @@ -281,8 +299,6 @@ public SingleOutputStreamOperator process( 1, 2, TypeExtractor.NO_INDEX, - TypeExtractor.NO_INDEX, - TypeExtractor.NO_INDEX, getType1(), getType2(), Utils.getCallLocationName(), diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/datastream/DataStream.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/datastream/DataStream.java index 78ba8e4a5bea6..8e24ad7355808 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/datastream/DataStream.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/datastream/DataStream.java @@ -286,9 +286,24 @@ public BroadcastConnectedStream connect(BroadcastStream broadcastSt * @return The {@link DataStream} with partitioned state (i.e. KeyedStream) */ public KeyedStream keyBy(KeySelector key) { + Preconditions.checkNotNull(key); return new KeyedStream<>(this, clean(key)); } + /** + * It creates a new {@link KeyedStream} that uses the provided key with explicit type information + * for partitioning its operator states. + * + * @param key The KeySelector to be used for extracting the key for partitioning. + * @param keyType The type information describing the key type. + * @return The {@link DataStream} with partitioned state (i.e. KeyedStream) + */ + public KeyedStream keyBy(KeySelector key, TypeInformation keyType) { + Preconditions.checkNotNull(key); + Preconditions.checkNotNull(keyType); + return new KeyedStream<>(this, clean(key), keyType); + } + /** * Partitions the operator state of a {@link DataStream} by the given key positions. * @@ -621,7 +636,6 @@ public SingleOutputStreamOperator process(ProcessFunction processFu 0, 1, TypeExtractor.NO_INDEX, - TypeExtractor.NO_INDEX, getType(), Utils.getCallLocationName(), true); diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/datastream/IterativeStream.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/datastream/IterativeStream.java index f614ab03f529f..3d2227553859e 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/datastream/IterativeStream.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/datastream/IterativeStream.java @@ -206,5 +206,9 @@ public ConnectedStreams keyBy(KeySelector keySelector1, KeySelector< throw groupingException; } + @Override + public ConnectedStreams keyBy(KeySelector keySelector1, KeySelector keySelector2, TypeInformation keyType) { + throw groupingException; + } } } diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/datastream/JoinedStreams.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/datastream/JoinedStreams.java index 088fab9478126..bb67c091423a6 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/datastream/JoinedStreams.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/datastream/JoinedStreams.java @@ -79,9 +79,24 @@ public JoinedStreams(DataStream input1, DataStream input2) { /** * Specifies a {@link KeySelector} for elements from the first input. + * + * @param keySelector The KeySelector to be used for extracting the key for partitioning. */ public Where where(KeySelector keySelector) { - TypeInformation keyType = TypeExtractor.getKeySelectorTypes(keySelector, input1.getType()); + requireNonNull(keySelector); + final TypeInformation keyType = TypeExtractor.getKeySelectorTypes(keySelector, input1.getType()); + return where(keySelector, keyType); + } + + /** + * Specifies a {@link KeySelector} for elements from the first input with explicit type information for the key type. + * + * @param keySelector The KeySelector to be used for extracting the first input's key for partitioning. + * @param keyType The type information describing the key type. + */ + public Where where(KeySelector keySelector, TypeInformation keyType) { + requireNonNull(keySelector); + requireNonNull(keyType); return new Where<>(input1.clean(keySelector), keyType); } @@ -105,12 +120,28 @@ public class Where { /** * Specifies a {@link KeySelector} for elements from the second input. + * + * @param keySelector The KeySelector to be used for extracting the second input's key for partitioning. */ public EqualTo equalTo(KeySelector keySelector) { - TypeInformation otherKey = TypeExtractor.getKeySelectorTypes(keySelector, input2.getType()); - if (!otherKey.equals(this.keyType)) { + requireNonNull(keySelector); + final TypeInformation otherKey = TypeExtractor.getKeySelectorTypes(keySelector, input2.getType()); + return equalTo(keySelector, otherKey); + } + + /** + * Specifies a {@link KeySelector} for elements from the second input with explicit type information for the key type. + * + * @param keySelector The KeySelector to be used for extracting the second input's key for partitioning. + * @param keyType The type information describing the key type. + */ + public EqualTo equalTo(KeySelector keySelector, TypeInformation keyType) { + requireNonNull(keySelector); + requireNonNull(keyType); + + if (!keyType.equals(this.keyType)) { throw new IllegalArgumentException("The keys for the two inputs are not equal: " + - "first key = " + this.keyType + " , second key = " + otherKey); + "first key = " + this.keyType + " , second key = " + keyType); } return new EqualTo(input2.clean(keySelector)); @@ -226,8 +257,6 @@ public DataStream apply(JoinFunction function) { 0, 1, 2, - new int[]{0}, - new int[]{1}, TypeExtractor.NO_INDEX, input1.getType(), input2.getType(), @@ -309,8 +338,6 @@ public DataStream apply(FlatJoinFunction function) { 0, 1, 2, - new int[]{0}, - new int[]{1}, new int[]{2, 0}, input1.getType(), input2.getType(), diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/datastream/KeyedStream.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/datastream/KeyedStream.java index 32a5c9678f2d1..84df716f40c91 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/datastream/KeyedStream.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/datastream/KeyedStream.java @@ -70,6 +70,7 @@ import org.apache.flink.streaming.api.windowing.windows.Window; import org.apache.flink.streaming.runtime.partitioner.KeyGroupStreamPartitioner; import org.apache.flink.streaming.runtime.partitioner.StreamPartitioner; +import org.apache.flink.util.Preconditions; import org.apache.commons.lang3.StringUtils; @@ -305,7 +306,6 @@ public SingleOutputStreamOperator process(ProcessFunction processFu 0, 1, TypeExtractor.NO_INDEX, - TypeExtractor.NO_INDEX, getType(), Utils.getCallLocationName(), true); @@ -366,7 +366,6 @@ public SingleOutputStreamOperator process(KeyedProcessFunction 1, 2, TypeExtractor.NO_INDEX, - TypeExtractor.NO_INDEX, getType(), Utils.getCallLocationName(), true); @@ -480,8 +479,6 @@ public IntervalJoined between(Time lowerBound, Time upperBound) { @PublicEvolving public static class IntervalJoined { - private static final String INTERVAL_JOIN_FUNC_NAME = "IntervalJoin"; - private final KeyedStream left; private final KeyedStream right; @@ -534,33 +531,52 @@ public IntervalJoined lowerBoundExclusive() { } /** - * Completes the join operation with the user function that is executed for each joined pair + * Completes the join operation with the given user function that is executed for each joined pair * of elements. - * @param udf The user-defined function - * @param The output type - * @return Returns a DataStream + * + * @param processJoinFunction The user-defined process join function. + * @param The output type. + * @return The transformed {@link DataStream}. */ @PublicEvolving - public DataStream process(ProcessJoinFunction udf) { - - ProcessJoinFunction cleanedUdf = left.getExecutionEnvironment().clean(udf); + public SingleOutputStreamOperator process(ProcessJoinFunction processJoinFunction) { + Preconditions.checkNotNull(processJoinFunction); - TypeInformation resultType = TypeExtractor.getBinaryOperatorReturnType( - cleanedUdf, - ProcessJoinFunction.class, // ProcessJoinFunction - 0, // 0 1 2 + final TypeInformation outputType = TypeExtractor.getBinaryOperatorReturnType( + processJoinFunction, + ProcessJoinFunction.class, + 0, 1, 2, - new int[]{0}, // lambda input 1 type arg indices - new int[]{1}, // lambda input 1 type arg indices - TypeExtractor.NO_INDEX, // output arg indices - left.getType(), // input 1 type information - right.getType(), // input 2 type information - INTERVAL_JOIN_FUNC_NAME , - false + TypeExtractor.NO_INDEX, + left.getType(), + right.getType(), + Utils.getCallLocationName(), + true ); - IntervalJoinOperator operator = + return process(processJoinFunction, outputType); + } + + /** + * Completes the join operation with the given user function that is executed for each joined pair + * of elements. This methods allows for passing explicit type information for the output type. + * + * @param processJoinFunction The user-defined process join function. + * @param outputType The type information for the output type. + * @param The output type. + * @return The transformed {@link DataStream}. + */ + @PublicEvolving + public SingleOutputStreamOperator process( + ProcessJoinFunction processJoinFunction, + TypeInformation outputType) { + Preconditions.checkNotNull(processJoinFunction); + Preconditions.checkNotNull(outputType); + + final ProcessJoinFunction cleanedUdf = left.getExecutionEnvironment().clean(processJoinFunction); + + final IntervalJoinOperator operator = new IntervalJoinOperator<>( lowerBound, upperBound, @@ -574,8 +590,7 @@ public DataStream process(ProcessJoinFunction udf) { return left .connect(right) .keyBy(keySelector1, keySelector2) - .transform(INTERVAL_JOIN_FUNC_NAME , resultType, operator); - + .transform("Interval Join", outputType, operator); } } diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/datastream/WindowedStream.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/datastream/WindowedStream.java index 50c6f1ae1eb4e..1f09b73708048 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/datastream/WindowedStream.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/datastream/WindowedStream.java @@ -896,7 +896,6 @@ private static TypeInformation getWindowFunctionReturnType( WindowFunction.class, 0, 1, - new int[]{2, 0}, new int[]{3, 0}, inType, null, @@ -913,7 +912,6 @@ private static TypeInformation getProcessWindowFunctionRetur 0, 1, TypeExtractor.NO_INDEX, - TypeExtractor.NO_INDEX, inType, functionName, false); diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/TypeFillTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/TypeFillTest.java index 5baa9805ae9fd..cfb5adc2fc4f1 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/TypeFillTest.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/TypeFillTest.java @@ -21,12 +21,16 @@ import org.apache.flink.api.common.functions.MapFunction; import org.apache.flink.api.common.typeinfo.BasicTypeInfo; import org.apache.flink.api.common.typeinfo.TypeHint; +import org.apache.flink.api.common.typeinfo.Types; +import org.apache.flink.api.java.functions.KeySelector; import org.apache.flink.streaming.api.datastream.DataStream; import org.apache.flink.streaming.api.datastream.SingleOutputStreamOperator; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; import org.apache.flink.streaming.api.functions.co.CoFlatMapFunction; import org.apache.flink.streaming.api.functions.co.CoMapFunction; +import org.apache.flink.streaming.api.functions.co.ProcessJoinFunction; import org.apache.flink.streaming.api.functions.source.SourceFunction; +import org.apache.flink.streaming.api.windowing.time.Time; import org.apache.flink.util.Collector; import org.junit.Test; @@ -35,7 +39,8 @@ import static org.junit.Assert.fail; /** - * Tests for {@link TypeFill}. + * Tests for handling missing type information either by calling {@code returns()} or having an + * explicit type information parameter. */ @SuppressWarnings("serial") public class TypeFillTest { @@ -43,6 +48,7 @@ public class TypeFillTest { @Test public void test() { StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + env.setStreamTimeCharacteristic(TimeCharacteristic.EventTime); try { env.addSource(new TestSource()).print(); @@ -71,12 +77,53 @@ public void test() { fail(); } catch (Exception ignored) {} + try { + source.keyBy(new TestKeySelector()).print(); + fail(); + } catch (Exception ignored) {} + + try { + source.connect(source).keyBy(new TestKeySelector(), new TestKeySelector<>()); + fail(); + } catch (Exception ignored) {} + + try { + source.coGroup(source).where(new TestKeySelector<>()).equalTo(new TestKeySelector<>()); + fail(); + } catch (Exception ignored) {} + + try { + source.join(source).where(new TestKeySelector<>()).equalTo(new TestKeySelector<>()); + fail(); + } catch (Exception ignored) {} + + try { + source.keyBy((in) -> in) + .intervalJoin(source.keyBy((in) -> in)) + .between(Time.milliseconds(10L), Time.milliseconds(10L)) + .process(new TestProcessJoinFunction<>()) + .print(); + fail(); + } catch (Exception ignored) {} + env.addSource(new TestSource()).returns(Integer.class); source.map(new TestMap()).returns(Long.class).print(); source.flatMap(new TestFlatMap()).returns(new TypeHint(){}).print(); source.connect(source).map(new TestCoMap()).returns(BasicTypeInfo.INT_TYPE_INFO).print(); source.connect(source).flatMap(new TestCoFlatMap()) .returns(BasicTypeInfo.INT_TYPE_INFO).print(); + source.connect(source).keyBy(new TestKeySelector<>(), new TestKeySelector<>(), Types.STRING); + source.coGroup(source).where(new TestKeySelector<>(), Types.STRING).equalTo(new TestKeySelector<>(), Types.STRING); + source.join(source).where(new TestKeySelector<>(), Types.STRING).equalTo(new TestKeySelector<>(), Types.STRING); + source.keyBy((in) -> in) + .intervalJoin(source.keyBy((in) -> in)) + .between(Time.milliseconds(10L), Time.milliseconds(10L)) + .process(new TestProcessJoinFunction()) + .returns(Types.STRING); + source.keyBy((in) -> in) + .intervalJoin(source.keyBy((in) -> in)) + .between(Time.milliseconds(10L), Time.milliseconds(10L)) + .process(new TestProcessJoinFunction<>(), Types.STRING); assertEquals(BasicTypeInfo.LONG_TYPE_INFO, source.map(new TestMap()).returns(Long.class).getType()); @@ -142,4 +189,20 @@ public void flatMap1(IN1 value, Collector out) throws Exception {} public void flatMap2(IN2 value, Collector out) throws Exception {} } + + private static class TestKeySelector implements KeySelector { + + @Override + public KEY getKey(IN value) throws Exception { + return null; + } + } + + private static class TestProcessJoinFunction extends ProcessJoinFunction { + + @Override + public void processElement(IN1 left, IN2 right, Context ctx, Collector out) throws Exception { + // nothing to do + } + } } diff --git a/flink-tests/src/test/java/org/apache/flink/test/operators/CoGroupITCase.java b/flink-tests/src/test/java/org/apache/flink/test/operators/CoGroupITCase.java index 453f525c8b957..21583fd2c3498 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/operators/CoGroupITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/operators/CoGroupITCase.java @@ -669,6 +669,39 @@ public void testCoGroupWithRangePartitioning() throws Exception { compareResultAsTuples(result, expected); } + @Test + public void testCoGroupLambda() throws Exception { + final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); + + DataSet> left = env.fromElements( + new Tuple2<>(1, "hello"), + new Tuple2<>(2, "what's"), + new Tuple2<>(2, "up") + ); + DataSet> right = env.fromElements( + new Tuple2<>(1, "not"), + new Tuple2<>(1, "much"), + new Tuple2<>(2, "really") + ); + DataSet joined = left.coGroup(right).where(0).equalTo(0) + .with((Iterable> values1, Iterable> values2, + Collector out) -> { + int sum = 0; + for (Tuple2 next : values1) { + sum += next.f0; + } + for (Tuple2 next : values2) { + sum += next.f0; + } + out.collect(sum); + }).returns(Integer.class); + List result = joined.collect(); + + String expected = "6\n3\n"; + + compareResultAsText(result, expected); + } + // -------------------------------------------------------------------------------------------- // UDF classes // -------------------------------------------------------------------------------------------- diff --git a/flink-tests/src/test/java/org/apache/flink/test/operators/MapITCase.java b/flink-tests/src/test/java/org/apache/flink/test/operators/MapITCase.java index dfb3efb0163ba..f145555d509e6 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/operators/MapITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/operators/MapITCase.java @@ -515,4 +515,38 @@ public Tuple3 map(Tuple3 value) { return value; } } + + @Test + public void testMapWithLambdas() throws Exception { + final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); + + DataSet stringDs = env.fromElements(11, 12, 13, 14); + DataSet mappedDs = stringDs + .map(Object::toString) + .map(s -> s.replace("1", "2")) + .map(Trade::new) + .map(Trade::toString); + List result = mappedDs.collect(); + + String expected = "22\n" + + "22\n" + + "23\n" + + "24\n"; + + compareResultAsText(result, expected); + } + + private static class Trade { + + public String v; + + public Trade(String v) { + this.v = v; + } + + @Override + public String toString() { + return v; + } + } } diff --git a/flink-tests/src/test/java/org/apache/flink/test/operators/ReduceITCase.java b/flink-tests/src/test/java/org/apache/flink/test/operators/ReduceITCase.java index 2d6897ba067fe..750769c5c5a0a 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/operators/ReduceITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/operators/ReduceITCase.java @@ -90,7 +90,11 @@ public void testReduceOnTupleWithMultipleKeyFieldSelectors() throws Exception{ DataSet> ds = CollectionDataSets.get5TupleDataSet(env); DataSet> reduceDs = ds. - groupBy(4, 0).reduce(new Tuple5Reduce()); + groupBy(4, 0).reduce((in1, in2) -> { + Tuple5 out = new Tuple5<>(); + out.setFields(in1.f0, in1.f1 + in2.f1, 0, "P-)", in1.f4); + return out; + }); List> result = reduceDs .collect(); diff --git a/pom.xml b/pom.xml index 874f72a78c908..65cfacb1b44e2 100644 --- a/pom.xml +++ b/pom.xml @@ -57,7 +57,6 @@ under the License. flink-shaded-curator flink-core flink-java - flink-java8 flink-scala flink-filesystems flink-runtime From 7bb07e4e74ed7167cd98599af1a851afa1d77252 Mon Sep 17 00:00:00 2001 From: Timo Walther Date: Fri, 20 Jul 2018 08:47:53 +0200 Subject: [PATCH 31/65] [FLINK-9852] [table] Expose descriptor-based sink creation and introduce update mode This commit exposes the new unified sink creation through the table environments and the external catalog table. It introduce a new update-mode property in order to distinguish between append, retract, and upsert table sources and sinks. This commit refactors the top-level API classes a last time and adds more documentation. This commit completes the unified table sources/sinks story from an API point of view. Brief change log: - Introduction of TableEnvironment.connect() and corresponding API builder classes - Introduction of property update-mode: and update of existing connectors - External catalog support with proper source/sink discovery and API This closes #6343. --- .../kafka/Kafka010AvroTableSource.java | 6 +- .../kafka/Kafka010JsonTableSource.java | 6 +- .../kafka/Kafka011AvroTableSource.java | 6 +- .../kafka/Kafka011JsonTableSource.java | 6 +- .../kafka/Kafka08AvroTableSource.java | 6 +- .../kafka/Kafka08JsonTableSource.java | 6 +- .../kafka/Kafka09AvroTableSource.java | 6 +- .../kafka/Kafka09JsonTableSource.java | 6 +- .../kafka/KafkaAvroTableSource.java | 6 +- .../kafka/KafkaJsonTableSource.java | 4 +- .../connectors/kafka/KafkaTableSource.java | 4 +- .../kafka/KafkaTableSourceFactory.java | 3 + .../KafkaJsonTableSourceFactoryTestBase.java | 11 +- .../KafkaTableSourceFactoryTestBase.java | 11 +- .../flink/table/client/config/Sink.java | 4 +- .../flink/table/client/config/Source.java | 4 +- .../flink/table/client/config/SourceSink.java | 2 +- .../client/gateway/local/EnvironmentTest.java | 14 +- .../gateway/local/ExecutionContextTest.java | 6 +- .../gateway/local/LocalExecutorITCase.java | 12 +- .../gateway/utils/TestTableSinkFactory.java | 3 + .../gateway/utils/TestTableSourceFactory.java | 3 + .../resources/test-sql-client-defaults.yaml | 3 + .../resources/test-sql-client-factory.yaml | 1 + ....apache.flink.table.factories.TableFactory | 6 +- .../table/api/BatchTableEnvironment.scala | 36 +- .../table/api/StreamTableEnvironment.scala | 26 +- .../flink/table/api/TableEnvironment.scala | 32 +- .../table/catalog/ExternalCatalogSchema.scala | 2 +- .../table/catalog/ExternalCatalogTable.scala | 335 ++++++++++++++++-- .../catalog/ExternalTableSourceUtil.scala | 70 ---- .../table/catalog/ExternalTableUtil.scala | 102 ++++++ ...iptor.scala => BatchTableDescriptor.scala} | 19 +- .../BatchTableSourceDescriptor.scala | 87 ----- .../descriptors/ConnectTableDescriptor.scala | 108 ++++++ .../flink/table/descriptors/Descriptor.scala | 9 +- .../descriptors/DescriptorProperties.scala | 10 + .../descriptors/RegistrableDescriptor.scala | 49 +++ ...idator.scala => SchematicDescriptor.scala} | 16 +- .../descriptors/StreamTableDescriptor.scala | 101 ++++++ .../StreamTableDescriptorValidator.scala | 48 +++ .../StreamTableSourceDescriptor.scala | 90 ----- .../descriptors/StreamableDescriptor.scala | 67 ++++ .../table/descriptors/TableDescriptor.scala | 20 +- .../descriptors/TableSourceDescriptor.scala | 57 --- .../flink/table/factories/TableFactory.scala | 6 + .../table/factories/TableFactoryService.scala | 8 +- .../table/factories/TableFactoryUtil.scala | 82 +++++ .../table/factories/TableFormatFactory.scala | 2 + .../sinks/CsvAppendTableSinkFactory.scala | 45 +++ .../sinks/CsvBatchTableSinkFactory.scala | 38 ++ ...ry.scala => CsvTableSinkFactoryBase.scala} | 24 +- .../sources/CsvAppendTableSourceFactory.scala | 45 +++ .../sources/CsvBatchTableSourceFactory.scala | 38 ++ ....scala => CsvTableSourceFactoryBase.scala} | 24 +- .../flink/table/api/ExternalCatalogTest.scala | 20 +- .../catalog/ExternalCatalogSchemaTest.scala | 2 +- .../catalog/InMemoryExternalCatalogTest.scala | 8 +- .../descriptors/DescriptorTestBase.scala | 37 +- ...orTest.scala => TableDescriptorTest.scala} | 38 +- .../table/runtime/utils/CommonTestData.scala | 32 +- .../table/utils/MockTableEnvironment.scala | 4 +- 62 files changed, 1319 insertions(+), 563 deletions(-) delete mode 100644 flink-libraries/flink-table/src/main/scala/org/apache/flink/table/catalog/ExternalTableSourceUtil.scala create mode 100644 flink-libraries/flink-table/src/main/scala/org/apache/flink/table/catalog/ExternalTableUtil.scala rename flink-libraries/flink-table/src/main/scala/org/apache/flink/table/descriptors/{TableSinkDescriptor.scala => BatchTableDescriptor.scala} (71%) delete mode 100644 flink-libraries/flink-table/src/main/scala/org/apache/flink/table/descriptors/BatchTableSourceDescriptor.scala create mode 100644 flink-libraries/flink-table/src/main/scala/org/apache/flink/table/descriptors/ConnectTableDescriptor.scala create mode 100644 flink-libraries/flink-table/src/main/scala/org/apache/flink/table/descriptors/RegistrableDescriptor.scala rename flink-libraries/flink-table/src/main/scala/org/apache/flink/table/descriptors/{TableDescriptorValidator.scala => SchematicDescriptor.scala} (69%) create mode 100644 flink-libraries/flink-table/src/main/scala/org/apache/flink/table/descriptors/StreamTableDescriptor.scala create mode 100644 flink-libraries/flink-table/src/main/scala/org/apache/flink/table/descriptors/StreamTableDescriptorValidator.scala delete mode 100644 flink-libraries/flink-table/src/main/scala/org/apache/flink/table/descriptors/StreamTableSourceDescriptor.scala create mode 100644 flink-libraries/flink-table/src/main/scala/org/apache/flink/table/descriptors/StreamableDescriptor.scala delete mode 100644 flink-libraries/flink-table/src/main/scala/org/apache/flink/table/descriptors/TableSourceDescriptor.scala create mode 100644 flink-libraries/flink-table/src/main/scala/org/apache/flink/table/factories/TableFactoryUtil.scala create mode 100644 flink-libraries/flink-table/src/main/scala/org/apache/flink/table/sinks/CsvAppendTableSinkFactory.scala create mode 100644 flink-libraries/flink-table/src/main/scala/org/apache/flink/table/sinks/CsvBatchTableSinkFactory.scala rename flink-libraries/flink-table/src/main/scala/org/apache/flink/table/sinks/{CsvTableSinkFactory.scala => CsvTableSinkFactoryBase.scala} (82%) create mode 100644 flink-libraries/flink-table/src/main/scala/org/apache/flink/table/sources/CsvAppendTableSourceFactory.scala create mode 100644 flink-libraries/flink-table/src/main/scala/org/apache/flink/table/sources/CsvBatchTableSourceFactory.scala rename flink-libraries/flink-table/src/main/scala/org/apache/flink/table/sources/{CsvTableSourceFactory.scala => CsvTableSourceFactoryBase.scala} (86%) rename flink-libraries/flink-table/src/test/scala/org/apache/flink/table/descriptors/{TableSourceDescriptorTest.scala => TableDescriptorTest.scala} (78%) diff --git a/flink-connectors/flink-connector-kafka-0.10/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka010AvroTableSource.java b/flink-connectors/flink-connector-kafka-0.10/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka010AvroTableSource.java index ebbadcff1e197..d9d0a91ced1c8 100644 --- a/flink-connectors/flink-connector-kafka-0.10/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka010AvroTableSource.java +++ b/flink-connectors/flink-connector-kafka-0.10/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka010AvroTableSource.java @@ -39,7 +39,7 @@ * @deprecated Use the {@link org.apache.flink.table.descriptors.Kafka} descriptor together * with descriptors for schema and format instead. Descriptors allow for * implementation-agnostic definition of tables. See also - * {@link org.apache.flink.table.api.TableEnvironment#from(ConnectorDescriptor)}. + * {@link org.apache.flink.table.api.TableEnvironment#connect(ConnectorDescriptor)}. */ @Deprecated public class Kafka010AvroTableSource extends KafkaAvroTableSource { @@ -120,7 +120,7 @@ protected FlinkKafkaConsumerBase createKafkaConsumer(String topic, Properti * @deprecated Use the {@link org.apache.flink.table.descriptors.Kafka} descriptor together * with descriptors for schema and format instead. Descriptors allow for * implementation-agnostic definition of tables. See also - * {@link org.apache.flink.table.api.TableEnvironment#from(ConnectorDescriptor)}. + * {@link org.apache.flink.table.api.TableEnvironment#connect(ConnectorDescriptor)}. */ @Deprecated public static Builder builder() { @@ -133,7 +133,7 @@ public static Builder builder() { * @deprecated Use the {@link org.apache.flink.table.descriptors.Kafka} descriptor together * with descriptors for schema and format instead. Descriptors allow for * implementation-agnostic definition of tables. See also - * {@link org.apache.flink.table.api.TableEnvironment#from(ConnectorDescriptor)}. + * {@link org.apache.flink.table.api.TableEnvironment#connect(ConnectorDescriptor)}. */ @Deprecated public static class Builder extends KafkaAvroTableSource.Builder { diff --git a/flink-connectors/flink-connector-kafka-0.10/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka010JsonTableSource.java b/flink-connectors/flink-connector-kafka-0.10/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka010JsonTableSource.java index a5e33a12715db..38d9034a9a7bf 100644 --- a/flink-connectors/flink-connector-kafka-0.10/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka010JsonTableSource.java +++ b/flink-connectors/flink-connector-kafka-0.10/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka010JsonTableSource.java @@ -36,7 +36,7 @@ * @deprecated Use the {@link org.apache.flink.table.descriptors.Kafka} descriptor together * with descriptors for schema and format instead. Descriptors allow for * implementation-agnostic definition of tables. See also - * {@link org.apache.flink.table.api.TableEnvironment#from(ConnectorDescriptor)}. + * {@link org.apache.flink.table.api.TableEnvironment#connect(ConnectorDescriptor)}. */ @Deprecated public class Kafka010JsonTableSource extends KafkaJsonTableSource { @@ -121,7 +121,7 @@ protected FlinkKafkaConsumerBase createKafkaConsumer(String topic, Properti * @deprecated Use the {@link org.apache.flink.table.descriptors.Kafka} descriptor together * with descriptors for schema and format instead. Descriptors allow for * implementation-agnostic definition of tables. See also - * {@link org.apache.flink.table.api.TableEnvironment#from(ConnectorDescriptor)}. + * {@link org.apache.flink.table.api.TableEnvironment#connect(ConnectorDescriptor)}. */ @Deprecated public static Kafka010JsonTableSource.Builder builder() { @@ -134,7 +134,7 @@ public static Kafka010JsonTableSource.Builder builder() { * @deprecated Use the {@link org.apache.flink.table.descriptors.Kafka} descriptor together * with descriptors for schema and format instead. Descriptors allow for * implementation-agnostic definition of tables. See also - * {@link org.apache.flink.table.api.TableEnvironment#from(ConnectorDescriptor)}. + * {@link org.apache.flink.table.api.TableEnvironment#connect(ConnectorDescriptor)}. */ @Deprecated public static class Builder extends KafkaJsonTableSource.Builder { diff --git a/flink-connectors/flink-connector-kafka-0.11/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka011AvroTableSource.java b/flink-connectors/flink-connector-kafka-0.11/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka011AvroTableSource.java index b3f4e0a6e7527..fab592f1bda60 100644 --- a/flink-connectors/flink-connector-kafka-0.11/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka011AvroTableSource.java +++ b/flink-connectors/flink-connector-kafka-0.11/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka011AvroTableSource.java @@ -39,7 +39,7 @@ * @deprecated Use the {@link org.apache.flink.table.descriptors.Kafka} descriptor together * with descriptors for schema and format instead. Descriptors allow for * implementation-agnostic definition of tables. See also - * {@link org.apache.flink.table.api.TableEnvironment#from(ConnectorDescriptor)}. + * {@link org.apache.flink.table.api.TableEnvironment#connect(ConnectorDescriptor)}. */ @Deprecated public class Kafka011AvroTableSource extends KafkaAvroTableSource { @@ -119,7 +119,7 @@ protected FlinkKafkaConsumerBase createKafkaConsumer(String topic, Properti * @deprecated Use the {@link org.apache.flink.table.descriptors.Kafka} descriptor together * with descriptors for schema and format instead. Descriptors allow for * implementation-agnostic definition of tables. See also - * {@link org.apache.flink.table.api.TableEnvironment#from(ConnectorDescriptor)}. + * {@link org.apache.flink.table.api.TableEnvironment#connect(ConnectorDescriptor)}. */ @Deprecated public static Builder builder() { @@ -132,7 +132,7 @@ public static Builder builder() { * @deprecated Use the {@link org.apache.flink.table.descriptors.Kafka} descriptor together * with descriptors for schema and format instead. Descriptors allow for * implementation-agnostic definition of tables. See also - * {@link org.apache.flink.table.api.TableEnvironment#from(ConnectorDescriptor)}. + * {@link org.apache.flink.table.api.TableEnvironment#connect(ConnectorDescriptor)}. */ @Deprecated public static class Builder extends KafkaAvroTableSource.Builder { diff --git a/flink-connectors/flink-connector-kafka-0.11/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka011JsonTableSource.java b/flink-connectors/flink-connector-kafka-0.11/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka011JsonTableSource.java index 74c5007d80aa9..375eeadffb9bd 100644 --- a/flink-connectors/flink-connector-kafka-0.11/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka011JsonTableSource.java +++ b/flink-connectors/flink-connector-kafka-0.11/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka011JsonTableSource.java @@ -36,7 +36,7 @@ * @deprecated Use the {@link org.apache.flink.table.descriptors.Kafka} descriptor together * with descriptors for schema and format instead. Descriptors allow for * implementation-agnostic definition of tables. See also - * {@link org.apache.flink.table.api.TableEnvironment#from(ConnectorDescriptor)}. + * {@link org.apache.flink.table.api.TableEnvironment#connect(ConnectorDescriptor)}. */ @Deprecated public class Kafka011JsonTableSource extends KafkaJsonTableSource { @@ -121,7 +121,7 @@ protected FlinkKafkaConsumerBase createKafkaConsumer(String topic, Properti * @deprecated Use the {@link org.apache.flink.table.descriptors.Kafka} descriptor together * with descriptors for schema and format instead. Descriptors allow for * implementation-agnostic definition of tables. See also - * {@link org.apache.flink.table.api.TableEnvironment#from(ConnectorDescriptor)}. + * {@link org.apache.flink.table.api.TableEnvironment#connect(ConnectorDescriptor)}. */ @Deprecated public static Kafka011JsonTableSource.Builder builder() { @@ -134,7 +134,7 @@ public static Kafka011JsonTableSource.Builder builder() { * @deprecated Use the {@link org.apache.flink.table.descriptors.Kafka} descriptor together * with descriptors for schema and format instead. Descriptors allow for * implementation-agnostic definition of tables. See also - * {@link org.apache.flink.table.api.TableEnvironment#from(ConnectorDescriptor)}. + * {@link org.apache.flink.table.api.TableEnvironment#connect(ConnectorDescriptor)}. */ @Deprecated public static class Builder extends KafkaJsonTableSource.Builder { diff --git a/flink-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka08AvroTableSource.java b/flink-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka08AvroTableSource.java index 820628796c5ef..61c96bf1e5a46 100644 --- a/flink-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka08AvroTableSource.java +++ b/flink-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka08AvroTableSource.java @@ -39,7 +39,7 @@ * @deprecated Use the {@link org.apache.flink.table.descriptors.Kafka} descriptor together * with descriptors for schema and format instead. Descriptors allow for * implementation-agnostic definition of tables. See also - * {@link org.apache.flink.table.api.TableEnvironment#from(ConnectorDescriptor)}. + * {@link org.apache.flink.table.api.TableEnvironment#connect(ConnectorDescriptor)}. */ @Deprecated public class Kafka08AvroTableSource extends KafkaAvroTableSource { @@ -120,7 +120,7 @@ protected FlinkKafkaConsumerBase createKafkaConsumer(String topic, Properti * @deprecated Use the {@link org.apache.flink.table.descriptors.Kafka} descriptor together * with descriptors for schema and format instead. Descriptors allow for * implementation-agnostic definition of tables. See also - * {@link org.apache.flink.table.api.TableEnvironment#from(ConnectorDescriptor)}. + * {@link org.apache.flink.table.api.TableEnvironment#connect(ConnectorDescriptor)}. */ @Deprecated public static Builder builder() { @@ -133,7 +133,7 @@ public static Builder builder() { * @deprecated Use the {@link org.apache.flink.table.descriptors.Kafka} descriptor together * with descriptors for schema and format instead. Descriptors allow for * implementation-agnostic definition of tables. See also - * {@link org.apache.flink.table.api.TableEnvironment#from(ConnectorDescriptor)}. + * {@link org.apache.flink.table.api.TableEnvironment#connect(ConnectorDescriptor)}. */ @Deprecated public static class Builder extends KafkaAvroTableSource.Builder { diff --git a/flink-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka08JsonTableSource.java b/flink-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka08JsonTableSource.java index acb5783c262e6..dc5a077a89cf7 100644 --- a/flink-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka08JsonTableSource.java +++ b/flink-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka08JsonTableSource.java @@ -36,7 +36,7 @@ * @deprecated Use the {@link org.apache.flink.table.descriptors.Kafka} descriptor together * with descriptors for schema and format instead. Descriptors allow for * implementation-agnostic definition of tables. See also - * {@link org.apache.flink.table.api.TableEnvironment#from(ConnectorDescriptor)}. + * {@link org.apache.flink.table.api.TableEnvironment#connect(ConnectorDescriptor)}. */ @Deprecated public class Kafka08JsonTableSource extends KafkaJsonTableSource { @@ -120,7 +120,7 @@ protected FlinkKafkaConsumerBase createKafkaConsumer(String topic, Properti * @deprecated Use the {@link org.apache.flink.table.descriptors.Kafka} descriptor together * with descriptors for schema and format instead. Descriptors allow for * implementation-agnostic definition of tables. See also - * {@link org.apache.flink.table.api.TableEnvironment#from(ConnectorDescriptor)}. + * {@link org.apache.flink.table.api.TableEnvironment#connect(ConnectorDescriptor)}. */ @Deprecated public static Kafka08JsonTableSource.Builder builder() { @@ -133,7 +133,7 @@ public static Kafka08JsonTableSource.Builder builder() { * @deprecated Use the {@link org.apache.flink.table.descriptors.Kafka} descriptor together * with descriptors for schema and format instead. Descriptors allow for * implementation-agnostic definition of tables. See also - * {@link org.apache.flink.table.api.TableEnvironment#from(ConnectorDescriptor)}. + * {@link org.apache.flink.table.api.TableEnvironment#connect(ConnectorDescriptor)}. */ @Deprecated public static class Builder extends KafkaJsonTableSource.Builder { diff --git a/flink-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka09AvroTableSource.java b/flink-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka09AvroTableSource.java index cd4bad9e7589a..4352d7e400d79 100644 --- a/flink-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka09AvroTableSource.java +++ b/flink-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka09AvroTableSource.java @@ -39,7 +39,7 @@ * @deprecated Use the {@link org.apache.flink.table.descriptors.Kafka} descriptor together * with descriptors for schema and format instead. Descriptors allow for * implementation-agnostic definition of tables. See also - * {@link org.apache.flink.table.api.TableEnvironment#from(ConnectorDescriptor)}. + * {@link org.apache.flink.table.api.TableEnvironment#connect(ConnectorDescriptor)}. */ @Deprecated public class Kafka09AvroTableSource extends KafkaAvroTableSource { @@ -118,7 +118,7 @@ protected FlinkKafkaConsumerBase createKafkaConsumer(String topic, Properti * @deprecated Use the {@link org.apache.flink.table.descriptors.Kafka} descriptor together * with descriptors for schema and format instead. Descriptors allow for * implementation-agnostic definition of tables. See also - * {@link org.apache.flink.table.api.TableEnvironment#from(ConnectorDescriptor)}. + * {@link org.apache.flink.table.api.TableEnvironment#connect(ConnectorDescriptor)}. */ @Deprecated public static Builder builder() { @@ -131,7 +131,7 @@ public static Builder builder() { * @deprecated Use the {@link org.apache.flink.table.descriptors.Kafka} descriptor together * with descriptors for schema and format instead. Descriptors allow for * implementation-agnostic definition of tables. See also - * {@link org.apache.flink.table.api.TableEnvironment#from(ConnectorDescriptor)}. + * {@link org.apache.flink.table.api.TableEnvironment#connect(ConnectorDescriptor)}. */ @Deprecated public static class Builder extends KafkaAvroTableSource.Builder { diff --git a/flink-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka09JsonTableSource.java b/flink-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka09JsonTableSource.java index ad4e0d897684a..db1df3d889e12 100644 --- a/flink-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka09JsonTableSource.java +++ b/flink-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka09JsonTableSource.java @@ -36,7 +36,7 @@ * @deprecated Use the {@link org.apache.flink.table.descriptors.Kafka} descriptor together * with descriptors for schema and format instead. Descriptors allow for * implementation-agnostic definition of tables. See also - * {@link org.apache.flink.table.api.TableEnvironment#from(ConnectorDescriptor)}. + * {@link org.apache.flink.table.api.TableEnvironment#connect(ConnectorDescriptor)}. */ @Deprecated public class Kafka09JsonTableSource extends KafkaJsonTableSource { @@ -120,7 +120,7 @@ protected FlinkKafkaConsumerBase createKafkaConsumer(String topic, Properti * @deprecated Use the {@link org.apache.flink.table.descriptors.Kafka} descriptor together * with descriptors for schema and format instead. Descriptors allow for * implementation-agnostic definition of tables. See also - * {@link org.apache.flink.table.api.TableEnvironment#from(ConnectorDescriptor)}. + * {@link org.apache.flink.table.api.TableEnvironment#connect(ConnectorDescriptor)}. */ @Deprecated public static Kafka09JsonTableSource.Builder builder() { @@ -133,7 +133,7 @@ public static Kafka09JsonTableSource.Builder builder() { * @deprecated Use the {@link org.apache.flink.table.descriptors.Kafka} descriptor together * with descriptors for schema and format instead. Descriptors allow for * implementation-agnostic definition of tables. See also - * {@link org.apache.flink.table.api.TableEnvironment#from(ConnectorDescriptor)}. + * {@link org.apache.flink.table.api.TableEnvironment#connect(ConnectorDescriptor)}. */ @Deprecated public static class Builder extends KafkaJsonTableSource.Builder { diff --git a/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaAvroTableSource.java b/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaAvroTableSource.java index 3e9f2b03e344f..86fd21d15c60c 100644 --- a/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaAvroTableSource.java +++ b/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaAvroTableSource.java @@ -40,7 +40,7 @@ * @deprecated Use the {@link org.apache.flink.table.descriptors.Kafka} descriptor together * with descriptors for schema and format instead. Descriptors allow for * implementation-agnostic definition of tables. See also - * {@link org.apache.flink.table.api.TableEnvironment#from(ConnectorDescriptor)}. + * {@link org.apache.flink.table.api.TableEnvironment#connect(ConnectorDescriptor)}. */ @Deprecated @Internal @@ -56,7 +56,7 @@ public abstract class KafkaAvroTableSource extends KafkaTableSource { * @deprecated Use the {@link org.apache.flink.table.descriptors.Kafka} descriptor together * with descriptors for schema and format instead. Descriptors allow for * implementation-agnostic definition of tables. See also - * {@link org.apache.flink.table.api.TableEnvironment#from(ConnectorDescriptor)}. + * {@link org.apache.flink.table.api.TableEnvironment#connect(ConnectorDescriptor)}. */ @Deprecated protected KafkaAvroTableSource( @@ -89,7 +89,7 @@ public String explainSource() { * @deprecated Use the {@link org.apache.flink.table.descriptors.Kafka} descriptor together * with descriptors for schema and format instead. Descriptors allow for * implementation-agnostic definition of tables. See also - * {@link org.apache.flink.table.api.TableEnvironment#from(ConnectorDescriptor)}. + * {@link org.apache.flink.table.api.TableEnvironment#connect(ConnectorDescriptor)}. */ @Deprecated protected abstract static class Builder diff --git a/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaJsonTableSource.java b/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaJsonTableSource.java index bd0d0dedf22e7..70b286bc296a7 100644 --- a/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaJsonTableSource.java +++ b/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaJsonTableSource.java @@ -39,7 +39,7 @@ * @deprecated Use the {@link org.apache.flink.table.descriptors.Kafka} descriptor together * with descriptors for schema and format instead. Descriptors allow for * implementation-agnostic definition of tables. See also - * {@link org.apache.flink.table.api.TableEnvironment#from(ConnectorDescriptor)}. + * {@link org.apache.flink.table.api.TableEnvironment#connect(ConnectorDescriptor)}. */ @Deprecated @Internal @@ -98,7 +98,7 @@ protected void setFailOnMissingField(boolean failOnMissingField) { * @deprecated Use the {@link org.apache.flink.table.descriptors.Kafka} descriptor together * with descriptors for schema and format instead. Descriptors allow for * implementation-agnostic definition of tables. See also - * {@link org.apache.flink.table.api.TableEnvironment#from(ConnectorDescriptor)}. + * {@link org.apache.flink.table.api.TableEnvironment#connect(ConnectorDescriptor)}. */ @Deprecated protected abstract static class Builder diff --git a/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaTableSource.java b/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaTableSource.java index 78b373bc2c486..474c22f704f71 100644 --- a/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaTableSource.java +++ b/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaTableSource.java @@ -406,7 +406,7 @@ protected abstract FlinkKafkaConsumerBase createKafkaConsumer( * @deprecated Use the {@link org.apache.flink.table.descriptors.Kafka} descriptor together * with descriptors for schema and format instead. Descriptors allow for * implementation-agnostic definition of tables. See also - * {@link org.apache.flink.table.api.TableEnvironment#from(ConnectorDescriptor)}. + * {@link org.apache.flink.table.api.TableEnvironment#connect(ConnectorDescriptor)}. */ @Deprecated protected abstract static class Builder { @@ -676,7 +676,7 @@ protected void configureTableSource(T tableSource) { * @deprecated Use the {@link org.apache.flink.table.descriptors.Kafka} descriptor together * with descriptors for schema and format instead. Descriptors allow for * implementation-agnostic definition of tables. See also - * {@link org.apache.flink.table.api.TableEnvironment#from(ConnectorDescriptor)}. + * {@link org.apache.flink.table.api.TableEnvironment#connect(ConnectorDescriptor)}. */ @Deprecated protected abstract B builder(); diff --git a/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaTableSourceFactory.java b/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaTableSourceFactory.java index 380d657e0f8ca..d7e42f543b391 100644 --- a/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaTableSourceFactory.java +++ b/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaTableSourceFactory.java @@ -67,6 +67,8 @@ import static org.apache.flink.table.descriptors.SchemaValidator.SCHEMA_NAME; import static org.apache.flink.table.descriptors.SchemaValidator.SCHEMA_PROCTIME; import static org.apache.flink.table.descriptors.SchemaValidator.SCHEMA_TYPE; +import static org.apache.flink.table.descriptors.StreamTableDescriptorValidator.UPDATE_MODE; +import static org.apache.flink.table.descriptors.StreamTableDescriptorValidator.UPDATE_MODE_VALUE_APPEND; /** * Factory for creating configured instances of {@link KafkaTableSource}. @@ -76,6 +78,7 @@ public abstract class KafkaTableSourceFactory implements StreamTableSourceFactor @Override public Map requiredContext() { Map context = new HashMap<>(); + context.put(UPDATE_MODE(), UPDATE_MODE_VALUE_APPEND()); // append mode context.put(CONNECTOR_TYPE(), CONNECTOR_TYPE_VALUE_KAFKA); // kafka context.put(CONNECTOR_VERSION(), kafkaVersion()); // version context.put(CONNECTOR_PROPERTY_VERSION(), "1"); // backwards compatibility diff --git a/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaJsonTableSourceFactoryTestBase.java b/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaJsonTableSourceFactoryTestBase.java index 51017f415d8ec..20da156cd52cc 100644 --- a/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaJsonTableSourceFactoryTestBase.java +++ b/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaJsonTableSourceFactoryTestBase.java @@ -28,7 +28,7 @@ import org.apache.flink.table.descriptors.Kafka; import org.apache.flink.table.descriptors.Rowtime; import org.apache.flink.table.descriptors.Schema; -import org.apache.flink.table.descriptors.TestTableSourceDescriptor; +import org.apache.flink.table.descriptors.TestTableDescriptor; import org.apache.flink.table.factories.StreamTableSourceFactory; import org.apache.flink.table.factories.TableFactoryService; import org.apache.flink.table.sources.TableSource; @@ -135,20 +135,21 @@ private void testTableSource(FormatDescriptor format) { offsets.put(0, 100L); offsets.put(1, 123L); - final TestTableSourceDescriptor testDesc = new TestTableSourceDescriptor( + final TestTableDescriptor testDesc = new TestTableDescriptor( new Kafka() .version(version()) .topic(TOPIC) .properties(props) .startFromSpecificOffsets(offsets)) - .addFormat(format) - .addSchema( + .withFormat(format) + .withSchema( new Schema() .field("fruit-name", Types.STRING).from("name") .field("count", Types.BIG_DEC) // no from so it must match with the input .field("event-time", Types.SQL_TIMESTAMP).rowtime( new Rowtime().timestampsFromField("time").watermarksPeriodicAscending()) - .field("proc-time", Types.SQL_TIMESTAMP).proctime()); + .field("proc-time", Types.SQL_TIMESTAMP).proctime()) + .inAppendMode(); DescriptorProperties properties = new DescriptorProperties(true); testDesc.addProperties(properties); diff --git a/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTableSourceFactoryTestBase.java b/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTableSourceFactoryTestBase.java index 1e8266d9025dc..96f160763413d 100644 --- a/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTableSourceFactoryTestBase.java +++ b/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTableSourceFactoryTestBase.java @@ -31,7 +31,7 @@ import org.apache.flink.table.descriptors.Kafka; import org.apache.flink.table.descriptors.Rowtime; import org.apache.flink.table.descriptors.Schema; -import org.apache.flink.table.descriptors.TestTableSourceDescriptor; +import org.apache.flink.table.descriptors.TestTableDescriptor; import org.apache.flink.table.factories.StreamTableSourceFactory; import org.apache.flink.table.factories.TableFactoryService; import org.apache.flink.table.factories.utils.TestDeserializationSchema; @@ -129,20 +129,21 @@ public void testTableSource() { offsets.put(PARTITION_0, OFFSET_0); offsets.put(PARTITION_1, OFFSET_1); - final TestTableSourceDescriptor testDesc = new TestTableSourceDescriptor( + final TestTableDescriptor testDesc = new TestTableDescriptor( new Kafka() .version(getKafkaVersion()) .topic(TOPIC) .properties(KAFKA_PROPERTIES) .startFromSpecificOffsets(offsets)) - .addFormat(new TestTableFormat()) - .addSchema( + .withFormat(new TestTableFormat()) + .withSchema( new Schema() .field(FRUIT_NAME, Types.STRING()).from(NAME) .field(COUNT, Types.DECIMAL()) // no from so it must match with the input .field(EVENT_TIME, Types.SQL_TIMESTAMP()).rowtime( new Rowtime().timestampsFromField(TIME).watermarksPeriodicAscending()) - .field(PROC_TIME, Types.SQL_TIMESTAMP()).proctime()); + .field(PROC_TIME, Types.SQL_TIMESTAMP()).proctime()) + .inAppendMode(); final DescriptorProperties descriptorProperties = new DescriptorProperties(true); testDesc.addProperties(descriptorProperties); final Map propertiesMap = descriptorProperties.asMap(); diff --git a/flink-libraries/flink-sql-client/src/main/java/org/apache/flink/table/client/config/Sink.java b/flink-libraries/flink-sql-client/src/main/java/org/apache/flink/table/client/config/Sink.java index 0de65fb92e5db..49ac14aaa048c 100644 --- a/flink-libraries/flink-sql-client/src/main/java/org/apache/flink/table/client/config/Sink.java +++ b/flink-libraries/flink-sql-client/src/main/java/org/apache/flink/table/client/config/Sink.java @@ -19,14 +19,14 @@ package org.apache.flink.table.client.config; import org.apache.flink.table.descriptors.DescriptorProperties; -import org.apache.flink.table.descriptors.TableSinkDescriptor; +import org.apache.flink.table.descriptors.TableDescriptor; import java.util.Map; /** * Configuration of a table sink. */ -public class Sink extends TableSinkDescriptor { +public class Sink implements TableDescriptor { private String name; private Map properties; diff --git a/flink-libraries/flink-sql-client/src/main/java/org/apache/flink/table/client/config/Source.java b/flink-libraries/flink-sql-client/src/main/java/org/apache/flink/table/client/config/Source.java index ef80596323dc9..1eb0ad9b6063f 100644 --- a/flink-libraries/flink-sql-client/src/main/java/org/apache/flink/table/client/config/Source.java +++ b/flink-libraries/flink-sql-client/src/main/java/org/apache/flink/table/client/config/Source.java @@ -19,14 +19,14 @@ package org.apache.flink.table.client.config; import org.apache.flink.table.descriptors.DescriptorProperties; -import org.apache.flink.table.descriptors.TableSourceDescriptor; +import org.apache.flink.table.descriptors.TableDescriptor; import java.util.Map; /** * Configuration of a table source. */ -public class Source extends TableSourceDescriptor { +public class Source implements TableDescriptor { private String name; private Map properties; diff --git a/flink-libraries/flink-sql-client/src/main/java/org/apache/flink/table/client/config/SourceSink.java b/flink-libraries/flink-sql-client/src/main/java/org/apache/flink/table/client/config/SourceSink.java index bfa3c4444629f..b6b9bea4ae738 100644 --- a/flink-libraries/flink-sql-client/src/main/java/org/apache/flink/table/client/config/SourceSink.java +++ b/flink-libraries/flink-sql-client/src/main/java/org/apache/flink/table/client/config/SourceSink.java @@ -26,7 +26,7 @@ /** * Common class for all descriptors describing a table source and sink together. */ -public class SourceSink extends TableDescriptor { +public class SourceSink implements TableDescriptor { private String name; private Map properties; diff --git a/flink-libraries/flink-sql-client/src/test/java/org/apache/flink/table/client/gateway/local/EnvironmentTest.java b/flink-libraries/flink-sql-client/src/test/java/org/apache/flink/table/client/gateway/local/EnvironmentTest.java index dcd94e2a82f1f..6fbf29d900f24 100644 --- a/flink-libraries/flink-sql-client/src/test/java/org/apache/flink/table/client/gateway/local/EnvironmentTest.java +++ b/flink-libraries/flink-sql-client/src/test/java/org/apache/flink/table/client/gateway/local/EnvironmentTest.java @@ -23,8 +23,9 @@ import org.junit.Test; -import java.util.Collections; +import java.util.HashMap; import java.util.HashSet; +import java.util.Map; import java.util.Set; import static org.junit.Assert.assertEquals; @@ -40,10 +41,17 @@ public class EnvironmentTest { @Test public void testMerging() throws Exception { - final Environment env1 = EnvironmentFileUtil.parseUnmodified(DEFAULTS_ENVIRONMENT_FILE); + final Map replaceVars1 = new HashMap<>(); + replaceVars1.put("$VAR_UPDATE_MODE", "update-mode: append"); + final Environment env1 = EnvironmentFileUtil.parseModified( + DEFAULTS_ENVIRONMENT_FILE, + replaceVars1); + + final Map replaceVars2 = new HashMap<>(replaceVars1); + replaceVars2.put("TableNumber1", "NewTable"); final Environment env2 = EnvironmentFileUtil.parseModified( FACTORY_ENVIRONMENT_FILE, - Collections.singletonMap("TableNumber1", "NewTable")); + replaceVars2); final Environment merged = Environment.merge(env1, env2); diff --git a/flink-libraries/flink-sql-client/src/test/java/org/apache/flink/table/client/gateway/local/ExecutionContextTest.java b/flink-libraries/flink-sql-client/src/test/java/org/apache/flink/table/client/gateway/local/ExecutionContextTest.java index f9bcaf3815b92..bc29f6f27955a 100644 --- a/flink-libraries/flink-sql-client/src/test/java/org/apache/flink/table/client/gateway/local/ExecutionContextTest.java +++ b/flink-libraries/flink-sql-client/src/test/java/org/apache/flink/table/client/gateway/local/ExecutionContextTest.java @@ -35,6 +35,7 @@ import java.util.Arrays; import java.util.Collections; +import java.util.HashMap; import java.util.HashSet; import java.util.Map; @@ -112,9 +113,12 @@ public void testSourceSinks() throws Exception { } private ExecutionContext createExecutionContext() throws Exception { + final Map replaceVars = new HashMap<>(); + replaceVars.put("$VAR_2", "streaming"); + replaceVars.put("$VAR_UPDATE_MODE", "update-mode: append"); final Environment env = EnvironmentFileUtil.parseModified( DEFAULTS_ENVIRONMENT_FILE, - Collections.singletonMap("$VAR_2", "streaming")); + replaceVars); final SessionContext session = new SessionContext("test-session", new Environment()); final Configuration flinkConfig = new Configuration(); return new ExecutionContext<>( diff --git a/flink-libraries/flink-sql-client/src/test/java/org/apache/flink/table/client/gateway/local/LocalExecutorITCase.java b/flink-libraries/flink-sql-client/src/test/java/org/apache/flink/table/client/gateway/local/LocalExecutorITCase.java index b1bf3041cd826..d8452e42dbfa3 100644 --- a/flink-libraries/flink-sql-client/src/test/java/org/apache/flink/table/client/gateway/local/LocalExecutorITCase.java +++ b/flink-libraries/flink-sql-client/src/test/java/org/apache/flink/table/client/gateway/local/LocalExecutorITCase.java @@ -136,13 +136,12 @@ public void testGetSessionProperties() throws Exception { executor.getSessionProperties(session); // modify defaults - session.setSessionProperty("execution.type", "streaming"); session.setSessionProperty("execution.result-mode", "table"); final Map actualProperties = executor.getSessionProperties(session); final Map expectedProperties = new HashMap<>(); - expectedProperties.put("execution.type", "streaming"); + expectedProperties.put("execution.type", "batch"); expectedProperties.put("execution.time-characteristic", "event-time"); expectedProperties.put("execution.periodic-watermarks-interval", "99"); expectedProperties.put("execution.parallelism", "1"); @@ -178,6 +177,7 @@ public void testStreamQueryExecutionChangelog() throws Exception { replaceVars.put("$VAR_1", "/"); replaceVars.put("$VAR_2", "streaming"); replaceVars.put("$VAR_3", "changelog"); + replaceVars.put("$VAR_UPDATE_MODE", "update-mode: append"); final Executor executor = createModifiedExecutor(clusterClient, replaceVars); final SessionContext session = new SessionContext("test-session", new Environment()); @@ -216,6 +216,7 @@ public void testStreamQueryExecutionTable() throws Exception { replaceVars.put("$VAR_1", "/"); replaceVars.put("$VAR_2", "streaming"); replaceVars.put("$VAR_3", "table"); + replaceVars.put("$VAR_UPDATE_MODE", "update-mode: append"); final Executor executor = createModifiedExecutor(clusterClient, replaceVars); final SessionContext session = new SessionContext("test-session", new Environment()); @@ -253,6 +254,7 @@ public void testBatchQueryExecution() throws Exception { replaceVars.put("$VAR_1", "/"); replaceVars.put("$VAR_2", "batch"); replaceVars.put("$VAR_3", "table"); + replaceVars.put("$VAR_UPDATE_MODE", ""); final Executor executor = createModifiedExecutor(clusterClient, replaceVars); final SessionContext session = new SessionContext("test-session", new Environment()); @@ -287,6 +289,7 @@ public void testStreamQueryExecutionSink() throws Exception { replaceVars.put("$VAR_0", url.getPath()); replaceVars.put("$VAR_2", "streaming"); replaceVars.put("$VAR_4", csvOutputPath); + replaceVars.put("$VAR_UPDATE_MODE", "update-mode: append"); final Executor executor = createModifiedExecutor(clusterClient, replaceVars); final SessionContext session = new SessionContext("test-session", new Environment()); @@ -333,8 +336,11 @@ private void verifySinkResult(String path) throws IOException { } private LocalExecutor createDefaultExecutor(ClusterClient clusterClient) throws Exception { + final Map replaceVars = new HashMap<>(); + replaceVars.put("$VAR_2", "batch"); + replaceVars.put("$VAR_UPDATE_MODE", ""); return new LocalExecutor( - EnvironmentFileUtil.parseModified(DEFAULTS_ENVIRONMENT_FILE, Collections.singletonMap("$VAR_2", "batch")), + EnvironmentFileUtil.parseModified(DEFAULTS_ENVIRONMENT_FILE, replaceVars), Collections.emptyList(), clusterClient.getFlinkConfiguration(), new DummyCustomCommandLine(clusterClient)); diff --git a/flink-libraries/flink-sql-client/src/test/java/org/apache/flink/table/client/gateway/utils/TestTableSinkFactory.java b/flink-libraries/flink-sql-client/src/test/java/org/apache/flink/table/client/gateway/utils/TestTableSinkFactory.java index 4b505817de1a8..207d5d2f1cbb3 100644 --- a/flink-libraries/flink-sql-client/src/test/java/org/apache/flink/table/client/gateway/utils/TestTableSinkFactory.java +++ b/flink-libraries/flink-sql-client/src/test/java/org/apache/flink/table/client/gateway/utils/TestTableSinkFactory.java @@ -45,6 +45,8 @@ import static org.apache.flink.table.descriptors.SchemaValidator.SCHEMA; import static org.apache.flink.table.descriptors.SchemaValidator.SCHEMA_NAME; import static org.apache.flink.table.descriptors.SchemaValidator.SCHEMA_TYPE; +import static org.apache.flink.table.descriptors.StreamTableDescriptorValidator.UPDATE_MODE; +import static org.apache.flink.table.descriptors.StreamTableDescriptorValidator.UPDATE_MODE_VALUE_APPEND; /** * Table sink factory for testing the classloading in {@link DependencyTest}. @@ -54,6 +56,7 @@ public class TestTableSinkFactory implements StreamTableSinkFactory { @Override public Map requiredContext() { final Map context = new HashMap<>(); + context.put(UPDATE_MODE(), UPDATE_MODE_VALUE_APPEND()); context.put(CONNECTOR_TYPE(), CONNECTOR_TYPE_VALUE); return context; } diff --git a/flink-libraries/flink-sql-client/src/test/java/org/apache/flink/table/client/gateway/utils/TestTableSourceFactory.java b/flink-libraries/flink-sql-client/src/test/java/org/apache/flink/table/client/gateway/utils/TestTableSourceFactory.java index a08472cbe9590..81f00e5145894 100644 --- a/flink-libraries/flink-sql-client/src/test/java/org/apache/flink/table/client/gateway/utils/TestTableSourceFactory.java +++ b/flink-libraries/flink-sql-client/src/test/java/org/apache/flink/table/client/gateway/utils/TestTableSourceFactory.java @@ -48,6 +48,8 @@ import static org.apache.flink.table.descriptors.SchemaValidator.SCHEMA; import static org.apache.flink.table.descriptors.SchemaValidator.SCHEMA_NAME; import static org.apache.flink.table.descriptors.SchemaValidator.SCHEMA_TYPE; +import static org.apache.flink.table.descriptors.StreamTableDescriptorValidator.UPDATE_MODE; +import static org.apache.flink.table.descriptors.StreamTableDescriptorValidator.UPDATE_MODE_VALUE_APPEND; /** * Table source factory for testing the classloading in {@link DependencyTest}. @@ -57,6 +59,7 @@ public class TestTableSourceFactory implements StreamTableSourceFactory { @Override public Map requiredContext() { final Map context = new HashMap<>(); + context.put(UPDATE_MODE(), UPDATE_MODE_VALUE_APPEND()); context.put(CONNECTOR_TYPE(), CONNECTOR_TYPE_VALUE); return context; } diff --git a/flink-libraries/flink-sql-client/src/test/resources/test-sql-client-defaults.yaml b/flink-libraries/flink-sql-client/src/test/resources/test-sql-client-defaults.yaml index 2a886f014a6ae..b759874939d31 100644 --- a/flink-libraries/flink-sql-client/src/test/resources/test-sql-client-defaults.yaml +++ b/flink-libraries/flink-sql-client/src/test/resources/test-sql-client-defaults.yaml @@ -26,6 +26,7 @@ tables: - name: TableNumber1 type: source + $VAR_UPDATE_MODE schema: - name: IntegerField1 type: INT @@ -45,6 +46,7 @@ tables: comment-prefix: "#" - name: TableNumber2 type: source + $VAR_UPDATE_MODE schema: - name: IntegerField2 type: INT @@ -64,6 +66,7 @@ tables: comment-prefix: "#" - name: TableSourceSink type: both + $VAR_UPDATE_MODE schema: - name: BooleanField type: BOOLEAN diff --git a/flink-libraries/flink-sql-client/src/test/resources/test-sql-client-factory.yaml b/flink-libraries/flink-sql-client/src/test/resources/test-sql-client-factory.yaml index 01ad63d9db0ae..3ce513ecbe7b6 100644 --- a/flink-libraries/flink-sql-client/src/test/resources/test-sql-client-factory.yaml +++ b/flink-libraries/flink-sql-client/src/test/resources/test-sql-client-factory.yaml @@ -26,6 +26,7 @@ tables: - name: TableNumber1 type: both + update-mode: append schema: - name: IntegerField1 type: INT diff --git a/flink-libraries/flink-table/src/main/resources/META-INF/services/org.apache.flink.table.factories.TableFactory b/flink-libraries/flink-table/src/main/resources/META-INF/services/org.apache.flink.table.factories.TableFactory index 4cda0ad2f40d9..ece39ebb981fb 100644 --- a/flink-libraries/flink-table/src/main/resources/META-INF/services/org.apache.flink.table.factories.TableFactory +++ b/flink-libraries/flink-table/src/main/resources/META-INF/services/org.apache.flink.table.factories.TableFactory @@ -13,5 +13,7 @@ # See the License for the specific language governing permissions and # limitations under the License. -org.apache.flink.table.sources.CsvTableSourceFactory -org.apache.flink.table.sinks.CsvTableSinkFactory +org.apache.flink.table.sources.CsvBatchTableSourceFactory +org.apache.flink.table.sources.CsvAppendTableSourceFactory +org.apache.flink.table.sinks.CsvBatchTableSinkFactory +org.apache.flink.table.sinks.CsvAppendTableSinkFactory diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/BatchTableEnvironment.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/BatchTableEnvironment.scala index a239ad56e1ed4..9265f0f6607d6 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/BatchTableEnvironment.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/BatchTableEnvironment.scala @@ -31,7 +31,7 @@ import org.apache.flink.api.common.typeinfo.TypeInformation import org.apache.flink.api.java.io.DiscardingOutputFormat import org.apache.flink.api.java.typeutils.GenericTypeInfo import org.apache.flink.api.java.{DataSet, ExecutionEnvironment} -import org.apache.flink.table.descriptors.{BatchTableSourceDescriptor, ConnectorDescriptor} +import org.apache.flink.table.descriptors.{BatchTableDescriptor, ConnectorDescriptor} import org.apache.flink.table.explain.PlanJsonParser import org.apache.flink.table.expressions.{Expression, TimeAttribute} import org.apache.flink.table.plan.nodes.FlinkConventions @@ -126,7 +126,7 @@ abstract class BatchTableEnvironment( } // no table is registered - case None => + case _ => val newTable = new TableSourceSinkTable( Some(new BatchTableSourceTable(batchTableSource)), None) @@ -141,26 +141,21 @@ abstract class BatchTableEnvironment( } /** - * Creates a table from a descriptor that describes the source connector, the source format, - * the resulting table schema, and other properties. + * Creates a table source and/or table sink from a descriptor. * - * Descriptors allow for declaring communication to external systems in an - * implementation-agnostic way. The classpath is scanned for connectors and matching connectors - * are configured accordingly. + * Descriptors allow for declaring the communication to external systems in an + * implementation-agnostic way. The classpath is scanned for suitable table factories that match + * the desired configuration. * - * The following example shows how to read from a Kafka connector using a JSON format and - * creating a table: + * The following example shows how to read from a connector using a JSON format and + * registering a table source as "MyTable": * * {{{ * * tableEnv - * .from( - * new Kafka() - * .version("0.11") - * .topic("clicks") - * .property("zookeeper.connect", "localhost") - * .property("group.id", "click-group") - * .startFromEarliest()) + * .connect( + * new ExternalSystemXYZ() + * .version("0.11")) * .withFormat( * new Json() * .jsonSchema("{...}") @@ -169,14 +164,13 @@ abstract class BatchTableEnvironment( * new Schema() * .field("user-name", "VARCHAR").from("u_name") * .field("count", "DECIMAL") - * .field("proc-time", "TIMESTAMP").proctime()) - * .toTable() + * .registerSource("MyTable") * }}} * - * @param connectorDescriptor connector descriptor describing the source of the table + * @param connectorDescriptor connector descriptor describing the external system */ - def from(connectorDescriptor: ConnectorDescriptor): BatchTableSourceDescriptor = { - new BatchTableSourceDescriptor(this, connectorDescriptor) + def connect(connectorDescriptor: ConnectorDescriptor): BatchTableDescriptor = { + new BatchTableDescriptor(this, connectorDescriptor) } /** diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/StreamTableEnvironment.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/StreamTableEnvironment.scala index 33b984dd8eb43..4c7303245a116 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/StreamTableEnvironment.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/StreamTableEnvironment.scala @@ -37,7 +37,7 @@ import org.apache.flink.streaming.api.TimeCharacteristic import org.apache.flink.streaming.api.datastream.DataStream import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment import org.apache.flink.table.calcite.{FlinkTypeFactory, RelTimeIndicatorConverter} -import org.apache.flink.table.descriptors.{ConnectorDescriptor, StreamTableSourceDescriptor} +import org.apache.flink.table.descriptors.{ConnectorDescriptor, StreamTableDescriptor} import org.apache.flink.table.explain.PlanJsonParser import org.apache.flink.table.expressions._ import org.apache.flink.table.plan.nodes.FlinkConventions @@ -145,7 +145,7 @@ abstract class StreamTableEnvironment( } // no table is registered - case None => + case _ => val newTable = new TableSourceSinkTable( Some(new StreamTableSourceTable(streamTableSource)), None) @@ -160,20 +160,19 @@ abstract class StreamTableEnvironment( } /** - * Creates a table from a descriptor that describes the source connector, the source format, - * the resulting table schema, and other properties. + * Creates a table source and/or table sink from a descriptor. * - * Descriptors allow for declaring communication to external systems in an - * implementation-agnostic way. The classpath is scanned for connectors and matching connectors - * are configured accordingly. + * Descriptors allow for declaring the communication to external systems in an + * implementation-agnostic way. The classpath is scanned for suitable table factories that match + * the desired configuration. * * The following example shows how to read from a Kafka connector using a JSON format and - * creating a table: + * registering a table source "MyTable" in append mode: * * {{{ * * tableEnv - * .from( + * .connect( * new Kafka() * .version("0.11") * .topic("clicks") @@ -189,13 +188,14 @@ abstract class StreamTableEnvironment( * .field("user-name", "VARCHAR").from("u_name") * .field("count", "DECIMAL") * .field("proc-time", "TIMESTAMP").proctime()) - * .toTable() + * .inAppendMode() + * .registerSource("MyTable") * }}} * - * @param connectorDescriptor connector descriptor describing the source of the table + * @param connectorDescriptor connector descriptor describing the external system */ - def from(connectorDescriptor: ConnectorDescriptor): StreamTableSourceDescriptor = { - new StreamTableSourceDescriptor(this, connectorDescriptor) + def connect(connectorDescriptor: ConnectorDescriptor): StreamTableDescriptor = { + new StreamTableDescriptor(this, connectorDescriptor) } /** diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/TableEnvironment.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/TableEnvironment.scala index 6a299dde95cc9..37f6d024a075e 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/TableEnvironment.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/TableEnvironment.scala @@ -49,7 +49,7 @@ import org.apache.flink.table.api.scala.{BatchTableEnvironment => ScalaBatchTabl import org.apache.flink.table.calcite.{FlinkPlannerImpl, FlinkRelBuilder, FlinkTypeFactory, FlinkTypeSystem} import org.apache.flink.table.catalog.{ExternalCatalog, ExternalCatalogSchema} import org.apache.flink.table.codegen.{ExpressionReducer, FunctionCodeGenerator, GeneratedFunction} -import org.apache.flink.table.descriptors.{ConnectorDescriptor, TableSourceDescriptor} +import org.apache.flink.table.descriptors.{ConnectorDescriptor, TableDescriptor} import org.apache.flink.table.expressions._ import org.apache.flink.table.functions.utils.UserDefinedFunctionUtils._ import org.apache.flink.table.functions.{AggregateFunction, ScalarFunction, TableFunction} @@ -523,26 +523,21 @@ abstract class TableEnvironment(val config: TableConfig) { } /** - * Creates a table from a descriptor that describes the source connector, the source format, - * the resulting table schema, and other properties. + * Creates a table source and/or table sink from a descriptor. * - * Descriptors allow for declaring communication to external systems in an - * implementation-agnostic way. The classpath is scanned for connectors and matching connectors - * are configured accordingly. + * Descriptors allow for declaring the communication to external systems in an + * implementation-agnostic way. The classpath is scanned for suitable table factories that match + * the desired configuration. * - * The following example shows how to read from a Kafka connector using a JSON format and - * creating table: + * The following example shows how to read from a connector using a JSON format and + * registering a table source as "MyTable": * * {{{ * * tableEnv - * .from( - * new Kafka() - * .version("0.11") - * .topic("clicks") - * .property("zookeeper.connect", "localhost") - * .property("group.id", "click-group") - * .startFromEarliest()) + * .connect( + * new ExternalSystemXYZ() + * .version("0.11")) * .withFormat( * new Json() * .jsonSchema("{...}") @@ -551,13 +546,12 @@ abstract class TableEnvironment(val config: TableConfig) { * new Schema() * .field("user-name", "VARCHAR").from("u_name") * .field("count", "DECIMAL") - * .field("proc-time", "TIMESTAMP").proctime()) - * .toTable() + * .registerSource("MyTable") * }}} * - * @param connectorDescriptor connector descriptor describing the source of the table + * @param connectorDescriptor connector descriptor describing the external system */ - def from(connectorDescriptor: ConnectorDescriptor): TableSourceDescriptor + def connect(connectorDescriptor: ConnectorDescriptor): TableDescriptor private[flink] def scanInternal(tablePath: Array[String]): Option[Table] = { require(tablePath != null && !tablePath.isEmpty, "tablePath must not be null or empty.") 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 776ddee21f795..adac93847ad22 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 @@ -77,7 +77,7 @@ class ExternalCatalogSchema( */ override def getTable(name: String): Table = try { val externalCatalogTable = catalog.getTable(name) - ExternalTableSourceUtil.fromExternalCatalogTable(tableEnv, externalCatalogTable) + ExternalTableUtil.fromExternalCatalogTable(tableEnv, externalCatalogTable) } catch { case TableNotExistException(table, _, _) => { LOG.warn(s"Table $table does not exist in externalCatalog $catalogIdentifier") diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/catalog/ExternalCatalogTable.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/catalog/ExternalCatalogTable.scala index c41a3505a3e97..79da852f7b560 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/catalog/ExternalCatalogTable.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/catalog/ExternalCatalogTable.scala @@ -18,33 +18,326 @@ package org.apache.flink.table.catalog +import org.apache.flink.table.descriptors.DescriptorProperties.toScala +import org.apache.flink.table.descriptors.StatisticsValidator.{STATISTICS_COLUMNS, STATISTICS_ROW_COUNT, readColumnStats} +import org.apache.flink.table.descriptors.StreamTableDescriptorValidator.{UPDATE_MODE, UPDATE_MODE_VALUE_APPEND, UPDATE_MODE_VALUE_RETRACT, UPDATE_MODE_VALUE_UPSERT} import org.apache.flink.table.descriptors._ +import org.apache.flink.table.factories.TableFactory import org.apache.flink.table.plan.stats.TableStats +import scala.collection.JavaConverters._ + /** - * Defines a table in an [[ExternalCatalog]]. + * Defines a table in an [[ExternalCatalog]]. External catalog tables describe table sources + * and/or sinks for both batch and stream environments. + * + * See also [[TableFactory]] for more information about how to target suitable factories. + * + * Use [[ExternalCatalogTableBuilder]] to integrate with the normalized descriptor-based API. * - * @param connectorDesc describes the system to connect to - * @param formatDesc describes the data format of a connector - * @param schemaDesc describes the schema of the result table - * @param statisticsDesc describes the estimated statistics of the result table - * @param metadataDesc describes additional metadata of a table + * @param isBatch Flag whether this external table is intended for batch environments. + * @param isStreaming Flag whether this external table is intended for streaming environments. + * @param isSource Flag whether this external table is declared as table source. + * @param isSink Flag whether this external table is declared as table sink. + * @param properties Properties that describe the table and should match with a [[TableFactory]]. */ class ExternalCatalogTable( - connectorDesc: ConnectorDescriptor, - formatDesc: Option[FormatDescriptor], - schemaDesc: Option[Schema], - statisticsDesc: Option[Statistics], - metadataDesc: Option[Metadata]) - extends TableSourceDescriptor { - - this.connectorDescriptor = Some(connectorDesc) - this.formatDescriptor = formatDesc - this.schemaDescriptor = schemaDesc - this.statisticsDescriptor = statisticsDesc - this.metaDescriptor = metadataDesc - - // expose statistics for external table source util - override def getTableStats: Option[TableStats] = super.getTableStats + private val isBatch: Boolean, + private val isStreaming: Boolean, + private val isSource: Boolean, + private val isSink: Boolean, + private val properties: java.util.Map[String, String]) + extends TableDescriptor { + + // ---------------------------------------------------------------------------------------------- + // Legacy code + // --------------------------------------------------------------------------------------------- + + /** + * Reads table statistics from the descriptors properties. + * + * @deprecated This method exists for backwards-compatibility only. + */ + @Deprecated + @deprecated + def getTableStats: Option[TableStats] = { + val normalizedProps = new DescriptorProperties() + addProperties(normalizedProps) + val rowCount = toScala(normalizedProps.getOptionalLong(STATISTICS_ROW_COUNT)) + rowCount match { + case Some(cnt) => + val columnStats = readColumnStats(normalizedProps, STATISTICS_COLUMNS) + Some(TableStats(cnt, columnStats.asJava)) + case None => + None + } + } + + // ---------------------------------------------------------------------------------------------- + // Getters + // ---------------------------------------------------------------------------------------------- + + /** + * Returns whether this external table is declared as table source. + */ + def isTableSource: Boolean = { + isSource + } + + /** + * Returns whether this external table is declared as table sink. + */ + def isTableSink: Boolean = { + isSource + } + + /** + * Returns whether this external table is intended for batch environments. + */ + def isBatchTable: Boolean = { + isBatch + } + + /** + * Returns whether this external table is intended for stream environments. + */ + def isStreamTable: Boolean = { + isStreaming + } + + // ---------------------------------------------------------------------------------------------- + + /** + * Internal method for properties conversion. + */ + override private[flink] def addProperties(descriptorProperties: DescriptorProperties): Unit = { + descriptorProperties.putProperties(properties) + } +} + +object ExternalCatalogTable { + + /** + * Creates a builder for creating an [[ExternalCatalogTable]]. + * + * It takes [[Descriptor]]s which allow for declaring the communication to external + * systems in an implementation-agnostic way. The classpath is scanned for suitable table + * factories that match the desired configuration. + * + * Use the provided builder methods to configure the external catalog table accordingly. + * + * The following example shows how to read from a connector using a JSON format and + * declaring it as a table source: + * + * {{{ + * ExternalCatalogTable( + * new ExternalSystemXYZ() + * .version("0.11")) + * .withFormat( + * new Json() + * .jsonSchema("{...}") + * .failOnMissingField(false)) + * .withSchema( + * new Schema() + * .field("user-name", "VARCHAR").from("u_name") + * .field("count", "DECIMAL") + * .supportsStreaming() + * .asTableSource() + * }}} + * + * @param connectorDescriptor Connector descriptor describing the external system + * @return External catalog builder + */ + def builder(connectorDescriptor: ConnectorDescriptor): ExternalCatalogTableBuilder = { + new ExternalCatalogTableBuilder(connectorDescriptor) + } +} + +/** + * Builder for an [[ExternalCatalogTable]]. + * + * @param connectorDescriptor Connector descriptor describing the external system + */ +class ExternalCatalogTableBuilder(private val connectorDescriptor: ConnectorDescriptor) + extends TableDescriptor + with SchematicDescriptor[ExternalCatalogTableBuilder] + with StreamableDescriptor[ExternalCatalogTableBuilder] { + + private var isBatch: Boolean = true + private var isStreaming: Boolean = true + + private var formatDescriptor: Option[FormatDescriptor] = None + private var schemaDescriptor: Option[Schema] = None + private var statisticsDescriptor: Option[Statistics] = None + private var metadataDescriptor: Option[Metadata] = None + private var updateMode: Option[String] = None + + /** + * Specifies the format that defines how to read data from a connector. + */ + override def withFormat(format: FormatDescriptor): ExternalCatalogTableBuilder = { + formatDescriptor = Some(format) + this + } + + /** + * Specifies the resulting table schema. + */ + override def withSchema(schema: Schema): ExternalCatalogTableBuilder = { + schemaDescriptor = Some(schema) + this + } + + /** + * Declares how to perform the conversion between a dynamic table and an external connector. + * + * In append mode, a dynamic table and an external connector only exchange INSERT messages. + * + * @see See also [[inRetractMode()]] and [[inUpsertMode()]]. + */ + override def inAppendMode(): ExternalCatalogTableBuilder = { + updateMode = Some(UPDATE_MODE_VALUE_APPEND) + this + } + + /** + * Declares how to perform the conversion between a dynamic table and an external connector. + * + * In retract mode, a dynamic table and an external connector exchange ADD and RETRACT messages. + * + * An INSERT change is encoded as an ADD message, a DELETE change as a RETRACT message, and an + * UPDATE change as a RETRACT message for the updated (previous) row and an ADD message for + * the updating (new) row. + * + * In this mode, a key must not be defined as opposed to upsert mode. However, every update + * consists of two messages which is less efficient. + * + * @see See also [[inAppendMode()]] and [[inUpsertMode()]]. + */ + override def inRetractMode(): ExternalCatalogTableBuilder = { + updateMode = Some(UPDATE_MODE_VALUE_RETRACT) + this + } + + /** + * Declares how to perform the conversion between a dynamic table and an external connector. + * + * In upsert mode, a dynamic table and an external connector exchange UPSERT and DELETE messages. + * + * This mode requires a (possibly composite) unique key by which updates can be propagated. The + * external connector needs to be aware of the unique key attribute in order to apply messages + * correctly. INSERT and UPDATE changes are encoded as UPSERT messages. DELETE changes as + * DELETE messages. + * + * The main difference to a retract stream is that UPDATE changes are encoded with a single + * message and are therefore more efficient. + * + * @see See also [[inAppendMode()]] and [[inRetractMode()]]. + */ + override def inUpsertMode(): ExternalCatalogTableBuilder = { + updateMode = Some(UPDATE_MODE_VALUE_UPSERT) + this + } + + /** + * Specifies the statistics for this external table. + */ + def withStatistics(statistics: Statistics): ExternalCatalogTableBuilder = { + statisticsDescriptor = Some(statistics) + this + } + + /** + * Specifies the metadata for this external table. + */ + def withMetadata(metadata: Metadata): ExternalCatalogTableBuilder = { + metadataDescriptor = Some(metadata) + this + } + + /** + * Explicitly declares this external table for supporting only stream environments. + */ + def supportsStreaming(): ExternalCatalogTableBuilder = { + isBatch = false + isStreaming = true + this + } + + /** + * Explicitly declares this external table for supporting only batch environments. + */ + def supportsBatch(): ExternalCatalogTableBuilder = { + isBatch = false + isStreaming = true + this + } + + /** + * Explicitly declares this external table for supporting both batch and stream environments. + */ + def supportsBatchAndStreaming(): ExternalCatalogTableBuilder = { + isBatch = true + isStreaming = true + this + } + + /** + * Declares this external table as a table source and returns the + * configured [[ExternalCatalogTable]]. + * + * @return External catalog table + */ + def asTableSource(): ExternalCatalogTable = { + new ExternalCatalogTable( + isBatch, + isStreaming, + isSource = true, + isSink = false, + DescriptorProperties.toJavaMap(this)) + } + + /** + * Declares this external table as a table sink and returns the + * configured [[ExternalCatalogTable]]. + * + * @return External catalog table + */ + def asTableSink(): ExternalCatalogTable = { + new ExternalCatalogTable( + isBatch, + isStreaming, + isSource = false, + isSink = true, + DescriptorProperties.toJavaMap(this)) + } + + /** + * Declares this external table as both a table source and sink. It returns the + * configured [[ExternalCatalogTable]]. + * + * @return External catalog table + */ + def asTableSourceAndSink(): ExternalCatalogTable = { + new ExternalCatalogTable( + isBatch, + isStreaming, + isSource = true, + isSink = true, + DescriptorProperties.toJavaMap(this)) + } + + // ---------------------------------------------------------------------------------------------- + /** + * Internal method for properties conversion. + */ + override private[flink] def addProperties(properties: DescriptorProperties): Unit = { + connectorDescriptor.addProperties(properties) + formatDescriptor.foreach(_.addProperties(properties)) + schemaDescriptor.foreach(_.addProperties(properties)) + statisticsDescriptor.foreach(_.addProperties(properties)) + metadataDescriptor.foreach(_.addProperties(properties)) + updateMode.foreach(mode => properties.putString(UPDATE_MODE, mode)) + } } 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 deleted file mode 100644 index 011cbecb454b1..0000000000000 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/catalog/ExternalTableSourceUtil.scala +++ /dev/null @@ -1,70 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.flink.table.catalog - -import org.apache.flink.table.api._ -import org.apache.flink.table.descriptors.DescriptorProperties -import org.apache.flink.table.factories.{BatchTableSourceFactory, StreamTableSourceFactory, TableFactoryService} -import org.apache.flink.table.plan.schema.{BatchTableSourceTable, StreamTableSourceTable, TableSourceSinkTable, TableSourceTable} -import org.apache.flink.table.plan.stats.FlinkStatistic -import org.apache.flink.table.util.Logging - -/** - * The utility class is used to convert ExternalCatalogTable to TableSourceTable. - */ -object ExternalTableSourceUtil extends Logging { - - /** - * Converts an [[ExternalCatalogTable]] instance to a [[TableSourceTable]] instance - * - * @param externalCatalogTable the [[ExternalCatalogTable]] instance which to convert - * @return converted [[TableSourceTable]] instance from the input catalog table - */ - def fromExternalCatalogTable( - tableEnv: TableEnvironment, - externalCatalogTable: ExternalCatalogTable) - : TableSourceSinkTable[_, _] = { - val properties = new DescriptorProperties() - externalCatalogTable.addProperties(properties) - val javaMap = properties.asMap - tableEnv match { - // check for a batch table source in this batch environment - case _: BatchTableEnvironment => - val source = TableFactoryService - .find(classOf[BatchTableSourceFactory[_]], javaMap) - .createBatchTableSource(javaMap) - val sourceTable = new BatchTableSourceTable( - source, - new FlinkStatistic(externalCatalogTable.getTableStats)) - new TableSourceSinkTable(Some(sourceTable), None) - - // check for a stream table source in this streaming environment - case _: StreamTableEnvironment => - val source = TableFactoryService - .find(classOf[StreamTableSourceFactory[_]], javaMap) - .createStreamTableSource(javaMap) - val sourceTable = new StreamTableSourceTable( - source, - new FlinkStatistic(externalCatalogTable.getTableStats)) - new TableSourceSinkTable(Some(sourceTable), None) - - case _ => throw new TableException("Unsupported table environment.") - } - } -} diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/catalog/ExternalTableUtil.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/catalog/ExternalTableUtil.scala new file mode 100644 index 0000000000000..ec57c5ed2c20d --- /dev/null +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/catalog/ExternalTableUtil.scala @@ -0,0 +1,102 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.table.catalog + +import org.apache.flink.table.api._ +import org.apache.flink.table.factories._ +import org.apache.flink.table.plan.schema._ +import org.apache.flink.table.plan.stats.FlinkStatistic +import org.apache.flink.table.sinks.{BatchTableSink, StreamTableSink} +import org.apache.flink.table.sources.{BatchTableSource, StreamTableSource} +import org.apache.flink.table.util.Logging + + +/** + * The utility class is used to convert [[ExternalCatalogTable]] to [[TableSourceSinkTable]]. + * + * It uses [[TableFactoryService]] for discovering. + */ +object ExternalTableUtil extends Logging { + + /** + * Converts an [[ExternalCatalogTable]] instance to a [[TableSourceTable]] instance + * + * @param externalTable the [[ExternalCatalogTable]] instance which to convert + * @return converted [[TableSourceTable]] instance from the input catalog table + */ + def fromExternalCatalogTable[T1, T2]( + tableEnv: TableEnvironment, + externalTable: ExternalCatalogTable) + : TableSourceSinkTable[T1, T2] = { + + val statistics = new FlinkStatistic(externalTable.getTableStats) + + val source: Option[TableSourceTable[T1]] = if (externalTable.isTableSource) { + Some(createTableSource(tableEnv, externalTable, statistics)) + } else { + None + } + + val sink: Option[TableSinkTable[T2]] = if (externalTable.isTableSink) { + Some(createTableSink(tableEnv, externalTable, statistics)) + } else { + None + } + + new TableSourceSinkTable[T1, T2](source, sink) + } + + private def createTableSource[T]( + tableEnv: TableEnvironment, + externalTable: ExternalCatalogTable, + statistics: FlinkStatistic) + : TableSourceTable[T] = tableEnv match { + + case _: BatchTableEnvironment if externalTable.isBatchTable => + val source = TableFactoryUtil.findAndCreateTableSource(tableEnv, externalTable) + new BatchTableSourceTable[T](source.asInstanceOf[BatchTableSource[T]], statistics) + + case _: StreamTableEnvironment if externalTable.isStreamTable => + val source = TableFactoryUtil.findAndCreateTableSource(tableEnv, externalTable) + new StreamTableSourceTable[T](source.asInstanceOf[StreamTableSource[T]], statistics) + + case _ => + throw new ValidationException( + "External catalog table does not support the current environment for a table source.") + } + + private def createTableSink[T]( + tableEnv: TableEnvironment, + externalTable: ExternalCatalogTable, + statistics: FlinkStatistic) + : TableSinkTable[T] = tableEnv match { + + case _: BatchTableEnvironment if externalTable.isBatchTable => + val sink = TableFactoryUtil.findAndCreateTableSink(tableEnv, externalTable) + new TableSinkTable[T](sink.asInstanceOf[BatchTableSink[T]], statistics) + + case _: StreamTableEnvironment if externalTable.isStreamTable => + val sink = TableFactoryUtil.findAndCreateTableSink(tableEnv, externalTable) + new TableSinkTable[T](sink.asInstanceOf[StreamTableSink[T]], statistics) + + case _ => + throw new ValidationException( + "External catalog table does not support the current environment for a table sink.") + } +} diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/descriptors/TableSinkDescriptor.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/descriptors/BatchTableDescriptor.scala similarity index 71% rename from flink-libraries/flink-table/src/main/scala/org/apache/flink/table/descriptors/TableSinkDescriptor.scala rename to flink-libraries/flink-table/src/main/scala/org/apache/flink/table/descriptors/BatchTableDescriptor.scala index 0a4d5044ff8a5..6bd2a711f0a07 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/descriptors/TableSinkDescriptor.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/descriptors/BatchTableDescriptor.scala @@ -18,15 +18,14 @@ package org.apache.flink.table.descriptors +import org.apache.flink.table.api.BatchTableEnvironment + /** - * Common class for all descriptors describing a table sink. + * Descriptor for specifying a table source and/or sink in a batch environment. */ -abstract class TableSinkDescriptor extends TableDescriptor { - - /** - * Internal method for properties conversion. - */ - override private[flink] def addProperties(properties: DescriptorProperties): Unit = { - super.addProperties(properties) - } -} +class BatchTableDescriptor( + tableEnv: BatchTableEnvironment, + connectorDescriptor: ConnectorDescriptor) + extends ConnectTableDescriptor[BatchTableDescriptor]( + tableEnv, + connectorDescriptor) diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/descriptors/BatchTableSourceDescriptor.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/descriptors/BatchTableSourceDescriptor.scala deleted file mode 100644 index c967291d25489..0000000000000 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/descriptors/BatchTableSourceDescriptor.scala +++ /dev/null @@ -1,87 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.flink.table.descriptors - -import org.apache.flink.table.api.{BatchTableEnvironment, Table, ValidationException} -import org.apache.flink.table.factories.{BatchTableSourceFactory, TableFactoryService} -import org.apache.flink.table.sources.TableSource - -class BatchTableSourceDescriptor(tableEnv: BatchTableEnvironment, connector: ConnectorDescriptor) - extends TableSourceDescriptor { - - connectorDescriptor = Some(connector) - - override private[flink] def addProperties(properties: DescriptorProperties): Unit = { - // check for a format - if (connector.needsFormat() && formatDescriptor.isEmpty) { - throw new ValidationException( - s"The connector '$connector' requires a format description.") - } else if (!connector.needsFormat() && formatDescriptor.isDefined) { - throw new ValidationException( - s"The connector '$connector' does not require a format description " + - s"but '${formatDescriptor.get}' found.") - } - super.addProperties(properties) - } - - /** - * Searches for the specified table source, configures it accordingly, and returns it. - */ - def toTableSource: TableSource[_] = { - val properties = new DescriptorProperties() - addProperties(properties) - val javaMap = properties.asMap - TableFactoryService - .find(classOf[BatchTableSourceFactory[_]], javaMap) - .createBatchTableSource(javaMap) - } - - /** - * Searches for the specified table source, configures it accordingly, and returns it as a table. - */ - def toTable: Table = { - tableEnv.fromTableSource(toTableSource) - } - - /** - * Searches for the specified table source, configures it accordingly, and registers it as - * a table under the given name. - * - * @param name table name to be registered in the table environment - */ - def register(name: String): Unit = { - tableEnv.registerTableSource(name, toTableSource) - } - - /** - * Specifies the format that defines how to read data from a connector. - */ - def withFormat(format: FormatDescriptor): BatchTableSourceDescriptor = { - formatDescriptor = Some(format) - this - } - - /** - * Specifies the resulting table schema. - */ - def withSchema(schema: Schema): BatchTableSourceDescriptor = { - schemaDescriptor = Some(schema) - this - } -} diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/descriptors/ConnectTableDescriptor.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/descriptors/ConnectTableDescriptor.scala new file mode 100644 index 0000000000000..569b825993880 --- /dev/null +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/descriptors/ConnectTableDescriptor.scala @@ -0,0 +1,108 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.table.descriptors + +import org.apache.flink.table.api.{TableEnvironment, ValidationException} +import org.apache.flink.table.factories.TableFactoryUtil + +/** + * Common class for table's created with [[TableEnvironment.connect(ConnectorDescriptor)]]. + */ +abstract class ConnectTableDescriptor[D <: ConnectTableDescriptor[D]]( + private val tableEnv: TableEnvironment, + private val connectorDescriptor: ConnectorDescriptor) + extends TableDescriptor + with SchematicDescriptor[D] + with RegistrableDescriptor { this: D => + + private var formatDescriptor: Option[FormatDescriptor] = None + private var schemaDescriptor: Option[Schema] = None + + /** + * Searches for the specified table source, configures it accordingly, and registers it as + * a table under the given name. + * + * @param name table name to be registered in the table environment + */ + override def registerTableSource(name: String): Unit = { + val tableSource = TableFactoryUtil.findAndCreateTableSource(tableEnv, this) + tableEnv.registerTableSource(name, tableSource) + } + + /** + * Searches for the specified table sink, configures it accordingly, and registers it as + * a table under the given name. + * + * @param name table name to be registered in the table environment + */ + override def registerTableSink(name: String): Unit = { + val tableSink = TableFactoryUtil.findAndCreateTableSink(tableEnv, this) + tableEnv.registerTableSink(name, tableSink) + } + + /** + * Searches for the specified table source and sink, configures them accordingly, and registers + * them as a table under the given name. + * + * @param name table name to be registered in the table environment + */ + override def registerTableSourceAndSink(name: String): Unit = { + registerTableSource(name) + registerTableSink(name) + } + + /** + * Specifies the format that defines how to read data from a connector. + */ + override def withFormat(format: FormatDescriptor): D = { + formatDescriptor = Some(format) + this + } + + /** + * Specifies the resulting table schema. + */ + override def withSchema(schema: Schema): D = { + schemaDescriptor = Some(schema) + this + } + + // ---------------------------------------------------------------------------------------------- + + /** + * Internal method for properties conversion. + */ + override private[flink] def addProperties(properties: DescriptorProperties): Unit = { + + // this performs only basic validation + // more validation can only happen within a factory + if (connectorDescriptor.needsFormat() && formatDescriptor.isEmpty) { + throw new ValidationException( + s"The connector '$connectorDescriptor' requires a format description.") + } else if (!connectorDescriptor.needsFormat() && formatDescriptor.isDefined) { + throw new ValidationException( + s"The connector '$connectorDescriptor' does not require a format description " + + s"but '${formatDescriptor.get}' found.") + } + + connectorDescriptor.addProperties(properties) + formatDescriptor.foreach(_.addProperties(properties)) + schemaDescriptor.foreach(_.addProperties(properties)) + } +} diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/descriptors/Descriptor.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/descriptors/Descriptor.scala index e21527bfa4f8d..aa96bc4dd3a9a 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/descriptors/Descriptor.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/descriptors/Descriptor.scala @@ -19,18 +19,23 @@ package org.apache.flink.table.descriptors /** - * A class that adds a set of string-based, normalized properties for describing DDL information. + * A trait that adds a set of string-based, normalized properties for describing DDL information. * * Typical characteristics of a descriptor are: * - descriptors have a default constructor and a default 'apply()' method for Scala * - descriptors themselves contain very little logic * - corresponding validators validate the correctness (goal: have a single point of validation) */ -abstract class Descriptor { +trait Descriptor { /** * Internal method for properties conversion. */ private[flink] def addProperties(properties: DescriptorProperties): Unit + override def toString: String = { + val descriptorProperties = new DescriptorProperties() + addProperties(descriptorProperties) + descriptorProperties.toString + } } diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/descriptors/DescriptorProperties.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/descriptors/DescriptorProperties.scala index 3ad3eac83d710..2c88dfd141e73 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/descriptors/DescriptorProperties.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/descriptors/DescriptorProperties.scala @@ -1064,6 +1064,10 @@ class DescriptorProperties(normalizeKeys: Boolean = true) { properties.toMap.asJava } + override def toString: String = { + DescriptorProperties.toString(properties.toMap) + } + // ---------------------------------------------------------------------------------------------- /** @@ -1283,6 +1287,12 @@ object DescriptorProperties { .mkString("\n") } + def toJavaMap(descriptor: Descriptor): util.Map[String, String] = { + val descriptorProperties = new DescriptorProperties() + descriptor.addProperties(descriptorProperties) + descriptorProperties.asMap + } + // the following methods help for Scala <-> Java interfaces // most of these methods are not necessary once we upgraded to Scala 2.12 diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/descriptors/RegistrableDescriptor.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/descriptors/RegistrableDescriptor.scala new file mode 100644 index 0000000000000..e89ca8cfd49b6 --- /dev/null +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/descriptors/RegistrableDescriptor.scala @@ -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.table.descriptors + +/** + * A trait for descriptors that allow to register table source and/or sinks. + */ +trait RegistrableDescriptor extends TableDescriptor { + + /** + * Searches for the specified table source, configures it accordingly, and registers it as + * a table under the given name. + * + * @param name table name to be registered in the table environment + */ + def registerTableSource(name: String): Unit + + /** + * Searches for the specified table sink, configures it accordingly, and registers it as + * a table under the given name. + * + * @param name table name to be registered in the table environment + */ + def registerTableSink(name: String): Unit + + /** + * Searches for the specified table source and sink, configures them accordingly, and registers + * them as a table under the given name. + * + * @param name table name to be registered in the table environment + */ + def registerTableSourceAndSink(name: String): Unit +} diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/descriptors/TableDescriptorValidator.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/descriptors/SchematicDescriptor.scala similarity index 69% rename from flink-libraries/flink-table/src/main/scala/org/apache/flink/table/descriptors/TableDescriptorValidator.scala rename to flink-libraries/flink-table/src/main/scala/org/apache/flink/table/descriptors/SchematicDescriptor.scala index e0fa6025811a3..794ff9e73bf99 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/descriptors/TableDescriptorValidator.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/descriptors/SchematicDescriptor.scala @@ -19,11 +19,17 @@ package org.apache.flink.table.descriptors /** - * Validator for [[TableDescriptor]]. + * A trait for descriptors that allow to define a format and schema. */ -class TableDescriptorValidator extends DescriptorValidator { +trait SchematicDescriptor[D <: SchematicDescriptor[D]] extends TableDescriptor { - override def validate(properties: DescriptorProperties): Unit = { - // nothing to do - } + /** + * Specifies the format that defines how to read data from a connector. + */ + def withFormat(format: FormatDescriptor): D + + /** + * Specifies the resulting table schema. + */ + def withSchema(schema: Schema): D } diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/descriptors/StreamTableDescriptor.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/descriptors/StreamTableDescriptor.scala new file mode 100644 index 0000000000000..b9e64f90c9030 --- /dev/null +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/descriptors/StreamTableDescriptor.scala @@ -0,0 +1,101 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.table.descriptors + +import org.apache.flink.table.api.StreamTableEnvironment +import org.apache.flink.table.descriptors.StreamTableDescriptorValidator._ + +/** + * Descriptor for specifying a table source and/or sink in a streaming environment. + */ +class StreamTableDescriptor( + tableEnv: StreamTableEnvironment, + connectorDescriptor: ConnectorDescriptor) + extends ConnectTableDescriptor[StreamTableDescriptor]( + tableEnv, + connectorDescriptor) + with StreamableDescriptor[StreamTableDescriptor] { + + private var updateMode: Option[String] = None + + /** + * Declares how to perform the conversion between a dynamic table and an external connector. + * + * In append mode, a dynamic table and an external connector only exchange INSERT messages. + * + * @see See also [[inRetractMode()]] and [[inUpsertMode()]]. + */ + override def inAppendMode(): StreamTableDescriptor = { + updateMode = Some(UPDATE_MODE_VALUE_APPEND) + this + } + + /** + * Declares how to perform the conversion between a dynamic table and an external connector. + * + * In retract mode, a dynamic table and an external connector exchange ADD and RETRACT messages. + * + * An INSERT change is encoded as an ADD message, a DELETE change as a RETRACT message, and an + * UPDATE change as a RETRACT message for the updated (previous) row and an ADD message for + * the updating (new) row. + * + * In this mode, a key must not be defined as opposed to upsert mode. However, every update + * consists of two messages which is less efficient. + * + * @see See also [[inAppendMode()]] and [[inUpsertMode()]]. + */ + override def inRetractMode(): StreamTableDescriptor = { + updateMode = Some(UPDATE_MODE_VALUE_RETRACT) + this + } + + /** + * Declares how to perform the conversion between a dynamic table and an external connector. + * + * In upsert mode, a dynamic table and an external connector exchange UPSERT and DELETE messages. + * + * This mode requires a (possibly composite) unique key by which updates can be propagated. The + * external connector needs to be aware of the unique key attribute in order to apply messages + * correctly. INSERT and UPDATE changes are encoded as UPSERT messages. DELETE changes as + * DELETE messages. + * + * The main difference to a retract stream is that UPDATE changes are encoded with a single + * message and are therefore more efficient. + * + * @see See also [[inAppendMode()]] and [[inRetractMode()]]. + */ + override def inUpsertMode(): StreamTableDescriptor = { + updateMode = Some(UPDATE_MODE_VALUE_UPSERT) + this + } + + // ---------------------------------------------------------------------------------------------- + + /** + * Internal method for properties conversion. + */ + override private[flink] def addProperties(properties: DescriptorProperties): Unit = { + super.addProperties(properties) + updateMode.foreach(mode => properties.putString(UPDATE_MODE, mode)) + + // this performs only basic validation + // more validation can only happen within a factory + new StreamTableDescriptorValidator().validate(properties) + } +} diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/descriptors/StreamTableDescriptorValidator.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/descriptors/StreamTableDescriptorValidator.scala new file mode 100644 index 0000000000000..5a6a946ea078b --- /dev/null +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/descriptors/StreamTableDescriptorValidator.scala @@ -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.table.descriptors + +import java.util + +import org.apache.flink.table.descriptors.StreamTableDescriptorValidator._ + +/** + * Validator for [[StreamTableDescriptor]]. + */ +class StreamTableDescriptorValidator extends DescriptorValidator { + + override def validate(properties: DescriptorProperties): Unit = { + properties.validateEnumValues( + UPDATE_MODE, + isOptional = false, + util.Arrays.asList( + UPDATE_MODE_VALUE_APPEND, + UPDATE_MODE_VALUE_RETRACT, + UPDATE_MODE_VALUE_UPSERT) + ) + } +} + +object StreamTableDescriptorValidator { + + val UPDATE_MODE = "update-mode" + val UPDATE_MODE_VALUE_APPEND = "append" + val UPDATE_MODE_VALUE_RETRACT = "retract" + val UPDATE_MODE_VALUE_UPSERT = "upsert" +} diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/descriptors/StreamTableSourceDescriptor.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/descriptors/StreamTableSourceDescriptor.scala deleted file mode 100644 index 6ade2d6699a42..0000000000000 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/descriptors/StreamTableSourceDescriptor.scala +++ /dev/null @@ -1,90 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.flink.table.descriptors - -import org.apache.flink.table.api.{StreamTableEnvironment, Table, ValidationException} -import org.apache.flink.table.factories.{StreamTableSourceFactory, TableFactoryService} -import org.apache.flink.table.sources.TableSource - -/** - * Descriptor for specifying a table source in a streaming environment. - */ -class StreamTableSourceDescriptor(tableEnv: StreamTableEnvironment, connector: ConnectorDescriptor) - extends TableSourceDescriptor { - - connectorDescriptor = Some(connector) - - override private[flink] def addProperties(properties: DescriptorProperties): Unit = { - // check for a format - if (connector.needsFormat() && formatDescriptor.isEmpty) { - throw new ValidationException( - s"The connector '$connector' requires a format description.") - } else if (!connector.needsFormat() && formatDescriptor.isDefined) { - throw new ValidationException( - s"The connector '$connector' does not require a format description " + - s"but '${formatDescriptor.get}' found.") - } - super.addProperties(properties) - } - - /** - * Searches for the specified table source, configures it accordingly, and returns it. - */ - def toTableSource: TableSource[_] = { - val properties = new DescriptorProperties() - addProperties(properties) - val javaMap = properties.asMap - TableFactoryService - .find(classOf[StreamTableSourceFactory[_]], javaMap) - .createStreamTableSource(javaMap) - } - - /** - * Searches for the specified table source, configures it accordingly, and returns it as a table. - */ - def toTable: Table = { - tableEnv.fromTableSource(toTableSource) - } - - /** - * Searches for the specified table source, configures it accordingly, and registers it as - * a table under the given name. - * - * @param name table name to be registered in the table environment - */ - def register(name: String): Unit = { - tableEnv.registerTableSource(name, toTableSource) - } - - /** - * Specifies the format that defines how to read data from a connector. - */ - def withFormat(format: FormatDescriptor): StreamTableSourceDescriptor = { - formatDescriptor = Some(format) - this - } - - /** - * Specifies the resulting table schema. - */ - def withSchema(schema: Schema): StreamTableSourceDescriptor = { - schemaDescriptor = Some(schema) - this - } -} diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/descriptors/StreamableDescriptor.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/descriptors/StreamableDescriptor.scala new file mode 100644 index 0000000000000..0d424bd65ff01 --- /dev/null +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/descriptors/StreamableDescriptor.scala @@ -0,0 +1,67 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.table.descriptors + +/** + * A trait for descriptors that allow to convert between a dynamic table and an external connector. + */ +trait StreamableDescriptor[D <: StreamableDescriptor[D]] extends TableDescriptor { + + /** + * Declares how to perform the conversion between a dynamic table and an external connector. + * + * In append mode, a dynamic table and an external connector only exchange INSERT messages. + * + * @see See also [[inRetractMode()]] and [[inUpsertMode()]]. + */ + def inAppendMode(): D + + /** + * Declares how to perform the conversion between a dynamic table and an external connector. + * + * In retract mode, a dynamic table and an external connector exchange ADD and RETRACT messages. + * + * An INSERT change is encoded as an ADD message, a DELETE change as a RETRACT message, and an + * UPDATE change as a RETRACT message for the updated (previous) row and an ADD message for + * the updating (new) row. + * + * In this mode, a key must not be defined as opposed to upsert mode. However, every update + * consists of two messages which is less efficient. + * + * @see See also [[inAppendMode()]] and [[inUpsertMode()]]. + */ + def inRetractMode(): D + + /** + * Declares how to perform the conversion between a dynamic table and an external connector. + * + * In upsert mode, a dynamic table and an external connector exchange UPSERT and DELETE messages. + * + * This mode requires a (possibly composite) unique key by which updates can be propagated. The + * external connector needs to be aware of the unique key attribute in order to apply messages + * correctly. INSERT and UPDATE changes are encoded as UPSERT messages. DELETE changes as + * DELETE messages. + * + * The main difference to a retract stream is that UPDATE changes are encoded with a single + * message and are therefore more efficient. + * + * @see See also [[inAppendMode()]] and [[inRetractMode()]]. + */ + def inUpsertMode(): D +} diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/descriptors/TableDescriptor.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/descriptors/TableDescriptor.scala index 7b864d874e5c6..b14a31077b80e 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/descriptors/TableDescriptor.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/descriptors/TableDescriptor.scala @@ -19,22 +19,6 @@ package org.apache.flink.table.descriptors /** - * Common class for all descriptors describing table sources and sinks. + * Common trait for all descriptors describing table sources and sinks. */ -abstract class TableDescriptor extends Descriptor { - - protected var connectorDescriptor: Option[ConnectorDescriptor] = None - protected var formatDescriptor: Option[FormatDescriptor] = None - protected var schemaDescriptor: Option[Schema] = None - protected var metaDescriptor: Option[Metadata] = None - - /** - * Internal method for properties conversion. - */ - override private[flink] def addProperties(properties: DescriptorProperties): Unit = { - connectorDescriptor.foreach(_.addProperties(properties)) - formatDescriptor.foreach(_.addProperties(properties)) - schemaDescriptor.foreach(_.addProperties(properties)) - metaDescriptor.foreach(_.addProperties(properties)) - } -} +trait TableDescriptor extends Descriptor diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/descriptors/TableSourceDescriptor.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/descriptors/TableSourceDescriptor.scala deleted file mode 100644 index 3ca39c2bbfca2..0000000000000 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/descriptors/TableSourceDescriptor.scala +++ /dev/null @@ -1,57 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.flink.table.descriptors - -import org.apache.flink.table.descriptors.DescriptorProperties.toScala -import org.apache.flink.table.descriptors.StatisticsValidator.{STATISTICS_COLUMNS, STATISTICS_ROW_COUNT, readColumnStats} -import org.apache.flink.table.plan.stats.TableStats - -import scala.collection.JavaConverters._ - -/** - * Common class for all descriptors describing a table source. - */ -abstract class TableSourceDescriptor extends TableDescriptor { - - protected var statisticsDescriptor: Option[Statistics] = None - - /** - * Internal method for properties conversion. - */ - override private[flink] def addProperties(properties: DescriptorProperties): Unit = { - super.addProperties(properties) - statisticsDescriptor.foreach(_.addProperties(properties)) - } - - /** - * Reads table statistics from the descriptors properties. - */ - protected def getTableStats: Option[TableStats] = { - val normalizedProps = new DescriptorProperties() - addProperties(normalizedProps) - val rowCount = toScala(normalizedProps.getOptionalLong(STATISTICS_ROW_COUNT)) - rowCount match { - case Some(cnt) => - val columnStats = readColumnStats(normalizedProps, STATISTICS_COLUMNS) - Some(TableStats(cnt, columnStats.asJava)) - case None => - None - } - } -} diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/factories/TableFactory.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/factories/TableFactory.scala index cc99eccfe243b..a5dcc60c1432c 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/factories/TableFactory.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/factories/TableFactory.scala @@ -29,6 +29,12 @@ import java.util * Classes that implement this interface can be added to the * "META_INF/services/org.apache.flink.table.factories.TableFactory" file of a JAR file in * the current classpath to be found. + * + * @see [[BatchTableSourceFactory]] + * @see [[BatchTableSinkFactory]] + * @see [[StreamTableSourceFactory]] + * @see [[StreamTableSinkFactory]] + * @see [[TableFormatFactory]] */ trait TableFactory { diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/factories/TableFactoryService.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/factories/TableFactoryService.scala index 3baff8e87a80b..26b7c6df15afe 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/factories/TableFactoryService.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/factories/TableFactoryService.scala @@ -50,9 +50,7 @@ object TableFactoryService extends Logging { def find[T](factoryClass: Class[T], descriptor: Descriptor): T = { Preconditions.checkNotNull(descriptor) - val descriptorProperties = new DescriptorProperties() - descriptor.addProperties(descriptorProperties) - findInternal(factoryClass, descriptorProperties.asMap, None) + findInternal(factoryClass, DescriptorProperties.toJavaMap(descriptor), None) } /** @@ -68,9 +66,7 @@ object TableFactoryService extends Logging { Preconditions.checkNotNull(descriptor) Preconditions.checkNotNull(classLoader) - val descriptorProperties = new DescriptorProperties() - descriptor.addProperties(descriptorProperties) - findInternal(factoryClass, descriptorProperties.asMap, Some(classLoader)) + findInternal(factoryClass, DescriptorProperties.toJavaMap(descriptor), Some(classLoader)) } /** diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/factories/TableFactoryUtil.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/factories/TableFactoryUtil.scala new file mode 100644 index 0000000000000..9989ebcd08d0f --- /dev/null +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/factories/TableFactoryUtil.scala @@ -0,0 +1,82 @@ +/* + * 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.factories + +import org.apache.flink.table.api.{BatchTableEnvironment, StreamTableEnvironment, TableEnvironment, TableException} +import org.apache.flink.table.descriptors.{Descriptor, DescriptorProperties} +import org.apache.flink.table.sinks.TableSink +import org.apache.flink.table.sources.TableSource + +/** + * Utility for dealing with [[TableFactory]] using the [[TableFactoryService]]. + */ +object TableFactoryUtil { + + /** + * Returns a table source for a table environment. + */ + def findAndCreateTableSource[T]( + tableEnvironment: TableEnvironment, + descriptor: Descriptor) + : TableSource[T] = { + + val javaMap = DescriptorProperties.toJavaMap(descriptor) + + tableEnvironment match { + case _: BatchTableEnvironment => + TableFactoryService + .find(classOf[BatchTableSourceFactory[T]], javaMap) + .createBatchTableSource(javaMap) + + case _: StreamTableEnvironment => + TableFactoryService + .find(classOf[StreamTableSourceFactory[T]], javaMap) + .createStreamTableSource(javaMap) + + case e@_ => + throw new TableException(s"Unsupported table environment: ${e.getClass.getName}") + } + } + + /** + * Returns a table sink for a table environment. + */ + def findAndCreateTableSink[T]( + tableEnvironment: TableEnvironment, + descriptor: Descriptor) + : TableSink[T] = { + + val javaMap = DescriptorProperties.toJavaMap(descriptor) + + tableEnvironment match { + case _: BatchTableEnvironment => + TableFactoryService + .find(classOf[BatchTableSinkFactory[T]], javaMap) + .createBatchTableSink(javaMap) + + case _: StreamTableEnvironment => + TableFactoryService + .find(classOf[StreamTableSinkFactory[T]], javaMap) + .createStreamTableSink(javaMap) + + case e@_ => + throw new TableException(s"Unsupported table environment: ${e.getClass.getName}") + } + } +} diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/factories/TableFormatFactory.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/factories/TableFormatFactory.scala index 8fa6fadabe767..9e42a1560eeaa 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/factories/TableFormatFactory.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/factories/TableFormatFactory.scala @@ -25,6 +25,8 @@ import java.util * also [[TableFactory]] for more information. * * @tparam T record type that the format produces or consumes + * @see [[DeserializationSchemaFactory]] + * @see [[SerializationSchemaFactory]] */ trait TableFormatFactory[T] extends TableFactory { diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/sinks/CsvAppendTableSinkFactory.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/sinks/CsvAppendTableSinkFactory.scala new file mode 100644 index 0000000000000..65a41bbfd0dee --- /dev/null +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/sinks/CsvAppendTableSinkFactory.scala @@ -0,0 +1,45 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.table.sinks + +import java.util + +import org.apache.flink.table.descriptors.StreamTableDescriptorValidator.{UPDATE_MODE, UPDATE_MODE_VALUE_APPEND} +import org.apache.flink.table.factories.StreamTableSinkFactory +import org.apache.flink.types.Row + +/** + * Factory base for creating configured instances of [[CsvTableSink]] in a stream environment. + */ +class CsvAppendTableSinkFactory + extends CsvTableSinkFactoryBase + with StreamTableSinkFactory[Row] { + + override def requiredContext(): util.Map[String, String] = { + val context = new util.HashMap[String, String](super.requiredContext()) + context.put(UPDATE_MODE, UPDATE_MODE_VALUE_APPEND) + context + } + + override def createStreamTableSink( + properties: util.Map[String, String]) + : StreamTableSink[Row] = { + createTableSink(isStreaming = true, properties) + } +} diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/sinks/CsvBatchTableSinkFactory.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/sinks/CsvBatchTableSinkFactory.scala new file mode 100644 index 0000000000000..2687ed2135bd2 --- /dev/null +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/sinks/CsvBatchTableSinkFactory.scala @@ -0,0 +1,38 @@ +/* + * 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.sinks + +import java.util + +import org.apache.flink.table.factories.BatchTableSinkFactory +import org.apache.flink.types.Row + +/** + * Factory base for creating configured instances of [[CsvTableSink]] in a batch environment. + */ +class CsvBatchTableSinkFactory + extends CsvTableSinkFactoryBase + with BatchTableSinkFactory[Row] { + + override def createBatchTableSink( + properties: util.Map[String, String]) + : BatchTableSink[Row] = { + createTableSink(isStreaming = false, properties) + } +} diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/sinks/CsvTableSinkFactory.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/sinks/CsvTableSinkFactoryBase.scala similarity index 82% rename from flink-libraries/flink-table/src/main/scala/org/apache/flink/table/sinks/CsvTableSinkFactory.scala rename to flink-libraries/flink-table/src/main/scala/org/apache/flink/table/sinks/CsvTableSinkFactoryBase.scala index eb99f028edec5..6ceba4c47e1f5 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/sinks/CsvTableSinkFactory.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/sinks/CsvTableSinkFactoryBase.scala @@ -28,16 +28,12 @@ import org.apache.flink.table.descriptors.FileSystemValidator._ import org.apache.flink.table.descriptors.FormatDescriptorValidator._ import org.apache.flink.table.descriptors.SchemaValidator._ import org.apache.flink.table.descriptors._ -import org.apache.flink.table.factories.{BatchTableSinkFactory, StreamTableSinkFactory, TableFactory} -import org.apache.flink.types.Row +import org.apache.flink.table.factories.TableFactory /** - * Factory for creating configured instances of [[CsvTableSink]]. + * Factory base for creating configured instances of [[CsvTableSink]]. */ -class CsvTableSinkFactory - extends TableFactory - with StreamTableSinkFactory[Row] - with BatchTableSinkFactory[Row] { +abstract class CsvTableSinkFactoryBase extends TableFactory { override def requiredContext(): util.Map[String, String] = { val context = new util.HashMap[String, String]() @@ -63,19 +59,7 @@ class CsvTableSinkFactory properties } - override def createStreamTableSink( - properties: util.Map[String, String]) - : StreamTableSink[Row] = { - createTableSink(isStreaming = true, properties) - } - - override def createBatchTableSink( - properties: util.Map[String, String]) - : BatchTableSink[Row] = { - createTableSink(isStreaming = false, properties) - } - - private def createTableSink( + protected def createTableSink( isStreaming: Boolean, properties: util.Map[String, String]) : CsvTableSink = { diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/sources/CsvAppendTableSourceFactory.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/sources/CsvAppendTableSourceFactory.scala new file mode 100644 index 0000000000000..afbe2ea2dd1ad --- /dev/null +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/sources/CsvAppendTableSourceFactory.scala @@ -0,0 +1,45 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.table.sources + +import java.util + +import org.apache.flink.table.descriptors.StreamTableDescriptorValidator.{UPDATE_MODE, UPDATE_MODE_VALUE_APPEND} +import org.apache.flink.table.factories.StreamTableSourceFactory +import org.apache.flink.types.Row + +/** + * Factory for creating configured instances of [[CsvTableSource]] in a stream environment. + */ +class CsvAppendTableSourceFactory + extends CsvTableSourceFactoryBase + with StreamTableSourceFactory[Row] { + + override def requiredContext(): util.Map[String, String] = { + val context = new util.HashMap[String, String](super.requiredContext()) + context.put(UPDATE_MODE, UPDATE_MODE_VALUE_APPEND) + context + } + + override def createStreamTableSource( + properties: util.Map[String, String]) + : StreamTableSource[Row] = { + createTableSource(isStreaming = true, properties) + } +} diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/sources/CsvBatchTableSourceFactory.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/sources/CsvBatchTableSourceFactory.scala new file mode 100644 index 0000000000000..9d8fa40b0893e --- /dev/null +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/sources/CsvBatchTableSourceFactory.scala @@ -0,0 +1,38 @@ +/* + * 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.sources + +import java.util + +import org.apache.flink.table.factories.BatchTableSourceFactory +import org.apache.flink.types.Row + +/** + * Factory for creating configured instances of [[CsvTableSource]] in a batch environment. + */ +class CsvBatchTableSourceFactory + extends CsvTableSourceFactoryBase + with BatchTableSourceFactory[Row] { + + override def createBatchTableSource( + properties: util.Map[String, String]) + : BatchTableSource[Row] = { + createTableSource(isStreaming = false, properties) + } +} diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/sources/CsvTableSourceFactory.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/sources/CsvTableSourceFactoryBase.scala similarity index 86% rename from flink-libraries/flink-table/src/main/scala/org/apache/flink/table/sources/CsvTableSourceFactory.scala rename to flink-libraries/flink-table/src/main/scala/org/apache/flink/table/sources/CsvTableSourceFactoryBase.scala index 96751ec650f16..d32022048e189 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/sources/CsvTableSourceFactory.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/sources/CsvTableSourceFactoryBase.scala @@ -28,16 +28,12 @@ import org.apache.flink.table.descriptors.FileSystemValidator.{CONNECTOR_PATH, C import org.apache.flink.table.descriptors.FormatDescriptorValidator.{FORMAT_PROPERTY_VERSION, FORMAT_TYPE} import org.apache.flink.table.descriptors.SchemaValidator.SCHEMA import org.apache.flink.table.descriptors._ -import org.apache.flink.table.factories.{BatchTableSourceFactory, StreamTableSourceFactory, TableFactory} -import org.apache.flink.types.Row +import org.apache.flink.table.factories.TableFactory /** - * Factory for creating configured instances of [[CsvTableSource]]. + * Factory base for creating configured instances of [[CsvTableSource]]. */ -class CsvTableSourceFactory - extends TableFactory - with StreamTableSourceFactory[Row] - with BatchTableSourceFactory[Row] { +abstract class CsvTableSourceFactoryBase extends TableFactory { override def requiredContext(): util.Map[String, String] = { val context = new util.HashMap[String, String]() @@ -68,19 +64,7 @@ class CsvTableSourceFactory properties } - override def createStreamTableSource( - properties: util.Map[String, String]) - : StreamTableSource[Row] = { - createTableSource(isStreaming = true, properties) - } - - override def createBatchTableSource( - properties: util.Map[String, String]) - : BatchTableSource[Row] = { - createTableSource(isStreaming = false, properties) - } - - private def createTableSource( + protected def createTableSource( isStreaming: Boolean, properties: util.Map[String, String]) : CsvTableSource = { diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/ExternalCatalogTest.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/ExternalCatalogTest.scala index 6df00e749047e..7f567f9d2d749 100644 --- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/ExternalCatalogTest.scala +++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/ExternalCatalogTest.scala @@ -39,7 +39,9 @@ class ExternalCatalogTest extends TableTestBase { val util = batchTestUtil() val tableEnv = util.tableEnv - tableEnv.registerExternalCatalog("test", CommonTestData.getInMemoryTestCatalog) + tableEnv.registerExternalCatalog( + "test", + CommonTestData.getInMemoryTestCatalog(isStreaming = false)) val table1 = tableEnv.scan("test", "db1", "tb1") val table2 = tableEnv.scan("test", "db2", "tb2") @@ -69,7 +71,9 @@ class ExternalCatalogTest extends TableTestBase { def testBatchSQL(): Unit = { val util = batchTestUtil() - util.tableEnv.registerExternalCatalog("test", CommonTestData.getInMemoryTestCatalog) + util.tableEnv.registerExternalCatalog( + "test", + CommonTestData.getInMemoryTestCatalog(isStreaming = false)) val sqlQuery = "SELECT d * 2, e, g FROM test.db2.tb2 WHERE d < 3 UNION ALL " + "(SELECT a * 2, b, c FROM test.db1.tb1)" @@ -96,7 +100,9 @@ class ExternalCatalogTest extends TableTestBase { val util = streamTestUtil() val tableEnv = util.tableEnv - util.tableEnv.registerExternalCatalog("test", CommonTestData.getInMemoryTestCatalog) + util.tableEnv.registerExternalCatalog( + "test", + CommonTestData.getInMemoryTestCatalog(isStreaming = true)) val table1 = tableEnv.scan("test", "db1", "tb1") val table2 = tableEnv.scan("test", "db2", "tb2") @@ -128,7 +134,9 @@ class ExternalCatalogTest extends TableTestBase { def testStreamSQL(): Unit = { val util = streamTestUtil() - util.tableEnv.registerExternalCatalog("test", CommonTestData.getInMemoryTestCatalog) + util.tableEnv.registerExternalCatalog( + "test", + CommonTestData.getInMemoryTestCatalog(isStreaming = true)) val sqlQuery = "SELECT d * 2, e, g FROM test.db2.tb2 WHERE d < 3 UNION ALL " + "(SELECT a * 2, b, c FROM test.db1.tb1)" @@ -155,7 +163,9 @@ class ExternalCatalogTest extends TableTestBase { val util = batchTestUtil() val tableEnv = util.tableEnv - tableEnv.registerExternalCatalog("test", CommonTestData.getInMemoryTestCatalog) + tableEnv.registerExternalCatalog( + "test", + CommonTestData.getInMemoryTestCatalog(isStreaming = false)) val table1 = tableEnv.scan("test", "tb1") val table2 = tableEnv.scan("test", "db2", "tb2") diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/catalog/ExternalCatalogSchemaTest.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/catalog/ExternalCatalogSchemaTest.scala index 58f51f91c446c..c98a7c176a692 100644 --- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/catalog/ExternalCatalogSchemaTest.scala +++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/catalog/ExternalCatalogSchemaTest.scala @@ -47,7 +47,7 @@ class ExternalCatalogSchemaTest extends TableTestBase { @Before def setUp(): Unit = { val rootSchemaPlus: SchemaPlus = CalciteSchema.createRootSchema(true, false).plus() - val catalog = CommonTestData.getInMemoryTestCatalog + val catalog = CommonTestData.getInMemoryTestCatalog(isStreaming = true) ExternalCatalogSchema.registerCatalog( streamTestUtil().tableEnv, rootSchemaPlus, schemaName, catalog) externalCatalogSchema = rootSchemaPlus.getSubSchema("schemaName") diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/catalog/InMemoryExternalCatalogTest.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/catalog/InMemoryExternalCatalogTest.scala index ac2a7292019f9..1f84b3d406157 100644 --- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/catalog/InMemoryExternalCatalogTest.scala +++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/catalog/InMemoryExternalCatalogTest.scala @@ -138,7 +138,9 @@ class InMemoryExternalCatalogTest { val schemaDesc = new Schema() .field("first", BasicTypeInfo.STRING_TYPE_INFO) .field("second", BasicTypeInfo.INT_TYPE_INFO) - new ExternalCatalogTable(connDesc, None, Some(schemaDesc), None, None) + ExternalCatalogTable.builder(connDesc) + .withSchema(schemaDesc) + .asTableSource() } private def createTableInstance( @@ -149,7 +151,9 @@ class InMemoryExternalCatalogTest { fieldNames.zipWithIndex.foreach { case (fieldName, index) => schemaDesc.field(fieldName, fieldTypes(index)) } - new ExternalCatalogTable(connDesc, None, Some(schemaDesc), None, None) + ExternalCatalogTable.builder(connDesc) + .withSchema(schemaDesc) + .asTableSource() } class TestConnectorDesc extends ConnectorDescriptor("test", version = 1, formatNeeded = false) { diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/descriptors/DescriptorTestBase.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/descriptors/DescriptorTestBase.scala index 7a98b0be23fe5..3f6426de4db6a 100644 --- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/descriptors/DescriptorTestBase.scala +++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/descriptors/DescriptorTestBase.scala @@ -18,6 +18,7 @@ package org.apache.flink.table.descriptors +import org.apache.flink.table.descriptors.StreamTableDescriptorValidator.{UPDATE_MODE, UPDATE_MODE_VALUE_APPEND, UPDATE_MODE_VALUE_RETRACT, UPDATE_MODE_VALUE_UPSERT} import org.apache.flink.util.Preconditions import org.junit.Assert.assertEquals import org.junit.Test @@ -84,18 +85,44 @@ abstract class DescriptorTestBase { } } -class TestTableSourceDescriptor(connector: ConnectorDescriptor) - extends TableSourceDescriptor { +class TestTableDescriptor(connector: ConnectorDescriptor) + extends TableDescriptor + with SchematicDescriptor[TestTableDescriptor] + with StreamableDescriptor[TestTableDescriptor] { - this.connectorDescriptor = Some(connector) + private var formatDescriptor: Option[FormatDescriptor] = None + private var schemaDescriptor: Option[Schema] = None + private var updateMode: Option[String] = None - def addFormat(format: FormatDescriptor): TestTableSourceDescriptor = { + override private[flink] def addProperties(properties: DescriptorProperties): Unit = { + connector.addProperties(properties) + formatDescriptor.foreach(_.addProperties(properties)) + schemaDescriptor.foreach(_.addProperties(properties)) + updateMode.foreach(mode => properties.putString(UPDATE_MODE, mode)) + } + + override def withFormat(format: FormatDescriptor): TestTableDescriptor = { this.formatDescriptor = Some(format) this } - def addSchema(schema: Schema): TestTableSourceDescriptor = { + override def withSchema(schema: Schema): TestTableDescriptor = { this.schemaDescriptor = Some(schema) this } + + override def inAppendMode(): TestTableDescriptor = { + updateMode = Some(UPDATE_MODE_VALUE_APPEND) + this + } + + override def inRetractMode(): TestTableDescriptor = { + updateMode = Some(UPDATE_MODE_VALUE_RETRACT) + this + } + + override def inUpsertMode(): TestTableDescriptor = { + updateMode = Some(UPDATE_MODE_VALUE_UPSERT) + this + } } diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/descriptors/TableSourceDescriptorTest.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/descriptors/TableDescriptorTest.scala similarity index 78% rename from flink-libraries/flink-table/src/test/scala/org/apache/flink/table/descriptors/TableSourceDescriptorTest.scala rename to flink-libraries/flink-table/src/test/scala/org/apache/flink/table/descriptors/TableDescriptorTest.scala index a7dd644f3aa25..ccac3170cbd3f 100644 --- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/descriptors/TableSourceDescriptorTest.scala +++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/descriptors/TableDescriptorTest.scala @@ -26,9 +26,9 @@ import org.junit.Test import scala.collection.JavaConverters._ /** - * Tests for [[TableSourceDescriptor]]. + * Tests for [[TableDescriptor]]. */ -class TableSourceDescriptorTest extends TableTestBase { +class TableDescriptorTest extends TableTestBase { @Test def testStreamTableSourceDescriptor(): Unit = { @@ -45,9 +45,10 @@ class TableSourceDescriptorTest extends TableTestBase { val schema = Schema() .field("myfield", Types.STRING) .field("myfield2", Types.INT) - if (isStreaming) { - schema.field("proctime", Types.SQL_TIMESTAMP).proctime() - } + // CSV table source and sink do not support proctime yet + //if (isStreaming) { + // schema.field("proctime", Types.SQL_TIMESTAMP).proctime() + //} val connector = FileSystem() .path("/path/to/csv") @@ -55,25 +56,24 @@ class TableSourceDescriptorTest extends TableTestBase { val format = Csv() .field("myfield", Types.STRING) .field("myfield2", Types.INT) - .quoteCharacter(';') .fieldDelimiter("#") - .lineDelimiter("\r\n") - .commentPrefix("%%") - .ignoreFirstLine() - .ignoreParseErrors() - val descriptor = if (isStreaming) { + val descriptor: RegistrableDescriptor = if (isStreaming) { streamTestUtil().tableEnv - .from(connector) + .connect(connector) .withFormat(format) .withSchema(schema) + .inAppendMode() } else { batchTestUtil().tableEnv - .from(connector) + .connect(connector) .withFormat(format) .withSchema(schema) } + // tests the table factory discovery and thus validates the result automatically + descriptor.registerTableSourceAndSink("MyTable") + val expectedCommonProperties = Seq( "connector.property-version" -> "1", "connector.type" -> "filesystem", @@ -84,12 +84,7 @@ class TableSourceDescriptorTest extends TableTestBase { "format.fields.0.type" -> "VARCHAR", "format.fields.1.name" -> "myfield2", "format.fields.1.type" -> "INT", - "format.quote-character" -> ";", "format.field-delimiter" -> "#", - "format.line-delimiter" -> "\r\n", - "format.comment-prefix" -> "%%", - "format.ignore-first-line" -> "true", - "format.ignore-parse-errors" -> "true", "schema.0.name" -> "myfield", "schema.0.type" -> "VARCHAR", "schema.1.name" -> "myfield2", @@ -98,9 +93,10 @@ class TableSourceDescriptorTest extends TableTestBase { val expectedProperties = if (isStreaming) { expectedCommonProperties ++ Seq( - "schema.2.name" -> "proctime", - "schema.2.type" -> "TIMESTAMP", - "schema.2.proctime" -> "true" + //"schema.2.name" -> "proctime", + //"schema.2.type" -> "TIMESTAMP", + //"schema.2.proctime" -> "true", + "update-mode" -> "append" ) } else { expectedCommonProperties diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/utils/CommonTestData.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/utils/CommonTestData.scala index ac9894c107f5d..e62396f451b31 100644 --- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/utils/CommonTestData.scala +++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/utils/CommonTestData.scala @@ -62,13 +62,13 @@ object CommonTestData { ) } - def getInMemoryTestCatalog: ExternalCatalog = { + def getInMemoryTestCatalog(isStreaming: Boolean): ExternalCatalog = { val csvRecord1 = Seq( "1#1#Hi", "2#2#Hello", "3#2#Hello world" ) - val tempFilePath1 = writeToTempFile(csvRecord1.mkString("$"), "csv-test1", "tmp") + val tempFilePath1 = writeToTempFile(csvRecord1.mkString("\n"), "csv-test1", "tmp") val connDesc1 = FileSystem().path(tempFilePath1) val formatDesc1 = Csv() @@ -76,13 +76,17 @@ object CommonTestData { .field("b", Types.LONG) .field("c", Types.STRING) .fieldDelimiter("#") - .lineDelimiter("$") val schemaDesc1 = Schema() .field("a", Types.INT) .field("b", Types.LONG) .field("c", Types.STRING) - val externalCatalogTable1 = new ExternalCatalogTable( - connDesc1, Some(formatDesc1), Some(schemaDesc1), None, None) + val externalTableBuilder1 = ExternalCatalogTable.builder(connDesc1) + .withFormat(formatDesc1) + .withSchema(schemaDesc1) + + if (isStreaming) { + externalTableBuilder1.inAppendMode() + } val csvRecord2 = Seq( "1#1#0#Hallo#1", @@ -101,7 +105,7 @@ object CommonTestData { "5#14#13#JKL#2", "5#15#14#KLM#2" ) - val tempFilePath2 = writeToTempFile(csvRecord2.mkString("$"), "csv-test2", "tmp") + val tempFilePath2 = writeToTempFile(csvRecord2.mkString("\n"), "csv-test2", "tmp") val connDesc2 = FileSystem().path(tempFilePath2) val formatDesc2 = Csv() @@ -111,15 +115,19 @@ object CommonTestData { .field("g", Types.STRING) .field("h", Types.LONG) .fieldDelimiter("#") - .lineDelimiter("$") val schemaDesc2 = Schema() .field("d", Types.INT) .field("e", Types.LONG) .field("f", Types.INT) .field("g", Types.STRING) .field("h", Types.LONG) - val externalCatalogTable2 = new ExternalCatalogTable( - connDesc2, Some(formatDesc2), Some(schemaDesc2), None, None) + val externalTableBuilder2 = ExternalCatalogTable.builder(connDesc2) + .withFormat(formatDesc2) + .withSchema(schemaDesc2) + + if (isStreaming) { + externalTableBuilder2.inAppendMode() + } val catalog = new InMemoryExternalCatalog("test") val db1 = new InMemoryExternalCatalog("db1") @@ -128,9 +136,9 @@ object CommonTestData { catalog.createSubCatalog("db2", db2, ignoreIfExists = false) // Register the table with both catalogs - catalog.createTable("tb1", externalCatalogTable1, ignoreIfExists = false) - db1.createTable("tb1", externalCatalogTable1, ignoreIfExists = false) - db2.createTable("tb2", externalCatalogTable2, ignoreIfExists = false) + catalog.createTable("tb1", externalTableBuilder1.asTableSource(), ignoreIfExists = false) + db1.createTable("tb1", externalTableBuilder1.asTableSource(), ignoreIfExists = false) + db2.createTable("tb2", externalTableBuilder2.asTableSource(), ignoreIfExists = false) catalog } diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/utils/MockTableEnvironment.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/utils/MockTableEnvironment.scala index c25f30f37fc8d..87dbc91b37bc9 100644 --- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/utils/MockTableEnvironment.scala +++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/utils/MockTableEnvironment.scala @@ -21,7 +21,7 @@ package org.apache.flink.table.utils import org.apache.calcite.tools.RuleSet import org.apache.flink.api.common.typeinfo.TypeInformation import org.apache.flink.table.api.{QueryConfig, Table, TableConfig, TableEnvironment} -import org.apache.flink.table.descriptors.{ConnectorDescriptor, TableSourceDescriptor} +import org.apache.flink.table.descriptors.{ConnectorDescriptor, TableDescriptor} import org.apache.flink.table.sinks.TableSink import org.apache.flink.table.sources.TableSource @@ -55,5 +55,5 @@ class MockTableEnvironment extends TableEnvironment(new TableConfig) { override def explain(table: Table): String = ??? - override def from(connectorDescriptor: ConnectorDescriptor): TableSourceDescriptor = ??? + override def connect(connectorDescriptor: ConnectorDescriptor): TableDescriptor = ??? } From 36ae5cd3ceb1a52b73ac8bfb845a15d361c9f390 Mon Sep 17 00:00:00 2001 From: Timo Walther Date: Wed, 18 Jul 2018 13:30:28 +0200 Subject: [PATCH 32/65] [FLINK-9886] [sql-client] Build SQL jars with every build This closes #6366. --- flink-connectors/flink-connector-kafka-0.10/pom.xml | 6 +++--- flink-connectors/flink-connector-kafka-0.11/pom.xml | 6 +++--- flink-connectors/flink-connector-kafka-0.9/pom.xml | 6 +++--- flink-formats/flink-avro/pom.xml | 9 +++++++-- flink-formats/flink-json/pom.xml | 6 +++--- 5 files changed, 19 insertions(+), 14 deletions(-) diff --git a/flink-connectors/flink-connector-kafka-0.10/pom.xml b/flink-connectors/flink-connector-kafka-0.10/pom.xml index 2fb7a32397646..9b2353dc8ff4a 100644 --- a/flink-connectors/flink-connector-kafka-0.10/pom.xml +++ b/flink-connectors/flink-connector-kafka-0.10/pom.xml @@ -203,12 +203,12 @@ under the License.
    + - - release + sql-jars - release + !skipSqlJars diff --git a/flink-connectors/flink-connector-kafka-0.11/pom.xml b/flink-connectors/flink-connector-kafka-0.11/pom.xml index aa6000495809a..4ff1d960629d9 100644 --- a/flink-connectors/flink-connector-kafka-0.11/pom.xml +++ b/flink-connectors/flink-connector-kafka-0.11/pom.xml @@ -212,12 +212,12 @@ under the License. + - - release + sql-jars - release + !skipSqlJars diff --git a/flink-connectors/flink-connector-kafka-0.9/pom.xml b/flink-connectors/flink-connector-kafka-0.9/pom.xml index 20d2991688068..74ada66f228e3 100644 --- a/flink-connectors/flink-connector-kafka-0.9/pom.xml +++ b/flink-connectors/flink-connector-kafka-0.9/pom.xml @@ -191,12 +191,12 @@ under the License. + - - release + sql-jars - release + !skipSqlJars diff --git a/flink-formats/flink-avro/pom.xml b/flink-formats/flink-avro/pom.xml index dbf7fd07337f5..f313978a8e775 100644 --- a/flink-formats/flink-avro/pom.xml +++ b/flink-formats/flink-avro/pom.xml @@ -109,9 +109,14 @@ under the License. + - - release + sql-jars + + + !skipSqlJars + + diff --git a/flink-formats/flink-json/pom.xml b/flink-formats/flink-json/pom.xml index 4e31766948fe4..9884ff6ef8e90 100644 --- a/flink-formats/flink-json/pom.xml +++ b/flink-formats/flink-json/pom.xml @@ -86,12 +86,12 @@ under the License. + - - release + sql-jars - release + !skipSqlJars From 6a36afd3a3db3a8c7b0c27aca525218793882239 Mon Sep 17 00:00:00 2001 From: Andrey Zagrebin Date: Fri, 13 Jul 2018 19:27:35 +0200 Subject: [PATCH 33/65] [FLINK-9858][tests] State TTL End-to-End Test This closes #6361. --- .../flink-stream-state-ttl-test/pom.xml | 104 ++++++++++ .../tests/DataStreamStateTTLTestProgram.java | 100 ++++++++++ .../flink/streaming/tests/TtlStateUpdate.java | 45 +++++ .../streaming/tests/TtlStateUpdateSource.java | 78 ++++++++ .../tests/TtlVerifyUpdateFunction.java | 187 ++++++++++++++++++ .../verify/AbstractTtlStateVerifier.java | 105 ++++++++++ .../verify/TtlAggregatingStateVerifier.java | 107 ++++++++++ .../tests/verify/TtlFoldingStateVerifier.java | 87 ++++++++ .../tests/verify/TtlListStateVerifier.java | 78 ++++++++ .../tests/verify/TtlMapStateVerifier.java | 94 +++++++++ .../verify/TtlReducingStateVerifier.java | 86 ++++++++ .../tests/verify/TtlStateVerifier.java | 60 ++++++ .../tests/verify/TtlUpdateContext.java | 71 +++++++ .../tests/verify/TtlValueStateVerifier.java | 66 +++++++ .../tests/verify/TtlVerificationContext.java | 69 +++++++ .../streaming/tests/verify/ValueWithTs.java | 107 ++++++++++ flink-end-to-end-tests/pom.xml | 1 + flink-end-to-end-tests/run-nightly-tests.sh | 3 + flink-end-to-end-tests/test-scripts/common.sh | 37 ++-- .../test-scripts/test_stream_state_ttl.sh | 83 ++++++++ 20 files changed, 1555 insertions(+), 13 deletions(-) create mode 100644 flink-end-to-end-tests/flink-stream-state-ttl-test/pom.xml create mode 100644 flink-end-to-end-tests/flink-stream-state-ttl-test/src/main/java/org/apache/flink/streaming/tests/DataStreamStateTTLTestProgram.java create mode 100644 flink-end-to-end-tests/flink-stream-state-ttl-test/src/main/java/org/apache/flink/streaming/tests/TtlStateUpdate.java create mode 100644 flink-end-to-end-tests/flink-stream-state-ttl-test/src/main/java/org/apache/flink/streaming/tests/TtlStateUpdateSource.java create mode 100644 flink-end-to-end-tests/flink-stream-state-ttl-test/src/main/java/org/apache/flink/streaming/tests/TtlVerifyUpdateFunction.java create mode 100644 flink-end-to-end-tests/flink-stream-state-ttl-test/src/main/java/org/apache/flink/streaming/tests/verify/AbstractTtlStateVerifier.java create mode 100644 flink-end-to-end-tests/flink-stream-state-ttl-test/src/main/java/org/apache/flink/streaming/tests/verify/TtlAggregatingStateVerifier.java create mode 100644 flink-end-to-end-tests/flink-stream-state-ttl-test/src/main/java/org/apache/flink/streaming/tests/verify/TtlFoldingStateVerifier.java create mode 100644 flink-end-to-end-tests/flink-stream-state-ttl-test/src/main/java/org/apache/flink/streaming/tests/verify/TtlListStateVerifier.java create mode 100644 flink-end-to-end-tests/flink-stream-state-ttl-test/src/main/java/org/apache/flink/streaming/tests/verify/TtlMapStateVerifier.java create mode 100644 flink-end-to-end-tests/flink-stream-state-ttl-test/src/main/java/org/apache/flink/streaming/tests/verify/TtlReducingStateVerifier.java create mode 100644 flink-end-to-end-tests/flink-stream-state-ttl-test/src/main/java/org/apache/flink/streaming/tests/verify/TtlStateVerifier.java create mode 100644 flink-end-to-end-tests/flink-stream-state-ttl-test/src/main/java/org/apache/flink/streaming/tests/verify/TtlUpdateContext.java create mode 100644 flink-end-to-end-tests/flink-stream-state-ttl-test/src/main/java/org/apache/flink/streaming/tests/verify/TtlValueStateVerifier.java create mode 100644 flink-end-to-end-tests/flink-stream-state-ttl-test/src/main/java/org/apache/flink/streaming/tests/verify/TtlVerificationContext.java create mode 100644 flink-end-to-end-tests/flink-stream-state-ttl-test/src/main/java/org/apache/flink/streaming/tests/verify/ValueWithTs.java create mode 100755 flink-end-to-end-tests/test-scripts/test_stream_state_ttl.sh diff --git a/flink-end-to-end-tests/flink-stream-state-ttl-test/pom.xml b/flink-end-to-end-tests/flink-stream-state-ttl-test/pom.xml new file mode 100644 index 0000000000000..38bfcee7c983c --- /dev/null +++ b/flink-end-to-end-tests/flink-stream-state-ttl-test/pom.xml @@ -0,0 +1,104 @@ + + + + 4.0.0 + + + org.apache.flink + flink-end-to-end-tests + 1.6-SNAPSHOT + .. + + + flink-stream-state-ttl-test + flink-stream-state-ttl-test + jar + + + + org.apache.flink + flink-core + ${project.version} + provided + + + org.apache.flink + flink-streaming-java_${scala.binary.version} + ${project.version} + provided + + + org.apache.flink + flink-statebackend-rocksdb_2.11 + ${project.version} + + + org.apache.flink + flink-datastream-allround-test + ${project.version} + + + + + + + org.apache.maven.plugins + maven-shade-plugin + 3.0.0 + + + package + + shade + + + DataStreamStateTTLTestProgram + + + com.google.code.findbugs:jsr305 + org.slf4j:* + log4j:* + + + + + *:* + + META-INF/*.SF + META-INF/*.DSA + META-INF/*.RSA + + + + + + org.apache.flink.streaming.tests.DataStreamStateTTLTestProgram + + + + + + + + + + diff --git a/flink-end-to-end-tests/flink-stream-state-ttl-test/src/main/java/org/apache/flink/streaming/tests/DataStreamStateTTLTestProgram.java b/flink-end-to-end-tests/flink-stream-state-ttl-test/src/main/java/org/apache/flink/streaming/tests/DataStreamStateTTLTestProgram.java new file mode 100644 index 0000000000000..f4c961937ea8f --- /dev/null +++ b/flink-end-to-end-tests/flink-stream-state-ttl-test/src/main/java/org/apache/flink/streaming/tests/DataStreamStateTTLTestProgram.java @@ -0,0 +1,100 @@ +/* + * 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.tests; + +import org.apache.flink.api.common.state.StateTtlConfiguration; +import org.apache.flink.api.common.time.Time; +import org.apache.flink.api.java.utils.ParameterTool; +import org.apache.flink.configuration.ConfigOption; +import org.apache.flink.configuration.ConfigOptions; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.streaming.api.functions.sink.PrintSinkFunction; + +import static org.apache.flink.streaming.tests.DataStreamAllroundTestJobFactory.setupEnvironment; + +/** + * A test job for State TTL feature. + * + *

    The test pipeline does the following: + * - generates random keyed state updates for each state TTL verifier (state type) + * - performs update of created state with TTL for each verifier + * - keeps previous updates in other state + * - verifies expected result of last update against preserved history of updates + * + *

    Program parameters: + *

      + *
    • update_generator_source.keyspace (int, default - 100): Number of different keys for updates emitted by the update generator.
    • + *
    • update_generator_source.sleep_time (long, default - 0): Milliseconds to sleep after emitting updates in the update generator. Set to 0 to disable sleeping.
    • + *
    • update_generator_source.sleep_after_elements (long, default - 0): Number of updates to emit before sleeping in the update generator. Set to 0 to disable sleeping.
    • + *
    • state_ttl_verifier.ttl_milli (long, default - 1000): State time-to-live.
    • + *
    • report_stat.after_updates_num (long, default - 200): Report state update statistics after certain number of updates (average update chain length and clashes).
    • + *
    + */ +public class DataStreamStateTTLTestProgram { + private static final ConfigOption UPDATE_GENERATOR_SRC_KEYSPACE = ConfigOptions + .key("update_generator_source.keyspace") + .defaultValue(100); + + private static final ConfigOption UPDATE_GENERATOR_SRC_SLEEP_TIME = ConfigOptions + .key("update_generator_source.sleep_time") + .defaultValue(0L); + + private static final ConfigOption UPDATE_GENERATOR_SRC_SLEEP_AFTER_ELEMENTS = ConfigOptions + .key("update_generator_source.sleep_after_elements") + .defaultValue(0L); + + private static final ConfigOption STATE_TTL_VERIFIER_TTL_MILLI = ConfigOptions + .key("state_ttl_verifier.ttl_milli") + .defaultValue(1000L); + + private static final ConfigOption REPORT_STAT_AFTER_UPDATES_NUM = ConfigOptions + .key("report_stat.after_updates_num") + .defaultValue(200L); + + public static void main(String[] args) throws Exception { + final ParameterTool pt = ParameterTool.fromArgs(args); + + final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + + setupEnvironment(env, pt); + + int keySpace = pt.getInt(UPDATE_GENERATOR_SRC_KEYSPACE.key(), UPDATE_GENERATOR_SRC_KEYSPACE.defaultValue()); + long sleepAfterElements = pt.getLong(UPDATE_GENERATOR_SRC_SLEEP_AFTER_ELEMENTS.key(), + UPDATE_GENERATOR_SRC_SLEEP_AFTER_ELEMENTS.defaultValue()); + long sleepTime = pt.getLong(UPDATE_GENERATOR_SRC_SLEEP_TIME.key(), + UPDATE_GENERATOR_SRC_SLEEP_TIME.defaultValue()); + Time ttl = Time.milliseconds(pt.getLong(STATE_TTL_VERIFIER_TTL_MILLI.key(), + STATE_TTL_VERIFIER_TTL_MILLI.defaultValue())); + long reportStatAfterUpdatesNum = pt.getLong(REPORT_STAT_AFTER_UPDATES_NUM.key(), + REPORT_STAT_AFTER_UPDATES_NUM.defaultValue()); + + StateTtlConfiguration ttlConfig = StateTtlConfiguration.newBuilder(ttl).build(); + + env + .addSource(new TtlStateUpdateSource(keySpace, sleepAfterElements, sleepTime)) + .name("TtlStateUpdateSource") + .keyBy(TtlStateUpdate::getKey) + .flatMap(new TtlVerifyUpdateFunction(ttlConfig, reportStatAfterUpdatesNum)) + .name("TtlVerifyUpdateFunction") + .addSink(new PrintSinkFunction<>()) + .name("PrintFailedVerifications"); + + env.execute("State TTL test job"); + } +} diff --git a/flink-end-to-end-tests/flink-stream-state-ttl-test/src/main/java/org/apache/flink/streaming/tests/TtlStateUpdate.java b/flink-end-to-end-tests/flink-stream-state-ttl-test/src/main/java/org/apache/flink/streaming/tests/TtlStateUpdate.java new file mode 100644 index 0000000000000..e89b544cc669a --- /dev/null +++ b/flink-end-to-end-tests/flink-stream-state-ttl-test/src/main/java/org/apache/flink/streaming/tests/TtlStateUpdate.java @@ -0,0 +1,45 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.streaming.tests; + +import javax.annotation.Nonnull; + +import java.io.Serializable; +import java.util.Map; + +/** Randomly generated keyed state updates per state type. */ +class TtlStateUpdate implements Serializable { + private final int key; + + @Nonnull + private final Map updates; + + TtlStateUpdate(int key, @Nonnull Map updates) { + this.key = key; + this.updates = updates; + } + + int getKey() { + return key; + } + + Object getUpdate(String verifierId) { + return updates.get(verifierId); + } +} diff --git a/flink-end-to-end-tests/flink-stream-state-ttl-test/src/main/java/org/apache/flink/streaming/tests/TtlStateUpdateSource.java b/flink-end-to-end-tests/flink-stream-state-ttl-test/src/main/java/org/apache/flink/streaming/tests/TtlStateUpdateSource.java new file mode 100644 index 0000000000000..6aff14e1cd3df --- /dev/null +++ b/flink-end-to-end-tests/flink-stream-state-ttl-test/src/main/java/org/apache/flink/streaming/tests/TtlStateUpdateSource.java @@ -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.streaming.tests; + +import org.apache.flink.streaming.api.functions.source.RichParallelSourceFunction; +import org.apache.flink.streaming.tests.verify.TtlStateVerifier; + +import java.util.Map; +import java.util.Random; +import java.util.stream.Collectors; + +/** + * Source of randomly generated keyed state updates. + * + *

    Internal loop generates {@code sleepAfterElements} state updates + * for each verifier from {@link TtlStateVerifier#VERIFIERS} using {@link TtlStateVerifier#generateRandomUpdate} + * and waits for {@code sleepTime} to continue generation. + */ +class TtlStateUpdateSource extends RichParallelSourceFunction { + private final int maxKey; + private final long sleepAfterElements; + private final long sleepTime; + + /** Flag that determines if this source is running, i.e. generating events. */ + private volatile boolean running = true; + + TtlStateUpdateSource(int maxKey, long sleepAfterElements, long sleepTime) { + this.maxKey = maxKey; + this.sleepAfterElements = sleepAfterElements; + this.sleepTime = sleepTime; + } + + @Override + public void run(SourceContext ctx) throws Exception { + Random random = new Random(); + long elementsBeforeSleep = sleepAfterElements; + while (running) { + for (int i = 0; i < sleepAfterElements; i++) { + synchronized (ctx.getCheckpointLock()) { + Map updates = TtlStateVerifier.VERIFIERS.stream() + .collect(Collectors.toMap(TtlStateVerifier::getId, TtlStateVerifier::generateRandomUpdate)); + ctx.collect(new TtlStateUpdate(random.nextInt(maxKey), updates)); + } + } + + if (sleepTime > 0) { + if (elementsBeforeSleep == 1) { + elementsBeforeSleep = sleepAfterElements; + long rnd = sleepTime < Integer.MAX_VALUE ? random.nextInt((int) sleepTime) : 0L; + Thread.sleep(rnd + sleepTime); + } else if (elementsBeforeSleep > 1) { + --elementsBeforeSleep; + } + } + } + } + + @Override + public void cancel() { + running = false; + } +} diff --git a/flink-end-to-end-tests/flink-stream-state-ttl-test/src/main/java/org/apache/flink/streaming/tests/TtlVerifyUpdateFunction.java b/flink-end-to-end-tests/flink-stream-state-ttl-test/src/main/java/org/apache/flink/streaming/tests/TtlVerifyUpdateFunction.java new file mode 100644 index 0000000000000..a99a45f8bce47 --- /dev/null +++ b/flink-end-to-end-tests/flink-stream-state-ttl-test/src/main/java/org/apache/flink/streaming/tests/TtlVerifyUpdateFunction.java @@ -0,0 +1,187 @@ +/* + * 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.tests; + +import org.apache.flink.api.common.functions.RichFlatMapFunction; +import org.apache.flink.api.common.state.KeyedStateStore; +import org.apache.flink.api.common.state.ListState; +import org.apache.flink.api.common.state.ListStateDescriptor; +import org.apache.flink.api.common.state.State; +import org.apache.flink.api.common.state.StateTtlConfiguration; +import org.apache.flink.api.common.typeutils.TypeSerializer; +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.tests.verify.TtlStateVerifier; +import org.apache.flink.streaming.tests.verify.TtlUpdateContext; +import org.apache.flink.streaming.tests.verify.TtlVerificationContext; +import org.apache.flink.streaming.tests.verify.ValueWithTs; +import org.apache.flink.util.Collector; +import org.apache.flink.util.Preconditions; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import javax.annotation.Nonnull; + +import java.io.Serializable; +import java.util.Collections; +import java.util.List; +import java.util.Map; +import java.util.stream.Collectors; +import java.util.stream.StreamSupport; + +/** + * Update state with TTL for each verifier. + * + *

    This function for each verifier from {@link TtlStateVerifier#VERIFIERS} + * - creates state with TTL + * - creates state of previous updates for further verification against it + * - receives random state update + * - gets state value before update + * - updates state with random value + * - gets state value after update + * - checks if this update clashes with any previous updates + * - if clashes, clears state and recreate update + * - verifies last update against previous updates + * - emits verification context in case of failure + */ +class TtlVerifyUpdateFunction + extends RichFlatMapFunction implements CheckpointedFunction { + private static final Logger LOG = LoggerFactory.getLogger(TtlVerifyUpdateFunction.class); + + @Nonnull + private final StateTtlConfiguration ttlConfig; + private final long ttl; + private final UpdateStat stat; + + private transient Map states; + private transient Map>> prevUpdatesByVerifierId; + + TtlVerifyUpdateFunction(@Nonnull StateTtlConfiguration ttlConfig, long reportStatAfterUpdatesNum) { + this.ttlConfig = ttlConfig; + this.ttl = ttlConfig.getTtl().toMilliseconds(); + this.stat = new UpdateStat(reportStatAfterUpdatesNum); + } + + @Override + public void flatMap(TtlStateUpdate updates, Collector out) throws Exception { + for (TtlStateVerifier verifier : TtlStateVerifier.VERIFIERS) { + TtlVerificationContext verificationContext = generateUpdateAndVerificationContext(updates, verifier); + if (!verifier.verify(verificationContext)) { + out.collect(verificationContext.toString()); + } + } + } + + private TtlVerificationContext generateUpdateAndVerificationContext( + TtlStateUpdate updates, TtlStateVerifier verifier) throws Exception { + List> prevUpdates = getPrevUpdates(verifier.getId()); + Object update = updates.getUpdate(verifier.getId()); + TtlUpdateContext updateContext = performUpdate(verifier, update); + boolean clashes = updateClashesWithPrevUpdates(updateContext.getUpdateWithTs(), prevUpdates); + if (clashes) { + resetState(verifier.getId()); + prevUpdates = Collections.emptyList(); + updateContext = performUpdate(verifier, update); + } + stat.update(clashes, prevUpdates.size()); + prevUpdatesByVerifierId.get(verifier.getId()).add(updateContext.getUpdateWithTs()); + return new TtlVerificationContext<>(updates.getKey(), verifier.getId(), prevUpdates, updateContext); + } + + private List> getPrevUpdates(String verifierId) throws Exception { + return StreamSupport + .stream(prevUpdatesByVerifierId.get(verifierId).get().spliterator(), false) + .collect(Collectors.toList()); + } + + private TtlUpdateContext performUpdate( + TtlStateVerifier verifier, Object update) throws Exception { + State state = states.get(verifier.getId()); + long timestampBeforeUpdate = System.currentTimeMillis(); + Object valueBeforeUpdate = verifier.get(state); + verifier.update(state, update); + Object updatedValue = verifier.get(state); + return new TtlUpdateContext<>(timestampBeforeUpdate, + valueBeforeUpdate, update, updatedValue, System.currentTimeMillis()); + } + + private boolean updateClashesWithPrevUpdates(ValueWithTs update, List> prevUpdates) { + return tooSlow(update) || + (!prevUpdates.isEmpty() && prevUpdates.stream().anyMatch(pu -> updatesClash(pu, update))); + } + + private boolean tooSlow(ValueWithTs update) { + return update.getTimestampAfterUpdate() - update.getTimestampBeforeUpdate() >= ttl; + } + + private boolean updatesClash(ValueWithTs prevUpdate, ValueWithTs nextUpdate) { + return prevUpdate.getTimestampAfterUpdate() + ttl >= nextUpdate.getTimestampBeforeUpdate() && + prevUpdate.getTimestampBeforeUpdate() + ttl <= nextUpdate.getTimestampAfterUpdate(); + } + + private void resetState(String verifierId) { + states.get(verifierId).clear(); + prevUpdatesByVerifierId.get(verifierId).clear(); + } + + @Override + public void snapshotState(FunctionSnapshotContext context) { + + } + + @Override + public void initializeState(FunctionInitializationContext context) { + states = TtlStateVerifier.VERIFIERS.stream() + .collect(Collectors.toMap(TtlStateVerifier::getId, v -> v.createState(context, ttlConfig))); + prevUpdatesByVerifierId = TtlStateVerifier.VERIFIERS.stream() + .collect(Collectors.toMap(TtlStateVerifier::getId, v -> { + Preconditions.checkNotNull(v); + TypeSerializer> typeSerializer = new ValueWithTs.Serializer(v.getUpdateSerializer()); + ListStateDescriptor> stateDesc = new ListStateDescriptor<>( + "TtlPrevValueState_" + v.getId(), typeSerializer); + KeyedStateStore store = context.getKeyedStateStore(); + return store.getListState(stateDesc); + })); + } + + private static class UpdateStat implements Serializable { + final long reportStatAfterUpdatesNum; + long updates = 0; + long clashes = 0; + long prevUpdatesNum = 0; + + UpdateStat(long reportStatAfterUpdatesNum) { + this.reportStatAfterUpdatesNum = reportStatAfterUpdatesNum; + } + + void update(boolean clash, long prevUpdatesSize) { + updates++; + if (clash) { + clashes++; + } + prevUpdatesNum += prevUpdatesSize; + if (updates % reportStatAfterUpdatesNum == 0) { + LOG.info(String.format("Avg update chain length: %d, clash stat: %d/%d", + prevUpdatesNum / updates, clashes, updates)); + } + } + } +} diff --git a/flink-end-to-end-tests/flink-stream-state-ttl-test/src/main/java/org/apache/flink/streaming/tests/verify/AbstractTtlStateVerifier.java b/flink-end-to-end-tests/flink-stream-state-ttl-test/src/main/java/org/apache/flink/streaming/tests/verify/AbstractTtlStateVerifier.java new file mode 100644 index 0000000000000..c56ff197142d2 --- /dev/null +++ b/flink-end-to-end-tests/flink-stream-state-ttl-test/src/main/java/org/apache/flink/streaming/tests/verify/AbstractTtlStateVerifier.java @@ -0,0 +1,105 @@ +/* + * 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.tests.verify; + +import org.apache.flink.api.common.state.State; +import org.apache.flink.api.common.state.StateDescriptor; +import org.apache.flink.api.common.state.StateTtlConfiguration; +import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.runtime.state.FunctionInitializationContext; +import org.apache.flink.util.StringUtils; + +import javax.annotation.Nonnull; + +import java.util.ArrayList; +import java.util.List; +import java.util.Objects; +import java.util.Random; + +/** Base class for State TTL verifiers. */ +abstract class AbstractTtlStateVerifier, S extends State, SV, UV, GV> + implements TtlStateVerifier { + static final Random RANDOM = new Random(); + + @Nonnull + final D stateDesc; + + AbstractTtlStateVerifier(@Nonnull D stateDesc) { + this.stateDesc = stateDesc; + } + + @Nonnull + static String randomString() { + return StringUtils.getRandomString(RANDOM, 2, 20); + } + + @SuppressWarnings("unchecked") + @Override + @Nonnull + public State createState(@Nonnull FunctionInitializationContext context, @Nonnull StateTtlConfiguration ttlConfig) { + stateDesc.enableTimeToLive(ttlConfig); + return createState(context); + } + + abstract State createState(FunctionInitializationContext context); + + @SuppressWarnings("unchecked") + @Override + @Nonnull + public TypeSerializer getUpdateSerializer() { + return (TypeSerializer) stateDesc.getSerializer(); + } + + @SuppressWarnings("unchecked") + @Override + public GV get(@Nonnull State state) throws Exception { + return getInternal((S) state); + } + + abstract GV getInternal(@Nonnull S state) throws Exception; + + @SuppressWarnings("unchecked") + @Override + public void update(@Nonnull State state, Object update) throws Exception { + updateInternal((S) state, (UV) update); + } + + abstract void updateInternal(@Nonnull S state, UV update) throws Exception; + + @SuppressWarnings("unchecked") + @Override + public boolean verify(@Nonnull TtlVerificationContext verificationContextRaw) { + TtlVerificationContext verificationContext = (TtlVerificationContext) verificationContextRaw; + List> updates = new ArrayList<>(verificationContext.getPrevUpdates()); + long currentTimestamp = verificationContext.getUpdateContext().getTimestampBeforeUpdate(); + GV prevValue = expected(updates, currentTimestamp); + GV valueBeforeUpdate = verificationContext.getUpdateContext().getValueBeforeUpdate(); + ValueWithTs update = verificationContext.getUpdateContext().getUpdateWithTs(); + GV updatedValue = verificationContext.getUpdateContext().getUpdatedValue(); + updates.add(update); + GV expectedValue = expected(updates, currentTimestamp); + return Objects.equals(valueBeforeUpdate, prevValue) && Objects.equals(updatedValue, expectedValue); + } + + abstract GV expected(@Nonnull List> updates, long currentTimestamp); + + boolean expired(long lastTimestamp, long currentTimestamp) { + return lastTimestamp + stateDesc.getTtlConfig().getTtl().toMilliseconds() <= currentTimestamp; + } +} diff --git a/flink-end-to-end-tests/flink-stream-state-ttl-test/src/main/java/org/apache/flink/streaming/tests/verify/TtlAggregatingStateVerifier.java b/flink-end-to-end-tests/flink-stream-state-ttl-test/src/main/java/org/apache/flink/streaming/tests/verify/TtlAggregatingStateVerifier.java new file mode 100644 index 0000000000000..960bbe72401ff --- /dev/null +++ b/flink-end-to-end-tests/flink-stream-state-ttl-test/src/main/java/org/apache/flink/streaming/tests/verify/TtlAggregatingStateVerifier.java @@ -0,0 +1,107 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.streaming.tests.verify; + +import org.apache.flink.api.common.functions.AggregateFunction; +import org.apache.flink.api.common.state.AggregatingState; +import org.apache.flink.api.common.state.AggregatingStateDescriptor; +import org.apache.flink.api.common.state.State; +import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.api.common.typeutils.base.IntSerializer; +import org.apache.flink.api.common.typeutils.base.LongSerializer; +import org.apache.flink.runtime.state.FunctionInitializationContext; + +import javax.annotation.Nonnull; + +import java.util.List; + +class TtlAggregatingStateVerifier extends AbstractTtlStateVerifier< + AggregatingStateDescriptor, AggregatingState, Long, Integer, String> { + TtlAggregatingStateVerifier() { + super(new AggregatingStateDescriptor<>("TtlAggregatingStateVerifier", AGG_FUNC, LongSerializer.INSTANCE)); + } + + @Override + @Nonnull + State createState(@Nonnull FunctionInitializationContext context) { + return context.getKeyedStateStore().getAggregatingState(stateDesc); + } + + @Override + @Nonnull + public TypeSerializer getUpdateSerializer() { + return IntSerializer.INSTANCE; + } + + @Override + @Nonnull + public Integer generateRandomUpdate() { + return RANDOM.nextInt(100); + } + + @Override + String getInternal(@Nonnull AggregatingState state) throws Exception { + return state.get(); + } + + @Override + void updateInternal(@Nonnull AggregatingState state, Integer update) throws Exception { + state.add(update); + } + + @Override + String expected(@Nonnull List> updates, long currentTimestamp) { + if (updates.isEmpty()) { + return null; + } + long acc = AGG_FUNC.createAccumulator(); + long lastTs = updates.get(0).getTimestampAfterUpdate(); + for (ValueWithTs update : updates) { + if (expired(lastTs, update.getTimestampAfterUpdate())) { + acc = AGG_FUNC.createAccumulator(); + } + acc = AGG_FUNC.add(update.getValue(), acc); + lastTs = update.getTimestampAfterUpdate(); + } + return expired(lastTs, currentTimestamp) ? null : AGG_FUNC.getResult(acc); + } + + private static final AggregateFunction AGG_FUNC = + new AggregateFunction() { + @Override + public Long createAccumulator() { + return 3L; + } + + @Override + public Long add(Integer value, Long accumulator) { + return accumulator + value; + } + + @Override + public String getResult(Long accumulator) { + return Long.toString(accumulator); + } + + @Override + public Long merge(Long a, Long b) { + return a + b; + } + }; +} diff --git a/flink-end-to-end-tests/flink-stream-state-ttl-test/src/main/java/org/apache/flink/streaming/tests/verify/TtlFoldingStateVerifier.java b/flink-end-to-end-tests/flink-stream-state-ttl-test/src/main/java/org/apache/flink/streaming/tests/verify/TtlFoldingStateVerifier.java new file mode 100644 index 0000000000000..c1cc761b0f3f1 --- /dev/null +++ b/flink-end-to-end-tests/flink-stream-state-ttl-test/src/main/java/org/apache/flink/streaming/tests/verify/TtlFoldingStateVerifier.java @@ -0,0 +1,87 @@ +/* + * 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.tests.verify; + +import org.apache.flink.api.common.state.FoldingState; +import org.apache.flink.api.common.state.FoldingStateDescriptor; +import org.apache.flink.api.common.state.State; +import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.api.common.typeutils.base.IntSerializer; +import org.apache.flink.api.common.typeutils.base.LongSerializer; +import org.apache.flink.runtime.state.FunctionInitializationContext; + +import javax.annotation.Nonnull; + +import java.util.List; + +@SuppressWarnings("deprecation") +class TtlFoldingStateVerifier extends AbstractTtlStateVerifier< + FoldingStateDescriptor, FoldingState, Long, Integer, Long> { + private static final long INIT_VAL = 5L; + + TtlFoldingStateVerifier() { + super(new FoldingStateDescriptor<>( + "TtlFoldingStateVerifier", INIT_VAL, (v, acc) -> acc + v, LongSerializer.INSTANCE)); + } + + @Override + @Nonnull + State createState(@Nonnull FunctionInitializationContext context) { + return context.getKeyedStateStore().getFoldingState(stateDesc); + } + + @Override + @Nonnull + public TypeSerializer getUpdateSerializer() { + return IntSerializer.INSTANCE; + } + + @Override + @Nonnull + public Integer generateRandomUpdate() { + return RANDOM.nextInt(100); + } + + @Override + Long getInternal(@Nonnull FoldingState state) throws Exception { + return state.get(); + } + + @Override + void updateInternal(@Nonnull FoldingState state, Integer update) throws Exception { + state.add(update); + } + + @Override + Long expected(@Nonnull List> updates, long currentTimestamp) { + if (updates.isEmpty()) { + return null; + } + long acc = INIT_VAL; + long lastTs = updates.get(0).getTimestampAfterUpdate(); + for (ValueWithTs update : updates) { + if (expired(lastTs, update.getTimestampAfterUpdate())) { + acc = INIT_VAL; + } + acc += update.getValue(); + lastTs = update.getTimestampAfterUpdate(); + } + return expired(lastTs, currentTimestamp) ? null : acc; + } +} diff --git a/flink-end-to-end-tests/flink-stream-state-ttl-test/src/main/java/org/apache/flink/streaming/tests/verify/TtlListStateVerifier.java b/flink-end-to-end-tests/flink-stream-state-ttl-test/src/main/java/org/apache/flink/streaming/tests/verify/TtlListStateVerifier.java new file mode 100644 index 0000000000000..b355aa9861a48 --- /dev/null +++ b/flink-end-to-end-tests/flink-stream-state-ttl-test/src/main/java/org/apache/flink/streaming/tests/verify/TtlListStateVerifier.java @@ -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.streaming.tests.verify; + +import org.apache.flink.api.common.state.ListState; +import org.apache.flink.api.common.state.ListStateDescriptor; +import org.apache.flink.api.common.state.State; +import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.api.common.typeutils.base.StringSerializer; +import org.apache.flink.runtime.state.FunctionInitializationContext; + +import javax.annotation.Nonnull; + +import java.util.List; +import java.util.stream.Collectors; +import java.util.stream.StreamSupport; + +class TtlListStateVerifier extends AbstractTtlStateVerifier< + ListStateDescriptor, ListState, List, String, List> { + TtlListStateVerifier() { + super(new ListStateDescriptor<>("TtlListStateVerifier", StringSerializer.INSTANCE)); + } + + @Override + @Nonnull + State createState(@Nonnull FunctionInitializationContext context) { + return context.getKeyedStateStore().getListState(stateDesc); + } + + @Override + @Nonnull + public TypeSerializer getUpdateSerializer() { + return StringSerializer.INSTANCE; + } + + @Override + @Nonnull + public String generateRandomUpdate() { + return randomString(); + } + + @Override + @Nonnull + List getInternal(@Nonnull ListState state) throws Exception { + return StreamSupport.stream(state.get().spliterator(), false) + .collect(Collectors.toList()); + } + + @Override + void updateInternal(@Nonnull ListState state, String update) throws Exception { + state.add(update); + } + + @Override + @Nonnull + List expected(@Nonnull List> updates, long currentTimestamp) { + return updates.stream() + .filter(u -> !expired(u.getTimestampAfterUpdate(), currentTimestamp)) + .map(ValueWithTs::getValue) + .collect(Collectors.toList()); + } +} diff --git a/flink-end-to-end-tests/flink-stream-state-ttl-test/src/main/java/org/apache/flink/streaming/tests/verify/TtlMapStateVerifier.java b/flink-end-to-end-tests/flink-stream-state-ttl-test/src/main/java/org/apache/flink/streaming/tests/verify/TtlMapStateVerifier.java new file mode 100644 index 0000000000000..a9d6b36f62f08 --- /dev/null +++ b/flink-end-to-end-tests/flink-stream-state-ttl-test/src/main/java/org/apache/flink/streaming/tests/verify/TtlMapStateVerifier.java @@ -0,0 +1,94 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.streaming.tests.verify; + +import org.apache.flink.api.common.state.MapState; +import org.apache.flink.api.common.state.MapStateDescriptor; +import org.apache.flink.api.common.state.State; +import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.api.common.typeutils.base.StringSerializer; +import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.api.java.typeutils.runtime.TupleSerializer; +import org.apache.flink.runtime.state.FunctionInitializationContext; + +import javax.annotation.Nonnull; + +import java.util.ArrayList; +import java.util.List; +import java.util.Map; +import java.util.stream.Collectors; +import java.util.stream.IntStream; +import java.util.stream.StreamSupport; + +class TtlMapStateVerifier extends AbstractTtlStateVerifier< + MapStateDescriptor, MapState, + Map, Tuple2, Map> { + private static final List KEYS = new ArrayList<>(); + static { + IntStream.range(0, RANDOM.nextInt(5) + 5).forEach(i -> KEYS.add(randomString())); + } + + TtlMapStateVerifier() { + super(new MapStateDescriptor<>("TtlMapStateVerifier", StringSerializer.INSTANCE, StringSerializer.INSTANCE)); + } + + @Override + @Nonnull + State createState(@Nonnull FunctionInitializationContext context) { + return context.getKeyedStateStore().getMapState(stateDesc); + } + + @SuppressWarnings("unchecked") + @Override + @Nonnull + public TypeSerializer> getUpdateSerializer() { + return new TupleSerializer( + Tuple2.class, new TypeSerializer[] {StringSerializer.INSTANCE, StringSerializer.INSTANCE}); + } + + @Override + @Nonnull + public Tuple2 generateRandomUpdate() { + return Tuple2.of(KEYS.get(RANDOM.nextInt(KEYS.size())), randomString()); + } + + @Override + @Nonnull + Map getInternal(@Nonnull MapState state) throws Exception { + return StreamSupport.stream(state.entries().spliterator(), false) + .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue)); + } + + @Override + void updateInternal(@Nonnull MapState state, Tuple2 update) throws Exception { + state.put(update.f0, update.f1); + } + + @Override + @Nonnull + Map expected(@Nonnull List>> updates, long currentTimestamp) { + return updates.stream() + .collect(Collectors.groupingBy(u -> u.getValue().f0)) + .entrySet().stream() + .map(e -> e.getValue().get(e.getValue().size() - 1)) + .filter(u -> !expired(u.getTimestampAfterUpdate(), currentTimestamp)) + .map(ValueWithTs::getValue) + .collect(Collectors.toMap(u -> u.f0, u -> u.f1)); + } +} diff --git a/flink-end-to-end-tests/flink-stream-state-ttl-test/src/main/java/org/apache/flink/streaming/tests/verify/TtlReducingStateVerifier.java b/flink-end-to-end-tests/flink-stream-state-ttl-test/src/main/java/org/apache/flink/streaming/tests/verify/TtlReducingStateVerifier.java new file mode 100644 index 0000000000000..773be05e04b5d --- /dev/null +++ b/flink-end-to-end-tests/flink-stream-state-ttl-test/src/main/java/org/apache/flink/streaming/tests/verify/TtlReducingStateVerifier.java @@ -0,0 +1,86 @@ +/* + * 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.tests.verify; + +import org.apache.flink.api.common.functions.ReduceFunction; +import org.apache.flink.api.common.state.ReducingState; +import org.apache.flink.api.common.state.ReducingStateDescriptor; +import org.apache.flink.api.common.state.State; +import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.api.common.typeutils.base.IntSerializer; +import org.apache.flink.runtime.state.FunctionInitializationContext; + +import javax.annotation.Nonnull; + +import java.util.List; + +class TtlReducingStateVerifier extends AbstractTtlStateVerifier< + ReducingStateDescriptor, ReducingState, Integer, Integer, Integer> { + TtlReducingStateVerifier() { + super(new ReducingStateDescriptor<>( + "TtlReducingStateVerifier", + (ReduceFunction) (value1, value2) -> value1 + value2, + IntSerializer.INSTANCE)); + } + + @Override + @Nonnull + State createState(@Nonnull FunctionInitializationContext context) { + return context.getKeyedStateStore().getReducingState(stateDesc); + } + + @Override + @Nonnull + public TypeSerializer getUpdateSerializer() { + return IntSerializer.INSTANCE; + } + + @Override + @Nonnull + public Integer generateRandomUpdate() { + return RANDOM.nextInt(100); + } + + @Override + Integer getInternal(@Nonnull ReducingState state) throws Exception { + return state.get(); + } + + @Override + void updateInternal(@Nonnull ReducingState state, Integer update) throws Exception { + state.add(update); + } + + @Override + Integer expected(@Nonnull List> updates, long currentTimestamp) { + if (updates.isEmpty()) { + return null; + } + int acc = 0; + long lastTs = updates.get(0).getTimestampAfterUpdate(); + for (ValueWithTs update : updates) { + if (expired(lastTs, update.getTimestampAfterUpdate())) { + acc = 0; + } + acc += update.getValue(); + lastTs = update.getTimestampAfterUpdate(); + } + return expired(lastTs, currentTimestamp) ? null : acc; + } +} diff --git a/flink-end-to-end-tests/flink-stream-state-ttl-test/src/main/java/org/apache/flink/streaming/tests/verify/TtlStateVerifier.java b/flink-end-to-end-tests/flink-stream-state-ttl-test/src/main/java/org/apache/flink/streaming/tests/verify/TtlStateVerifier.java new file mode 100644 index 0000000000000..e1c2e070842fd --- /dev/null +++ b/flink-end-to-end-tests/flink-stream-state-ttl-test/src/main/java/org/apache/flink/streaming/tests/verify/TtlStateVerifier.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.streaming.tests.verify; + +import org.apache.flink.api.common.state.State; +import org.apache.flink.api.common.state.StateTtlConfiguration; +import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.runtime.state.FunctionInitializationContext; + +import javax.annotation.Nonnull; + +import java.util.Arrays; +import java.util.List; + +/** TTL state verifier interface. */ +public interface TtlStateVerifier { + List> VERIFIERS = Arrays.asList( + new TtlValueStateVerifier(), + new TtlListStateVerifier(), + new TtlMapStateVerifier(), + new TtlAggregatingStateVerifier(), + new TtlReducingStateVerifier(), + new TtlFoldingStateVerifier() + ); + + @Nonnull + default String getId() { + return this.getClass().getSimpleName(); + } + + @Nonnull + State createState(@Nonnull FunctionInitializationContext context, @Nonnull StateTtlConfiguration ttlConfig); + + @Nonnull + TypeSerializer getUpdateSerializer(); + + UV generateRandomUpdate(); + + GV get(@Nonnull State state) throws Exception; + + void update(@Nonnull State state, Object update) throws Exception; + + boolean verify(@Nonnull TtlVerificationContext verificationContext); +} diff --git a/flink-end-to-end-tests/flink-stream-state-ttl-test/src/main/java/org/apache/flink/streaming/tests/verify/TtlUpdateContext.java b/flink-end-to-end-tests/flink-stream-state-ttl-test/src/main/java/org/apache/flink/streaming/tests/verify/TtlUpdateContext.java new file mode 100644 index 0000000000000..959340b408d9c --- /dev/null +++ b/flink-end-to-end-tests/flink-stream-state-ttl-test/src/main/java/org/apache/flink/streaming/tests/verify/TtlUpdateContext.java @@ -0,0 +1,71 @@ +/* + * 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.tests.verify; + +import javax.annotation.Nonnull; + +import java.io.Serializable; + +/** Contains context relevant for state update with TTL. */ +public class TtlUpdateContext implements Serializable { + private final long timestampBeforeUpdate; + private final GV valueBeforeUpdate; + private final UV update; + private final GV updatedValue; + private final long timestampAfterUpdate; + + public TtlUpdateContext( + long timestampBeforeUpdate, + GV valueBeforeUpdate, UV update, GV updatedValue, + long timestampAfterUpdate) { + this.valueBeforeUpdate = valueBeforeUpdate; + this.update = update; + this.updatedValue = updatedValue; + this.timestampBeforeUpdate = timestampBeforeUpdate; + this.timestampAfterUpdate = timestampAfterUpdate; + } + + long getTimestampBeforeUpdate() { + return timestampBeforeUpdate; + } + + GV getValueBeforeUpdate() { + return valueBeforeUpdate; + } + + @Nonnull + public ValueWithTs getUpdateWithTs() { + return new ValueWithTs<>(update, timestampBeforeUpdate, timestampAfterUpdate); + } + + GV getUpdatedValue() { + return updatedValue; + } + + @Override + public String toString() { + return "TtlUpdateContext{" + + "timestampBeforeUpdate=" + timestampBeforeUpdate + + ", valueBeforeUpdate=" + valueBeforeUpdate + + ", update=" + update + + ", updatedValue=" + updatedValue + + ", timestampAfterUpdate=" + timestampAfterUpdate + + '}'; + } +} diff --git a/flink-end-to-end-tests/flink-stream-state-ttl-test/src/main/java/org/apache/flink/streaming/tests/verify/TtlValueStateVerifier.java b/flink-end-to-end-tests/flink-stream-state-ttl-test/src/main/java/org/apache/flink/streaming/tests/verify/TtlValueStateVerifier.java new file mode 100644 index 0000000000000..fa4929b933f41 --- /dev/null +++ b/flink-end-to-end-tests/flink-stream-state-ttl-test/src/main/java/org/apache/flink/streaming/tests/verify/TtlValueStateVerifier.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.tests.verify; + +import org.apache.flink.api.common.state.State; +import org.apache.flink.api.common.state.ValueState; +import org.apache.flink.api.common.state.ValueStateDescriptor; +import org.apache.flink.api.common.typeutils.base.StringSerializer; +import org.apache.flink.runtime.state.FunctionInitializationContext; + +import javax.annotation.Nonnull; + +import java.util.List; + +class TtlValueStateVerifier + extends AbstractTtlStateVerifier, ValueState, String, String, String> { + TtlValueStateVerifier() { + super(new ValueStateDescriptor<>("TtlValueStateVerifier", StringSerializer.INSTANCE)); + } + + @Override + @Nonnull + State createState(FunctionInitializationContext context) { + return context.getKeyedStateStore().getState(stateDesc); + } + + @Nonnull + public String generateRandomUpdate() { + return randomString(); + } + + @Override + String getInternal(@Nonnull ValueState state) throws Exception { + return state.value(); + } + + @Override + void updateInternal(@Nonnull ValueState state, String update) throws Exception { + state.update(update); + } + + @Override + String expected(@Nonnull List> updates, long currentTimestamp) { + if (updates.isEmpty()) { + return null; + } + ValueWithTs lastUpdate = updates.get(updates.size() - 1); + return expired(lastUpdate.getTimestampAfterUpdate(), currentTimestamp) ? null : lastUpdate.getValue(); + } +} diff --git a/flink-end-to-end-tests/flink-stream-state-ttl-test/src/main/java/org/apache/flink/streaming/tests/verify/TtlVerificationContext.java b/flink-end-to-end-tests/flink-stream-state-ttl-test/src/main/java/org/apache/flink/streaming/tests/verify/TtlVerificationContext.java new file mode 100644 index 0000000000000..4c985cd525b31 --- /dev/null +++ b/flink-end-to-end-tests/flink-stream-state-ttl-test/src/main/java/org/apache/flink/streaming/tests/verify/TtlVerificationContext.java @@ -0,0 +1,69 @@ +/* + * 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.tests.verify; + +import javax.annotation.Nonnull; + +import java.io.Serializable; +import java.util.ArrayList; +import java.util.List; + +/** Data to verify state update with TTL. */ +public class TtlVerificationContext implements Serializable { + private final int key; + @Nonnull + private final String verifierId; + @Nonnull + private final List> prevUpdates; + @Nonnull + private final TtlUpdateContext updateContext; + + @SuppressWarnings("unchecked") + public TtlVerificationContext( + int key, + @Nonnull String verifierId, + @Nonnull List> prevUpdates, + @Nonnull TtlUpdateContext updateContext) { + this.key = key; + this.verifierId = verifierId; + this.prevUpdates = new ArrayList<>(); + prevUpdates.forEach(pu -> this.prevUpdates.add((ValueWithTs) pu)); + this.updateContext = (TtlUpdateContext) updateContext; + } + + @Nonnull + List> getPrevUpdates() { + return prevUpdates; + } + + @Nonnull + TtlUpdateContext getUpdateContext() { + return updateContext; + } + + @Override + public String toString() { + return "TtlVerificationContext{" + + "key=" + key + + ", verifierId='" + verifierId + '\'' + + ", prevUpdates=" + prevUpdates + + ", updateContext=" + updateContext + + '}'; + } +} diff --git a/flink-end-to-end-tests/flink-stream-state-ttl-test/src/main/java/org/apache/flink/streaming/tests/verify/ValueWithTs.java b/flink-end-to-end-tests/flink-stream-state-ttl-test/src/main/java/org/apache/flink/streaming/tests/verify/ValueWithTs.java new file mode 100644 index 0000000000000..9302377d9eddb --- /dev/null +++ b/flink-end-to-end-tests/flink-stream-state-ttl-test/src/main/java/org/apache/flink/streaming/tests/verify/ValueWithTs.java @@ -0,0 +1,107 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.streaming.tests.verify; + +import org.apache.flink.api.common.typeutils.CompositeSerializer; +import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.api.common.typeutils.base.LongSerializer; +import org.apache.flink.util.FlinkRuntimeException; + +import javax.annotation.Nonnull; + +import java.io.Serializable; + +/** User state value with timestamps before and after update. */ +public class ValueWithTs implements Serializable { + private final V value; + private final long timestampBeforeUpdate; + private final long timestampAfterUpdate; + + public ValueWithTs(V value, long timestampBeforeUpdate, long timestampAfterUpdate) { + this.value = value; + this.timestampBeforeUpdate = timestampBeforeUpdate; + this.timestampAfterUpdate = timestampAfterUpdate; + } + + V getValue() { + return value; + } + + public long getTimestampBeforeUpdate() { + return timestampBeforeUpdate; + } + + public long getTimestampAfterUpdate() { + return timestampAfterUpdate; + } + + @Override + public String toString() { + return "ValueWithTs{" + + "value=" + value + + ", timestampBeforeUpdate=" + timestampBeforeUpdate + + ", timestampAfterUpdate=" + timestampAfterUpdate + + '}'; + } + + /** Serializer for Serializer. */ + public static class Serializer extends CompositeSerializer> { + + public Serializer(TypeSerializer userValueSerializer) { + super(true, userValueSerializer, LongSerializer.INSTANCE, LongSerializer.INSTANCE); + } + + @SuppressWarnings("unchecked") + Serializer(PrecomputedParameters precomputed, TypeSerializer... fieldSerializers) { + super(precomputed, fieldSerializers); + } + + @Override + public ValueWithTs createInstance(@Nonnull Object ... values) { + return new ValueWithTs<>(values[0], (Long) values[1], (Long) values[2]); + } + + @Override + protected void setField(@Nonnull ValueWithTs value, int index, Object fieldValue) { + throw new UnsupportedOperationException(); + } + + @Override + protected Object getField(@Nonnull ValueWithTs value, int index) { + switch (index) { + case 0: + return value.getValue(); + case 1: + return value.getTimestampBeforeUpdate(); + case 2: + return value.getTimestampAfterUpdate(); + default: + throw new FlinkRuntimeException("Unexpected field index for ValueWithTs"); + } + } + + @SuppressWarnings("unchecked") + @Override + protected CompositeSerializer> createSerializerInstance( + PrecomputedParameters precomputed, + TypeSerializer... originalSerializers) { + return new Serializer(precomputed, (TypeSerializer) originalSerializers[0]); + } + } +} diff --git a/flink-end-to-end-tests/pom.xml b/flink-end-to-end-tests/pom.xml index 091dd0b0f51a8..d80ac32dedca0 100644 --- a/flink-end-to-end-tests/pom.xml +++ b/flink-end-to-end-tests/pom.xml @@ -50,6 +50,7 @@ under the License. flink-elasticsearch5-test flink-quickstart-test flink-confluent-schema-registry + flink-stream-state-ttl-test diff --git a/flink-end-to-end-tests/run-nightly-tests.sh b/flink-end-to-end-tests/run-nightly-tests.sh index 15c73d5b7a429..dc8424f25eccf 100755 --- a/flink-end-to-end-tests/run-nightly-tests.sh +++ b/flink-end-to-end-tests/run-nightly-tests.sh @@ -102,5 +102,8 @@ run_test "Quickstarts Scala nightly end-to-end test" "$END_TO_END_DIR/test-scrip run_test "Avro Confluent Schema Registry nightly end-to-end test" "$END_TO_END_DIR/test-scripts/test_confluent_schema_registry.sh" +run_test "State TTL Heap backend end-to-end test" "$END_TO_END_DIR/test-scripts/test_stream_state_ttl.sh file" +run_test "State TTL RocksDb backend end-to-end test" "$END_TO_END_DIR/test-scripts/test_stream_state_ttl.sh rocks" + printf "\n[PASS] All tests passed\n" exit 0 diff --git a/flink-end-to-end-tests/test-scripts/common.sh b/flink-end-to-end-tests/test-scripts/common.sh index c78afe74b6d3b..f4563cc3ea451 100644 --- a/flink-end-to-end-tests/test-scripts/common.sh +++ b/flink-end-to-end-tests/test-scripts/common.sh @@ -240,6 +240,15 @@ function start_cluster { done } +function start_taskmanagers { + tmnum=$1 + echo "Start ${tmnum} more task managers" + for (( c=0; c /dev/null - done + if (( count != 0 )); then + start_taskmanagers ${count} > /dev/null + fi sleep 5; done } @@ -508,7 +516,8 @@ function rollback_flink_slf4j_metric_reporter() { function get_metric_processed_records { OPERATOR=$1 - N=$(grep ".General purpose test job.$OPERATOR.numRecordsIn:" $FLINK_DIR/log/*taskexecutor*.log | sed 's/.* //g' | tail -1) + JOB_NAME="${2:-General purpose test job}" + N=$(grep ".${JOB_NAME}.$OPERATOR.numRecordsIn:" $FLINK_DIR/log/*taskexecutor*.log | sed 's/.* //g' | tail -1) if [ -z $N ]; then N=0 fi @@ -517,7 +526,8 @@ function get_metric_processed_records { function get_num_metric_samples { OPERATOR=$1 - N=$(grep ".General purpose test job.$OPERATOR.numRecordsIn:" $FLINK_DIR/log/*taskexecutor*.log | wc -l) + JOB_NAME="${2:-General purpose test job}" + N=$(grep ".${JOB_NAME}.$OPERATOR.numRecordsIn:" $FLINK_DIR/log/*taskexecutor*.log | wc -l) if [ -z $N ]; then N=0 fi @@ -527,13 +537,14 @@ function get_num_metric_samples { function wait_oper_metric_num_in_records { OPERATOR=$1 MAX_NUM_METRICS="${2:-200}" - NUM_METRICS=$(get_num_metric_samples ${OPERATOR}) - OLD_NUM_METRICS=${3:-${NUM_METRICS}} + JOB_NAME="${3:-General purpose test job}" + NUM_METRICS=$(get_num_metric_samples ${OPERATOR} '${JOB_NAME}') + OLD_NUM_METRICS=${4:-${NUM_METRICS}} # monitor the numRecordsIn metric of the state machine operator in the second execution # we let the test finish once the second restore execution has processed 200 records while : ; do - NUM_METRICS=$(get_num_metric_samples ${OPERATOR}) - NUM_RECORDS=$(get_metric_processed_records ${OPERATOR}) + NUM_METRICS=$(get_num_metric_samples ${OPERATOR} "${JOB_NAME}") + NUM_RECORDS=$(get_metric_processed_records ${OPERATOR} "${JOB_NAME}") # only account for metrics that appeared in the second execution if (( $OLD_NUM_METRICS >= $NUM_METRICS )) ; then @@ -541,7 +552,7 @@ function wait_oper_metric_num_in_records { fi if (( $NUM_RECORDS < $MAX_NUM_METRICS )); then - echo "Waiting for job to process up to 200 records, current progress: $NUM_RECORDS records ..." + echo "Waiting for job to process up to ${MAX_NUM_METRICS} records, current progress: ${NUM_RECORDS} records ..." sleep 1 else break diff --git a/flink-end-to-end-tests/test-scripts/test_stream_state_ttl.sh b/flink-end-to-end-tests/test-scripts/test_stream_state_ttl.sh new file mode 100755 index 0000000000000..fb911f327ff90 --- /dev/null +++ b/flink-end-to-end-tests/test-scripts/test_stream_state_ttl.sh @@ -0,0 +1,83 @@ +#!/usr/bin/env bash +################################################################################ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +################################################################################ + +source "$(dirname "$0")"/common.sh + +STATE_BACKEND_TYPE="${1:-file}" +STATE_BACKEND_FILE_ASYNC="${2:-false}" +TTL="${3:-1000}" +PRECISION="${4:-5}" +PARALLELISM="${5-3}" +UPDATE_NUM="${6-1000}" + +CHECKPOINT_DIR="file://$TEST_DATA_DIR/savepoint-e2e-test-chckpt-dir" + +TEST=flink-stream-state-ttl-test +TEST_PROGRAM_NAME=DataStreamStateTTLTestProgram +TEST_PROGRAM_JAR=${END_TO_END_DIR}/$TEST/target/$TEST_PROGRAM_NAME.jar + +setup_flink_slf4j_metric_reporter +function test_cleanup { + # don't call ourselves again for another signal interruption + trap "exit -1" INT + # don't call ourselves again for normal exit + trap "" EXIT + + # revert our modifications to the Flink distribution + rm ${FLINK_DIR}/lib/flink-metrics-slf4j-*.jar +} +trap test_cleanup INT +trap test_cleanup EXIT + +start_cluster +start_taskmanagers $PARALLELISM + +function job_id() { + CMD="${FLINK_DIR}/bin/flink run -d -p ${PARALLELISM} ${TEST_PROGRAM_JAR} \ + --test.semantics exactly-once \ + --environment.parallelism ${PARALLELISM} \ + --state_backend ${STATE_BACKEND_TYPE} \ + --state_ttl_verifier.ttl_milli ${TTL} \ + --state_ttl_verifier.precision_milli ${PRECISION} \ + --state_backend.checkpoint_directory ${CHECKPOINT_DIR} \ + --state_backend.file.async ${STATE_BACKEND_FILE_ASYNC} \ + --update_generator_source.sleep_time 10 \ + --update_generator_source.sleep_after_elements 1" + echo "${CMD}" +} + +JOB_CMD=$(job_id) +echo ${JOB_CMD} +JOB=$(${JOB_CMD} | grep 'Job has been submitted with JobID' | sed 's/.* //g') +wait_job_running ${JOB} +wait_oper_metric_num_in_records TtlVerifyUpdateFunction.0 ${UPDATE_NUM} 'State TTL test job' + +SAVEPOINT_PATH=$(take_savepoint ${JOB} ${TEST_DATA_DIR} \ + | grep "Savepoint completed. Path:" | sed 's/.* //g') + +cancel_job ${JOB} + +JOB_CMD=$(job_id) +echo ${JOB_CMD} +JOB=$(${JOB_CMD} | grep 'Job has been submitted with JobID' | sed 's/.* //g') +wait_job_running ${JOB} +wait_oper_metric_num_in_records TtlVerifyUpdateFunction.0 ${UPDATE_NUM} "State TTL test job" + +# if verification fails job produces failed TTL'ed state updates, +# output would be non-empty and the test will not pass From f19337a7d7576ef905a59088ba8279c2d0397a1f Mon Sep 17 00:00:00 2001 From: Stefan Richter Date: Wed, 18 Jul 2018 23:29:56 +0200 Subject: [PATCH 34/65] [FLINK-9902][tests] Improve and refactor window checkpointing IT cases This closes #6376. --- ...EventTimeAllWindowCheckpointingITCase.java | 296 ++------------ .../EventTimeWindowCheckpointingITCase.java | 375 +++++------------- .../WindowCheckpointingITCase.java | 229 +++-------- .../checkpointing/utils/FailingSource.java | 155 ++++++++ .../test/checkpointing/utils/IntType.java | 38 ++ .../checkpointing/utils/ValidatingSink.java | 128 ++++++ .../flink/test/util/SuccessException.java | 2 +- 7 files changed, 505 insertions(+), 718 deletions(-) create mode 100644 flink-tests/src/test/java/org/apache/flink/test/checkpointing/utils/FailingSource.java create mode 100644 flink-tests/src/test/java/org/apache/flink/test/checkpointing/utils/IntType.java create mode 100644 flink-tests/src/test/java/org/apache/flink/test/checkpointing/utils/ValidatingSink.java diff --git a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/EventTimeAllWindowCheckpointingITCase.java b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/EventTimeAllWindowCheckpointingITCase.java index 9e14b2660e25f..5dc2aa0e87f05 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/EventTimeAllWindowCheckpointingITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/EventTimeAllWindowCheckpointingITCase.java @@ -26,31 +26,23 @@ import org.apache.flink.configuration.AkkaOptions; import org.apache.flink.configuration.Configuration; import org.apache.flink.configuration.TaskManagerOptions; -import org.apache.flink.runtime.state.CheckpointListener; import org.apache.flink.streaming.api.TimeCharacteristic; -import org.apache.flink.streaming.api.checkpoint.ListCheckpointed; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; -import org.apache.flink.streaming.api.functions.sink.RichSinkFunction; -import org.apache.flink.streaming.api.functions.source.RichSourceFunction; import org.apache.flink.streaming.api.functions.windowing.RichAllWindowFunction; -import org.apache.flink.streaming.api.watermark.Watermark; import org.apache.flink.streaming.api.windowing.time.Time; import org.apache.flink.streaming.api.windowing.windows.TimeWindow; +import org.apache.flink.test.checkpointing.utils.FailingSource; +import org.apache.flink.test.checkpointing.utils.IntType; +import org.apache.flink.test.checkpointing.utils.ValidatingSink; import org.apache.flink.test.util.MiniClusterResource; import org.apache.flink.test.util.MiniClusterResourceConfiguration; -import org.apache.flink.test.util.SuccessException; import org.apache.flink.util.Collector; import org.apache.flink.util.TestLogger; import org.junit.ClassRule; import org.junit.Test; -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.util.TestUtils.tryExecute; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; @@ -88,20 +80,17 @@ public void testTumblingTimeWindow() { final int numElementsPerKey = 3000; final int windowSize = 100; final int numKeys = 1; - FailingSource.reset(); try { StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); env.setParallelism(PARALLELISM); env.setStreamTimeCharacteristic(TimeCharacteristic.EventTime); env.enableCheckpointing(100); - env.setRestartStrategy(RestartStrategies.fixedDelayRestart(3, 0)); + env.setRestartStrategy(RestartStrategies.fixedDelayRestart(1, 0)); env.getConfig().disableSysoutLogging(); env - .addSource(new FailingSource(numKeys, - numElementsPerKey, - numElementsPerKey / 3)) + .addSource(new FailingSource(new EventTimeWindowCheckpointingITCase.KeyedEventTimeGenerator(numKeys, windowSize), numElementsPerKey)) .rebalance() .timeWindowAll(Time.of(windowSize, MILLISECONDS)) .apply(new RichAllWindowFunction, Tuple4, TimeWindow>() { @@ -133,9 +122,12 @@ public void apply( out.collect(new Tuple4<>(key, window.getStart(), window.getEnd(), new IntType(sum))); } }) - .addSink(new ValidatingSink(numKeys, numElementsPerKey / windowSize)).setParallelism(1); + .addSink(new ValidatingSink<>( + new EventTimeWindowCheckpointingITCase.SinkValidatorUpdateFun(numElementsPerKey), + new EventTimeWindowCheckpointingITCase.SinkValidatorCheckFun(numKeys, numElementsPerKey, windowSize))) + .setParallelism(1); - tryExecute(env, "Tumbling Window Test"); + env.execute("Tumbling Window Test"); } catch (Exception e) { e.printStackTrace(); @@ -149,7 +141,6 @@ public void testSlidingTimeWindow() { final int windowSize = 1000; final int windowSlide = 100; final int numKeys = 1; - FailingSource.reset(); try { StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); @@ -160,7 +151,7 @@ public void testSlidingTimeWindow() { env.getConfig().disableSysoutLogging(); env - .addSource(new FailingSource(numKeys, numElementsPerKey, numElementsPerKey / 3)) + .addSource(new FailingSource(new EventTimeWindowCheckpointingITCase.KeyedEventTimeGenerator(numKeys, windowSlide), numElementsPerKey)) .rebalance() .timeWindowAll(Time.of(windowSize, MILLISECONDS), Time.of(windowSlide, MILLISECONDS)) .apply(new RichAllWindowFunction, Tuple4, TimeWindow>() { @@ -192,9 +183,12 @@ public void apply( out.collect(new Tuple4<>(key, window.getStart(), window.getEnd(), new IntType(sum))); } }) - .addSink(new ValidatingSink(numKeys, numElementsPerKey / windowSlide)).setParallelism(1); + .addSink(new ValidatingSink<>( + new EventTimeWindowCheckpointingITCase.SinkValidatorUpdateFun(numElementsPerKey), + new EventTimeWindowCheckpointingITCase.SinkValidatorCheckFun(numKeys, numElementsPerKey, windowSlide))) + .setParallelism(1); - tryExecute(env, "Sliding Window Test"); + env.execute("Sliding Window Test"); } catch (Exception e) { e.printStackTrace(); @@ -207,20 +201,17 @@ public void testPreAggregatedTumblingTimeWindow() { final int numElementsPerKey = 3000; final int windowSize = 100; final int numKeys = 1; - FailingSource.reset(); try { StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); env.setParallelism(PARALLELISM); env.setStreamTimeCharacteristic(TimeCharacteristic.EventTime); env.enableCheckpointing(100); - env.setRestartStrategy(RestartStrategies.fixedDelayRestart(3, 0)); + env.setRestartStrategy(RestartStrategies.fixedDelayRestart(1, 0)); env.getConfig().disableSysoutLogging(); env - .addSource(new FailingSource(numKeys, - numElementsPerKey, - numElementsPerKey / 3)) + .addSource(new FailingSource(new EventTimeWindowCheckpointingITCase.KeyedEventTimeGenerator(numKeys, windowSize), numElementsPerKey)) .rebalance() .timeWindowAll(Time.of(windowSize, MILLISECONDS)) .reduce( @@ -261,9 +252,12 @@ public void apply( } } }) - .addSink(new ValidatingSink(numKeys, numElementsPerKey / windowSize)).setParallelism(1); + .addSink(new ValidatingSink<>( + new EventTimeWindowCheckpointingITCase.SinkValidatorUpdateFun(numElementsPerKey), + new EventTimeWindowCheckpointingITCase.SinkValidatorCheckFun(numKeys, numElementsPerKey, windowSize))) + .setParallelism(1); - tryExecute(env, "Tumbling Window Test"); + env.execute("Tumbling Window Test"); } catch (Exception e) { e.printStackTrace(); @@ -276,20 +270,17 @@ public void testPreAggregatedFoldingTumblingTimeWindow() { final int numElementsPerKey = 3000; final int windowSize = 100; final int numKeys = 1; - FailingSource.reset(); try { StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); env.setParallelism(PARALLELISM); env.setStreamTimeCharacteristic(TimeCharacteristic.EventTime); env.enableCheckpointing(100); - env.setRestartStrategy(RestartStrategies.fixedDelayRestart(3, 0)); + env.setRestartStrategy(RestartStrategies.fixedDelayRestart(1, 0)); env.getConfig().disableSysoutLogging(); env - .addSource(new FailingSource(numKeys, - numElementsPerKey, - numElementsPerKey / 3)) + .addSource(new FailingSource(new EventTimeWindowCheckpointingITCase.KeyedEventTimeGenerator(numKeys, windowSize), numElementsPerKey)) .rebalance() .timeWindowAll(Time.of(windowSize, MILLISECONDS)) .fold(new Tuple4<>(0L, 0L, 0L, new IntType(0)), @@ -329,9 +320,12 @@ public void apply( } } }) - .addSink(new ValidatingSink(numKeys, numElementsPerKey / windowSize)).setParallelism(1); + .addSink(new ValidatingSink<>( + new EventTimeWindowCheckpointingITCase.SinkValidatorUpdateFun(numElementsPerKey), + new EventTimeWindowCheckpointingITCase.SinkValidatorCheckFun(numKeys, numElementsPerKey, windowSize))) + .setParallelism(1); - tryExecute(env, "Tumbling Window Test"); + env.execute("Tumbling Window Test"); } catch (Exception e) { e.printStackTrace(); @@ -345,20 +339,17 @@ public void testPreAggregatedSlidingTimeWindow() { final int windowSize = 1000; final int windowSlide = 100; final int numKeys = 1; - FailingSource.reset(); try { StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); env.setParallelism(PARALLELISM); env.setStreamTimeCharacteristic(TimeCharacteristic.EventTime); env.enableCheckpointing(100); - env.setRestartStrategy(RestartStrategies.fixedDelayRestart(3, 0)); + env.setRestartStrategy(RestartStrategies.fixedDelayRestart(1, 0)); env.getConfig().disableSysoutLogging(); env - .addSource(new FailingSource(numKeys, - numElementsPerKey, - numElementsPerKey / 3)) + .addSource(new FailingSource(new EventTimeWindowCheckpointingITCase.KeyedEventTimeGenerator(numKeys, windowSlide), numElementsPerKey)) .rebalance() .timeWindowAll(Time.of(windowSize, MILLISECONDS), Time.of(windowSlide, MILLISECONDS)) @@ -400,229 +391,16 @@ public void apply( } } }) - .addSink(new ValidatingSink(numKeys, numElementsPerKey / windowSlide)).setParallelism(1); + .addSink(new ValidatingSink<>( + new EventTimeWindowCheckpointingITCase.SinkValidatorUpdateFun(numElementsPerKey), + new EventTimeWindowCheckpointingITCase.SinkValidatorCheckFun(numKeys, numElementsPerKey, windowSlide))) + .setParallelism(1); - tryExecute(env, "Tumbling Window Test"); + env.execute("Tumbling Window Test"); } catch (Exception e) { e.printStackTrace(); fail(e.getMessage()); } } - - // ------------------------------------------------------------------------ - // Utilities - // ------------------------------------------------------------------------ - - private static class FailingSource extends RichSourceFunction> - implements ListCheckpointed, CheckpointListener { - private static volatile boolean failedBefore = false; - - private final int numKeys; - private final int numElementsToEmit; - private final int failureAfterNumElements; - - private volatile int numElementsEmitted; - private volatile int numSuccessfulCheckpoints; - private volatile boolean running = true; - - private FailingSource(int numKeys, int numElementsToEmitPerKey, int failureAfterNumElements) { - this.numKeys = numKeys; - this.numElementsToEmit = numElementsToEmitPerKey; - this.failureAfterNumElements = failureAfterNumElements; - } - - @Override - public void open(Configuration parameters) { - // non-parallel source - assertEquals(1, getRuntimeContext().getNumberOfParallelSubtasks()); - } - - @Override - public void run(SourceContext> ctx) throws Exception { - // we loop longer than we have elements, to permit delayed checkpoints - // to still cause a failure - while (running) { - - if (!failedBefore) { - // delay a bit, if we have not failed before - Thread.sleep(1); - if (numSuccessfulCheckpoints >= 2 && numElementsEmitted >= failureAfterNumElements) { - // cause a failure if we have not failed before and have reached - // enough completed checkpoints and elements - failedBefore = true; - throw new Exception("Artificial Failure"); - } - } - - if (numElementsEmitted < numElementsToEmit && - (failedBefore || numElementsEmitted <= failureAfterNumElements)) { - // the function failed before, or we are in the elements before the failure - synchronized (ctx.getCheckpointLock()) { - int next = numElementsEmitted++; - for (long i = 0; i < numKeys; i++) { - ctx.collectWithTimestamp(new Tuple2<>(i, new IntType(next)), next); - } - ctx.emitWatermark(new Watermark(next)); - } - } - else { - // if our work is done, delay a bit to prevent busy waiting - Thread.sleep(1); - } - } - } - - @Override - public void cancel() { - running = false; - } - - @Override - public void notifyCheckpointComplete(long checkpointId) { - numSuccessfulCheckpoints++; - } - - public static void reset() { - failedBefore = false; - } - - @Override - public List snapshotState(long checkpointId, long timestamp) throws Exception { - return Collections.singletonList(this.numElementsEmitted); - } - - @Override - public void restoreState(List state) throws Exception { - if (state.isEmpty() || state.size() > 1) { - throw new RuntimeException("Test failed due to unexpected recovered state size " + state.size()); - } - this.numElementsEmitted = state.get(0); - } - } - - private static class ValidatingSink extends RichSinkFunction> - implements ListCheckpointed> { - - private final HashMap windowCounts = new HashMap<>(); - - private final int numKeys; - private final int numWindowsExpected; - - private ValidatingSink(int numKeys, int numWindowsExpected) { - this.numKeys = numKeys; - this.numWindowsExpected = numWindowsExpected; - } - - @Override - public void open(Configuration parameters) throws Exception { - // this sink can only work with DOP 1 - assertEquals(1, getRuntimeContext().getNumberOfParallelSubtasks()); - - // it can happen that a checkpoint happens when the complete success state is - // already set. In that case we restart with the final state and would never - // finish because no more elements arrive. - if (windowCounts.size() == numKeys) { - boolean seenAll = true; - for (Integer windowCount: windowCounts.values()) { - if (windowCount != numWindowsExpected) { - seenAll = false; - break; - } - } - if (seenAll) { - throw new SuccessException(); - } - } - } - - @Override - public void close() throws Exception { - boolean seenAll = true; - if (windowCounts.size() == numKeys) { - for (Integer windowCount: windowCounts.values()) { - if (windowCount != numWindowsExpected) { - seenAll = false; - break; - } - } - } - assertTrue("The source must see all expected windows.", seenAll); - } - - @Override - public void invoke(Tuple4 value) throws Exception { - - // verify the contents of that window, Tuple4.f1 and .f2 are the window start/end - // the sum should be "sum (start .. end-1)" - - int expectedSum = 0; - for (long i = value.f1; i < value.f2; i++) { - // only sum up positive vals, to filter out the negative start of the - // first sliding windows - if (i > 0) { - expectedSum += i; - } - } - - assertEquals("Window start: " + value.f1 + " end: " + value.f2, expectedSum, value.f3.value); - - Integer curr = windowCounts.get(value.f0); - if (curr != null) { - windowCounts.put(value.f0, curr + 1); - } - else { - windowCounts.put(value.f0, 1); - } - - boolean seenAll = true; - if (windowCounts.size() == numKeys) { - for (Integer windowCount: windowCounts.values()) { - if (windowCount < numWindowsExpected) { - seenAll = false; - break; - } else if (windowCount > numWindowsExpected) { - fail("Window count to high: " + windowCount); - } - } - - if (seenAll) { - // exit - throw new SuccessException(); - } - - } - } - - @Override - public List> snapshotState(long checkpointId, long timestamp) throws Exception { - return Collections.singletonList(this.windowCounts); - } - - @Override - public void restoreState(List> state) throws Exception { - if (state.isEmpty() || state.size() > 1) { - throw new RuntimeException("Test failed due to unexpected recovered state size " + state.size()); - } - this.windowCounts.putAll(state.get(0)); - } - } - - // ------------------------------------------------------------------------ - // Utilities - // ------------------------------------------------------------------------ - - /** - * Custom boxed integer type. - */ - public static class IntType { - - public int value; - - public IntType() {} - - public IntType(int value) { - this.value = value; - } - } } diff --git a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/EventTimeWindowCheckpointingITCase.java b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/EventTimeWindowCheckpointingITCase.java index c3d93d7f7689b..e9a2e45d2601f 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/EventTimeWindowCheckpointingITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/EventTimeWindowCheckpointingITCase.java @@ -32,21 +32,20 @@ import org.apache.flink.contrib.streaming.state.RocksDBStateBackend; import org.apache.flink.core.fs.Path; import org.apache.flink.runtime.state.AbstractStateBackend; -import org.apache.flink.runtime.state.CheckpointListener; import org.apache.flink.runtime.state.filesystem.FsStateBackend; import org.apache.flink.runtime.state.memory.MemoryStateBackend; import org.apache.flink.streaming.api.TimeCharacteristic; -import org.apache.flink.streaming.api.checkpoint.ListCheckpointed; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; -import org.apache.flink.streaming.api.functions.sink.RichSinkFunction; -import org.apache.flink.streaming.api.functions.source.RichSourceFunction; +import org.apache.flink.streaming.api.functions.source.SourceFunction; import org.apache.flink.streaming.api.functions.windowing.RichWindowFunction; import org.apache.flink.streaming.api.watermark.Watermark; import org.apache.flink.streaming.api.windowing.time.Time; import org.apache.flink.streaming.api.windowing.windows.TimeWindow; +import org.apache.flink.test.checkpointing.utils.FailingSource; +import org.apache.flink.test.checkpointing.utils.IntType; +import org.apache.flink.test.checkpointing.utils.ValidatingSink; import org.apache.flink.test.util.MiniClusterResource; import org.apache.flink.test.util.MiniClusterResourceConfiguration; -import org.apache.flink.test.util.SuccessException; import org.apache.flink.util.Collector; import org.apache.flink.util.TestLogger; @@ -65,19 +64,10 @@ import java.io.IOException; import java.util.Arrays; import java.util.Collection; -import java.util.Collections; -import java.util.HashMap; -import java.util.List; +import java.util.Map; import static java.util.concurrent.TimeUnit.MILLISECONDS; -import static org.apache.flink.test.checkpointing.EventTimeWindowCheckpointingITCase.StateBackendEnum.FILE; -import static org.apache.flink.test.checkpointing.EventTimeWindowCheckpointingITCase.StateBackendEnum.FILE_ASYNC; -import static org.apache.flink.test.checkpointing.EventTimeWindowCheckpointingITCase.StateBackendEnum.MEM; -import static org.apache.flink.test.checkpointing.EventTimeWindowCheckpointingITCase.StateBackendEnum.MEM_ASYNC; -import static org.apache.flink.test.checkpointing.EventTimeWindowCheckpointingITCase.StateBackendEnum.ROCKSDB_FULLY_ASYNC; -import static org.apache.flink.test.checkpointing.EventTimeWindowCheckpointingITCase.StateBackendEnum.ROCKSDB_INCREMENTAL; import static org.apache.flink.test.checkpointing.EventTimeWindowCheckpointingITCase.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; import static org.junit.Assert.fail; @@ -255,20 +245,19 @@ public void testTumblingTimeWindow() { final int numElementsPerKey = numElementsPerKey(); final int windowSize = windowSize(); final int numKeys = numKeys(); - FailingSource.reset(); try { StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); env.setParallelism(PARALLELISM); env.setStreamTimeCharacteristic(TimeCharacteristic.EventTime); env.enableCheckpointing(100); - env.setRestartStrategy(RestartStrategies.fixedDelayRestart(3, 0)); + env.setRestartStrategy(RestartStrategies.fixedDelayRestart(1, 0)); env.getConfig().disableSysoutLogging(); env.setStateBackend(this.stateBackend); env.getConfig().setUseSnapshotCompression(true); env - .addSource(new FailingSource(numKeys, numElementsPerKey, numElementsPerKey / 3)) + .addSource(new FailingSource(new KeyedEventTimeGenerator(numKeys, windowSize), numElementsPerKey)) .rebalance() .keyBy(0) .timeWindow(Time.of(windowSize, MILLISECONDS)) @@ -299,12 +288,17 @@ public void apply( sum += value.f1.value; key = value.f0; } - out.collect(new Tuple4<>(key, window.getStart(), window.getEnd(), new IntType(sum))); + + final Tuple4 result = + new Tuple4<>(key, window.getStart(), window.getEnd(), new IntType(sum)); + out.collect(result); } }) - .addSink(new ValidatingSink(numKeys, numElementsPerKey / windowSize)).setParallelism(1); + .addSink(new ValidatingSink<>( + new SinkValidatorUpdateFun(numElementsPerKey), + new SinkValidatorCheckFun(numKeys, numElementsPerKey, windowSize))).setParallelism(1); - tryExecute(env, "Tumbling Window Test"); + env.execute("Tumbling Window Test"); } catch (Exception e) { e.printStackTrace(); @@ -326,7 +320,6 @@ public void doTestTumblingTimeWindowWithKVState(int maxParallelism) { final int numElementsPerKey = numElementsPerKey(); final int windowSize = windowSize(); final int numKeys = numKeys(); - FailingSource.reset(); try { StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); @@ -334,13 +327,13 @@ public void doTestTumblingTimeWindowWithKVState(int maxParallelism) { env.setMaxParallelism(maxParallelism); env.setStreamTimeCharacteristic(TimeCharacteristic.EventTime); env.enableCheckpointing(100); - env.setRestartStrategy(RestartStrategies.fixedDelayRestart(3, 0)); + env.setRestartStrategy(RestartStrategies.fixedDelayRestart(1, 0)); env.getConfig().disableSysoutLogging(); env.setStateBackend(this.stateBackend); env.getConfig().setUseSnapshotCompression(true); env - .addSource(new FailingSource(numKeys, numElementsPerKey, numElementsPerKey / 3)) + .addSource(new FailingSource(new KeyedEventTimeGenerator(numKeys, windowSize), numElementsPerKey)) .rebalance() .keyBy(0) .timeWindow(Time.of(windowSize, MILLISECONDS)) @@ -378,9 +371,11 @@ public void apply( out.collect(new Tuple4<>(tuple.getField(0), window.getStart(), window.getEnd(), new IntType(count.value()))); } }) - .addSink(new CountValidatingSink(numKeys, numElementsPerKey / windowSize)).setParallelism(1); + .addSink(new ValidatingSink<>( + new CountingSinkValidatorUpdateFun(), + new SinkValidatorCheckFun(numKeys, numElementsPerKey, windowSize))).setParallelism(1); - tryExecute(env, "Tumbling Window Test"); + env.execute("Tumbling Window Test"); } catch (Exception e) { e.printStackTrace(); @@ -394,7 +389,6 @@ public void testSlidingTimeWindow() { final int windowSize = windowSize(); final int windowSlide = windowSlide(); final int numKeys = numKeys(); - FailingSource.reset(); try { StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); @@ -402,13 +396,13 @@ public void testSlidingTimeWindow() { env.setParallelism(PARALLELISM); env.setStreamTimeCharacteristic(TimeCharacteristic.EventTime); env.enableCheckpointing(100); - env.setRestartStrategy(RestartStrategies.fixedDelayRestart(3, 0)); + env.setRestartStrategy(RestartStrategies.fixedDelayRestart(1, 0)); env.getConfig().disableSysoutLogging(); env.setStateBackend(this.stateBackend); env.getConfig().setUseSnapshotCompression(true); env - .addSource(new FailingSource(numKeys, numElementsPerKey, numElementsPerKey / 3)) + .addSource(new FailingSource(new KeyedEventTimeGenerator(numKeys, windowSlide), numElementsPerKey)) .rebalance() .keyBy(0) .timeWindow(Time.of(windowSize, MILLISECONDS), Time.of(windowSlide, MILLISECONDS)) @@ -439,12 +433,16 @@ public void apply( sum += value.f1.value; key = value.f0; } - out.collect(new Tuple4<>(key, window.getStart(), window.getEnd(), new IntType(sum))); + final Tuple4 output = + new Tuple4<>(key, window.getStart(), window.getEnd(), new IntType(sum)); + out.collect(output); } }) - .addSink(new ValidatingSink(numKeys, numElementsPerKey / windowSlide)).setParallelism(1); + .addSink(new ValidatingSink<>( + new SinkValidatorUpdateFun(numElementsPerKey), + new SinkValidatorCheckFun(numKeys, numElementsPerKey, windowSlide))).setParallelism(1); - tryExecute(env, "Tumbling Window Test"); + env.execute("Tumbling Window Test"); } catch (Exception e) { e.printStackTrace(); @@ -457,20 +455,19 @@ public void testPreAggregatedTumblingTimeWindow() { final int numElementsPerKey = numElementsPerKey(); final int windowSize = windowSize(); final int numKeys = numKeys(); - FailingSource.reset(); try { StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); env.setParallelism(PARALLELISM); env.setStreamTimeCharacteristic(TimeCharacteristic.EventTime); env.enableCheckpointing(100); - env.setRestartStrategy(RestartStrategies.fixedDelayRestart(3, 0)); + env.setRestartStrategy(RestartStrategies.fixedDelayRestart(1, 0)); env.getConfig().disableSysoutLogging(); env.setStateBackend(this.stateBackend); env.getConfig().setUseSnapshotCompression(true); env - .addSource(new FailingSource(numKeys, numElementsPerKey, numElementsPerKey / 3)) + .addSource(new FailingSource(new KeyedEventTimeGenerator(numKeys, windowSize), numElementsPerKey)) .rebalance() .keyBy(0) .timeWindow(Time.of(windowSize, MILLISECONDS)) @@ -505,16 +502,19 @@ public void apply( assertTrue(open); for (Tuple2 in: input) { - out.collect(new Tuple4<>(in.f0, - window.getStart(), - window.getEnd(), - in.f1)); + final Tuple4 output = new Tuple4<>(in.f0, + window.getStart(), + window.getEnd(), + in.f1); + out.collect(output); } } }) - .addSink(new ValidatingSink(numKeys, numElementsPerKey / windowSize)).setParallelism(1); + .addSink(new ValidatingSink<>( + new SinkValidatorUpdateFun(numElementsPerKey), + new SinkValidatorCheckFun(numKeys, numElementsPerKey, windowSize))).setParallelism(1); - tryExecute(env, "Tumbling Window Test"); + env.execute("Tumbling Window Test"); } catch (Exception e) { e.printStackTrace(); @@ -528,20 +528,19 @@ public void testPreAggregatedSlidingTimeWindow() { final int windowSize = windowSize(); final int windowSlide = windowSlide(); final int numKeys = numKeys(); - FailingSource.reset(); try { StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); env.setParallelism(PARALLELISM); env.setStreamTimeCharacteristic(TimeCharacteristic.EventTime); env.enableCheckpointing(100); - env.setRestartStrategy(RestartStrategies.fixedDelayRestart(3, 0)); + env.setRestartStrategy(RestartStrategies.fixedDelayRestart(1, 0)); env.getConfig().disableSysoutLogging(); env.setStateBackend(this.stateBackend); env.getConfig().setUseSnapshotCompression(true); env - .addSource(new FailingSource(numKeys, numElementsPerKey, numElementsPerKey / 3)) + .addSource(new FailingSource(new KeyedEventTimeGenerator(numKeys, windowSlide), numElementsPerKey)) .rebalance() .keyBy(0) .timeWindow(Time.of(windowSize, MILLISECONDS), Time.of(windowSlide, MILLISECONDS)) @@ -585,9 +584,11 @@ public void apply( } } }) - .addSink(new ValidatingSink(numKeys, numElementsPerKey / windowSlide)).setParallelism(1); + .addSink(new ValidatingSink<>( + new SinkValidatorUpdateFun(numElementsPerKey), + new SinkValidatorCheckFun(numKeys, numElementsPerKey, windowSlide))).setParallelism(1); - tryExecute(env, "Tumbling Window Test"); + env.execute("Tumbling Window Test"); } catch (Exception e) { e.printStackTrace(); @@ -599,151 +600,42 @@ public void apply( // Utilities // ------------------------------------------------------------------------ - private static class FailingSource extends RichSourceFunction> - implements ListCheckpointed, CheckpointListener { - private static volatile boolean failedBefore = false; - - private final int numKeys; - private final int numElementsToEmit; - private final int failureAfterNumElements; - - private volatile int numElementsEmitted; - private volatile int numSuccessfulCheckpoints; - private volatile boolean running = true; - - private FailingSource(int numKeys, int numElementsToEmitPerKey, int failureAfterNumElements) { - this.numKeys = numKeys; - this.numElementsToEmit = numElementsToEmitPerKey; - this.failureAfterNumElements = failureAfterNumElements; - } - - @Override - public void open(Configuration parameters) { - // non-parallel source - assertEquals(1, getRuntimeContext().getNumberOfParallelSubtasks()); - } - - @Override - public void run(SourceContext> ctx) throws Exception { - // we loop longer than we have elements, to permit delayed checkpoints - // to still cause a failure - while (running) { - - if (!failedBefore) { - // delay a bit, if we have not failed before - Thread.sleep(1); - if (numSuccessfulCheckpoints >= 2 && numElementsEmitted >= failureAfterNumElements) { - // cause a failure if we have not failed before and have reached - // enough completed checkpoints and elements - failedBefore = true; - throw new Exception("Artificial Failure"); - } - } - - if (numElementsEmitted < numElementsToEmit && - (failedBefore || numElementsEmitted <= failureAfterNumElements)) { - // the function failed before, or we are in the elements before the failure - synchronized (ctx.getCheckpointLock()) { - int next = numElementsEmitted++; - for (long i = 0; i < numKeys; i++) { - ctx.collectWithTimestamp(new Tuple2(i, new IntType(next)), next); - } - ctx.emitWatermark(new Watermark(next)); - } - } - else { - - // if our work is done, delay a bit to prevent busy waiting - Thread.sleep(1); - } - } - } - - @Override - public void cancel() { - running = false; - } - - @Override - public void notifyCheckpointComplete(long checkpointId) { - numSuccessfulCheckpoints++; - } + /** + * For validating the stateful window counts. + */ + static class CountingSinkValidatorUpdateFun + implements ValidatingSink.CountUpdater> { @Override - public List snapshotState(long checkpointId, long timestamp) throws Exception { - return Collections.singletonList(this.numElementsEmitted); - } + public void updateCount(Tuple4 value, Map windowCounts) { - @Override - public void restoreState(List state) throws Exception { - if (state.isEmpty() || state.size() > 1) { - throw new RuntimeException("Test failed due to unexpected recovered state size " + state.size()); - } - this.numElementsEmitted = state.get(0); - } + windowCounts.merge(value.f0, 1, (a, b) -> a + b); - public static void reset() { - failedBefore = false; + // verify the contents of that window, the contents should be: + // (key + num windows so far) + assertEquals("Window counts don't match for key " + value.f0 + ".", value.f0.intValue() + windowCounts.get(value.f0), value.f3.value); } } - private static class ValidatingSink extends RichSinkFunction> - implements ListCheckpointed> { - - private final HashMap windowCounts = new HashMap<>(); - - private final int numKeys; - private final int numWindowsExpected; + //------------------------------------ - private ValidatingSink(int numKeys, int numWindowsExpected) { - this.numKeys = numKeys; - this.numWindowsExpected = numWindowsExpected; - } + static class SinkValidatorUpdateFun implements ValidatingSink.CountUpdater> { - @Override - public void open(Configuration parameters) throws Exception { - // this sink can only work with DOP 1 - assertEquals(1, getRuntimeContext().getNumberOfParallelSubtasks()); + private final int elementsPerKey; - // it can happen that a checkpoint happens when the complete success state is - // already set. In that case we restart with the final state and would never - // finish because no more elements arrive. - if (windowCounts.size() == numKeys) { - boolean seenAll = true; - for (Integer windowCount: windowCounts.values()) { - if (windowCount != numWindowsExpected) { - seenAll = false; - break; - } - } - if (seenAll) { - throw new SuccessException(); - } - } + SinkValidatorUpdateFun(int elementsPerKey) { + this.elementsPerKey = elementsPerKey; } @Override - public void close() throws Exception { - boolean seenAll = true; - if (windowCounts.size() == numKeys) { - for (Integer windowCount: windowCounts.values()) { - if (windowCount < numWindowsExpected) { - seenAll = false; - break; - } - } - } - assertTrue("The sink must see all expected windows.", seenAll); - } - - @Override - public void invoke(Tuple4 value) throws Exception { - + public void updateCount(Tuple4 value, Map windowCounts) { // verify the contents of that window, Tuple4.f1 and .f2 are the window start/end // the sum should be "sum (start .. end-1)" int expectedSum = 0; - for (long i = value.f1; i < value.f2; i++) { + // we shorten the range if it goes beyond elementsPerKey, because those are "incomplete" sliding windows + long countUntil = Math.min(value.f2, elementsPerKey); + for (long i = value.f1; i < countUntil; i++) { // only sum up positive vals, to filter out the negative start of the // first sliding windows if (i > 0) { @@ -753,142 +645,55 @@ public void invoke(Tuple4 value) throws Exception { assertEquals("Window start: " + value.f1 + " end: " + value.f2, expectedSum, value.f3.value); - Integer curr = windowCounts.get(value.f0); - if (curr != null) { - windowCounts.put(value.f0, curr + 1); - } - else { - windowCounts.put(value.f0, 1); - } - - if (windowCounts.size() == numKeys) { - boolean seenAll = true; - for (Integer windowCount: windowCounts.values()) { - if (windowCount < numWindowsExpected) { - seenAll = false; - break; - } else if (windowCount > numWindowsExpected) { - fail("Window count to high: " + windowCount); - } - } - - if (seenAll) { - // exit - throw new SuccessException(); - } - - } - } - - @Override - public List> snapshotState(long checkpointId, long timestamp) throws Exception { - return Collections.singletonList(this.windowCounts); - } - - @Override - public void restoreState(List> state) throws Exception { - if (state.isEmpty() || state.size() > 1) { - throw new RuntimeException("Test failed due to unexpected recovered state size " + state.size()); - } - windowCounts.putAll(state.get(0)); + windowCounts.merge(value.f0, 1, (val, increment) -> val + increment); } } - // Sink for validating the stateful window counts - private static class CountValidatingSink extends RichSinkFunction> - implements ListCheckpointed> { - - private final HashMap windowCounts = new HashMap<>(); + static class SinkValidatorCheckFun implements ValidatingSink.ResultChecker { private final int numKeys; private final int numWindowsExpected; - private CountValidatingSink(int numKeys, int numWindowsExpected) { + SinkValidatorCheckFun(int numKeys, int elementsPerKey, int elementsPerWindow) { this.numKeys = numKeys; - this.numWindowsExpected = numWindowsExpected; + this.numWindowsExpected = elementsPerKey / elementsPerWindow; } @Override - public void open(Configuration parameters) throws Exception { - // this sink can only work with DOP 1 - assertEquals(1, getRuntimeContext().getNumberOfParallelSubtasks()); - } - - @Override - public void close() throws Exception { - boolean seenAll = true; + public boolean checkResult(Map windowCounts) { if (windowCounts.size() == numKeys) { - for (Integer windowCount: windowCounts.values()) { + for (Integer windowCount : windowCounts.values()) { if (windowCount < numWindowsExpected) { - seenAll = false; - break; + return false; } } + return true; } - assertTrue("The source must see all expected windows.", seenAll); + return false; } + } - @Override - public void invoke(Tuple4 value) throws Exception { - - Integer curr = windowCounts.get(value.f0); - if (curr != null) { - windowCounts.put(value.f0, curr + 1); - } - else { - windowCounts.put(value.f0, 1); - } - - // verify the contents of that window, the contents should be: - // (key + num windows so far) - - assertEquals("Window counts don't match for key " + value.f0 + ".", value.f0.intValue() + windowCounts.get(value.f0), value.f3.value); - - boolean seenAll = true; - if (windowCounts.size() == numKeys) { - for (Integer windowCount: windowCounts.values()) { - if (windowCount < numWindowsExpected) { - seenAll = false; - break; - } else if (windowCount > numWindowsExpected) { - fail("Window count to high: " + windowCount); - } - } + static class KeyedEventTimeGenerator implements FailingSource.EventEmittingGenerator { - if (seenAll) { - // exit - throw new SuccessException(); - } + private final int keyUniverseSize; + private final int watermarkTrailing; - } - } - - @Override - public List> snapshotState(long checkpointId, long timestamp) throws Exception { - return Collections.singletonList(this.windowCounts); + public KeyedEventTimeGenerator(int keyUniverseSize, int numElementsPerWindow) { + this.keyUniverseSize = keyUniverseSize; + // we let the watermark a bit behind, so that there can be in-flight timers that required checkpointing + // to include correct timer snapshots in our testing. + this.watermarkTrailing = 4 * numElementsPerWindow / 3; } @Override - public void restoreState(List> state) throws Exception { - if (state.isEmpty() || state.size() > 1) { - throw new RuntimeException("Test failed due to unexpected recovered state size " + state.size()); + public void emitEvent(SourceFunction.SourceContext> ctx, int eventSequenceNo) { + final IntType intTypeNext = new IntType(eventSequenceNo); + for (long i = 0; i < keyUniverseSize; i++) { + final Tuple2 generatedEvent = new Tuple2<>(i, intTypeNext); + ctx.collectWithTimestamp(generatedEvent, eventSequenceNo); } - this.windowCounts.putAll(state.get(0)); - } - } - - // ------------------------------------------------------------------------ - // Utilities - // ------------------------------------------------------------------------ - - private static class IntType { - - public int value; - - public IntType() {} - public IntType(int value) { - this.value = value; + ctx.emitWatermark(new Watermark(eventSequenceNo - watermarkTrailing)); } } diff --git a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/WindowCheckpointingITCase.java b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/WindowCheckpointingITCase.java index b6163e8dc2571..b0e2967d4b19a 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/WindowCheckpointingITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/WindowCheckpointingITCase.java @@ -25,18 +25,17 @@ import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.configuration.Configuration; import org.apache.flink.configuration.TaskManagerOptions; -import org.apache.flink.runtime.state.CheckpointListener; import org.apache.flink.streaming.api.TimeCharacteristic; -import org.apache.flink.streaming.api.checkpoint.ListCheckpointed; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; -import org.apache.flink.streaming.api.functions.sink.RichSinkFunction; -import org.apache.flink.streaming.api.functions.source.RichSourceFunction; +import org.apache.flink.streaming.api.functions.source.SourceFunction; import org.apache.flink.streaming.api.functions.windowing.RichWindowFunction; import org.apache.flink.streaming.api.windowing.time.Time; import org.apache.flink.streaming.api.windowing.windows.TimeWindow; +import org.apache.flink.test.checkpointing.utils.FailingSource; +import org.apache.flink.test.checkpointing.utils.IntType; +import org.apache.flink.test.checkpointing.utils.ValidatingSink; import org.apache.flink.test.util.MiniClusterResource; import org.apache.flink.test.util.MiniClusterResourceConfiguration; -import org.apache.flink.test.util.SuccessException; import org.apache.flink.util.Collector; import org.apache.flink.util.TestLogger; @@ -47,9 +46,7 @@ import java.util.Arrays; import java.util.Collection; -import java.util.Collections; -import java.util.HashMap; -import java.util.List; +import java.util.Map; import static java.util.concurrent.TimeUnit.MILLISECONDS; import static org.apache.flink.test.util.TestUtils.tryExecute; @@ -92,7 +89,6 @@ private static Configuration getConfiguration() { @Test public void testTumblingProcessingTimeWindow() { final int numElements = 3000; - FailingSource.reset(); try { StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); @@ -100,11 +96,14 @@ public void testTumblingProcessingTimeWindow() { env.setStreamTimeCharacteristic(timeCharacteristic); env.getConfig().setAutoWatermarkInterval(10); env.enableCheckpointing(100); - env.setRestartStrategy(RestartStrategies.fixedDelayRestart(3, 0)); + env.setRestartStrategy(RestartStrategies.fixedDelayRestart(1, 0)); env.getConfig().disableSysoutLogging(); + SinkValidatorUpdaterAndChecker updaterAndChecker = + new SinkValidatorUpdaterAndChecker(numElements, 1); + env - .addSource(new FailingSource(numElements, numElements / 3)) + .addSource(new FailingSource(new Generator(), numElements, timeCharacteristic)) .rebalance() .keyBy(0) .timeWindow(Time.of(100, MILLISECONDS)) @@ -130,11 +129,12 @@ public void apply( for (Tuple2 value : values) { assertEquals(value.f0.intValue(), value.f1.value); - out.collect(new Tuple2(value.f0, new IntType(1))); + out.collect(new Tuple2<>(value.f0, new IntType(1))); } } }) - .addSink(new ValidatingSink(numElements, 1)).setParallelism(1); + .addSink(new ValidatingSink<>(updaterAndChecker, updaterAndChecker, timeCharacteristic)) + .setParallelism(1); tryExecute(env, "Tumbling Window Test"); } @@ -147,7 +147,6 @@ public void apply( @Test public void testSlidingProcessingTimeWindow() { final int numElements = 3000; - FailingSource.reset(); try { StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); @@ -155,11 +154,12 @@ public void testSlidingProcessingTimeWindow() { env.setStreamTimeCharacteristic(timeCharacteristic); env.getConfig().setAutoWatermarkInterval(10); env.enableCheckpointing(100); - env.setRestartStrategy(RestartStrategies.fixedDelayRestart(3, 0)); + env.setRestartStrategy(RestartStrategies.fixedDelayRestart(1, 0)); env.getConfig().disableSysoutLogging(); - + SinkValidatorUpdaterAndChecker updaterAndChecker = + new SinkValidatorUpdaterAndChecker(numElements, 3); env - .addSource(new FailingSource(numElements, numElements / 3)) + .addSource(new FailingSource(new Generator(), numElements, timeCharacteristic)) .rebalance() .keyBy(0) .timeWindow(Time.of(150, MILLISECONDS), Time.of(50, MILLISECONDS)) @@ -185,13 +185,14 @@ public void apply( for (Tuple2 value : values) { assertEquals(value.f0.intValue(), value.f1.value); - out.collect(new Tuple2(value.f0, new IntType(1))); + out.collect(new Tuple2<>(value.f0, new IntType(1))); } } }) - .addSink(new ValidatingSink(numElements, 3)).setParallelism(1); + .addSink(new ValidatingSink<>(updaterAndChecker, updaterAndChecker, timeCharacteristic)) + .setParallelism(1); - tryExecute(env, "Tumbling Window Test"); + tryExecute(env, "Sliding Window Test"); } catch (Exception e) { e.printStackTrace(); @@ -202,7 +203,6 @@ public void apply( @Test public void testAggregatingTumblingProcessingTimeWindow() { final int numElements = 3000; - FailingSource.reset(); try { StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); @@ -210,11 +210,12 @@ public void testAggregatingTumblingProcessingTimeWindow() { env.setStreamTimeCharacteristic(timeCharacteristic); env.getConfig().setAutoWatermarkInterval(10); env.enableCheckpointing(100); - env.setRestartStrategy(RestartStrategies.fixedDelayRestart(3, 0)); + env.setRestartStrategy(RestartStrategies.fixedDelayRestart(1, 0)); env.getConfig().disableSysoutLogging(); - + SinkValidatorUpdaterAndChecker updaterAndChecker = + new SinkValidatorUpdaterAndChecker(numElements, 1); env - .addSource(new FailingSource(numElements, numElements / 3)) + .addSource(new FailingSource(new Generator(), numElements, timeCharacteristic)) .map(new MapFunction, Tuple2>() { @Override public Tuple2 map(Tuple2 value) { @@ -234,9 +235,10 @@ public Tuple2 reduce( return new Tuple2<>(a.f0, new IntType(1)); } }) - .addSink(new ValidatingSink(numElements, 1)).setParallelism(1); + .addSink(new ValidatingSink<>(updaterAndChecker, updaterAndChecker, timeCharacteristic)) + .setParallelism(1); - tryExecute(env, "Tumbling Window Test"); + tryExecute(env, "Aggregating Tumbling Window Test"); } catch (Exception e) { e.printStackTrace(); @@ -247,7 +249,6 @@ public Tuple2 reduce( @Test public void testAggregatingSlidingProcessingTimeWindow() { final int numElements = 3000; - FailingSource.reset(); try { StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); @@ -255,11 +256,12 @@ public void testAggregatingSlidingProcessingTimeWindow() { env.setStreamTimeCharacteristic(timeCharacteristic); env.getConfig().setAutoWatermarkInterval(10); env.enableCheckpointing(100); - env.setRestartStrategy(RestartStrategies.fixedDelayRestart(3, 0)); + env.setRestartStrategy(RestartStrategies.fixedDelayRestart(1, 0)); env.getConfig().disableSysoutLogging(); - + SinkValidatorUpdaterAndChecker updaterAndChecker = + new SinkValidatorUpdaterAndChecker(numElements, 3); env - .addSource(new FailingSource(numElements, numElements / 3)) + .addSource(new FailingSource(new Generator(), numElements, timeCharacteristic)) .map(new MapFunction, Tuple2>() { @Override public Tuple2 map(Tuple2 value) { @@ -278,9 +280,10 @@ public Tuple2 reduce( return new Tuple2<>(a.f0, new IntType(1)); } }) - .addSink(new ValidatingSink(numElements, 3)).setParallelism(1); + .addSink(new ValidatingSink<>(updaterAndChecker, updaterAndChecker, timeCharacteristic)) + .setParallelism(1); - tryExecute(env, "Tumbling Window Test"); + tryExecute(env, "Aggregating Sliding Window Test"); } catch (Exception e) { e.printStackTrace(); @@ -292,152 +295,50 @@ public Tuple2 reduce( // Utilities // ------------------------------------------------------------------------ - private static class FailingSource extends RichSourceFunction> - implements ListCheckpointed, CheckpointListener { - private static volatile boolean failedBefore = false; - - private final int numElementsToEmit; - private final int failureAfterNumElements; - - private volatile int numElementsEmitted; - private volatile int numSuccessfulCheckpoints; - private volatile boolean running = true; - - private FailingSource(int numElementsToEmit, int failureAfterNumElements) { - this.numElementsToEmit = numElementsToEmit; - this.failureAfterNumElements = failureAfterNumElements; - } - - @Override - public void open(Configuration parameters) { - // non-parallel source - assertEquals(1, getRuntimeContext().getNumberOfParallelSubtasks()); - } - - @Override - public void run(SourceContext> ctx) throws Exception { - // we loop longer than we have elements, to permit delayed checkpoints - // to still cause a failure - while (running) { - if (!failedBefore) { - // delay a bit, if we have not failed before - Thread.sleep(1); - if (numSuccessfulCheckpoints >= 2 && numElementsEmitted >= failureAfterNumElements) { - // cause a failure if we have not failed before and have reached - // enough completed checkpoints and elements - failedBefore = true; - throw new Exception("Artificial Failure"); - } - } - - if (numElementsEmitted < numElementsToEmit && - (failedBefore || numElementsEmitted <= failureAfterNumElements)) { - // the function failed before, or we are in the elements before the failure - synchronized (ctx.getCheckpointLock()) { - int next = numElementsEmitted++; - ctx.collect(new Tuple2((long) next, new IntType(next))); - } - } else { - // if our work is done, delay a bit to prevent busy waiting - Thread.sleep(10); - } - } - } + static class Generator implements FailingSource.EventEmittingGenerator { @Override - public void cancel() { - running = false; - } - - @Override - public void notifyCheckpointComplete(long checkpointId) { - numSuccessfulCheckpoints++; - } - - @Override - public List snapshotState(long checkpointId, long timestamp) throws Exception { - return Collections.singletonList(this.numElementsEmitted); - } - - @Override - public void restoreState(List state) throws Exception { - if (state.isEmpty() || state.size() > 1) { - throw new RuntimeException("Test failed due to unexpected recovered state size " + state.size()); - } - this.numElementsEmitted = state.get(0); - } - - public static void reset() { - failedBefore = false; + public void emitEvent(SourceFunction.SourceContext> ctx, int eventSequenceNo) { + ctx.collect(new Tuple2<>((long) eventSequenceNo, new IntType(eventSequenceNo))); } } - private static class ValidatingSink extends RichSinkFunction> - implements ListCheckpointed> { - - private final HashMap counts = new HashMap<>(); + static class SinkValidatorUpdaterAndChecker + implements ValidatingSink.CountUpdater>, ValidatingSink.ResultChecker { private final int elementCountExpected; private final int countPerElementExpected; - private int aggCount; - - private ValidatingSink(int elementCountExpected, int countPerElementExpected) { + SinkValidatorUpdaterAndChecker(int elementCountExpected, int countPerElementExpected) { this.elementCountExpected = elementCountExpected; this.countPerElementExpected = countPerElementExpected; } @Override - public void open(Configuration parameters) throws Exception { - // this sink can only work with DOP 1 - assertEquals(1, getRuntimeContext().getNumberOfParallelSubtasks()); - checkSuccess(); + public void updateCount(Tuple2 value, Map windowCounts) { + windowCounts.merge(value.f0, value.f1.value, (a, b) -> a + b); } @Override - public void invoke(Tuple2 value) throws Exception { - Integer curr = counts.get(value.f0); - if (curr != null) { - counts.put(value.f0, curr + value.f1.value); - } - else { - counts.put(value.f0, value.f1.value); - } + public boolean checkResult(Map windowCounts) { + int aggCount = 0; - // check if we have seen all we expect - aggCount += value.f1.value; - checkSuccess(); - } - - private void checkSuccess() throws SuccessException { - if (aggCount >= elementCountExpected * countPerElementExpected) { - // we are done. validate - assertEquals(elementCountExpected, counts.size()); - - for (Integer i : counts.values()) { - assertEquals(countPerElementExpected, i.intValue()); - } - - // exit - throw new SuccessException(); + for (Integer i : windowCounts.values()) { + aggCount += i; } - } - @Override - public List> snapshotState(long checkpointId, long timestamp) throws Exception { - return Collections.singletonList(this.counts); - } - - @Override - public void restoreState(List> state) throws Exception { - if (state.isEmpty() || state.size() > 1) { - throw new RuntimeException("Test failed due to unexpected recovered state size " + state.size()); + if (aggCount < elementCountExpected * countPerElementExpected + || elementCountExpected != windowCounts.size()) { + return false; } - this.counts.putAll(state.get(0)); - for (Integer i : state.get(0).values()) { - this.aggCount += i; + for (int i : windowCounts.values()) { + if (countPerElementExpected != i) { + return false; + } } + + return true; } } @@ -452,22 +353,4 @@ public static Collection timeCharacteristic(){ new TimeCharacteristic[]{TimeCharacteristic.IngestionTime} ); } - - // ------------------------------------------------------------------------ - // Utilities - // ------------------------------------------------------------------------ - - /** - * POJO with int value. - */ - public static class IntType { - - public int value; - - public IntType() {} - - public IntType(int value) { - this.value = value; - } - } } diff --git a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/utils/FailingSource.java b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/utils/FailingSource.java new file mode 100644 index 0000000000000..822d73b12c8bf --- /dev/null +++ b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/utils/FailingSource.java @@ -0,0 +1,155 @@ +/* + * 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.utils; + +import org.apache.flink.api.common.functions.RuntimeContext; +import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.runtime.state.CheckpointListener; +import org.apache.flink.streaming.api.TimeCharacteristic; +import org.apache.flink.streaming.api.checkpoint.ListCheckpointed; +import org.apache.flink.streaming.api.functions.source.RichSourceFunction; + +import javax.annotation.Nonnegative; +import javax.annotation.Nonnull; + +import java.io.Serializable; +import java.util.Collections; +import java.util.List; +import java.util.concurrent.atomic.AtomicLong; + +import static org.junit.Assert.assertEquals; + +/** + * Source for window checkpointing IT cases that can introduce artificial failures. + */ +public class FailingSource extends RichSourceFunction> + implements ListCheckpointed, CheckpointListener { + + /** + * Function to generate and emit the test events (and watermarks if required). + */ + @FunctionalInterface + public interface EventEmittingGenerator extends Serializable { + void emitEvent(SourceContext> ctx, int eventSequenceNo); + } + + private static final long INITIAL = Long.MIN_VALUE; + private static final long STATEFUL_CHECKPOINT_COMPLETED = Long.MIN_VALUE; + + @Nonnull + private final EventEmittingGenerator eventEmittingGenerator; + private final int expectedEmitCalls; + private final int failureAfterNumElements; + private final boolean usingProcessingTime; + private final AtomicLong checkpointStatus; + + private int emitCallCount; + private volatile boolean running; + + public FailingSource( + @Nonnull EventEmittingGenerator eventEmittingGenerator, + @Nonnegative int numberOfGeneratorInvocations) { + this(eventEmittingGenerator, numberOfGeneratorInvocations, TimeCharacteristic.EventTime); + } + + public FailingSource( + @Nonnull EventEmittingGenerator eventEmittingGenerator, + @Nonnegative int numberOfGeneratorInvocations, + @Nonnull TimeCharacteristic timeCharacteristic) { + this.eventEmittingGenerator = eventEmittingGenerator; + this.running = true; + this.emitCallCount = 0; + this.expectedEmitCalls = numberOfGeneratorInvocations; + this.failureAfterNumElements = numberOfGeneratorInvocations / 2; + this.checkpointStatus = new AtomicLong(INITIAL); + this.usingProcessingTime = timeCharacteristic == TimeCharacteristic.ProcessingTime; + } + + @Override + public void open(Configuration parameters) { + // non-parallel source + assertEquals(1, getRuntimeContext().getNumberOfParallelSubtasks()); + } + + @Override + public void run(SourceContext> ctx) throws Exception { + + final RuntimeContext runtimeContext = getRuntimeContext(); + // detect if this task is "the chosen one" and should fail (via subtaskidx), if it did not fail before (via attempt) + final boolean failThisTask = + runtimeContext.getAttemptNumber() == 0 && runtimeContext.getIndexOfThisSubtask() == 0; + + // we loop longer than we have elements, to permit delayed checkpoints + // to still cause a failure + while (running) { + + // the function failed before, or we are in the elements before the failure + synchronized (ctx.getCheckpointLock()) { + eventEmittingGenerator.emitEvent(ctx, emitCallCount++); + running &= (emitCallCount < expectedEmitCalls); + } + + if (emitCallCount < failureAfterNumElements) { + Thread.sleep(1); + } else if (failThisTask && emitCallCount == failureAfterNumElements) { + // wait for a pending checkpoint that fulfills our requirements if needed + while (checkpointStatus.get() != STATEFUL_CHECKPOINT_COMPLETED) { + Thread.sleep(1); + } + throw new Exception("Artificial Failure"); + } + } + + if (usingProcessingTime) { + while (running) { + Thread.sleep(10); + } + } + } + + @Override + public void cancel() { + running = false; + } + + @Override + public void notifyCheckpointComplete(long checkpointId) { + // This will unblock the task for failing, if this is the checkpoint we are waiting for + checkpointStatus.compareAndSet(checkpointId, STATEFUL_CHECKPOINT_COMPLETED); + } + + @Override + public List snapshotState(long checkpointId, long timestamp) throws Exception { + // We accept a checkpoint as basis if it should have a "decent amount" of state + if (emitCallCount > failureAfterNumElements / 2) { + // This means we are waiting for notification of this checkpoint to completed now. + checkpointStatus.compareAndSet(INITIAL, checkpointId); + } + return Collections.singletonList(this.emitCallCount); + } + + @Override + public void restoreState(List state) throws Exception { + if (state.isEmpty() || state.size() > 1) { + throw new RuntimeException("Test failed due to unexpected recovered state size " + state.size()); + } + this.emitCallCount = state.get(0); + } +} diff --git a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/utils/IntType.java b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/utils/IntType.java new file mode 100644 index 0000000000000..3bc4ea0e05da4 --- /dev/null +++ b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/utils/IntType.java @@ -0,0 +1,38 @@ +/* + * 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.utils; + +/** + * Test type that wraps an int. + */ +public class IntType { + + public int value; + + public IntType(int value) { + this.value = value; + } + + @Override + public String toString() { + return "IntType{" + + "value=" + value + + '}'; + } +} diff --git a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/utils/ValidatingSink.java b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/utils/ValidatingSink.java new file mode 100644 index 0000000000000..b35273894705e --- /dev/null +++ b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/utils/ValidatingSink.java @@ -0,0 +1,128 @@ +/* + * 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.utils; + +import org.apache.flink.configuration.Configuration; +import org.apache.flink.streaming.api.TimeCharacteristic; +import org.apache.flink.streaming.api.checkpoint.ListCheckpointed; +import org.apache.flink.streaming.api.functions.sink.RichSinkFunction; +import org.apache.flink.test.util.SuccessException; + +import javax.annotation.Nonnull; + +import java.io.Serializable; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +import static org.junit.Assert.assertEquals; + +/** + * Generalized sink for validation of window checkpointing IT cases. + */ +public class ValidatingSink extends RichSinkFunction + implements ListCheckpointed> { + + /** + * Function to check if the window counts are as expected. + */ + @FunctionalInterface + public interface ResultChecker extends Serializable { + boolean checkResult(Map windowCounts); + } + + /** + * Function that updates the window counts from an update event. + * + * @param type of the update event. + */ + public interface CountUpdater extends Serializable { + void updateCount(T update, Map windowCounts); + } + + @Nonnull + private final ResultChecker resultChecker; + + @Nonnull + private final CountUpdater countUpdater; + + @Nonnull + private final HashMap windowCounts; + + private final boolean usingProcessingTime; + + public ValidatingSink( + @Nonnull CountUpdater countUpdater, + @Nonnull ResultChecker resultChecker) { + this(countUpdater, resultChecker, TimeCharacteristic.EventTime); + } + + public ValidatingSink( + @Nonnull CountUpdater countUpdater, + @Nonnull ResultChecker resultChecker, + @Nonnull TimeCharacteristic timeCharacteristic) { + + this.resultChecker = resultChecker; + this.countUpdater = countUpdater; + this.usingProcessingTime = TimeCharacteristic.ProcessingTime == timeCharacteristic; + this.windowCounts = new HashMap<>(); + } + + @Override + public void open(Configuration parameters) throws Exception { + // this sink can only work with DOP 1 + assertEquals(1, getRuntimeContext().getNumberOfParallelSubtasks()); + if (usingProcessingTime && resultChecker.checkResult(windowCounts)) { + throw new SuccessException(); + } + } + + @Override + public void close() { + if (resultChecker.checkResult(windowCounts)) { + if (usingProcessingTime) { + throw new SuccessException(); + } + } else { + throw new AssertionError("Test failed check."); + } + } + + @Override + public void invoke(T value, Context context) throws Exception { + countUpdater.updateCount(value, windowCounts); + if (usingProcessingTime && resultChecker.checkResult(windowCounts)) { + throw new SuccessException(); + } + } + + @Override + public List> snapshotState(long checkpointId, long timestamp) throws Exception { + return Collections.singletonList(this.windowCounts); + } + + @Override + public void restoreState(List> state) throws Exception { + if (state.isEmpty() || state.size() > 1) { + throw new RuntimeException("Test failed due to unexpected recovered state size " + state.size()); + } + windowCounts.putAll(state.get(0)); + } +} diff --git a/flink-tests/src/test/java/org/apache/flink/test/util/SuccessException.java b/flink-tests/src/test/java/org/apache/flink/test/util/SuccessException.java index 22ac02b76535e..d8e2a8fb64fc3 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/util/SuccessException.java +++ b/flink-tests/src/test/java/org/apache/flink/test/util/SuccessException.java @@ -21,6 +21,6 @@ /** * Exception that is thrown to terminate a program and indicate success. */ -public class SuccessException extends Exception { +public class SuccessException extends RuntimeException { private static final long serialVersionUID = -7011865671593955887L; } From 1ee705afa7122e1b92fd3f1b12fe4c97c66ffd5b Mon Sep 17 00:00:00 2001 From: Stephan Ewen Date: Fri, 20 Jul 2018 17:24:52 +0200 Subject: [PATCH 35/65] [hotfix] [core] Align serialization methods in SimpleVersionedSerialization --- .../core/io/SimpleVersionedSerialization.java | 33 ++++++-- .../io/SimpleVersionedSerializationTest.java | 81 ++++--------------- 2 files changed, 42 insertions(+), 72 deletions(-) diff --git a/flink-core/src/main/java/org/apache/flink/core/io/SimpleVersionedSerialization.java b/flink-core/src/main/java/org/apache/flink/core/io/SimpleVersionedSerialization.java index 8bead11caeb30..2c5b68c592c0a 100644 --- a/flink-core/src/main/java/org/apache/flink/core/io/SimpleVersionedSerialization.java +++ b/flink-core/src/main/java/org/apache/flink/core/io/SimpleVersionedSerialization.java @@ -110,7 +110,7 @@ public static byte[] writeVersionAndSerialize(SimpleVersionedSerializer s checkNotNull(datum, "datum"); final byte[] data = serializer.serialize(datum); - final byte[] versionAndData = new byte[data.length + 4]; + final byte[] versionAndData = new byte[data.length + 8]; final int version = serializer.getVersion(); versionAndData[0] = (byte) (version >> 24); @@ -118,8 +118,14 @@ public static byte[] writeVersionAndSerialize(SimpleVersionedSerializer s versionAndData[2] = (byte) (version >> 8); versionAndData[3] = (byte) version; + final int length = data.length; + versionAndData[4] = (byte) (length >> 24); + versionAndData[5] = (byte) (length >> 16); + versionAndData[6] = (byte) (length >> 8); + versionAndData[7] = (byte) length; + // move the data to the array - System.arraycopy(data, 0, versionAndData, 4, data.length); + System.arraycopy(data, 0, versionAndData, 8, data.length); return versionAndData; } @@ -142,14 +148,25 @@ public static T readVersionAndDeSerialize(SimpleVersionedSerializer seria checkNotNull(bytes, "bytes"); checkArgument(bytes.length >= 4, "byte array below minimum length (4 bytes)"); - final byte[] dataOnly = Arrays.copyOfRange(bytes, 4, bytes.length); + final byte[] dataOnly = Arrays.copyOfRange(bytes, 8, bytes.length); final int version = ((bytes[0] & 0xff) << 24) | - ((bytes[1] & 0xff) << 16) | - ((bytes[2] & 0xff) << 8) | - (bytes[3] & 0xff); - - return serializer.deserialize(version, dataOnly); + ((bytes[1] & 0xff) << 16) | + ((bytes[2] & 0xff) << 8) | + (bytes[3] & 0xff); + + final int length = + ((bytes[4] & 0xff) << 24) | + ((bytes[5] & 0xff) << 16) | + ((bytes[6] & 0xff) << 8) | + (bytes[7] & 0xff); + + if (length == dataOnly.length) { + return serializer.deserialize(version, dataOnly); + } + else { + throw new IOException("Corrupt data, conflicting lengths. Length fields: " + length + ", data: " + dataOnly.length); + } } // ------------------------------------------------------------------------ diff --git a/flink-core/src/test/java/org/apache/flink/core/io/SimpleVersionedSerializationTest.java b/flink-core/src/test/java/org/apache/flink/core/io/SimpleVersionedSerializationTest.java index 89a6b27bb6aab..116d37ca9640b 100644 --- a/flink-core/src/test/java/org/apache/flink/core/io/SimpleVersionedSerializationTest.java +++ b/flink-core/src/test/java/org/apache/flink/core/io/SimpleVersionedSerializationTest.java @@ -26,8 +26,8 @@ import java.io.IOException; import java.nio.charset.StandardCharsets; +import static org.junit.Assert.assertArrayEquals; import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertNotNull; /** * Tests for the {@link SimpleVersionedSerialization} class. @@ -35,7 +35,7 @@ public class SimpleVersionedSerializationTest { @Test - public void testStreamSerializationRoundTrip() throws IOException { + public void testSerializationRoundTrip() throws IOException { final SimpleVersionedSerializer utfEncoder = new SimpleVersionedSerializer() { private static final int VERSION = Integer.MAX_VALUE / 2; // version should occupy many bytes @@ -60,14 +60,20 @@ public String deserialize(int version, byte[] serialized) throws IOException { final String testString = "dugfakgs"; final DataOutputSerializer out = new DataOutputSerializer(32); SimpleVersionedSerialization.writeVersionAndSerialize(utfEncoder, testString, out); + final byte[] outBytes = out.getCopyOfBuffer(); + + final byte[] bytes = SimpleVersionedSerialization.writeVersionAndSerialize(utfEncoder, testString); + assertArrayEquals(bytes, outBytes); - final DataInputDeserializer in = new DataInputDeserializer(out.getCopyOfBuffer()); + final DataInputDeserializer in = new DataInputDeserializer(bytes); final String deserialized = SimpleVersionedSerialization.readVersionAndDeSerialize(utfEncoder, in); + final String deserializedFromBytes = SimpleVersionedSerialization.readVersionAndDeSerialize(utfEncoder, outBytes); assertEquals(testString, deserialized); + assertEquals(testString, deserializedFromBytes); } @Test - public void testStreamSerializeEmpty() throws IOException { + public void testSerializeEmpty() throws IOException { final String testString = "beeeep!"; SimpleVersionedSerializer emptySerializer = new SimpleVersionedSerializer() { @@ -92,68 +98,15 @@ public String deserialize(int version, byte[] serialized) throws IOException { final DataOutputSerializer out = new DataOutputSerializer(32); SimpleVersionedSerialization.writeVersionAndSerialize(emptySerializer, "abc", out); + final byte[] outBytes = out.getCopyOfBuffer(); - final DataInputDeserializer in = new DataInputDeserializer(out.getCopyOfBuffer()); - assertEquals(testString, SimpleVersionedSerialization.readVersionAndDeSerialize(emptySerializer, in)); - } - - @Test - public void testSerializationRoundTrip() throws IOException { - final SimpleVersionedSerializer utfEncoder = new SimpleVersionedSerializer() { - - private static final int VERSION = Integer.MAX_VALUE / 2; // version should occupy many bytes - - @Override - public int getVersion() { - return VERSION; - } - - @Override - public byte[] serialize(String str) throws IOException { - return str.getBytes(StandardCharsets.UTF_8); - } - - @Override - public String deserialize(int version, byte[] serialized) throws IOException { - assertEquals(VERSION, version); - return new String(serialized, StandardCharsets.UTF_8); - } - }; - - final String testString = "dugfakgs"; - byte[] serialized = SimpleVersionedSerialization.writeVersionAndSerialize(utfEncoder, testString); + final byte[] bytes = SimpleVersionedSerialization.writeVersionAndSerialize(emptySerializer, "abc"); + assertArrayEquals(bytes, outBytes); - final String deserialized = SimpleVersionedSerialization.readVersionAndDeSerialize(utfEncoder, serialized); + final DataInputDeserializer in = new DataInputDeserializer(bytes); + final String deserialized = SimpleVersionedSerialization.readVersionAndDeSerialize(emptySerializer, in); + final String deserializedFromBytes = SimpleVersionedSerialization.readVersionAndDeSerialize(emptySerializer, outBytes); assertEquals(testString, deserialized); - } - - @Test - public void testSerializeEmpty() throws IOException { - final String testString = "beeeep!"; - - SimpleVersionedSerializer emptySerializer = new SimpleVersionedSerializer() { - - @Override - public int getVersion() { - return 42; - } - - @Override - public byte[] serialize(String obj) throws IOException { - return new byte[0]; - } - - @Override - public String deserialize(int version, byte[] serialized) throws IOException { - assertEquals(42, version); - assertEquals(0, serialized.length); - return testString; - } - }; - - byte[] serialized = SimpleVersionedSerialization.writeVersionAndSerialize(emptySerializer, "abc"); - assertNotNull(serialized); - - assertEquals(testString, SimpleVersionedSerialization.readVersionAndDeSerialize(emptySerializer, serialized)); + assertEquals(testString, deserializedFromBytes); } } From 8b1cc16745ce2759d1bcf72f9bcb55da6ab2ae1b Mon Sep 17 00:00:00 2001 From: Stefan Richter Date: Fri, 20 Jul 2018 20:28:55 +0200 Subject: [PATCH 36/65] [hotfix] Fix loop in FailingSource --- .../apache/flink/test/checkpointing/utils/FailingSource.java | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/utils/FailingSource.java b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/utils/FailingSource.java index 822d73b12c8bf..ff4995996e6c2 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/utils/FailingSource.java +++ b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/utils/FailingSource.java @@ -98,12 +98,11 @@ public void run(SourceContext> ctx) throws Exception { // we loop longer than we have elements, to permit delayed checkpoints // to still cause a failure - while (running) { + while (running && emitCallCount < expectedEmitCalls) { // the function failed before, or we are in the elements before the failure synchronized (ctx.getCheckpointLock()) { eventEmittingGenerator.emitEvent(ctx, emitCallCount++); - running &= (emitCallCount < expectedEmitCalls); } if (emitCallCount < failureAfterNumElements) { From b6fb4f636716be12445ff9facb76382cac1e1e56 Mon Sep 17 00:00:00 2001 From: Timo Walther Date: Fri, 20 Jul 2018 12:59:41 +0200 Subject: [PATCH 37/65] [hotfix] [table] Deprecate SchemaValidator#deriveTableSinkSchema The method combines two separate concepts of table schema and field mapping. This should be split into two methods once we have support for the corresponding interfaces (see FLINK-9870). --- .../gateway/utils/TestTableSourceFactory.java | 2 +- .../flink/table/descriptors/SchemaValidator.scala | 13 +++++-------- .../flink/table/sinks/CsvTableSinkFactoryBase.scala | 2 +- .../flink/table/utils/InMemoryTableFactory.scala | 2 +- 4 files changed, 8 insertions(+), 11 deletions(-) diff --git a/flink-libraries/flink-sql-client/src/test/java/org/apache/flink/table/client/gateway/utils/TestTableSourceFactory.java b/flink-libraries/flink-sql-client/src/test/java/org/apache/flink/table/client/gateway/utils/TestTableSourceFactory.java index 81f00e5145894..b0b88486b9554 100644 --- a/flink-libraries/flink-sql-client/src/test/java/org/apache/flink/table/client/gateway/utils/TestTableSourceFactory.java +++ b/flink-libraries/flink-sql-client/src/test/java/org/apache/flink/table/client/gateway/utils/TestTableSourceFactory.java @@ -83,7 +83,7 @@ public StreamTableSource createStreamTableSource(Map proper final Optional proctime = SchemaValidator.deriveProctimeAttribute(params); final List rowtime = SchemaValidator.deriveRowtimeAttributes(params); return new TestTableSource( - SchemaValidator.deriveTableSourceSchema(params), + params.getTableSchema(SCHEMA()), properties.get(CONNECTOR_TEST_PROPERTY), proctime.orElse(null), rowtime); diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/descriptors/SchemaValidator.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/descriptors/SchemaValidator.scala index ec83b3c62a1ca..af2baba25eac4 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/descriptors/SchemaValidator.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/descriptors/SchemaValidator.scala @@ -174,18 +174,15 @@ object SchemaValidator { attributes.asJava } - /** - * Derives the table schema for a table source. A table source can directly use "name" and - * "type" and needs no special handling for time attributes or aliasing. - */ - def deriveTableSourceSchema(properties: DescriptorProperties): TableSchema = { - properties.getTableSchema(SCHEMA) - } - /** * Derives the table schema for a table sink. A sink ignores a proctime attribute and * needs to track the origin of a rowtime field. + * + * @deprecated This method combines two separate concepts of table schema and field mapping. + * This should be split into two methods once we have support for + * the corresponding interfaces (see FLINK-9870). */ + @deprecated def deriveTableSinkSchema(properties: DescriptorProperties): TableSchema = { val builder = TableSchema.builder() diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/sinks/CsvTableSinkFactoryBase.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/sinks/CsvTableSinkFactoryBase.scala index 6ceba4c47e1f5..849d16c94746c 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/sinks/CsvTableSinkFactoryBase.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/sinks/CsvTableSinkFactoryBase.scala @@ -77,7 +77,7 @@ abstract class CsvTableSinkFactoryBase extends TableFactory { // build val formatSchema = params.getTableSchema(FORMAT_FIELDS) - val tableSchema = SchemaValidator.deriveTableSinkSchema(params) + val tableSchema = params.getTableSchema(SCHEMA) if (!formatSchema.equals(tableSchema)) { throw new TableException( diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/utils/InMemoryTableFactory.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/utils/InMemoryTableFactory.scala index 21dfb194fff32..6b86c08e69be8 100644 --- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/utils/InMemoryTableFactory.scala +++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/utils/InMemoryTableFactory.scala @@ -76,7 +76,7 @@ class InMemoryTableFactory(terminationCount: Int) supportsSourceTimestamps = true, supportsSourceWatermarks = true).validate(params) - val tableSchema = SchemaValidator.deriveTableSourceSchema(params) + val tableSchema = params.getTableSchema(SCHEMA) // proctime val proctimeAttributeOpt = SchemaValidator.deriveProctimeAttribute(params) From 1dd683544e29004e28b53aa2856d6449ec58142c Mon Sep 17 00:00:00 2001 From: zentol Date: Thu, 19 Jul 2018 13:47:53 +0200 Subject: [PATCH 38/65] [FLINK-9895][tests] Ensure error logging for NettyLeakDetectionResource This closes #6374. --- .../io/network/netty/NettyLeakDetectionResource.java | 8 ++++++++ 1 file changed, 8 insertions(+) diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/NettyLeakDetectionResource.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/NettyLeakDetectionResource.java index 027f454d2fe54..1dbeb25cac266 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/NettyLeakDetectionResource.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/NettyLeakDetectionResource.java @@ -21,7 +21,9 @@ import org.apache.flink.shaded.netty4.io.netty.util.ResourceLeakDetector; import org.apache.flink.shaded.netty4.io.netty.util.ResourceLeakDetectorFactory; +import org.junit.Assert; import org.junit.rules.ExternalResource; +import org.slf4j.LoggerFactory; import javax.annotation.concurrent.GuardedBy; @@ -48,6 +50,12 @@ public class NettyLeakDetectionResource extends ExternalResource { private static final Object refCountLock = new Object(); private static int refCount = 0; + public NettyLeakDetectionResource() { + Assert.assertTrue( + "Error logging must be enabled for the ResourceLeakDetector.", + LoggerFactory.getLogger(ResourceLeakDetector.class).isErrorEnabled()); + } + @Override protected void before() { synchronized (refCountLock) { From 29c76ce21ac56dd657d2ae6ea97ff3675964e6dd Mon Sep 17 00:00:00 2001 From: zentol Date: Wed, 18 Jul 2018 13:42:06 +0200 Subject: [PATCH 39/65] [FLINK-9888][release] Remove unsafe defaults from release scripts This closes #6362. --- tools/releasing/create_binary_release.sh | 6 +++++- tools/releasing/create_release_branch.sh | 12 ++++++++++-- tools/releasing/create_source_release.sh | 6 +++++- tools/releasing/update_branch_version.sh | 12 ++++++++++-- 4 files changed, 30 insertions(+), 6 deletions(-) diff --git a/tools/releasing/create_binary_release.sh b/tools/releasing/create_binary_release.sh index 0bdff60425b36..4dbc6fab8c5ff 100755 --- a/tools/releasing/create_binary_release.sh +++ b/tools/releasing/create_binary_release.sh @@ -20,12 +20,16 @@ ## ## Variables with defaults (if not overwritten by environment) ## -RELEASE_VERSION=${RELEASE_VERSION:-1.3-SNAPSHOT} SCALA_VERSION=none HADOOP_VERSION=none SKIP_GPG=${SKIP_GPG:-false} MVN=${MVN:-mvn} +if [ -z "${RELEASE_VERSION}" ]; then + echo "RELEASE_VERSION was not set." + exit 1 +fi + # fail immediately set -o errexit set -o nounset diff --git a/tools/releasing/create_release_branch.sh b/tools/releasing/create_release_branch.sh index 98532523aff39..7e16483ec6cae 100755 --- a/tools/releasing/create_release_branch.sh +++ b/tools/releasing/create_release_branch.sh @@ -20,11 +20,19 @@ ## ## Variables with defaults (if not overwritten by environment) ## -OLD_VERSION=${OLD_VERSION:-1.2-SNAPSHOT} -NEW_VERSION=${NEW_VERSION:-1.3-SNAPSHOT} RELEASE_CANDIDATE=${RELEASE_CANDIDATE:-none} MVN=${MVN:-mvn} +if [ -z "${OLD_VERSION}" ]; then + echo "OLD_VERSION was not set." + exit 1 +fi + +if [ -z "${NEW_VERSION}" ]; then + echo "NEW_VERSION was not set." + exit 1 +fi + # fail immediately set -o errexit set -o nounset diff --git a/tools/releasing/create_source_release.sh b/tools/releasing/create_source_release.sh index 0cc32feed130b..93298027cbc85 100755 --- a/tools/releasing/create_source_release.sh +++ b/tools/releasing/create_source_release.sh @@ -20,9 +20,13 @@ ## ## Variables with defaults (if not overwritten by environment) ## -RELEASE_VERSION=${RELEASE_VERSION:-1.3-SNAPSHOT} MVN=${MVN:-mvn} +if [ -z "${RELEASE_VERSION}" ]; then + echo "RELEASE_VERSION was not set." + exit 1 +fi + # fail immediately set -o errexit set -o nounset diff --git a/tools/releasing/update_branch_version.sh b/tools/releasing/update_branch_version.sh index e4dc710f50f11..951ea7151c8bf 100755 --- a/tools/releasing/update_branch_version.sh +++ b/tools/releasing/update_branch_version.sh @@ -20,10 +20,18 @@ ## ## Variables with defaults (if not overwritten by environment) ## -OLD_VERSION=${OLD_VERSION:-1.4-SNAPSHOT} -NEW_VERSION=${NEW_VERSION:-1.5-SNAPSHOT} MVN=${MVN:-mvn} +if [ -z "${OLD_VERSION}" ]; then + echo "OLD_VERSION was not set." + exit 1 +fi + +if [ -z "${NEW_VERSION}" ]; then + echo "NEW_VERSION was not set." + exit 1 +fi + # fail immediately set -o errexit set -o nounset From 87cb185b5d270f1cc6f05756fb25d00c5eaf5ff7 Mon Sep 17 00:00:00 2001 From: zentol Date: Tue, 17 Jul 2018 09:34:45 +0200 Subject: [PATCH 40/65] [FLINK-9873][runtime] Log task state when aborting checkpoint This closes #6350. --- .../runtime/checkpoint/CheckpointCoordinator.java | 13 ++++++++++--- 1 file changed, 10 insertions(+), 3 deletions(-) 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 803b2ca66d845..e936b2462225a 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 @@ -457,13 +457,20 @@ public CheckpointTriggerResult triggerCheckpoint( Execution[] executions = new Execution[tasksToTrigger.length]; for (int i = 0; i < tasksToTrigger.length; i++) { Execution ee = tasksToTrigger[i].getCurrentExecutionAttempt(); - if (ee != null && ee.getState() == ExecutionState.RUNNING) { - executions[i] = ee; - } else { + if (ee == null) { LOG.info("Checkpoint triggering task {} of job {} is not being executed at the moment. Aborting checkpoint.", tasksToTrigger[i].getTaskNameWithSubtaskIndex(), job); return new CheckpointTriggerResult(CheckpointDeclineReason.NOT_ALL_REQUIRED_TASKS_RUNNING); + } else if (ee.getState() == ExecutionState.RUNNING) { + executions[i] = ee; + } else { + LOG.info("Checkpoint triggering task {} of job {} is not in state {} but {} instead. Aborting checkpoint.", + tasksToTrigger[i].getTaskNameWithSubtaskIndex(), + job, + ExecutionState.RUNNING, + ee.getState()); + return new CheckpointTriggerResult(CheckpointDeclineReason.NOT_ALL_REQUIRED_TASKS_RUNNING); } } From b08414bc041e42d968e8289a8412e77efe14dc94 Mon Sep 17 00:00:00 2001 From: yanghua Date: Fri, 13 Jul 2018 16:48:04 +0800 Subject: [PATCH 41/65] [FLINK-9841][rest] Close log file channel after response was fully written This closes #6329. --- .../AbstractTaskManagerFileHandler.java | 25 ++++++++++++++++--- 1 file changed, 22 insertions(+), 3 deletions(-) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/taskmanager/AbstractTaskManagerFileHandler.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/taskmanager/AbstractTaskManagerFileHandler.java index edefa1565b069..4c7ac9406b1a2 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/taskmanager/AbstractTaskManagerFileHandler.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/taskmanager/AbstractTaskManagerFileHandler.java @@ -63,6 +63,7 @@ import javax.annotation.Nonnull; import java.io.File; +import java.io.FileNotFoundException; import java.io.IOException; import java.io.RandomAccessFile; import java.nio.channels.FileChannel; @@ -208,11 +209,20 @@ private void removeBlob(RemovalNotification Date: Wed, 11 Jul 2018 15:41:24 +0200 Subject: [PATCH 42/65] [FLINK-9805][rest] Catch JsonProcessingException in RestClient This closes #6307. --- .../main/java/org/apache/flink/runtime/rest/RestClient.java | 3 +-- 1 file changed, 1 insertion(+), 2 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 a1195366c823b..052b9b121ed3b 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 @@ -34,7 +34,6 @@ import org.apache.flink.util.FlinkException; import org.apache.flink.util.Preconditions; -import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.core.JsonParseException; import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.core.JsonParser; import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.core.JsonProcessingException; import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.JavaType; @@ -418,7 +417,7 @@ private void readRawResponse(FullHttpResponse msg) { try (InputStream in = new ByteBufInputStream(content)) { rawResponse = objectMapper.readTree(in); LOG.debug("Received response {}.", rawResponse); - } catch (JsonParseException je) { + } catch (JsonProcessingException je) { LOG.error("Response was not valid JSON.", je); // let's see if it was a plain-text message instead content.readerIndex(0); From 5d8431474b6e1c082fc06c83f739a85211cb1482 Mon Sep 17 00:00:00 2001 From: Timo Walther Date: Mon, 23 Jul 2018 13:48:45 +0200 Subject: [PATCH 43/65] [hotfix] [sql-client] Wrap exceptions thrown during environment instance creation --- .../flink/table/client/gateway/local/ExecutionContext.java | 7 ++++++- 1 file changed, 6 insertions(+), 1 deletion(-) diff --git a/flink-libraries/flink-sql-client/src/main/java/org/apache/flink/table/client/gateway/local/ExecutionContext.java b/flink-libraries/flink-sql-client/src/main/java/org/apache/flink/table/client/gateway/local/ExecutionContext.java index 9152908645a49..4283953446e17 100644 --- a/flink-libraries/flink-sql-client/src/main/java/org/apache/flink/table/client/gateway/local/ExecutionContext.java +++ b/flink-libraries/flink-sql-client/src/main/java/org/apache/flink/table/client/gateway/local/ExecutionContext.java @@ -166,7 +166,12 @@ public ClusterDescriptor createClusterDescriptor() throws Exception { } public EnvironmentInstance createEnvironmentInstance() { - return new EnvironmentInstance(); + try { + return new EnvironmentInstance(); + } catch (Throwable t) { + // catch everything such that a wrong environment does not affect invocations + throw new SqlExecutionException("Could not create environment instance.", t); + } } public Map> getTableSources() { From f998f0f96086491f4781d5b92f431483feb64859 Mon Sep 17 00:00:00 2001 From: kkloudas Date: Tue, 17 Jul 2018 11:52:02 +0200 Subject: [PATCH 44/65] [FLINK-9903] [DataStream API] Refactor StreamingFileSink / add bulk encoders * Add supports for bulk encoders. * Expose more options in the rolling policy and * Allows to return any object as bucket id from the bucketer. --- .../api/common/serialization/BulkWriter.java | 95 ++++ .../core/fs/SafetyNetWrapperFileSystem.java | 5 + .../api/functions/sink/filesystem/Bucket.java | 48 +- .../sink/filesystem/BucketFactory.java | 29 +- .../sink/filesystem/BucketState.java | 12 +- .../filesystem/BucketStateSerializer.java | 39 +- .../functions/sink/filesystem/Buckets.java | 339 +++++++++++++ .../sink/filesystem/BulkPartWriter.java | 110 +++++ .../sink/filesystem/DefaultBucketFactory.java | 35 +- .../sink/filesystem/PartFileInfo.java | 4 +- ...rtFileHandler.java => PartFileWriter.java} | 97 ++-- .../sink/filesystem/RollingPolicy.java | 20 +- .../sink/filesystem/RowWisePartWriter.java | 96 ++++ .../sink/filesystem/StreamingFileSink.java | 457 ++++++++---------- .../bucketers/BasePathBucketer.java | 11 +- .../sink/filesystem/bucketers/Bucketer.java | 22 +- .../bucketers/DateTimeBucketer.java | 12 +- .../SimpleVersionedStringSerializer.java | 77 +++ .../policies}/DefaultRollingPolicy.java | 49 +- .../policies/OnCheckpointRollingPolicy.java | 45 ++ .../filesystem/BucketStateSerializerTest.java | 47 +- .../sink/filesystem/BucketsTest.java | 119 +++++ .../sink/filesystem/BulkWriterTest.java | 144 ++++++ .../LocalStreamingFileSinkTest.java | 267 ++++------ .../sink/filesystem/RollingPolicyTest.java | 225 +++++++++ .../functions/sink/filesystem/TestUtils.java | 164 +++++++ 26 files changed, 1969 insertions(+), 599 deletions(-) create mode 100644 flink-core/src/main/java/org/apache/flink/api/common/serialization/BulkWriter.java create mode 100644 flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/sink/filesystem/Buckets.java create mode 100644 flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/sink/filesystem/BulkPartWriter.java rename flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/sink/filesystem/{PartFileHandler.java => PartFileWriter.java} (54%) create mode 100644 flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/sink/filesystem/RowWisePartWriter.java create mode 100644 flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/sink/filesystem/bucketers/SimpleVersionedStringSerializer.java rename flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/sink/filesystem/{ => rolling/policies}/DefaultRollingPolicy.java (68%) create mode 100644 flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/sink/filesystem/rolling/policies/OnCheckpointRollingPolicy.java create mode 100644 flink-streaming-java/src/test/java/org/apache/flink/streaming/api/functions/sink/filesystem/BucketsTest.java create mode 100644 flink-streaming-java/src/test/java/org/apache/flink/streaming/api/functions/sink/filesystem/BulkWriterTest.java create mode 100644 flink-streaming-java/src/test/java/org/apache/flink/streaming/api/functions/sink/filesystem/RollingPolicyTest.java create mode 100644 flink-streaming-java/src/test/java/org/apache/flink/streaming/api/functions/sink/filesystem/TestUtils.java diff --git a/flink-core/src/main/java/org/apache/flink/api/common/serialization/BulkWriter.java b/flink-core/src/main/java/org/apache/flink/api/common/serialization/BulkWriter.java new file mode 100644 index 0000000000000..44f5fbe987afe --- /dev/null +++ b/flink-core/src/main/java/org/apache/flink/api/common/serialization/BulkWriter.java @@ -0,0 +1,95 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.api.common.serialization; + +import org.apache.flink.annotation.PublicEvolving; +import org.apache.flink.core.fs.FSDataOutputStream; + +import java.io.IOException; +import java.io.Serializable; + +/** + * An encoder that encodes data in a bulk fashion, encoding many records together at a time. + * + *

    Examples for bulk encoding are most compressed formats, including formats like + * Parquet and ORC which encode batches of records into blocks of column vectors. + * + *

    The bulk encoder may be stateful and is bound to a single stream during its + * lifetime. + * + * @param The type of the elements encoded through this encoder. + */ +@PublicEvolving +public interface BulkWriter { + + /** + * Adds an element to the encoder. The encoder may temporarily buffer the element, + * or immediately write it to the stream. + * + *

    It may be that adding this element fills up an internal buffer and causes the + * encoding and flushing of a batch of internally buffered elements. + * + * @param element The element to add. + * @throws IOException Thrown, if the element cannot be added to the encoder, + * or if the output stream throws an exception. + */ + void addElement(T element) throws IOException; + + /** + * Flushes all intermediate buffered data to the output stream. + * It is expected that flushing often may reduce the efficiency of the encoding. + * + * @throws IOException Thrown if the encoder cannot be flushed, or if the output + * stream throws an exception. + */ + void flush() throws IOException; + + /** + * Finishes the writing. This must flush all internal buffer, finish encoding, and write + * footers. + * + *

    The writer is not expected to handle any more records via {@link #addElement(Object)} after + * this method is called. + * + *

    Important: This method MUST NOT close the stream that the writer writes to. + * Closing the stream is expected to happen through the invoker of this method afterwards. + * + * @throws IOException Thrown if the finalization fails. + */ + void finish() throws IOException; + + // ------------------------------------------------------------------------ + + /** + * A factory that creates a {@link BulkWriter}. + * @param The type of record to write. + */ + @FunctionalInterface + interface Factory extends Serializable { + + /** + * Creates a writer that writes to the given stream. + * + * @param out The output stream to write the encoded data to. + * @throws IOException Thrown if the writer cannot be opened, or if the output + * stream throws an exception. + */ + BulkWriter create(FSDataOutputStream out) throws IOException; + } +} diff --git a/flink-core/src/main/java/org/apache/flink/core/fs/SafetyNetWrapperFileSystem.java b/flink-core/src/main/java/org/apache/flink/core/fs/SafetyNetWrapperFileSystem.java index 92b3a74c2ccb2..04e6315589321 100644 --- a/flink-core/src/main/java/org/apache/flink/core/fs/SafetyNetWrapperFileSystem.java +++ b/flink-core/src/main/java/org/apache/flink/core/fs/SafetyNetWrapperFileSystem.java @@ -64,6 +64,11 @@ public FileStatus getFileStatus(Path f) throws IOException { return unsafeFileSystem.getFileStatus(f); } + @Override + public RecoverableWriter createRecoverableWriter() throws IOException { + return unsafeFileSystem.createRecoverableWriter(); + } + @Override public BlockLocation[] getFileBlockLocations(FileStatus file, long start, long len) throws IOException { return unsafeFileSystem.getFileBlockLocations(file, start, len); diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/sink/filesystem/Bucket.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/sink/filesystem/Bucket.java index d9a6d751c3820..3e2d22c09b840 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/sink/filesystem/Bucket.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/sink/filesystem/Bucket.java @@ -19,7 +19,6 @@ package org.apache.flink.streaming.api.functions.sink.filesystem; import org.apache.flink.annotation.PublicEvolving; -import org.apache.flink.api.common.serialization.Encoder; import org.apache.flink.core.fs.Path; import org.apache.flink.core.fs.RecoverableWriter; import org.apache.flink.util.Preconditions; @@ -30,6 +29,7 @@ import java.util.Iterator; import java.util.List; import java.util.Map; +import java.util.Objects; /** * A bucket is the directory organization of the output of the {@link StreamingFileSink}. @@ -39,17 +39,17 @@ * queried to see in which bucket this element should be written to. */ @PublicEvolving -public class Bucket { +public class Bucket { private static final String PART_PREFIX = "part"; - private final String bucketId; + private final BucketID bucketId; private final Path bucketPath; private final int subtaskIndex; - private final Encoder encoder; + private final PartFileWriter.PartFileFactory partFileFactory; private final RecoverableWriter fsWriter; @@ -57,7 +57,7 @@ public class Bucket { private long partCounter; - private PartFileHandler currentPart; + private PartFileWriter currentPart; private List pending; @@ -68,10 +68,10 @@ public Bucket( RecoverableWriter fsWriter, int subtaskIndex, long initialPartCounter, - Encoder writer, - BucketState bucketstate) throws IOException { + PartFileWriter.PartFileFactory partFileFactory, + BucketState bucketState) throws IOException { - this(fsWriter, subtaskIndex, bucketstate.getBucketId(), bucketstate.getBucketPath(), initialPartCounter, writer); + this(fsWriter, subtaskIndex, bucketState.getBucketId(), bucketState.getBucketPath(), initialPartCounter, partFileFactory); // the constructor must have already initialized the filesystem writer Preconditions.checkState(fsWriter != null); @@ -79,15 +79,15 @@ public Bucket( // we try to resume the previous in-progress file, if the filesystem // supports such operation. If not, we just commit the file and start fresh. - final RecoverableWriter.ResumeRecoverable resumable = bucketstate.getCurrentInProgress(); + final RecoverableWriter.ResumeRecoverable resumable = bucketState.getInProgress(); if (resumable != null) { - currentPart = PartFileHandler.resumeFrom( - bucketId, fsWriter, resumable, bucketstate.getCreationTime()); + currentPart = partFileFactory.resumeFrom( + bucketId, fsWriter, resumable, bucketState.getCreationTime()); } // we commit pending files for previous checkpoints to the last successful one // (from which we are recovering from) - for (List commitables: bucketstate.getPendingPerCheckpoint().values()) { + for (List commitables: bucketState.getPendingPerCheckpoint().values()) { for (RecoverableWriter.CommitRecoverable commitable: commitables) { fsWriter.recoverForCommit(commitable).commitAfterRecovery(); } @@ -100,26 +100,26 @@ public Bucket( public Bucket( RecoverableWriter fsWriter, int subtaskIndex, - String bucketId, + BucketID bucketId, Path bucketPath, long initialPartCounter, - Encoder writer) { + PartFileWriter.PartFileFactory partFileFactory) { this.fsWriter = Preconditions.checkNotNull(fsWriter); this.subtaskIndex = subtaskIndex; this.bucketId = Preconditions.checkNotNull(bucketId); this.bucketPath = Preconditions.checkNotNull(bucketPath); this.partCounter = initialPartCounter; - this.encoder = Preconditions.checkNotNull(writer); + this.partFileFactory = Preconditions.checkNotNull(partFileFactory); this.pending = new ArrayList<>(); } - public PartFileInfo getInProgressPartInfo() { + public PartFileInfo getInProgressPartInfo() { return currentPart; } - public String getBucketId() { + public BucketID getBucketId() { return bucketId; } @@ -137,18 +137,18 @@ public boolean isActive() { void write(IN element, long currentTime) throws IOException { Preconditions.checkState(currentPart != null, "bucket has been closed"); - currentPart.write(element, encoder, currentTime); + currentPart.write(element, currentTime); } void rollPartFile(final long currentTime) throws IOException { closePartFile(); - currentPart = PartFileHandler.openNew(bucketId, fsWriter, getNewPartPath(), currentTime); + currentPart = partFileFactory.openNew(bucketId, fsWriter, getNewPartPath(), currentTime); partCounter++; } - void merge(final Bucket bucket) throws IOException { + void merge(final Bucket bucket) throws IOException { Preconditions.checkNotNull(bucket); - Preconditions.checkState(bucket.getBucketPath().equals(getBucketPath())); + Preconditions.checkState(Objects.equals(bucket.getBucketPath(), bucketPath)); // there should be no pending files in the "to-merge" states. Preconditions.checkState(bucket.pending.isEmpty()); @@ -176,7 +176,7 @@ public void dispose() { } } - public void commitUpToCheckpoint(long checkpointId) throws IOException { + public void onCheckpointAcknowledgment(long checkpointId) throws IOException { Preconditions.checkNotNull(fsWriter); Iterator>> it = @@ -193,7 +193,7 @@ public void commitUpToCheckpoint(long checkpointId) throws IOException { } } - public BucketState snapshot(long checkpointId) throws IOException { + public BucketState onCheckpoint(long checkpointId) throws IOException { RecoverableWriter.ResumeRecoverable resumable = null; long creationTime = Long.MAX_VALUE; @@ -206,7 +206,7 @@ public BucketState snapshot(long checkpointId) throws IOException { pendingPerCheckpoint.put(checkpointId, pending); pending = new ArrayList<>(); } - return new BucketState(bucketId, bucketPath, creationTime, resumable, pendingPerCheckpoint); + return new BucketState<>(bucketId, bucketPath, creationTime, resumable, pendingPerCheckpoint); } private Path getNewPartPath() { diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/sink/filesystem/BucketFactory.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/sink/filesystem/BucketFactory.java index 88f3c1abe7ddc..0c6b587b421f0 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/sink/filesystem/BucketFactory.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/sink/filesystem/BucketFactory.java @@ -19,7 +19,6 @@ package org.apache.flink.streaming.api.functions.sink.filesystem; import org.apache.flink.annotation.Internal; -import org.apache.flink.api.common.serialization.Encoder; import org.apache.flink.core.fs.Path; import org.apache.flink.core.fs.RecoverableWriter; @@ -30,20 +29,20 @@ * A factory able to create {@link Bucket buckets} for the {@link StreamingFileSink}. */ @Internal -public interface BucketFactory extends Serializable { +interface BucketFactory extends Serializable { - Bucket getNewBucket( - RecoverableWriter fsWriter, - int subtaskIndex, - String bucketId, - Path bucketPath, - long initialPartCounter, - Encoder writer) throws IOException; + Bucket getNewBucket( + final RecoverableWriter fsWriter, + final int subtaskIndex, + final BucketID bucketId, + final Path bucketPath, + final long initialPartCounter, + final PartFileWriter.PartFileFactory partFileWriterFactory) throws IOException; - Bucket restoreBucket( - RecoverableWriter fsWriter, - int subtaskIndex, - long initialPartCounter, - Encoder writer, - BucketState bucketstate) throws IOException; + Bucket restoreBucket( + final RecoverableWriter fsWriter, + final int subtaskIndex, + final long initialPartCounter, + final PartFileWriter.PartFileFactory partFileWriterFactory, + final BucketState bucketState) throws IOException; } diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/sink/filesystem/BucketState.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/sink/filesystem/BucketState.java index 5ebc46c46650d..bb49e3a5d2593 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/sink/filesystem/BucketState.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/sink/filesystem/BucketState.java @@ -32,9 +32,9 @@ * The state of the {@link Bucket} that is to be checkpointed. */ @Internal -public class BucketState { +public class BucketState { - private final String bucketId; + private final BucketID bucketId; /** * The base path for the bucket, i.e. the directory where all the part files are stored. @@ -59,10 +59,10 @@ public class BucketState { private final Map> pendingPerCheckpoint; public BucketState( - final String bucketId, + final BucketID bucketId, final Path bucketPath, final long creationTime, - final @Nullable RecoverableWriter.ResumeRecoverable inProgress, + @Nullable final RecoverableWriter.ResumeRecoverable inProgress, final Map> pendingPerCheckpoint ) { this.bucketId = Preconditions.checkNotNull(bucketId); @@ -72,7 +72,7 @@ public BucketState( this.pendingPerCheckpoint = Preconditions.checkNotNull(pendingPerCheckpoint); } - public String getBucketId() { + public BucketID getBucketId() { return bucketId; } @@ -85,7 +85,7 @@ public long getCreationTime() { } @Nullable - public RecoverableWriter.ResumeRecoverable getCurrentInProgress() { + public RecoverableWriter.ResumeRecoverable getInProgress() { return inProgress; } diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/sink/filesystem/BucketStateSerializer.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/sink/filesystem/BucketStateSerializer.java index a167ec97650a8..cf9b8057bd107 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/sink/filesystem/BucketStateSerializer.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/sink/filesystem/BucketStateSerializer.java @@ -22,7 +22,6 @@ import org.apache.flink.annotation.VisibleForTesting; import org.apache.flink.core.fs.Path; import org.apache.flink.core.fs.RecoverableWriter; -import org.apache.flink.core.fs.RecoverableWriter.CommitRecoverable; import org.apache.flink.core.io.SimpleVersionedSerialization; import org.apache.flink.core.io.SimpleVersionedSerializer; import org.apache.flink.core.memory.DataInputDeserializer; @@ -42,7 +41,7 @@ * A {@code SimpleVersionedSerializer} used to serialize the {@link BucketState BucketState}. */ @Internal -class BucketStateSerializer implements SimpleVersionedSerializer { +class BucketStateSerializer implements SimpleVersionedSerializer> { private static final int MAGIC_NUMBER = 0x1e764b79; @@ -50,12 +49,16 @@ class BucketStateSerializer implements SimpleVersionedSerializer { private final SimpleVersionedSerializer commitableSerializer; - public BucketStateSerializer( - final SimpleVersionedSerializer resumableSerializer, - final SimpleVersionedSerializer commitableSerializer) { + private final SimpleVersionedSerializer bucketIdSerializer; + BucketStateSerializer( + final SimpleVersionedSerializer resumableSerializer, + final SimpleVersionedSerializer commitableSerializer, + final SimpleVersionedSerializer bucketIdSerializer + ) { this.resumableSerializer = Preconditions.checkNotNull(resumableSerializer); this.commitableSerializer = Preconditions.checkNotNull(commitableSerializer); + this.bucketIdSerializer = Preconditions.checkNotNull(bucketIdSerializer); } @Override @@ -64,7 +67,7 @@ public int getVersion() { } @Override - public byte[] serialize(BucketState state) throws IOException { + public byte[] serialize(BucketState state) throws IOException { DataOutputSerializer out = new DataOutputSerializer(256); out.writeInt(MAGIC_NUMBER); serializeV1(state, out); @@ -72,7 +75,7 @@ public byte[] serialize(BucketState state) throws IOException { } @Override - public BucketState deserialize(int version, byte[] serialized) throws IOException { + public BucketState deserialize(int version, byte[] serialized) throws IOException { switch (version) { case 1: DataInputDeserializer in = new DataInputDeserializer(serialized); @@ -84,13 +87,13 @@ public BucketState deserialize(int version, byte[] serialized) throws IOExceptio } @VisibleForTesting - void serializeV1(BucketState state, DataOutputView out) throws IOException { - out.writeUTF(state.getBucketId()); + void serializeV1(BucketState state, DataOutputView out) throws IOException { + SimpleVersionedSerialization.writeVersionAndSerialize(bucketIdSerializer, state.getBucketId(), out); out.writeUTF(state.getBucketPath().toString()); out.writeLong(state.getCreationTime()); // put the current open part file - final RecoverableWriter.ResumeRecoverable currentPart = state.getCurrentInProgress(); + final RecoverableWriter.ResumeRecoverable currentPart = state.getInProgress(); if (currentPart != null) { out.writeBoolean(true); SimpleVersionedSerialization.writeVersionAndSerialize(resumableSerializer, currentPart, out); @@ -100,19 +103,19 @@ void serializeV1(BucketState state, DataOutputView out) throws IOException { } // put the map of pending files per checkpoint - final Map> pendingCommitters = state.getPendingPerCheckpoint(); + final Map> pendingCommitters = state.getPendingPerCheckpoint(); // manually keep the version here to safe some bytes out.writeInt(commitableSerializer.getVersion()); out.writeInt(pendingCommitters.size()); - for (Entry> resumablesForCheckpoint : pendingCommitters.entrySet()) { - List resumables = resumablesForCheckpoint.getValue(); + for (Entry> resumablesForCheckpoint : pendingCommitters.entrySet()) { + List resumables = resumablesForCheckpoint.getValue(); out.writeLong(resumablesForCheckpoint.getKey()); out.writeInt(resumables.size()); - for (CommitRecoverable resumable : resumables) { + for (RecoverableWriter.CommitRecoverable resumable : resumables) { byte[] serialized = commitableSerializer.serialize(resumable); out.writeInt(serialized.length); out.write(serialized); @@ -121,8 +124,8 @@ void serializeV1(BucketState state, DataOutputView out) throws IOException { } @VisibleForTesting - BucketState deserializeV1(DataInputView in) throws IOException { - final String bucketId = in.readUTF(); + BucketState deserializeV1(DataInputView in) throws IOException { + final BucketID bucketId = SimpleVersionedSerialization.readVersionAndDeSerialize(bucketIdSerializer, in); final String bucketPathStr = in.readUTF(); final long creationTime = in.readLong(); @@ -140,7 +143,7 @@ BucketState deserializeV1(DataInputView in) throws IOException { final long checkpointId = in.readLong(); final int noOfResumables = in.readInt(); - final ArrayList resumables = new ArrayList<>(noOfResumables); + final List resumables = new ArrayList<>(noOfResumables); for (int j = 0; j < noOfResumables; j++) { final byte[] bytes = new byte[in.readInt()]; in.readFully(bytes); @@ -149,7 +152,7 @@ BucketState deserializeV1(DataInputView in) throws IOException { resumablesPerCheckpoint.put(checkpointId, resumables); } - return new BucketState( + return new BucketState<>( bucketId, new Path(bucketPathStr), creationTime, diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/sink/filesystem/Buckets.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/sink/filesystem/Buckets.java new file mode 100644 index 0000000000000..6afba170c50b1 --- /dev/null +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/sink/filesystem/Buckets.java @@ -0,0 +1,339 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.streaming.api.functions.sink.filesystem; + +import org.apache.flink.api.common.state.ListState; +import org.apache.flink.core.fs.FileSystem; +import org.apache.flink.core.fs.Path; +import org.apache.flink.core.fs.RecoverableWriter; +import org.apache.flink.core.io.SimpleVersionedSerialization; +import org.apache.flink.streaming.api.functions.sink.SinkFunction; +import org.apache.flink.streaming.api.functions.sink.filesystem.bucketers.Bucketer; +import org.apache.flink.util.Preconditions; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import javax.annotation.Nullable; + +import java.io.IOException; +import java.util.HashMap; +import java.util.Iterator; +import java.util.Map; + +/** + * The manager of the different active buckets in the {@link StreamingFileSink}. + * + *

    This class is responsible for all bucket-related operations and the actual + * {@link StreamingFileSink} is just plugging in the functionality offered by + * this class to the lifecycle of the operator. + * + * @param The type of input elements. + * @param The type of ids for the buckets, as returned by the {@link Bucketer}. + */ +public class Buckets { + + private static final Logger LOG = LoggerFactory.getLogger(Buckets.class); + + // ------------------------ configuration fields -------------------------- + + private final Path basePath; + + private final BucketFactory bucketFactory; + + private final Bucketer bucketer; + + private final PartFileWriter.PartFileFactory partFileWriterFactory; + + private final RollingPolicy rollingPolicy; + + // --------------------------- runtime fields ----------------------------- + + private final int subtaskIndex; + + private final Buckets.BucketerContext bucketerContext; + + private final Map> activeBuckets; + + private long initMaxPartCounter; + + private long maxPartCounterUsed; + + private final RecoverableWriter fileSystemWriter; + + // --------------------------- State Related Fields ----------------------------- + + private final BucketStateSerializer bucketStateSerializer; + + /** + * A private constructor creating a new empty bucket manager. + * + * @param basePath The base path for our buckets. + * @param bucketer The {@link Bucketer} provided by the user. + * @param bucketFactory The {@link BucketFactory} to be used to create buckets. + * @param partFileWriterFactory The {@link PartFileWriter.PartFileFactory} to be used when writing data. + * @param rollingPolicy The {@link RollingPolicy} as specified by the user. + */ + Buckets( + final Path basePath, + final Bucketer bucketer, + final BucketFactory bucketFactory, + final PartFileWriter.PartFileFactory partFileWriterFactory, + final RollingPolicy rollingPolicy, + final int subtaskIndex) throws IOException { + + this.basePath = Preconditions.checkNotNull(basePath); + this.bucketer = Preconditions.checkNotNull(bucketer); + this.bucketFactory = Preconditions.checkNotNull(bucketFactory); + this.partFileWriterFactory = Preconditions.checkNotNull(partFileWriterFactory); + this.rollingPolicy = Preconditions.checkNotNull(rollingPolicy); + this.subtaskIndex = subtaskIndex; + + this.activeBuckets = new HashMap<>(); + this.bucketerContext = new Buckets.BucketerContext(); + + this.fileSystemWriter = FileSystem.get(basePath.toUri()).createRecoverableWriter(); + this.bucketStateSerializer = new BucketStateSerializer<>( + fileSystemWriter.getResumeRecoverableSerializer(), + fileSystemWriter.getCommitRecoverableSerializer(), + bucketer.getSerializer() + ); + + this.initMaxPartCounter = 0L; + this.maxPartCounterUsed = 0L; + } + + /** + * Initializes the state after recovery from a failure. + * @param bucketStates the state holding recovered state about active buckets. + * @param partCounterState the state holding the max previously used part counters. + * @throws Exception + */ + void initializeState(final ListState bucketStates, final ListState partCounterState) throws Exception { + + // When resuming after a failure: + // 1) we get the max part counter used before in order to make sure that we do not overwrite valid data + // 2) we commit any pending files for previous checkpoints (previous to the last successful one) + // 3) we resume writing to the previous in-progress file of each bucket, and + // 4) if we receive multiple states for the same bucket, we merge them. + + // get the max counter + long maxCounter = 0L; + for (long partCounter: partCounterState.get()) { + maxCounter = Math.max(partCounter, maxCounter); + } + initMaxPartCounter = maxCounter; + + // get the restored buckets + for (byte[] recoveredState : bucketStates.get()) { + final BucketState bucketState = SimpleVersionedSerialization.readVersionAndDeSerialize( + bucketStateSerializer, recoveredState); + + final BucketID bucketId = bucketState.getBucketId(); + + LOG.info("Recovered bucket for {}", bucketId); + + final Bucket restoredBucket = bucketFactory.restoreBucket( + fileSystemWriter, + subtaskIndex, + initMaxPartCounter, + partFileWriterFactory, + bucketState + ); + + final Bucket existingBucket = activeBuckets.get(bucketId); + if (existingBucket == null) { + activeBuckets.put(bucketId, restoredBucket); + } else { + existingBucket.merge(restoredBucket); + } + + if (LOG.isDebugEnabled()) { + LOG.debug("{} idx {} restored state for bucket {}", getClass().getSimpleName(), + subtaskIndex, assembleBucketPath(bucketId)); + } + } + } + + void publishUpToCheckpoint(long checkpointId) throws IOException { + final Iterator>> activeBucketIt = + activeBuckets.entrySet().iterator(); + + while (activeBucketIt.hasNext()) { + Bucket bucket = activeBucketIt.next().getValue(); + bucket.onCheckpointAcknowledgment(checkpointId); + + if (!bucket.isActive()) { + // We've dealt with all the pending files and the writer for this bucket is not currently open. + // Therefore this bucket is currently inactive and we can remove it from our state. + activeBucketIt.remove(); + } + } + } + + void snapshotState( + final long checkpointId, + final long checkpointTimestamp, + final ListState bucketStates, + final ListState partCounterState) throws Exception { + + Preconditions.checkState( + fileSystemWriter != null && bucketStateSerializer != null, + "sink has not been initialized" + ); + + for (Bucket bucket : activeBuckets.values()) { + final PartFileInfo info = bucket.getInProgressPartInfo(); + + if (info != null && + (rollingPolicy.shouldRollOnCheckpoint(info) || + rollingPolicy.shouldRollOnEvent(info) || + rollingPolicy.shouldRollOnProcessingTime(info, checkpointTimestamp)) + ) { + // we also check here so that we do not have to always + // wait for the "next" element to arrive. + bucket.closePartFile(); + } + + final BucketState bucketState = bucket.onCheckpoint(checkpointId); + bucketStates.add(SimpleVersionedSerialization.writeVersionAndSerialize(bucketStateSerializer, bucketState)); + } + + partCounterState.add(maxPartCounterUsed); + } + + /** + * Called on every incoming element to write it to its final location. + * @param value the element itself. + * @param context the {@link SinkFunction.Context context} available to the sink function. + * @throws Exception + */ + void onElement(IN value, SinkFunction.Context context) throws Exception { + final long currentProcessingTime = context.currentProcessingTime(); + + // setting the values in the bucketer context + bucketerContext.update( + context.timestamp(), + context.currentWatermark(), + currentProcessingTime); + + final BucketID bucketId = bucketer.getBucketId(value, bucketerContext); + + Bucket bucket = activeBuckets.get(bucketId); + if (bucket == null) { + final Path bucketPath = assembleBucketPath(bucketId); + bucket = bucketFactory.getNewBucket( + fileSystemWriter, + subtaskIndex, + bucketId, + bucketPath, + initMaxPartCounter, + partFileWriterFactory); + activeBuckets.put(bucketId, bucket); + } + + final PartFileInfo info = bucket.getInProgressPartInfo(); + if (info == null || rollingPolicy.shouldRollOnEvent(info)) { + bucket.rollPartFile(currentProcessingTime); + } + bucket.write(value, currentProcessingTime); + + // we update the counter here because as buckets become inactive and + // get removed in the initializeState(), at the time we snapshot they + // may not be there to take them into account during checkpointing. + updateMaxPartCounter(bucket.getPartCounter()); + } + + void onProcessingTime(long timestamp) throws Exception { + for (Bucket bucket : activeBuckets.values()) { + final PartFileInfo info = bucket.getInProgressPartInfo(); + if (info != null && rollingPolicy.shouldRollOnProcessingTime(info, timestamp)) { + bucket.closePartFile(); + } + } + } + + void close() { + if (activeBuckets != null) { + activeBuckets.values().forEach(Bucket::dispose); + } + } + + /** + * Assembles the final bucket {@link Path} that will be used for the provided bucket in the + * underlying filesystem. + * @param bucketId the id of the bucket as returned by the {@link Bucketer}. + * @return The resulting path. + */ + private Path assembleBucketPath(BucketID bucketId) { + return new Path(basePath, bucketId.toString()); + } + + /** + * Updates the state keeping track of the maximum used part + * counter across all local active buckets. + * @param candidate the part counter that will potentially replace the current {@link #maxPartCounterUsed}. + */ + private void updateMaxPartCounter(long candidate) { + maxPartCounterUsed = Math.max(maxPartCounterUsed, candidate); + } + + /** + * The {@link Bucketer.Context} exposed to the + * {@link Bucketer#getBucketId(Object, Bucketer.Context)} + * whenever a new incoming element arrives. + */ + private static final class BucketerContext implements Bucketer.Context { + + @Nullable + private Long elementTimestamp; + + private long currentWatermark; + + private long currentProcessingTime; + + private BucketerContext() { + this.elementTimestamp = null; + this.currentWatermark = Long.MIN_VALUE; + this.currentProcessingTime = Long.MIN_VALUE; + } + + void update(@Nullable Long element, long watermark, long processingTime) { + this.elementTimestamp = element; + this.currentWatermark = watermark; + this.currentProcessingTime = processingTime; + } + + @Override + public long currentProcessingTime() { + return currentProcessingTime; + } + + @Override + public long currentWatermark() { + return currentWatermark; + } + + @Override + @Nullable + public Long timestamp() { + return elementTimestamp; + } + } +} diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/sink/filesystem/BulkPartWriter.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/sink/filesystem/BulkPartWriter.java new file mode 100644 index 0000000000000..558b1bf628dc2 --- /dev/null +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/sink/filesystem/BulkPartWriter.java @@ -0,0 +1,110 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.streaming.api.functions.sink.filesystem; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.api.common.serialization.BulkWriter; +import org.apache.flink.core.fs.Path; +import org.apache.flink.core.fs.RecoverableFsDataOutputStream; +import org.apache.flink.core.fs.RecoverableWriter; +import org.apache.flink.streaming.api.functions.sink.filesystem.bucketers.Bucketer; +import org.apache.flink.util.Preconditions; + +import java.io.IOException; + +/** + * A {@link PartFileWriter} for bulk-encoding formats that use an {@link BulkPartWriter}. + * This also implements the {@link PartFileInfo}. + */ +@Internal +final class BulkPartWriter extends PartFileWriter { + + private final BulkWriter writer; + + private BulkPartWriter( + final BucketID bucketId, + final RecoverableFsDataOutputStream currentPartStream, + final BulkWriter writer, + final long creationTime) { + super(bucketId, currentPartStream, creationTime); + this.writer = Preconditions.checkNotNull(writer); + } + + @Override + void write(IN element, long currentTime) throws IOException { + writer.addElement(element); + markWrite(currentTime); + } + + @Override + RecoverableWriter.ResumeRecoverable persist() { + throw new UnsupportedOperationException("Bulk Part Writers do not support \"pause and resume\" operations."); + } + + @Override + RecoverableWriter.CommitRecoverable closeForCommit() throws IOException { + writer.flush(); + writer.finish(); + return super.closeForCommit(); + } + + /** + * A factory that creates {@link BulkPartWriter BulkPartWriters}. + * @param The type of input elements. + * @param The type of ids for the buckets, as returned by the {@link Bucketer}. + */ + static class Factory implements PartFileWriter.PartFileFactory { + + private final BulkWriter.Factory writerFactory; + + Factory(BulkWriter.Factory writerFactory) { + this.writerFactory = writerFactory; + } + + @Override + public PartFileWriter resumeFrom( + final BucketID bucketId, + final RecoverableWriter fileSystemWriter, + final RecoverableWriter.ResumeRecoverable resumable, + final long creationTime) throws IOException { + + Preconditions.checkNotNull(fileSystemWriter); + Preconditions.checkNotNull(resumable); + + final RecoverableFsDataOutputStream stream = fileSystemWriter.recover(resumable); + final BulkWriter writer = writerFactory.create(stream); + return new BulkPartWriter<>(bucketId, stream, writer, creationTime); + } + + @Override + public PartFileWriter openNew( + final BucketID bucketId, + final RecoverableWriter fileSystemWriter, + final Path path, + final long creationTime) throws IOException { + + Preconditions.checkNotNull(fileSystemWriter); + Preconditions.checkNotNull(path); + + final RecoverableFsDataOutputStream stream = fileSystemWriter.open(path); + final BulkWriter writer = writerFactory.create(stream); + return new BulkPartWriter<>(bucketId, stream, writer, creationTime); + } + } +} diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/sink/filesystem/DefaultBucketFactory.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/sink/filesystem/DefaultBucketFactory.java index 795ba7490360e..532138f1ba20c 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/sink/filesystem/DefaultBucketFactory.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/sink/filesystem/DefaultBucketFactory.java @@ -19,7 +19,6 @@ package org.apache.flink.streaming.api.functions.sink.filesystem; import org.apache.flink.annotation.Internal; -import org.apache.flink.api.common.serialization.Encoder; import org.apache.flink.core.fs.Path; import org.apache.flink.core.fs.RecoverableWriter; @@ -29,18 +28,18 @@ * A factory returning {@link Bucket buckets}. */ @Internal -public class DefaultBucketFactory implements BucketFactory { +class DefaultBucketFactory implements BucketFactory { - private static final long serialVersionUID = 3372881359208513357L; + private static final long serialVersionUID = 1L; @Override - public Bucket getNewBucket( - RecoverableWriter fsWriter, - int subtaskIndex, - String bucketId, - Path bucketPath, - long initialPartCounter, - Encoder writer) throws IOException { + public Bucket getNewBucket( + final RecoverableWriter fsWriter, + final int subtaskIndex, + final BucketID bucketId, + final Path bucketPath, + final long initialPartCounter, + final PartFileWriter.PartFileFactory partFileWriterFactory) { return new Bucket<>( fsWriter, @@ -48,22 +47,22 @@ public Bucket getNewBucket( bucketId, bucketPath, initialPartCounter, - writer); + partFileWriterFactory); } @Override - public Bucket restoreBucket( - RecoverableWriter fsWriter, - int subtaskIndex, - long initialPartCounter, - Encoder writer, - BucketState bucketState) throws IOException { + public Bucket restoreBucket( + final RecoverableWriter fsWriter, + final int subtaskIndex, + final long initialPartCounter, + final PartFileWriter.PartFileFactory partFileWriterFactory, + final BucketState bucketState) throws IOException { return new Bucket<>( fsWriter, subtaskIndex, initialPartCounter, - writer, + partFileWriterFactory, bucketState); } } diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/sink/filesystem/PartFileInfo.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/sink/filesystem/PartFileInfo.java index 9c3d047634eb0..5e72ea039d8f9 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/sink/filesystem/PartFileInfo.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/sink/filesystem/PartFileInfo.java @@ -29,13 +29,13 @@ * should roll the part file or not. */ @PublicEvolving -public interface PartFileInfo { +public interface PartFileInfo { /** * @return The bucket identifier of the current buffer, as returned by the * {@link Bucketer#getBucketId(Object, Bucketer.Context)}. */ - String getBucketId(); + BucketID getBucketId(); /** * @return The creation time (in ms) of the currently open part file. diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/sink/filesystem/PartFileHandler.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/sink/filesystem/PartFileWriter.java similarity index 54% rename from flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/sink/filesystem/PartFileHandler.java rename to flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/sink/filesystem/PartFileWriter.java index 10fd12b4664b7..662454ba200b3 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/sink/filesystem/PartFileHandler.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/sink/filesystem/PartFileWriter.java @@ -19,7 +19,6 @@ package org.apache.flink.streaming.api.functions.sink.filesystem; import org.apache.flink.annotation.Internal; -import org.apache.flink.api.common.serialization.Encoder; import org.apache.flink.core.fs.Path; import org.apache.flink.core.fs.RecoverableFsDataOutputStream; import org.apache.flink.core.fs.RecoverableWriter; @@ -29,22 +28,29 @@ import java.io.IOException; /** - * A handler for the currently open part file in a specific {@link Bucket}. - * This also implements the {@link PartFileInfo}. + * An abstract writer for the currently open part file in a specific {@link Bucket}. + * + *

    Currently, there are two subclasses, of this class: + *

      + *
    1. One for row-wise formats: the {@link RowWisePartWriter}.
    2. + *
    3. One for bulk encoding formats: the {@link BulkPartWriter}.
    4. + *
    + * + *

    This also implements the {@link PartFileInfo}. */ @Internal -class PartFileHandler implements PartFileInfo { +abstract class PartFileWriter implements PartFileInfo { - private final String bucketId; + private final BucketID bucketId; private final long creationTime; - private final RecoverableFsDataOutputStream currentPartStream; + protected final RecoverableFsDataOutputStream currentPartStream; private long lastUpdateTime; - private PartFileHandler( - final String bucketId, + protected PartFileWriter( + final BucketID bucketId, final RecoverableFsDataOutputStream currentPartStream, final long creationTime) { @@ -55,36 +61,7 @@ private PartFileHandler( this.lastUpdateTime = creationTime; } - public static PartFileHandler resumeFrom( - final String bucketId, - final RecoverableWriter fileSystemWriter, - final RecoverableWriter.ResumeRecoverable resumable, - final long creationTime) throws IOException { - Preconditions.checkNotNull(bucketId); - Preconditions.checkNotNull(fileSystemWriter); - Preconditions.checkNotNull(resumable); - - final RecoverableFsDataOutputStream stream = fileSystemWriter.recover(resumable); - return new PartFileHandler<>(bucketId, stream, creationTime); - } - - public static PartFileHandler openNew( - final String bucketId, - final RecoverableWriter fileSystemWriter, - final Path path, - final long creationTime) throws IOException { - Preconditions.checkNotNull(bucketId); - Preconditions.checkNotNull(fileSystemWriter); - Preconditions.checkNotNull(path); - - final RecoverableFsDataOutputStream stream = fileSystemWriter.open(path); - return new PartFileHandler<>(bucketId, stream, creationTime); - } - - void write(IN element, Encoder encoder, long currentTime) throws IOException { - encoder.encode(element, currentPartStream); - this.lastUpdateTime = currentTime; - } + abstract void write(IN element, long currentTime) throws IOException; RecoverableWriter.ResumeRecoverable persist() throws IOException { return currentPartStream.persist(); @@ -100,8 +77,12 @@ void dispose() { IOUtils.closeQuietly(currentPartStream); } + void markWrite(long now) { + this.lastUpdateTime = now; + } + @Override - public String getBucketId() { + public BucketID getBucketId() { return bucketId; } @@ -119,4 +100,42 @@ public long getSize() throws IOException { public long getLastUpdateTime() { return lastUpdateTime; } + + // ------------------------------------------------------------------------ + + /** + * An interface for factories that create the different {@link PartFileWriter writers}. + */ + interface PartFileFactory { + + /** + * Used upon recovery from a failure to recover a {@link PartFileWriter writer}. + * @param bucketId the id of the bucket this writer is writing to. + * @param fileSystemWriter the filesystem-specific writer to use when writing to the filesystem. + * @param resumable the state of the stream we are resurrecting. + * @param creationTime the creation time of the stream. + * @return the recovered {@link PartFileWriter writer}. + * @throws IOException + */ + PartFileWriter resumeFrom( + final BucketID bucketId, + final RecoverableWriter fileSystemWriter, + final RecoverableWriter.ResumeRecoverable resumable, + final long creationTime) throws IOException; + + /** + * Used to create a new {@link PartFileWriter writer}. + * @param bucketId the id of the bucket this writer is writing to. + * @param fileSystemWriter the filesystem-specific writer to use when writing to the filesystem. + * @param path the part this writer will write to. + * @param creationTime the creation time of the stream. + * @return the new {@link PartFileWriter writer}. + * @throws IOException + */ + PartFileWriter openNew( + final BucketID bucketId, + final RecoverableWriter fileSystemWriter, + final Path path, + final long creationTime) throws IOException; + } } diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/sink/filesystem/RollingPolicy.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/sink/filesystem/RollingPolicy.java index 936377e34dd1f..24c38aa58b7eb 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/sink/filesystem/RollingPolicy.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/sink/filesystem/RollingPolicy.java @@ -28,13 +28,27 @@ * rolls its currently open part file and opens a new one. */ @PublicEvolving -public interface RollingPolicy extends Serializable { +public interface RollingPolicy extends Serializable { /** - * Determines if the in-progress part file for a bucket should roll. + * Determines if the in-progress part file for a bucket should roll on every checkpoint. + * @param partFileState the state of the currently open part file of the bucket. + * @return {@code True} if the part file should roll, {@link false} otherwise. + */ + boolean shouldRollOnCheckpoint(final PartFileInfo partFileState) throws IOException; + + /** + * Determines if the in-progress part file for a bucket should roll based on its current state, e.g. its size. + * @param partFileState the state of the currently open part file of the bucket. + * @return {@code True} if the part file should roll, {@link false} otherwise. + */ + boolean shouldRollOnEvent(final PartFileInfo partFileState) throws IOException; + + /** + * Determines if the in-progress part file for a bucket should roll based on a time condition. * @param partFileState the state of the currently open part file of the bucket. * @param currentTime the current processing time. * @return {@code True} if the part file should roll, {@link false} otherwise. */ - boolean shouldRoll(final PartFileInfo partFileState, final long currentTime) throws IOException; + boolean shouldRollOnProcessingTime(final PartFileInfo partFileState, final long currentTime) throws IOException; } diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/sink/filesystem/RowWisePartWriter.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/sink/filesystem/RowWisePartWriter.java new file mode 100644 index 0000000000000..0b00b43c86e61 --- /dev/null +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/sink/filesystem/RowWisePartWriter.java @@ -0,0 +1,96 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.streaming.api.functions.sink.filesystem; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.api.common.serialization.Encoder; +import org.apache.flink.core.fs.Path; +import org.apache.flink.core.fs.RecoverableFsDataOutputStream; +import org.apache.flink.core.fs.RecoverableWriter; +import org.apache.flink.streaming.api.functions.sink.filesystem.bucketers.Bucketer; +import org.apache.flink.util.Preconditions; + +import java.io.IOException; + +/** + * A {@link PartFileWriter} for row-wise formats that use an {@link Encoder}. + * This also implements the {@link PartFileInfo}. + */ +@Internal +final class RowWisePartWriter extends PartFileWriter { + + private final Encoder encoder; + + private RowWisePartWriter( + final BucketID bucketId, + final RecoverableFsDataOutputStream currentPartStream, + final Encoder encoder, + final long creationTime) { + super(bucketId, currentPartStream, creationTime); + this.encoder = Preconditions.checkNotNull(encoder); + } + + @Override + void write(IN element, long currentTime) throws IOException { + encoder.encode(element, currentPartStream); + markWrite(currentTime); + } + + /** + * A factory that creates {@link RowWisePartWriter RowWisePartWriters}. + * @param The type of input elements. + * @param The type of ids for the buckets, as returned by the {@link Bucketer}. + */ + static class Factory implements PartFileWriter.PartFileFactory { + + private final Encoder encoder; + + Factory(Encoder encoder) { + this.encoder = encoder; + } + + @Override + public PartFileWriter resumeFrom( + final BucketID bucketId, + final RecoverableWriter fileSystemWriter, + final RecoverableWriter.ResumeRecoverable resumable, + final long creationTime) throws IOException { + + Preconditions.checkNotNull(fileSystemWriter); + Preconditions.checkNotNull(resumable); + + final RecoverableFsDataOutputStream stream = fileSystemWriter.recover(resumable); + return new RowWisePartWriter<>(bucketId, stream, encoder, creationTime); + } + + @Override + public PartFileWriter openNew( + final BucketID bucketId, + final RecoverableWriter fileSystemWriter, + final Path path, + final long creationTime) throws IOException { + + Preconditions.checkNotNull(fileSystemWriter); + Preconditions.checkNotNull(path); + + final RecoverableFsDataOutputStream stream = fileSystemWriter.open(path); + return new RowWisePartWriter<>(bucketId, stream, encoder, creationTime); + } + } +} diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/sink/filesystem/StreamingFileSink.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/sink/filesystem/StreamingFileSink.java index 0406afcf78be3..c2080796e4b65 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/sink/filesystem/StreamingFileSink.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/sink/filesystem/StreamingFileSink.java @@ -20,8 +20,8 @@ import org.apache.flink.annotation.PublicEvolving; import org.apache.flink.annotation.VisibleForTesting; +import org.apache.flink.api.common.serialization.BulkWriter; import org.apache.flink.api.common.serialization.Encoder; -import org.apache.flink.api.common.serialization.SimpleStringEncoder; import org.apache.flink.api.common.state.ListState; import org.apache.flink.api.common.state.ListStateDescriptor; import org.apache.flink.api.common.state.OperatorStateStore; @@ -30,29 +30,23 @@ import org.apache.flink.configuration.Configuration; import org.apache.flink.core.fs.FileSystem; import org.apache.flink.core.fs.Path; -import org.apache.flink.core.fs.RecoverableWriter; -import org.apache.flink.core.io.SimpleVersionedSerialization; import org.apache.flink.runtime.state.CheckpointListener; 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.api.functions.sink.SinkFunction; import org.apache.flink.streaming.api.functions.sink.filesystem.bucketers.Bucketer; import org.apache.flink.streaming.api.functions.sink.filesystem.bucketers.DateTimeBucketer; +import org.apache.flink.streaming.api.functions.sink.filesystem.rolling.policies.DefaultRollingPolicy; +import org.apache.flink.streaming.api.functions.sink.filesystem.rolling.policies.OnCheckpointRollingPolicy; import org.apache.flink.streaming.api.operators.StreamingRuntimeContext; import org.apache.flink.streaming.runtime.tasks.ProcessingTimeCallback; import org.apache.flink.streaming.runtime.tasks.ProcessingTimeService; import org.apache.flink.util.Preconditions; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import javax.annotation.Nullable; - import java.io.IOException; -import java.util.HashMap; -import java.util.Iterator; -import java.util.Map; +import java.io.Serializable; /** * Sink that emits its input elements to {@link FileSystem} files within buckets. This is @@ -69,7 +63,9 @@ * be written to inside the base directory. The {@code Bucketer} can, for example, use time or * a property of the element to determine the bucket directory. The default {@code Bucketer} is a * {@link DateTimeBucketer} which will create one new bucket every hour. You can specify - * a custom {@code Bucketer} using {@link #setBucketer(Bucketer)}. + * a custom {@code Bucketer} using the {@code setBucketer(Bucketer)} method, after calling + * {@link StreamingFileSink#forRowFormat(Path, Encoder)} or + * {@link StreamingFileSink#forBulkFormat(Path, BulkWriter.Factory)}. * * *

    The filenames of the part files contain the part prefix, "part-", the parallel subtask index of the sink @@ -94,19 +90,6 @@ * state are transferred into the {@code finished} state while any {@code in-progress} files are rolled back, so that * they do not contain data that arrived after the checkpoint from which we restore. * - *

    NOTE: - *

      - *
    1. - * If checkpointing is not enabled the pending files will never be moved to the finished state. - *
    2. - *
    3. - * The part files are written using an instance of {@link Encoder}. By default, a - * {@link SimpleStringEncoder} is used, which writes the result of {@code toString()} for - * every element, separated by newlines. You can configure the writer using the - * {@link #setEncoder(Encoder)}. - *
    4. - *
    - * * @param Type of the elements emitted by this sink */ @PublicEvolving @@ -116,8 +99,6 @@ public class StreamingFileSink private static final long serialVersionUID = 1L; - private static final Logger LOG = LoggerFactory.getLogger(StreamingFileSink.class); - // -------------------------- state descriptors --------------------------- private static final ListStateDescriptor BUCKET_STATE_DESC = @@ -128,298 +109,264 @@ public class StreamingFileSink // ------------------------ configuration fields -------------------------- - private final Path basePath; - - private final BucketFactory bucketFactory; - - private long bucketCheckInterval = 60L * 1000L; - - private Bucketer bucketer; + private final long bucketCheckInterval; - private Encoder encoder; - - private RollingPolicy rollingPolicy; + private final StreamingFileSink.BucketsBuilder bucketsBuilder; // --------------------------- runtime fields ----------------------------- - private transient BucketerContext bucketerContext; - - private transient RecoverableWriter fileSystemWriter; + private transient Buckets buckets; private transient ProcessingTimeService processingTimeService; - private transient Map> activeBuckets; + // --------------------------- State Related Fields ----------------------------- - ////////////////// State Related Fields ///////////////////// + private transient ListState bucketStates; - private transient BucketStateSerializer bucketStateSerializer; + private transient ListState maxPartCountersState; - private transient ListState restoredBucketStates; + /** + * Creates a new {@code StreamingFileSink} that writes files to the given base directory. + */ + private StreamingFileSink( + final StreamingFileSink.BucketsBuilder bucketsBuilder, + final long bucketCheckInterval) { + + Preconditions.checkArgument(bucketCheckInterval > 0L); - private transient ListState restoredMaxCounters; + this.bucketsBuilder = Preconditions.checkNotNull(bucketsBuilder); + this.bucketCheckInterval = bucketCheckInterval; + } - private transient long initMaxPartCounter; + // ------------------------------------------------------------------------ - private transient long maxPartCounterUsed; + // --------------------------- Sink Builders ----------------------------- /** - * Creates a new {@code StreamingFileSink} that writes files to the given base directory. - * - *

    This uses a {@link DateTimeBucketer} as {@link Bucketer} and a {@link SimpleStringEncoder} as a writer. - * - * @param basePath The directory to which to write the bucket files. + * Creates the builder for a {@code StreamingFileSink} with row-encoding format. + * @param basePath the base path where all the buckets are going to be created as sub-directories. + * @param encoder the {@link Encoder} to be used when writing elements in the buckets. + * @param the type of incoming elements + * @return The builder where the remaining of the configuration parameters for the sink can be configured. + * In order to instantiate the sink, call {@link RowFormatBuilder#build()} after specifying the desired parameters. */ - public StreamingFileSink(Path basePath) { - this(basePath, new DefaultBucketFactory<>()); + public static StreamingFileSink.RowFormatBuilder forRowFormat( + final Path basePath, final Encoder encoder) { + return new StreamingFileSink.RowFormatBuilder<>(basePath, encoder, new DateTimeBucketer<>()); } - @VisibleForTesting - StreamingFileSink(Path basePath, BucketFactory bucketFactory) { - this.basePath = Preconditions.checkNotNull(basePath); - this.bucketer = new DateTimeBucketer<>(); - this.encoder = new SimpleStringEncoder<>(); - this.rollingPolicy = DefaultRollingPolicy.create().build(); - this.bucketFactory = Preconditions.checkNotNull(bucketFactory); + /** + * Creates the builder for a {@link StreamingFileSink} with row-encoding format. + * @param basePath the base path where all the buckets are going to be created as sub-directories. + * @param writerFactory the {@link BulkWriter.Factory} to be used when writing elements in the buckets. + * @param the type of incoming elements + * @return The builder where the remaining of the configuration parameters for the sink can be configured. + * In order to instantiate the sink, call {@link RowFormatBuilder#build()} after specifying the desired parameters. + */ + public static StreamingFileSink.BulkFormatBuilder forBulkFormat( + final Path basePath, final BulkWriter.Factory writerFactory) { + return new StreamingFileSink.BulkFormatBuilder<>(basePath, writerFactory, new DateTimeBucketer<>()); } - public StreamingFileSink setEncoder(Encoder encoder) { - this.encoder = Preconditions.checkNotNull(encoder); - return this; - } + /** + * The base abstract class for the {@link RowFormatBuilder} and {@link BulkFormatBuilder}. + */ + private abstract static class BucketsBuilder implements Serializable { - public StreamingFileSink setBucketer(Bucketer bucketer) { - this.bucketer = Preconditions.checkNotNull(bucketer); - return this; - } + private static final long serialVersionUID = 1L; - public StreamingFileSink setBucketCheckInterval(long interval) { - this.bucketCheckInterval = interval; - return this; + abstract Buckets createBuckets(final int subtaskIndex) throws IOException; } - public StreamingFileSink setRollingPolicy(RollingPolicy policy) { - this.rollingPolicy = Preconditions.checkNotNull(policy); - return this; - } + /** + * A builder for configuring the sink for row-wise encoding formats. + */ + @PublicEvolving + public static class RowFormatBuilder extends StreamingFileSink.BucketsBuilder { - @Override - public void notifyCheckpointComplete(long checkpointId) throws Exception { - final Iterator>> activeBucketIt = - activeBuckets.entrySet().iterator(); - - while (activeBucketIt.hasNext()) { - Bucket bucket = activeBucketIt.next().getValue(); - bucket.commitUpToCheckpoint(checkpointId); - - if (!bucket.isActive()) { - // We've dealt with all the pending files and the writer for this bucket is not currently open. - // Therefore this bucket is currently inactive and we can remove it from our state. - activeBucketIt.remove(); - } - } - } + private static final long serialVersionUID = 1L; - @Override - public void snapshotState(FunctionSnapshotContext context) throws Exception { - Preconditions.checkState( - restoredBucketStates != null && fileSystemWriter != null && bucketStateSerializer != null, - "sink has not been initialized"); + private long bucketCheckInterval = 60L * 1000L; + + private final Path basePath; - restoredBucketStates.clear(); - for (Bucket bucket : activeBuckets.values()) { + private final Encoder encoder; - final PartFileInfo info = bucket.getInProgressPartInfo(); - final long checkpointTimestamp = context.getCheckpointTimestamp(); + private Bucketer bucketer; - if (info != null && rollingPolicy.shouldRoll(info, checkpointTimestamp)) { - // we also check here so that we do not have to always - // wait for the "next" element to arrive. - bucket.closePartFile(); - } + private RollingPolicy rollingPolicy; - final BucketState bucketState = bucket.snapshot(context.getCheckpointId()); - restoredBucketStates.add(SimpleVersionedSerialization.writeVersionAndSerialize(bucketStateSerializer, bucketState)); + private BucketFactory bucketFactory = new DefaultBucketFactory<>(); + + RowFormatBuilder(Path basePath, Encoder encoder, Bucketer bucketer) { + this.basePath = Preconditions.checkNotNull(basePath); + this.encoder = Preconditions.checkNotNull(encoder); + this.bucketer = Preconditions.checkNotNull(bucketer); + this.rollingPolicy = DefaultRollingPolicy.create().build(); } - restoredMaxCounters.clear(); - restoredMaxCounters.add(maxPartCounterUsed); - } + public StreamingFileSink.RowFormatBuilder withBucketCheckInterval(final long interval) { + this.bucketCheckInterval = interval; + return this; + } - @Override - public void initializeState(FunctionInitializationContext context) throws Exception { - initFileSystemWriter(); + public StreamingFileSink.RowFormatBuilder withBucketer(final Bucketer bucketer) { + this.bucketer = Preconditions.checkNotNull(bucketer); + return this; + } - this.activeBuckets = new HashMap<>(); + public StreamingFileSink.RowFormatBuilder withRollingPolicy(final RollingPolicy policy) { + this.rollingPolicy = Preconditions.checkNotNull(policy); + return this; + } - // When resuming after a failure: - // 1) we get the max part counter used before in order to make sure that we do not overwrite valid data - // 2) we commit any pending files for previous checkpoints (previous to the last successful one) - // 3) we resume writing to the previous in-progress file of each bucket, and - // 4) if we receive multiple states for the same bucket, we merge them. + public StreamingFileSink.RowFormatBuilder withBucketerAndPolicy(final Bucketer bucketer, final RollingPolicy policy) { + @SuppressWarnings("unchecked") + StreamingFileSink.RowFormatBuilder reInterpreted = (StreamingFileSink.RowFormatBuilder) this; + reInterpreted.bucketer = Preconditions.checkNotNull(bucketer); + reInterpreted.rollingPolicy = Preconditions.checkNotNull(policy); + return reInterpreted; + } - final OperatorStateStore stateStore = context.getOperatorStateStore(); + @VisibleForTesting + StreamingFileSink.RowFormatBuilder withBucketFactory(final BucketFactory factory) { + this.bucketFactory = Preconditions.checkNotNull(factory); + return this; + } - restoredBucketStates = stateStore.getListState(BUCKET_STATE_DESC); - restoredMaxCounters = stateStore.getUnionListState(MAX_PART_COUNTER_STATE_DESC); + /** Creates the actual sink. */ + public StreamingFileSink build() { + return new StreamingFileSink<>(this, bucketCheckInterval); + } - if (context.isRestored()) { - final int subtaskIndex = getRuntimeContext().getIndexOfThisSubtask(); - - LOG.info("Restoring state for the {} (taskIdx={}).", getClass().getSimpleName(), subtaskIndex); - - long maxCounter = 0L; - for (long partCounter: restoredMaxCounters.get()) { - maxCounter = Math.max(partCounter, maxCounter); - } - initMaxPartCounter = maxCounter; - - for (byte[] recoveredState : restoredBucketStates.get()) { - final BucketState bucketState = SimpleVersionedSerialization.readVersionAndDeSerialize( - bucketStateSerializer, recoveredState); - - final String bucketId = bucketState.getBucketId(); - - LOG.info("Recovered bucket for {}", bucketId); - - final Bucket restoredBucket = bucketFactory.restoreBucket( - fileSystemWriter, - subtaskIndex, - initMaxPartCounter, - encoder, - bucketState - ); - - final Bucket existingBucket = activeBuckets.get(bucketId); - if (existingBucket == null) { - activeBuckets.put(bucketId, restoredBucket); - } else { - existingBucket.merge(restoredBucket); - } - - if (LOG.isDebugEnabled()) { - LOG.debug("{} idx {} restored state for bucket {}", getClass().getSimpleName(), - subtaskIndex, assembleBucketPath(bucketId)); - } - } + @Override + Buckets createBuckets(int subtaskIndex) throws IOException { + return new Buckets<>( + basePath, + bucketer, + bucketFactory, + new RowWisePartWriter.Factory<>(encoder), + rollingPolicy, + subtaskIndex); } } - @Override - public void open(Configuration parameters) throws Exception { - super.open(parameters); + /** + * A builder for configuring the sink for bulk-encoding formats, e.g. Parquet/ORC. + */ + @PublicEvolving + public static class BulkFormatBuilder extends StreamingFileSink.BucketsBuilder { - processingTimeService = ((StreamingRuntimeContext) getRuntimeContext()).getProcessingTimeService(); - long currentProcessingTime = processingTimeService.getCurrentProcessingTime(); - processingTimeService.registerTimer(currentProcessingTime + bucketCheckInterval, this); - this.bucketerContext = new BucketerContext(); - } + private static final long serialVersionUID = 1L; - @Override - public void onProcessingTime(long timestamp) throws Exception { - final long currentTime = processingTimeService.getCurrentProcessingTime(); - for (Bucket bucket : activeBuckets.values()) { - final PartFileInfo info = bucket.getInProgressPartInfo(); - if (info != null && rollingPolicy.shouldRoll(info, currentTime)) { - bucket.closePartFile(); - } - } - processingTimeService.registerTimer(timestamp + bucketCheckInterval, this); - } + private long bucketCheckInterval = 60L * 1000L; - @Override - public void invoke(IN value, Context context) throws Exception { - final long currentProcessingTime = processingTimeService.getCurrentProcessingTime(); - final int subtaskIndex = getRuntimeContext().getIndexOfThisSubtask(); + private final Path basePath; + + private final BulkWriter.Factory writerFactory; + + private Bucketer bucketer; - // setting the values in the bucketer context - bucketerContext.update(context.timestamp(), context.currentWatermark(), currentProcessingTime); - - final String bucketId = bucketer.getBucketId(value, bucketerContext); - - Bucket bucket = activeBuckets.get(bucketId); - if (bucket == null) { - final Path bucketPath = assembleBucketPath(bucketId); - bucket = bucketFactory.getNewBucket( - fileSystemWriter, - subtaskIndex, - bucketId, - bucketPath, - initMaxPartCounter, - encoder); - activeBuckets.put(bucketId, bucket); + private BucketFactory bucketFactory = new DefaultBucketFactory<>(); + + BulkFormatBuilder(Path basePath, BulkWriter.Factory writerFactory, Bucketer bucketer) { + this.basePath = Preconditions.checkNotNull(basePath); + this.writerFactory = Preconditions.checkNotNull(writerFactory); + this.bucketer = Preconditions.checkNotNull(bucketer); } - final PartFileInfo info = bucket.getInProgressPartInfo(); - if (info == null || rollingPolicy.shouldRoll(info, currentProcessingTime)) { - bucket.rollPartFile(currentProcessingTime); + public StreamingFileSink.BulkFormatBuilder withBucketCheckInterval(long interval) { + this.bucketCheckInterval = interval; + return this; } - bucket.write(value, currentProcessingTime); - // we update the counter here because as buckets become inactive and - // get removed in the initializeState(), at the time we snapshot they - // may not be there to take them into account during checkpointing. - updateMaxPartCounter(bucket.getPartCounter()); - } + public StreamingFileSink.BulkFormatBuilder withBucketer(Bucketer bucketer) { + @SuppressWarnings("unchecked") + StreamingFileSink.BulkFormatBuilder reInterpreted = (StreamingFileSink.BulkFormatBuilder) this; + reInterpreted.bucketer = Preconditions.checkNotNull(bucketer); + return reInterpreted; + } - @Override - public void close() throws Exception { - if (activeBuckets != null) { - activeBuckets.values().forEach(Bucket::dispose); + @VisibleForTesting + StreamingFileSink.BulkFormatBuilder withBucketFactory(final BucketFactory factory) { + this.bucketFactory = Preconditions.checkNotNull(factory); + return this; } - } - private void initFileSystemWriter() throws IOException { - if (fileSystemWriter == null) { - fileSystemWriter = FileSystem.get(basePath.toUri()).createRecoverableWriter(); - bucketStateSerializer = new BucketStateSerializer( - fileSystemWriter.getResumeRecoverableSerializer(), - fileSystemWriter.getCommitRecoverableSerializer() - ); + /** Creates the actual sink. */ + public StreamingFileSink build() { + return new StreamingFileSink<>(this, bucketCheckInterval); } - } - private void updateMaxPartCounter(long candidate) { - maxPartCounterUsed = Math.max(maxPartCounterUsed, candidate); + @Override + Buckets createBuckets(int subtaskIndex) throws IOException { + return new Buckets<>( + basePath, + bucketer, + bucketFactory, + new BulkPartWriter.Factory<>(writerFactory), + new OnCheckpointRollingPolicy<>(), + subtaskIndex); + } } - private Path assembleBucketPath(String bucketId) { - return new Path(basePath, bucketId); + // --------------------------- Sink Methods ----------------------------- + + @Override + public void initializeState(FunctionInitializationContext context) throws Exception { + final int subtaskIndex = getRuntimeContext().getIndexOfThisSubtask(); + this.buckets = bucketsBuilder.createBuckets(subtaskIndex); + + final OperatorStateStore stateStore = context.getOperatorStateStore(); + bucketStates = stateStore.getListState(BUCKET_STATE_DESC); + maxPartCountersState = stateStore.getUnionListState(MAX_PART_COUNTER_STATE_DESC); + + if (context.isRestored()) { + buckets.initializeState(bucketStates, maxPartCountersState); + } } - /** - * The {@link Bucketer.Context} exposed to the - * {@link Bucketer#getBucketId(Object, Bucketer.Context)} - * whenever a new incoming element arrives. - */ - private static class BucketerContext implements Bucketer.Context { + @Override + public void notifyCheckpointComplete(long checkpointId) throws Exception { + buckets.publishUpToCheckpoint(checkpointId); + } - @Nullable - private Long elementTimestamp; + @Override + public void snapshotState(FunctionSnapshotContext context) throws Exception { + Preconditions.checkState(bucketStates != null && maxPartCountersState != null, "sink has not been initialized"); - private long currentWatermark; + bucketStates.clear(); + maxPartCountersState.clear(); - private long currentProcessingTime; + buckets.snapshotState( + context.getCheckpointId(), + context.getCheckpointTimestamp(), + bucketStates, + maxPartCountersState); + } - void update(@Nullable Long elementTimestamp, long currentWatermark, long currentProcessingTime) { - this.elementTimestamp = elementTimestamp; - this.currentWatermark = currentWatermark; - this.currentProcessingTime = currentProcessingTime; - } + @Override + public void open(Configuration parameters) throws Exception { + super.open(parameters); + this.processingTimeService = ((StreamingRuntimeContext) getRuntimeContext()).getProcessingTimeService(); + long currentProcessingTime = processingTimeService.getCurrentProcessingTime(); + processingTimeService.registerTimer(currentProcessingTime + bucketCheckInterval, this); + } - @Override - public long currentProcessingTime() { - return currentProcessingTime; - } + @Override + public void onProcessingTime(long timestamp) throws Exception { + final long currentTime = processingTimeService.getCurrentProcessingTime(); + buckets.onProcessingTime(currentTime); + processingTimeService.registerTimer(currentTime + bucketCheckInterval, this); + } - @Override - public long currentWatermark() { - return currentWatermark; - } + @Override + public void invoke(IN value, SinkFunction.Context context) throws Exception { + buckets.onElement(value, context); + } - @Override - @Nullable - public Long timestamp() { - return elementTimestamp; - } + @Override + public void close() throws Exception { + buckets.close(); } } diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/sink/filesystem/bucketers/BasePathBucketer.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/sink/filesystem/bucketers/BasePathBucketer.java index 5ffe15284cea4..d7b201325903c 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/sink/filesystem/bucketers/BasePathBucketer.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/sink/filesystem/bucketers/BasePathBucketer.java @@ -19,21 +19,28 @@ package org.apache.flink.streaming.api.functions.sink.filesystem.bucketers; import org.apache.flink.annotation.PublicEvolving; +import org.apache.flink.core.io.SimpleVersionedSerializer; /** * A {@link Bucketer} that does not perform any * bucketing of files. All files are written to the base path. */ @PublicEvolving -public class BasePathBucketer implements Bucketer { +public class BasePathBucketer implements Bucketer { private static final long serialVersionUID = -6033643155550226022L; @Override - public String getBucketId(T element, Context context) { + public String getBucketId(T element, Bucketer.Context context) { return ""; } + @Override + public SimpleVersionedSerializer getSerializer() { + // in the future this could be optimized as it is the empty string. + return SimpleVersionedStringSerializer.INSTANCE; + } + @Override public String toString() { return "BasePathBucketer"; diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/sink/filesystem/bucketers/Bucketer.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/sink/filesystem/bucketers/Bucketer.java index 5c30927705354..503e361d93a01 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/sink/filesystem/bucketers/Bucketer.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/sink/filesystem/bucketers/Bucketer.java @@ -19,6 +19,8 @@ package org.apache.flink.streaming.api.functions.sink.filesystem.bucketers; import org.apache.flink.annotation.PublicEvolving; +import org.apache.flink.core.fs.Path; +import org.apache.flink.core.io.SimpleVersionedSerializer; import org.apache.flink.streaming.api.functions.sink.SinkFunction; import javax.annotation.Nullable; @@ -33,9 +35,16 @@ *

    The {@code StreamingFileSink} can be writing to many buckets at a time, and it is responsible for managing * a set of active buckets. Whenever a new element arrives it will ask the {@code Bucketer} for the bucket the * element should fall in. The {@code Bucketer} can, for example, determine buckets based on system time. + * + * @param The type of input elements. + * @param The type of the object returned by the {@link #getBucketId(Object, Bucketer.Context)}. This has to have + * a correct {@link #hashCode()} and {@link #equals(Object)} method. In addition, the {@link Path} + * to the created bucket will be the result of the {@link #toString()} of this method, appended to + * the {@code basePath} specified in the + * {@link org.apache.flink.streaming.api.functions.sink.filesystem.StreamingFileSink StreamingFileSink} */ @PublicEvolving -public interface Bucketer extends Serializable { +public interface Bucketer extends Serializable { /** * Returns the identifier of the bucket the provided element should be put into. @@ -48,13 +57,20 @@ public interface Bucketer extends Serializable { * and the {@code base path} provided during the initialization of the * {@link org.apache.flink.streaming.api.functions.sink.filesystem.StreamingFileSink sink}. */ - String getBucketId(T element, Context context); + BucketID getBucketId(IN element, Bucketer.Context context); + + /** + * @return A {@link SimpleVersionedSerializer} capable of serializing/deserializing the elements + * of type {@code BucketID}. That is the type of the objects returned by the + * {@link #getBucketId(Object, Bucketer.Context)}. + */ + SimpleVersionedSerializer getSerializer(); /** * Context that the {@link Bucketer} can use for getting additional data about * an input record. * - *

    The context is only valid for the duration of a {@link Bucketer#getBucketId(Object, Context)} call. + *

    The context is only valid for the duration of a {@link Bucketer#getBucketId(Object, Bucketer.Context)} call. * Do not store the context and use afterwards! */ @PublicEvolving diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/sink/filesystem/bucketers/DateTimeBucketer.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/sink/filesystem/bucketers/DateTimeBucketer.java index 515468c72beee..eed0b79d0235f 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/sink/filesystem/bucketers/DateTimeBucketer.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/sink/filesystem/bucketers/DateTimeBucketer.java @@ -19,6 +19,7 @@ package org.apache.flink.streaming.api.functions.sink.filesystem.bucketers; import org.apache.flink.annotation.PublicEvolving; +import org.apache.flink.core.io.SimpleVersionedSerializer; import java.text.SimpleDateFormat; import java.util.Date; @@ -50,9 +51,9 @@ * */ @PublicEvolving -public class DateTimeBucketer implements Bucketer { +public class DateTimeBucketer implements Bucketer { - private static final long serialVersionUID = 3284420879277893804L; + private static final long serialVersionUID = 1L; private static final String DEFAULT_FORMAT_STRING = "yyyy-MM-dd--HH"; @@ -78,13 +79,18 @@ public DateTimeBucketer(String formatString) { } @Override - public String getBucketId(T element, Context context) { + public String getBucketId(IN element, Bucketer.Context context) { if (dateFormatter == null) { dateFormatter = new SimpleDateFormat(formatString); } return dateFormatter.format(new Date(context.currentProcessingTime())); } + @Override + public SimpleVersionedSerializer getSerializer() { + return SimpleVersionedStringSerializer.INSTANCE; + } + @Override public String toString() { return "DateTimeBucketer{" + diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/sink/filesystem/bucketers/SimpleVersionedStringSerializer.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/sink/filesystem/bucketers/SimpleVersionedStringSerializer.java new file mode 100644 index 0000000000000..d025af976507e --- /dev/null +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/sink/filesystem/bucketers/SimpleVersionedStringSerializer.java @@ -0,0 +1,77 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.streaming.api.functions.sink.filesystem.bucketers; + +import org.apache.flink.core.io.SimpleVersionedSerializer; + +import java.io.IOException; +import java.nio.ByteBuffer; +import java.nio.ByteOrder; +import java.nio.charset.Charset; +import java.nio.charset.StandardCharsets; + +/** + * A {@link SimpleVersionedSerializer} implementation for Strings. + */ +public final class SimpleVersionedStringSerializer implements SimpleVersionedSerializer { + + private static final Charset CHARSET = StandardCharsets.UTF_8; + + public static final SimpleVersionedStringSerializer INSTANCE = new SimpleVersionedStringSerializer(); + + @Override + public int getVersion() { + return 1; + } + + @Override + public byte[] serialize(String value) { + final byte[] serialized = value.getBytes(StandardCharsets.UTF_8); + final byte[] targetBytes = new byte[Integer.BYTES + serialized.length]; + + final ByteBuffer bb = ByteBuffer.wrap(targetBytes).order(ByteOrder.LITTLE_ENDIAN); + bb.putInt(serialized.length); + bb.put(serialized); + return targetBytes; + } + + @Override + public String deserialize(int version, byte[] serialized) throws IOException { + switch (version) { + case 1: + return deserializeV1(serialized); + default: + throw new IOException("Unrecognized version or corrupt state: " + version); + } + } + + private static String deserializeV1(byte[] serialized) { + final ByteBuffer bb = ByteBuffer.wrap(serialized).order(ByteOrder.LITTLE_ENDIAN); + final byte[] targetStringBytes = new byte[bb.getInt()]; + bb.get(targetStringBytes); + return new String(targetStringBytes, CHARSET); + } + + /** + * Private constructor to prevent instantiation. + * Access the serializer through the {@link #INSTANCE}. + */ + private SimpleVersionedStringSerializer() {} +} + diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/sink/filesystem/DefaultRollingPolicy.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/sink/filesystem/rolling/policies/DefaultRollingPolicy.java similarity index 68% rename from flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/sink/filesystem/DefaultRollingPolicy.java rename to flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/sink/filesystem/rolling/policies/DefaultRollingPolicy.java index 026ac70b79f5d..a9ff617bc223f 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/sink/filesystem/DefaultRollingPolicy.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/sink/filesystem/rolling/policies/DefaultRollingPolicy.java @@ -16,9 +16,11 @@ * limitations under the License. */ -package org.apache.flink.streaming.api.functions.sink.filesystem; +package org.apache.flink.streaming.api.functions.sink.filesystem.rolling.policies; import org.apache.flink.annotation.PublicEvolving; +import org.apache.flink.streaming.api.functions.sink.filesystem.PartFileInfo; +import org.apache.flink.streaming.api.functions.sink.filesystem.RollingPolicy; import org.apache.flink.util.Preconditions; import java.io.IOException; @@ -35,9 +37,9 @@ * */ @PublicEvolving -public final class DefaultRollingPolicy implements RollingPolicy { +public final class DefaultRollingPolicy implements RollingPolicy { - private static final long serialVersionUID = 1318929857047767030L; + private static final long serialVersionUID = 1L; private static final long DEFAULT_INACTIVITY_INTERVAL = 60L * 1000L; @@ -65,36 +67,47 @@ private DefaultRollingPolicy(long partSize, long rolloverInterval, long inactivi } @Override - public boolean shouldRoll(final PartFileInfo state, final long currentTime) throws IOException { - if (state == null) { + public boolean shouldRollOnCheckpoint(PartFileInfo partFileState) { + return false; + } + + @Override + public boolean shouldRollOnEvent(PartFileInfo partFileState) throws IOException { + if (partFileState == null) { // this means that there is no currently open part file. return true; } - if (state.getSize() > partSize) { + return partFileState.getSize() > partSize; + } + + @Override + public boolean shouldRollOnProcessingTime(final PartFileInfo partFileState, final long currentTime) { + if (partFileState == null) { + // this means that there is no currently open part file. return true; } - if (currentTime - state.getCreationTime() > rolloverInterval) { + if (currentTime - partFileState.getCreationTime() > rolloverInterval) { return true; } - return currentTime - state.getLastUpdateTime() > inactivityInterval; + return currentTime - partFileState.getLastUpdateTime() > inactivityInterval; } /** - * Initiates the instantiation of a {@link DefaultRollingPolicy}. + * Initiates the instantiation of a {@code DefaultRollingPolicy}. * To finalize it and have the actual policy, call {@code .create()}. */ - public static PolicyBuilder create() { - return new PolicyBuilder(); + public static DefaultRollingPolicy.PolicyBuilder create() { + return new DefaultRollingPolicy.PolicyBuilder(); } /** * A helper class that holds the configuration properties for the {@link DefaultRollingPolicy}. */ @PublicEvolving - public static class PolicyBuilder { + public static final class PolicyBuilder { private long partSize = DEFAULT_MAX_PART_SIZE; @@ -102,11 +115,13 @@ public static class PolicyBuilder { private long inactivityInterval = DEFAULT_INACTIVITY_INTERVAL; + private PolicyBuilder() {} + /** * Sets the part size above which a part file will have to roll. * @param size the allowed part size. */ - public PolicyBuilder withMaxPartSize(long size) { + public DefaultRollingPolicy.PolicyBuilder withMaxPartSize(long size) { Preconditions.checkState(size > 0L); this.partSize = size; return this; @@ -116,7 +131,7 @@ public PolicyBuilder withMaxPartSize(long size) { * Sets the interval of allowed inactivity after which a part file will have to roll. * @param interval the allowed inactivity interval. */ - public PolicyBuilder withInactivityInterval(long interval) { + public DefaultRollingPolicy.PolicyBuilder withInactivityInterval(long interval) { Preconditions.checkState(interval > 0L); this.inactivityInterval = interval; return this; @@ -126,7 +141,7 @@ public PolicyBuilder withInactivityInterval(long interval) { * Sets the max time a part file can stay open before having to roll. * @param interval the desired rollover interval. */ - public PolicyBuilder withRolloverInterval(long interval) { + public DefaultRollingPolicy.PolicyBuilder withRolloverInterval(long interval) { Preconditions.checkState(interval > 0L); this.rolloverInterval = interval; return this; @@ -135,8 +150,8 @@ public PolicyBuilder withRolloverInterval(long interval) { /** * Creates the actual policy. */ - public DefaultRollingPolicy build() { - return new DefaultRollingPolicy(partSize, rolloverInterval, inactivityInterval); + public DefaultRollingPolicy build() { + return new DefaultRollingPolicy<>(partSize, rolloverInterval, inactivityInterval); } } } diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/sink/filesystem/rolling/policies/OnCheckpointRollingPolicy.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/sink/filesystem/rolling/policies/OnCheckpointRollingPolicy.java new file mode 100644 index 0000000000000..4361941951a3c --- /dev/null +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/sink/filesystem/rolling/policies/OnCheckpointRollingPolicy.java @@ -0,0 +1,45 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.streaming.api.functions.sink.filesystem.rolling.policies; + +import org.apache.flink.streaming.api.functions.sink.filesystem.PartFileInfo; +import org.apache.flink.streaming.api.functions.sink.filesystem.RollingPolicy; + +/** + * A {@link RollingPolicy} which rolls on every checkpoint. + */ +public class OnCheckpointRollingPolicy implements RollingPolicy { + + private static final long serialVersionUID = 1L; + + @Override + public boolean shouldRollOnCheckpoint(PartFileInfo partFileState) { + return true; + } + + @Override + public boolean shouldRollOnEvent(PartFileInfo partFileState) { + return false; + } + + @Override + public boolean shouldRollOnProcessingTime(PartFileInfo partFileState, long currentTime) { + return false; + } +} diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/functions/sink/filesystem/BucketStateSerializerTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/functions/sink/filesystem/BucketStateSerializerTest.java index 353ac00fd9020..3d5be6340ff15 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/functions/sink/filesystem/BucketStateSerializerTest.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/functions/sink/filesystem/BucketStateSerializerTest.java @@ -25,6 +25,7 @@ import org.apache.flink.core.fs.RecoverableWriter; import org.apache.flink.core.io.SimpleVersionedSerialization; import org.apache.flink.core.io.SimpleVersionedSerializer; +import org.apache.flink.streaming.api.functions.sink.filesystem.bucketers.SimpleVersionedStringSerializer; import org.junit.Assert; import org.junit.ClassRule; @@ -60,20 +61,21 @@ public void testSerializationEmpty() throws IOException { final Path testBucket = new Path(testFolder.getPath(), "test"); - final BucketState bucketState = new BucketState( + final BucketState bucketState = new BucketState<>( "test", testBucket, Long.MAX_VALUE, null, new HashMap<>()); - final SimpleVersionedSerializer serializer = - new BucketStateSerializer( + final SimpleVersionedSerializer> serializer = + new BucketStateSerializer<>( writer.getResumeRecoverableSerializer(), - writer.getCommitRecoverableSerializer() + writer.getCommitRecoverableSerializer(), + SimpleVersionedStringSerializer.INSTANCE ); byte[] bytes = SimpleVersionedSerialization.writeVersionAndSerialize(serializer, bucketState); - final BucketState recoveredState = SimpleVersionedSerialization.readVersionAndDeSerialize(serializer, bytes); + final BucketState recoveredState = SimpleVersionedSerialization.readVersionAndDeSerialize(serializer, bytes); Assert.assertEquals(testBucket, recoveredState.getBucketPath()); - Assert.assertNull(recoveredState.getCurrentInProgress()); + Assert.assertNull(recoveredState.getInProgress()); Assert.assertTrue(recoveredState.getPendingPerCheckpoint().isEmpty()); } @@ -90,13 +92,14 @@ public void testSerializationOnlyInProgress() throws IOException { final RecoverableWriter.ResumeRecoverable current = stream.persist(); - final BucketState bucketState = new BucketState( + final BucketState bucketState = new BucketState<>( "test", testBucket, Long.MAX_VALUE, current, new HashMap<>()); - final SimpleVersionedSerializer serializer = - new BucketStateSerializer( + final SimpleVersionedSerializer> serializer = + new BucketStateSerializer<>( writer.getResumeRecoverableSerializer(), - writer.getCommitRecoverableSerializer() + writer.getCommitRecoverableSerializer(), + SimpleVersionedStringSerializer.INSTANCE ); final byte[] bytes = SimpleVersionedSerialization.writeVersionAndSerialize(serializer, bucketState); @@ -104,7 +107,7 @@ public void testSerializationOnlyInProgress() throws IOException { // to simulate that everything is over for file. stream.close(); - final BucketState recoveredState = SimpleVersionedSerialization.readVersionAndDeSerialize(serializer, bytes); + final BucketState recoveredState = SimpleVersionedSerialization.readVersionAndDeSerialize(serializer, bytes); Assert.assertEquals(testBucket, recoveredState.getBucketPath()); @@ -147,18 +150,19 @@ public void testSerializationFull() throws IOException { final RecoverableWriter.ResumeRecoverable current = stream.persist(); - final BucketState bucketState = new BucketState( + final BucketState bucketState = new BucketState<>( "test-2", bucketPath, Long.MAX_VALUE, current, commitRecoverables); - final SimpleVersionedSerializer serializer = - new BucketStateSerializer( + final SimpleVersionedSerializer> serializer = + new BucketStateSerializer<>( writer.getResumeRecoverableSerializer(), - writer.getCommitRecoverableSerializer() + writer.getCommitRecoverableSerializer(), + SimpleVersionedStringSerializer.INSTANCE ); stream.close(); byte[] bytes = SimpleVersionedSerialization.writeVersionAndSerialize(serializer, bucketState); - final BucketState recoveredState = SimpleVersionedSerialization.readVersionAndDeSerialize(serializer, bytes); + final BucketState recoveredState = SimpleVersionedSerialization.readVersionAndDeSerialize(serializer, bytes); Assert.assertEquals(bucketPath, recoveredState.getBucketPath()); @@ -220,20 +224,21 @@ public void testSerializationNullInProgress() throws IOException { final RecoverableWriter.ResumeRecoverable current = null; - final BucketState bucketState = new BucketState( + final BucketState bucketState = new BucketState<>( "", bucketPath, Long.MAX_VALUE, current, commitRecoverables); - final SimpleVersionedSerializer serializer = new BucketStateSerializer( + final SimpleVersionedSerializer> serializer = new BucketStateSerializer<>( writer.getResumeRecoverableSerializer(), - writer.getCommitRecoverableSerializer() + writer.getCommitRecoverableSerializer(), + SimpleVersionedStringSerializer.INSTANCE ); byte[] bytes = SimpleVersionedSerialization.writeVersionAndSerialize(serializer, bucketState); - final BucketState recoveredState = SimpleVersionedSerialization.readVersionAndDeSerialize(serializer, bytes); + final BucketState recoveredState = SimpleVersionedSerialization.readVersionAndDeSerialize(serializer, bytes); Assert.assertEquals(bucketPath, recoveredState.getBucketPath()); - Assert.assertNull(recoveredState.getCurrentInProgress()); + Assert.assertNull(recoveredState.getInProgress()); final Map> recoveredRecoverables = recoveredState.getPendingPerCheckpoint(); Assert.assertEquals(5L, recoveredRecoverables.size()); diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/functions/sink/filesystem/BucketsTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/functions/sink/filesystem/BucketsTest.java new file mode 100644 index 0000000000000..042ba4ec9ccf6 --- /dev/null +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/functions/sink/filesystem/BucketsTest.java @@ -0,0 +1,119 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.streaming.api.functions.sink.filesystem; + +import org.apache.flink.api.common.serialization.SimpleStringEncoder; +import org.apache.flink.core.fs.Path; +import org.apache.flink.core.io.SimpleVersionedSerializer; +import org.apache.flink.streaming.api.functions.sink.SinkFunction; +import org.apache.flink.streaming.api.functions.sink.filesystem.bucketers.Bucketer; +import org.apache.flink.streaming.api.functions.sink.filesystem.bucketers.SimpleVersionedStringSerializer; + +import org.junit.Assert; +import org.junit.ClassRule; +import org.junit.Test; +import org.junit.rules.TemporaryFolder; + +import java.io.File; + +/** + * Tests for {@link Buckets}. + */ +public class BucketsTest { + + @ClassRule + public static final TemporaryFolder TEMP_FOLDER = new TemporaryFolder(); + + @Test + public void testContextPassingNormalExecution() throws Exception { + testCorrectPassingOfContext(1L, 2L, 3L); + } + + @Test + public void testContextPassingNullTimestamp() throws Exception { + testCorrectPassingOfContext(null, 2L, 3L); + } + + private void testCorrectPassingOfContext(Long timestamp, long watermark, long processingTime) throws Exception { + final File outDir = TEMP_FOLDER.newFolder(); + + final Long expectedTimestamp = timestamp; + final long expectedWatermark = watermark; + final long expectedProcessingTime = processingTime; + + final Buckets buckets = StreamingFileSink + .forRowFormat(new Path(outDir.toURI()), new SimpleStringEncoder<>()) + .withBucketer(new VarifyingBucketer(expectedTimestamp, expectedWatermark, expectedProcessingTime)) + .createBuckets(2); + + buckets.onElement("TEST", new SinkFunction.Context() { + @Override + public long currentProcessingTime() { + return expectedProcessingTime; + } + + @Override + public long currentWatermark() { + return expectedWatermark; + } + + @Override + public Long timestamp() { + return expectedTimestamp; + } + }); + } + + private static class VarifyingBucketer implements Bucketer { + + private static final long serialVersionUID = 7729086510972377578L; + + private final Long expectedTimestamp; + private final long expectedWatermark; + private final long expectedProcessingTime; + + VarifyingBucketer( + final Long expectedTimestamp, + final long expectedWatermark, + final long expectedProcessingTime + ) { + this.expectedTimestamp = expectedTimestamp; + this.expectedWatermark = expectedWatermark; + this.expectedProcessingTime = expectedProcessingTime; + } + + @Override + public String getBucketId(String element, Context context) { + final Long elementTimestamp = context.timestamp(); + final long watermark = context.currentWatermark(); + final long processingTime = context.currentProcessingTime(); + + Assert.assertEquals(expectedTimestamp, elementTimestamp); + Assert.assertEquals(expectedProcessingTime, processingTime); + Assert.assertEquals(expectedWatermark, watermark); + + return element; + } + + @Override + public SimpleVersionedSerializer getSerializer() { + return SimpleVersionedStringSerializer.INSTANCE; + } + } +} diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/functions/sink/filesystem/BulkWriterTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/functions/sink/filesystem/BulkWriterTest.java new file mode 100644 index 0000000000000..7b6b82cd4eb72 --- /dev/null +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/functions/sink/filesystem/BulkWriterTest.java @@ -0,0 +1,144 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.streaming.api.functions.sink.filesystem; + +import org.apache.flink.api.common.serialization.BulkWriter; +import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.core.fs.FSDataOutputStream; +import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; +import org.apache.flink.streaming.util.OneInputStreamOperatorTestHarness; +import org.apache.flink.util.Preconditions; +import org.apache.flink.util.TestLogger; + +import org.junit.Assert; +import org.junit.ClassRule; +import org.junit.Test; +import org.junit.rules.TemporaryFolder; + +import java.io.File; +import java.io.IOException; +import java.nio.charset.Charset; +import java.nio.charset.StandardCharsets; +import java.util.Map; + +/** + * Tests for the {@link StreamingFileSink} with {@link BulkWriter}. + */ +public class BulkWriterTest extends TestLogger { + + @ClassRule + public static final TemporaryFolder TEMP_FOLDER = new TemporaryFolder(); + + @Test + public void testCustomBulkWriter() throws Exception { + final File outDir = TEMP_FOLDER.newFolder(); + + // we set the max bucket size to small so that we can know when it rolls + try ( + OneInputStreamOperatorTestHarness, Object> testHarness = + TestUtils.createTestSinkWithBulkEncoder( + outDir, + 1, + 0, + 10L, + new TestUtils.TupleToStringBucketer(), + new TestBulkWriterFactory(), + new DefaultBucketFactory<>()) + ) { + + testHarness.setup(); + testHarness.open(); + + // this creates a new bucket "test1" and part-0-0 + testHarness.processElement(new StreamRecord<>(Tuple2.of("test1", 1), 1L)); + TestUtils.checkLocalFs(outDir, 1, 0); + + // we take a checkpoint so we roll. + testHarness.snapshot(1L, 1L); + + // these will close part-0-0 and open part-0-1 + testHarness.processElement(new StreamRecord<>(Tuple2.of("test1", 2), 2L)); + testHarness.processElement(new StreamRecord<>(Tuple2.of("test1", 3), 3L)); + + // we take a checkpoint so we roll again. + testHarness.snapshot(2L, 2L); + + TestUtils.checkLocalFs(outDir, 2, 0); + + Map contents = TestUtils.getFileContentByPath(outDir); + int fileCounter = 0; + for (Map.Entry fileContents : contents.entrySet()) { + if (fileContents.getKey().getName().contains(".part-0-0.inprogress")) { + fileCounter++; + Assert.assertEquals("test1@1\n", fileContents.getValue()); + } else if (fileContents.getKey().getName().contains(".part-0-1.inprogress")) { + fileCounter++; + Assert.assertEquals("test1@2\ntest1@3\n", fileContents.getValue()); + } + } + Assert.assertEquals(2L, fileCounter); + + // we acknowledge the latest checkpoint, so everything should be published. + testHarness.notifyOfCompletedCheckpoint(2L); + TestUtils.checkLocalFs(outDir, 0, 2); + } + } + + /** + * A {@link BulkWriter} used for the tests. + */ + private static class TestBulkWriter implements BulkWriter> { + + private static final Charset CHARSET = StandardCharsets.UTF_8; + + private final FSDataOutputStream stream; + + TestBulkWriter(final FSDataOutputStream stream) { + this.stream = Preconditions.checkNotNull(stream); + } + + @Override + public void addElement(Tuple2 element) throws IOException { + stream.write((element.f0 + '@' + element.f1 + '\n').getBytes(CHARSET)); + } + + @Override + public void flush() throws IOException { + stream.flush(); + } + + @Override + public void finish() throws IOException { + flush(); + } + } + + /** + * A {@link BulkWriter.Factory} used for the tests. + */ + private static class TestBulkWriterFactory implements BulkWriter.Factory> { + + private static final long serialVersionUID = 1L; + + @Override + public BulkWriter> create(FSDataOutputStream out) { + return new TestBulkWriter(out); + } + } +} diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/functions/sink/filesystem/LocalStreamingFileSinkTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/functions/sink/filesystem/LocalStreamingFileSinkTest.java index b6f73ac11cb85..6e942e95d4637 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/functions/sink/filesystem/LocalStreamingFileSinkTest.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/functions/sink/filesystem/LocalStreamingFileSinkTest.java @@ -18,20 +18,17 @@ package org.apache.flink.streaming.api.functions.sink.filesystem; -import org.apache.flink.api.common.serialization.Encoder; import org.apache.flink.api.common.serialization.SimpleStringEncoder; import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.core.fs.Path; import org.apache.flink.core.fs.RecoverableWriter; import org.apache.flink.runtime.checkpoint.OperatorSubtaskState; -import org.apache.flink.streaming.api.functions.sink.filesystem.bucketers.Bucketer; -import org.apache.flink.streaming.api.operators.StreamSink; +import org.apache.flink.streaming.api.functions.sink.filesystem.rolling.policies.DefaultRollingPolicy; import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; import org.apache.flink.streaming.util.AbstractStreamOperatorTestHarness; import org.apache.flink.streaming.util.OneInputStreamOperatorTestHarness; import org.apache.flink.util.TestLogger; -import org.apache.commons.io.FileUtils; import org.junit.Assert; import org.junit.ClassRule; import org.junit.Test; @@ -39,9 +36,6 @@ import java.io.File; import java.io.IOException; -import java.nio.charset.StandardCharsets; -import java.util.Collection; -import java.util.HashMap; import java.util.Map; /** @@ -56,12 +50,13 @@ public class LocalStreamingFileSinkTest extends TestLogger { public void testClosingWithoutInput() throws Exception { final File outDir = TEMP_FOLDER.newFolder(); - OneInputStreamOperatorTestHarness, Object> testHarness = - createRescalingTestSink(outDir, 1, 0, 100L, 124L); - testHarness.setup(); - testHarness.open(); - - testHarness.close(); + try ( + OneInputStreamOperatorTestHarness, Object> testHarness = + TestUtils.createRescalingTestSink(outDir, 1, 0, 100L, 124L); + ) { + testHarness.setup(); + testHarness.open(); + } } @Test @@ -70,7 +65,7 @@ public void testTruncateAfterRecoveryAndOverwrite() throws Exception { OperatorSubtaskState snapshot; // we set the max bucket size to small so that we can know when it rolls - try (OneInputStreamOperatorTestHarness, Object> testHarness = createRescalingTestSink( + try (OneInputStreamOperatorTestHarness, Object> testHarness = TestUtils.createRescalingTestSink( outDir, 1, 0, 100L, 10L)) { testHarness.setup(); @@ -78,7 +73,7 @@ public void testTruncateAfterRecoveryAndOverwrite() throws Exception { // this creates a new bucket "test1" and part-0-0 testHarness.processElement(new StreamRecord<>(Tuple2.of("test1", 1), 1L)); - checkLocalFs(outDir, 1, 0); + TestUtils.checkLocalFs(outDir, 1, 0); // we take a checkpoint so that we keep the in-progress file offset. snapshot = testHarness.snapshot(1L, 1L); @@ -87,9 +82,9 @@ public void testTruncateAfterRecoveryAndOverwrite() throws Exception { testHarness.processElement(new StreamRecord<>(Tuple2.of("test1", 2), 2L)); testHarness.processElement(new StreamRecord<>(Tuple2.of("test1", 3), 3L)); - checkLocalFs(outDir, 2, 0); + TestUtils.checkLocalFs(outDir, 2, 0); - Map contents = getFileContentByPath(outDir); + Map contents = TestUtils.getFileContentByPath(outDir); int fileCounter = 0; for (Map.Entry fileContents : contents.entrySet()) { if (fileContents.getKey().getName().contains(".part-0-0.inprogress")) { @@ -103,7 +98,7 @@ public void testTruncateAfterRecoveryAndOverwrite() throws Exception { Assert.assertEquals(2L, fileCounter); } - try (OneInputStreamOperatorTestHarness, Object> testHarness = createRescalingTestSink( + try (OneInputStreamOperatorTestHarness, Object> testHarness = TestUtils.createRescalingTestSink( outDir, 1, 0, 100L, 10L)) { testHarness.setup(); @@ -111,11 +106,11 @@ public void testTruncateAfterRecoveryAndOverwrite() throws Exception { testHarness.open(); // the in-progress is the not cleaned up one and the pending is truncated and finalized - checkLocalFs(outDir, 2, 0); + TestUtils.checkLocalFs(outDir, 2, 0); // now we go back to the first checkpoint so it should truncate part-0-0 and restart part-0-1 int fileCounter = 0; - for (Map.Entry fileContents : getFileContentByPath(outDir).entrySet()) { + for (Map.Entry fileContents : TestUtils.getFileContentByPath(outDir).entrySet()) { if (fileContents.getKey().getName().contains(".part-0-0.inprogress")) { // truncated fileCounter++; @@ -132,7 +127,7 @@ public void testTruncateAfterRecoveryAndOverwrite() throws Exception { testHarness.processElement(new StreamRecord<>(Tuple2.of("test1", 4), 4L)); fileCounter = 0; - for (Map.Entry fileContents : getFileContentByPath(outDir).entrySet()) { + for (Map.Entry fileContents : TestUtils.getFileContentByPath(outDir).entrySet()) { if (fileContents.getKey().getName().contains(".part-0-0.inprogress")) { fileCounter++; Assert.assertEquals("test1@1\ntest1@4\n", fileContents.getValue()); @@ -145,16 +140,16 @@ public void testTruncateAfterRecoveryAndOverwrite() throws Exception { Assert.assertEquals(2L, fileCounter); testHarness.processElement(new StreamRecord<>(Tuple2.of("test1", 5), 5L)); - checkLocalFs(outDir, 3, 0); // the previous part-0-1 in progress is simply ignored (random extension) + TestUtils.checkLocalFs(outDir, 3, 0); // the previous part-0-1 in progress is simply ignored (random extension) testHarness.snapshot(2L, 2L); // this will close the new part-0-1 testHarness.processElement(new StreamRecord<>(Tuple2.of("test1", 6), 6L)); - checkLocalFs(outDir, 3, 0); + TestUtils.checkLocalFs(outDir, 3, 0); fileCounter = 0; - for (Map.Entry fileContents : getFileContentByPath(outDir).entrySet()) { + for (Map.Entry fileContents : TestUtils.getFileContentByPath(outDir).entrySet()) { if (fileContents.getKey().getName().contains(".part-0-0.inprogress")) { fileCounter++; Assert.assertEquals("test1@1\ntest1@4\n", fileContents.getValue()); @@ -169,10 +164,10 @@ public void testTruncateAfterRecoveryAndOverwrite() throws Exception { // this will publish part-0-0 testHarness.notifyOfCompletedCheckpoint(2L); - checkLocalFs(outDir, 2, 1); + TestUtils.checkLocalFs(outDir, 2, 1); fileCounter = 0; - for (Map.Entry fileContents : getFileContentByPath(outDir).entrySet()) { + for (Map.Entry fileContents : TestUtils.getFileContentByPath(outDir).entrySet()) { if (fileContents.getKey().getName().equals("part-0-0")) { fileCounter++; Assert.assertEquals("test1@1\ntest1@4\n", fileContents.getValue()); @@ -192,7 +187,7 @@ public void testCommitStagedFilesInCorrectOrder() throws Exception { final File outDir = TEMP_FOLDER.newFolder(); // we set the max bucket size to small so that we can know when it rolls - try (OneInputStreamOperatorTestHarness, Object> testHarness = createRescalingTestSink( + try (OneInputStreamOperatorTestHarness, Object> testHarness = TestUtils.createRescalingTestSink( outDir, 1, 0, 100L, 10L)) { testHarness.setup(); @@ -203,11 +198,11 @@ public void testCommitStagedFilesInCorrectOrder() throws Exception { // these 2 create a new bucket "test1", with a .part-0-0.inprogress and also fill it testHarness.processElement(new StreamRecord<>(Tuple2.of("test1", 1), 1L)); testHarness.processElement(new StreamRecord<>(Tuple2.of("test1", 2), 2L)); - checkLocalFs(outDir, 1, 0); + TestUtils.checkLocalFs(outDir, 1, 0); // this will open .part-0-1.inprogress testHarness.processElement(new StreamRecord<>(Tuple2.of("test1", 3), 3L)); - checkLocalFs(outDir, 2, 0); + TestUtils.checkLocalFs(outDir, 2, 0); // we take a checkpoint so that we keep the in-progress file offset. testHarness.snapshot(1L, 1L); @@ -218,13 +213,13 @@ public void testCommitStagedFilesInCorrectOrder() throws Exception { // and open and fill .part-0-2.inprogress testHarness.processElement(new StreamRecord<>(Tuple2.of("test1", 5), 5L)); testHarness.processElement(new StreamRecord<>(Tuple2.of("test1", 6), 6L)); - checkLocalFs(outDir, 3, 0); // nothing committed yet + TestUtils.checkLocalFs(outDir, 3, 0); // nothing committed yet testHarness.snapshot(2L, 2L); // open .part-0-3.inprogress testHarness.processElement(new StreamRecord<>(Tuple2.of("test1", 7), 7L)); - checkLocalFs(outDir, 4, 0); + TestUtils.checkLocalFs(outDir, 4, 0); // this will close the part file (time) testHarness.setProcessingTime(101L); @@ -232,10 +227,10 @@ public void testCommitStagedFilesInCorrectOrder() throws Exception { testHarness.snapshot(3L, 3L); testHarness.notifyOfCompletedCheckpoint(1L); // the pending for checkpoint 1 are committed - checkLocalFs(outDir, 3, 1); + TestUtils.checkLocalFs(outDir, 3, 1); int fileCounter = 0; - for (Map.Entry fileContents : getFileContentByPath(outDir).entrySet()) { + for (Map.Entry fileContents : TestUtils.getFileContentByPath(outDir).entrySet()) { if (fileContents.getKey().getName().equals("part-0-0")) { fileCounter++; Assert.assertEquals("test1@1\ntest1@2\n", fileContents.getValue()); @@ -253,10 +248,10 @@ public void testCommitStagedFilesInCorrectOrder() throws Exception { Assert.assertEquals(4L, fileCounter); testHarness.notifyOfCompletedCheckpoint(3L); // all the pending for checkpoint 2 and 3 are committed - checkLocalFs(outDir, 0, 4); + TestUtils.checkLocalFs(outDir, 0, 4); fileCounter = 0; - for (Map.Entry fileContents : getFileContentByPath(outDir).entrySet()) { + for (Map.Entry fileContents : TestUtils.getFileContentByPath(outDir).entrySet()) { if (fileContents.getKey().getName().equals("part-0-0")) { fileCounter++; Assert.assertEquals("test1@1\ntest1@2\n", fileContents.getValue()); @@ -280,7 +275,7 @@ public void testInactivityPeriodWithLateNotify() throws Exception { final File outDir = TEMP_FOLDER.newFolder(); // we set a big bucket size so that it does not close by size, but by timers. - try (OneInputStreamOperatorTestHarness, Object> testHarness = createRescalingTestSink( + try (OneInputStreamOperatorTestHarness, Object> testHarness = TestUtils.createRescalingTestSink( outDir, 1, 0, 100L, 124L)) { testHarness.setup(); @@ -290,10 +285,10 @@ public void testInactivityPeriodWithLateNotify() throws Exception { testHarness.processElement(new StreamRecord<>(Tuple2.of("test1", 1), 1L)); testHarness.processElement(new StreamRecord<>(Tuple2.of("test2", 1), 1L)); - checkLocalFs(outDir, 2, 0); + TestUtils.checkLocalFs(outDir, 2, 0); int bucketCounter = 0; - for (Map.Entry fileContents : getFileContentByPath(outDir).entrySet()) { + for (Map.Entry fileContents : TestUtils.getFileContentByPath(outDir).entrySet()) { if (fileContents.getKey().getParentFile().getName().equals("test1")) { bucketCounter++; } else if (fileContents.getKey().getParentFile().getName().equals("test2")) { @@ -303,10 +298,10 @@ public void testInactivityPeriodWithLateNotify() throws Exception { Assert.assertEquals(2L, bucketCounter); // verifies that we have 2 buckets, "test1" and "test2" testHarness.setProcessingTime(101L); // put them in pending - checkLocalFs(outDir, 2, 0); + TestUtils.checkLocalFs(outDir, 2, 0); testHarness.snapshot(0L, 0L); // put them in pending for 0 - checkLocalFs(outDir, 2, 0); + TestUtils.checkLocalFs(outDir, 2, 0); // create another 2 buckets with 1 inprogress file each testHarness.processElement(new StreamRecord<>(Tuple2.of("test3", 1), 1L)); @@ -315,13 +310,13 @@ public void testInactivityPeriodWithLateNotify() throws Exception { testHarness.setProcessingTime(202L); // put them in pending testHarness.snapshot(1L, 0L); // put them in pending for 1 - checkLocalFs(outDir, 4, 0); + TestUtils.checkLocalFs(outDir, 4, 0); testHarness.notifyOfCompletedCheckpoint(0L); // put the pending for 0 to the "committed" state - checkLocalFs(outDir, 2, 2); + TestUtils.checkLocalFs(outDir, 2, 2); bucketCounter = 0; - for (Map.Entry fileContents : getFileContentByPath(outDir).entrySet()) { + for (Map.Entry fileContents : TestUtils.getFileContentByPath(outDir).entrySet()) { if (fileContents.getKey().getParentFile().getName().equals("test1")) { bucketCounter++; Assert.assertEquals("part-0-0", fileContents.getKey().getName()); @@ -339,10 +334,10 @@ public void testInactivityPeriodWithLateNotify() throws Exception { Assert.assertEquals(4L, bucketCounter); testHarness.notifyOfCompletedCheckpoint(1L); // put the pending for 1 to the "committed" state - checkLocalFs(outDir, 0, 4); + TestUtils.checkLocalFs(outDir, 0, 4); bucketCounter = 0; - for (Map.Entry fileContents : getFileContentByPath(outDir).entrySet()) { + for (Map.Entry fileContents : TestUtils.getFileContentByPath(outDir).entrySet()) { if (fileContents.getKey().getParentFile().getName().equals("test1")) { bucketCounter++; Assert.assertEquals("test1@1\n", fileContents.getValue()); @@ -367,9 +362,10 @@ public void testInactivityPeriodWithLateNotify() throws Exception { public void testClosingOnSnapshot() throws Exception { final File outDir = TEMP_FOLDER.newFolder(); - try (OneInputStreamOperatorTestHarness, Object> testHarness = - createRescalingTestSink(outDir, 1, 0, 100L, 2L)) { - + try ( + OneInputStreamOperatorTestHarness, Object> testHarness = + TestUtils.createRescalingTestSink(outDir, 1, 0, 100L, 2L) + ) { testHarness.setup(); testHarness.open(); @@ -377,29 +373,29 @@ public void testClosingOnSnapshot() throws Exception { testHarness.processElement(new StreamRecord<>(Tuple2.of("test1", 1), 1L)); testHarness.processElement(new StreamRecord<>(Tuple2.of("test2", 1), 1L)); - checkLocalFs(outDir, 2, 0); + TestUtils.checkLocalFs(outDir, 2, 0); // this is to check the inactivity threshold testHarness.setProcessingTime(101L); - checkLocalFs(outDir, 2, 0); + TestUtils.checkLocalFs(outDir, 2, 0); testHarness.processElement(new StreamRecord<>(Tuple2.of("test3", 1), 1L)); - checkLocalFs(outDir, 3, 0); + TestUtils.checkLocalFs(outDir, 3, 0); testHarness.snapshot(0L, 1L); - checkLocalFs(outDir, 3, 0); + TestUtils.checkLocalFs(outDir, 3, 0); testHarness.notifyOfCompletedCheckpoint(0L); - checkLocalFs(outDir, 0, 3); + TestUtils.checkLocalFs(outDir, 0, 3); testHarness.snapshot(1L, 0L); testHarness.processElement(new StreamRecord<>(Tuple2.of("test4", 10), 10L)); - checkLocalFs(outDir, 1, 3); + TestUtils.checkLocalFs(outDir, 1, 3); } // at close it is not moved to final. - checkLocalFs(outDir, 1, 3); + TestUtils.checkLocalFs(outDir, 1, 3); } @Test @@ -408,11 +404,11 @@ public void testScalingDownAndMergingOfStates() throws Exception { OperatorSubtaskState mergedSnapshot; - // we set small file size so that the part file rolls. + // we set small file size so that the part file rolls on every element. try ( - OneInputStreamOperatorTestHarness, Object> testHarness1 = createRescalingTestSink( + OneInputStreamOperatorTestHarness, Object> testHarness1 = TestUtils.createRescalingTestSink( outDir, 2, 0, 100L, 10L); - OneInputStreamOperatorTestHarness, Object> testHarness2 = createRescalingTestSink( + OneInputStreamOperatorTestHarness, Object> testHarness2 = TestUtils.createRescalingTestSink( outDir, 2, 1, 100L, 10L) ) { testHarness1.setup(); @@ -422,16 +418,16 @@ public void testScalingDownAndMergingOfStates() throws Exception { testHarness2.open(); testHarness1.processElement(new StreamRecord<>(Tuple2.of("test1", 0), 0L)); - checkLocalFs(outDir, 1, 0); + TestUtils.checkLocalFs(outDir, 1, 0); testHarness2.processElement(new StreamRecord<>(Tuple2.of("test1", 1), 1L)); testHarness2.processElement(new StreamRecord<>(Tuple2.of("test2", 1), 1L)); // all the files are in-progress - checkLocalFs(outDir, 3, 0); + TestUtils.checkLocalFs(outDir, 3, 0); int counter = 0; - for (Map.Entry fileContents : getFileContentByPath(outDir).entrySet()) { + for (Map.Entry fileContents : TestUtils.getFileContentByPath(outDir).entrySet()) { final String parentFilename = fileContents.getKey().getParentFile().getName(); final String inProgressFilename = fileContents.getKey().getName(); @@ -456,7 +452,7 @@ public void testScalingDownAndMergingOfStates() throws Exception { } try ( - OneInputStreamOperatorTestHarness, Object> testHarness = createRescalingTestSink( + OneInputStreamOperatorTestHarness, Object> testHarness = TestUtils.createRescalingTestSink( outDir, 1, 0, 100L, 10L) ) { testHarness.setup(); @@ -464,13 +460,13 @@ public void testScalingDownAndMergingOfStates() throws Exception { testHarness.open(); // still everything in-progress but the in-progress for prev task 1 should be put in pending now - checkLocalFs(outDir, 3, 0); + TestUtils.checkLocalFs(outDir, 3, 0); testHarness.snapshot(2L, 2L); testHarness.notifyOfCompletedCheckpoint(2L); int counter = 0; - for (Map.Entry fileContents : getFileContentByPath(outDir).entrySet()) { + for (Map.Entry fileContents : TestUtils.getFileContentByPath(outDir).entrySet()) { final String parentFilename = fileContents.getKey().getParentFile().getName(); final String filename = fileContents.getKey().getName(); @@ -498,11 +494,18 @@ public void testMaxCounterUponRecovery() throws Exception { final TestBucketFactory first = new TestBucketFactory(); final TestBucketFactory second = new TestBucketFactory(); + final RollingPolicy rollingPolicy = DefaultRollingPolicy + .create() + .withMaxPartSize(2L) + .withRolloverInterval(100L) + .withInactivityInterval(100L) + .build(); + try ( - OneInputStreamOperatorTestHarness, Object> testHarness1 = createCustomRescalingTestSink( - outDir, 2, 0, 100L, 2L, first, new SimpleStringEncoder<>()); - OneInputStreamOperatorTestHarness, Object> testHarness2 = createCustomRescalingTestSink( - outDir, 2, 1, 100L, 2L, second, new SimpleStringEncoder<>()) + OneInputStreamOperatorTestHarness, Object> testHarness1 = TestUtils.createCustomRescalingTestSink( + outDir, 2, 0, 10L, new TestUtils.TupleToStringBucketer(), new SimpleStringEncoder<>(), rollingPolicy, first); + OneInputStreamOperatorTestHarness, Object> testHarness2 = TestUtils.createCustomRescalingTestSink( + outDir, 2, 1, 10L, new TestUtils.TupleToStringBucketer(), new SimpleStringEncoder<>(), rollingPolicy, second) ) { testHarness1.setup(); testHarness1.open(); @@ -514,7 +517,7 @@ public void testMaxCounterUponRecovery() throws Exception { testHarness1.processElement(new StreamRecord<>(Tuple2.of("test1", 0), 0L)); testHarness1.processElement(new StreamRecord<>(Tuple2.of("test1", 0), 0L)); testHarness1.processElement(new StreamRecord<>(Tuple2.of("test1", 0), 0L)); - checkLocalFs(outDir, 3, 0); + TestUtils.checkLocalFs(outDir, 3, 0); // intentionally we snapshot them in the reverse order so that the states are shuffled mergedSnapshot = AbstractStreamOperatorTestHarness.repackageState( @@ -527,10 +530,10 @@ public void testMaxCounterUponRecovery() throws Exception { final TestBucketFactory secondRecovered = new TestBucketFactory(); try ( - OneInputStreamOperatorTestHarness, Object> testHarness1 = createCustomRescalingTestSink( - outDir, 2, 0, 100L, 2L, firstRecovered, new SimpleStringEncoder<>()); - OneInputStreamOperatorTestHarness, Object> testHarness2 = createCustomRescalingTestSink( - outDir, 2, 1, 100L, 2L, secondRecovered, new SimpleStringEncoder<>()) + OneInputStreamOperatorTestHarness, Object> testHarness1 = TestUtils.createCustomRescalingTestSink( + outDir, 2, 0, 10L, new TestUtils.TupleToStringBucketer(), new SimpleStringEncoder<>(), rollingPolicy, firstRecovered); + OneInputStreamOperatorTestHarness, Object> testHarness2 = TestUtils.createCustomRescalingTestSink( + outDir, 2, 1, 10L, new TestUtils.TupleToStringBucketer(), new SimpleStringEncoder<>(), rollingPolicy, secondRecovered) ) { testHarness1.setup(); testHarness1.initializeState(mergedSnapshot); @@ -540,7 +543,7 @@ public void testMaxCounterUponRecovery() throws Exception { testHarness1.processElement(new StreamRecord<>(Tuple2.of("test4", 0), 0L)); Assert.assertEquals(3L, firstRecovered.getInitialCounter()); - checkLocalFs(outDir, 1, 3); + TestUtils.checkLocalFs(outDir, 1, 3); testHarness2.setup(); testHarness2.initializeState(mergedSnapshot); @@ -550,78 +553,26 @@ public void testMaxCounterUponRecovery() throws Exception { testHarness2.processElement(new StreamRecord<>(Tuple2.of("test2", 0), 0L)); Assert.assertEquals(3L, secondRecovered.getInitialCounter()); - checkLocalFs(outDir, 2, 3); + TestUtils.checkLocalFs(outDir, 2, 3); } } ////////////////////// Helper Methods ////////////////////// - private OneInputStreamOperatorTestHarness, Object> createRescalingTestSink( - File outDir, - int totalParallelism, - int taskIdx, - long inactivityInterval, - long partMaxSize) throws Exception { - - return createCustomRescalingTestSink( - outDir, - totalParallelism, - taskIdx, - inactivityInterval, - partMaxSize, - new DefaultBucketFactory<>(), - (Encoder>) (element, stream) -> { - stream.write((element.f0 + '@' + element.f1).getBytes(StandardCharsets.UTF_8)); - stream.write('\n'); - }); - } - - private OneInputStreamOperatorTestHarness, Object> createCustomRescalingTestSink( - File outDir, - int totalParallelism, - int taskIdx, - long inactivityInterval, - long partMaxSize, - BucketFactory> factory, - Encoder> writer) throws Exception { - - StreamingFileSink> sink = new StreamingFileSink<>(new Path(outDir.toURI()), factory) - .setBucketer(new Bucketer>() { - - private static final long serialVersionUID = -3086487303018372007L; - - @Override - public String getBucketId(Tuple2 element, Context context) { - return element.f0; - } - }) - .setEncoder(writer) - .setRollingPolicy( - DefaultRollingPolicy - .create() - .withMaxPartSize(partMaxSize) - .withRolloverInterval(inactivityInterval) - .withInactivityInterval(inactivityInterval) - .build()) - .setBucketCheckInterval(10L); - - return new OneInputStreamOperatorTestHarness<>(new StreamSink<>(sink), 10, totalParallelism, taskIdx); - } - - static class TestBucketFactory extends DefaultBucketFactory> { + static class TestBucketFactory extends DefaultBucketFactory, String> { private static final long serialVersionUID = 2794824980604027930L; private long initialCounter = -1L; @Override - public Bucket> getNewBucket( - RecoverableWriter fsWriter, - int subtaskIndex, - String bucketId, - Path bucketPath, - long initialPartCounter, - Encoder> writer) throws IOException { + public Bucket, String> getNewBucket( + final RecoverableWriter fsWriter, + final int subtaskIndex, + final String bucketId, + final Path bucketPath, + final long initialPartCounter, + final PartFileWriter.PartFileFactory, String> partFileWriterFactory) { this.initialCounter = initialPartCounter; @@ -631,16 +582,16 @@ public Bucket> getNewBucket( bucketId, bucketPath, initialPartCounter, - writer); + partFileWriterFactory); } @Override - public Bucket> restoreBucket( - RecoverableWriter fsWriter, - int subtaskIndex, - long initialPartCounter, - Encoder> writer, - BucketState bucketState) throws IOException { + public Bucket, String> restoreBucket( + final RecoverableWriter fsWriter, + final int subtaskIndex, + final long initialPartCounter, + final PartFileWriter.PartFileFactory, String> partFileWriterFactory, + final BucketState bucketState) throws IOException { this.initialCounter = initialPartCounter; @@ -648,7 +599,7 @@ public Bucket> restoreBucket( fsWriter, subtaskIndex, initialPartCounter, - writer, + partFileWriterFactory, bucketState); } @@ -656,34 +607,4 @@ public long getInitialCounter() { return initialCounter; } } - - private static void checkLocalFs(File outDir, int expectedInProgress, int expectedCompleted) { - int inProgress = 0; - int finished = 0; - - for (File file: FileUtils.listFiles(outDir, null, true)) { - if (file.getAbsolutePath().endsWith("crc")) { - continue; - } - - if (file.toPath().getFileName().toString().startsWith(".")) { - inProgress++; - } else { - finished++; - } - } - - Assert.assertEquals(expectedInProgress, inProgress); - Assert.assertEquals(expectedCompleted, finished); - } - - private static Map getFileContentByPath(File directory) throws IOException { - Map contents = new HashMap<>(4); - - final Collection filesInBucket = FileUtils.listFiles(directory, null, true); - for (File file : filesInBucket) { - contents.put(file, FileUtils.readFileToString(file)); - } - return contents; - } } diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/functions/sink/filesystem/RollingPolicyTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/functions/sink/filesystem/RollingPolicyTest.java new file mode 100644 index 0000000000000..61e143350ee2f --- /dev/null +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/functions/sink/filesystem/RollingPolicyTest.java @@ -0,0 +1,225 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.streaming.api.functions.sink.filesystem; + +import org.apache.flink.api.common.serialization.SimpleStringEncoder; +import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.streaming.api.functions.sink.filesystem.rolling.policies.DefaultRollingPolicy; +import org.apache.flink.streaming.api.functions.sink.filesystem.rolling.policies.OnCheckpointRollingPolicy; +import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; +import org.apache.flink.streaming.util.OneInputStreamOperatorTestHarness; + +import org.junit.ClassRule; +import org.junit.Test; +import org.junit.rules.TemporaryFolder; + +import java.io.File; +import java.io.IOException; + +/** + * Tests for different {@link RollingPolicy rolling policies}. + */ +public class RollingPolicyTest { + + @ClassRule + public static final TemporaryFolder TEMP_FOLDER = new TemporaryFolder(); + + @Test + public void testDefaultRollingPolicy() throws Exception { + final File outDir = TEMP_FOLDER.newFolder(); + + final RollingPolicy rollingPolicy = DefaultRollingPolicy + .create() + .withMaxPartSize(10L) + .withInactivityInterval(4L) + .withRolloverInterval(11L) + .build(); + + try ( + OneInputStreamOperatorTestHarness, Object> testHarness = TestUtils.createCustomRescalingTestSink( + outDir, + 1, + 0, + 1L, + new TestUtils.TupleToStringBucketer(), + new SimpleStringEncoder<>(), + rollingPolicy, + new DefaultBucketFactory<>()) + ) { + testHarness.setup(); + testHarness.open(); + + testHarness.setProcessingTime(0L); + + testHarness.processElement(new StreamRecord<>(Tuple2.of("test1", 1), 1L)); + TestUtils.checkLocalFs(outDir, 1, 0); + + // roll due to size + testHarness.processElement(new StreamRecord<>(Tuple2.of("test1", 2), 2L)); + TestUtils.checkLocalFs(outDir, 1, 0); + + testHarness.processElement(new StreamRecord<>(Tuple2.of("test1", 3), 3L)); + TestUtils.checkLocalFs(outDir, 2, 0); + + // roll due to inactivity + testHarness.setProcessingTime(7L); + + testHarness.processElement(new StreamRecord<>(Tuple2.of("test1", 4), 4L)); + TestUtils.checkLocalFs(outDir, 3, 0); + + // roll due to rollover interval + testHarness.setProcessingTime(20L); + + testHarness.processElement(new StreamRecord<>(Tuple2.of("test1", 5), 5L)); + TestUtils.checkLocalFs(outDir, 4, 0); + + // we take a checkpoint but we should not roll. + testHarness.snapshot(1L, 1L); + + TestUtils.checkLocalFs(outDir, 4, 0); + + // acknowledge the checkpoint, so publish the 3 closed files, but not the open one. + testHarness.notifyOfCompletedCheckpoint(1L); + TestUtils.checkLocalFs(outDir, 1, 3); + } + } + + @Test + public void testRollOnCheckpointPolicy() throws Exception { + final File outDir = TEMP_FOLDER.newFolder(); + + final RollingPolicy rollingPolicy = new OnCheckpointRollingPolicy<>(); + + try ( + OneInputStreamOperatorTestHarness, Object> testHarness = TestUtils.createCustomRescalingTestSink( + outDir, + 1, + 0, + 10L, + new TestUtils.TupleToStringBucketer(), + new SimpleStringEncoder<>(), + rollingPolicy, + new DefaultBucketFactory<>()) + ) { + testHarness.setup(); + testHarness.open(); + + testHarness.setProcessingTime(0L); + + testHarness.processElement(new StreamRecord<>(Tuple2.of("test2", 1), 1L)); + + testHarness.processElement(new StreamRecord<>(Tuple2.of("test1", 1), 1L)); + testHarness.processElement(new StreamRecord<>(Tuple2.of("test1", 2), 2L)); + TestUtils.checkLocalFs(outDir, 2, 0); + + testHarness.processElement(new StreamRecord<>(Tuple2.of("test1", 3), 3L)); + TestUtils.checkLocalFs(outDir, 2, 0); + + // we take a checkpoint so we roll. + testHarness.snapshot(1L, 1L); + + // this will create a new part file + testHarness.processElement(new StreamRecord<>(Tuple2.of("test1", 4), 4L)); + TestUtils.checkLocalFs(outDir, 3, 0); + + // and open and fill .part-0-2.inprogress + testHarness.processElement(new StreamRecord<>(Tuple2.of("test1", 5), 5L)); + testHarness.processElement(new StreamRecord<>(Tuple2.of("test1", 6), 6L)); + TestUtils.checkLocalFs(outDir, 3, 0); // nothing committed yet + + // we take a checkpoint so we roll. + testHarness.snapshot(2L, 2L); + + testHarness.processElement(new StreamRecord<>(Tuple2.of("test2", 7), 7L)); + TestUtils.checkLocalFs(outDir, 4, 0); + + // we acknowledge the last checkpoint so we should publish all but the latest in-progress file + testHarness.notifyOfCompletedCheckpoint(2L); + TestUtils.checkLocalFs(outDir, 1, 3); + } + } + + @Test + public void testCustomRollingPolicy() throws Exception { + final File outDir = TEMP_FOLDER.newFolder(); + + final RollingPolicy rollingPolicy = new RollingPolicy() { + + private static final long serialVersionUID = 1L; + + @Override + public boolean shouldRollOnCheckpoint(PartFileInfo partFileState) { + return true; + } + + @Override + public boolean shouldRollOnEvent(PartFileInfo partFileState) throws IOException { + // this means that 2 elements will close the part file. + return partFileState.getSize() > 12L; + } + + @Override + public boolean shouldRollOnProcessingTime(PartFileInfo partFileState, long currentTime) { + return false; + } + }; + + try ( + OneInputStreamOperatorTestHarness, Object> testHarness = TestUtils.createCustomRescalingTestSink( + outDir, + 1, + 0, + 10L, + new TestUtils.TupleToStringBucketer(), + new SimpleStringEncoder<>(), + rollingPolicy, + new DefaultBucketFactory<>()) + ) { + testHarness.setup(); + testHarness.open(); + + testHarness.setProcessingTime(0L); + + testHarness.processElement(new StreamRecord<>(Tuple2.of("test2", 1), 1L)); + + // the following 2 elements will close a part file ... + testHarness.processElement(new StreamRecord<>(Tuple2.of("test1", 1), 1L)); + testHarness.processElement(new StreamRecord<>(Tuple2.of("test1", 2), 2L)); + + // ... and this one will open a new ... + testHarness.processElement(new StreamRecord<>(Tuple2.of("test1", 3), 2L)); + TestUtils.checkLocalFs(outDir, 3, 0); + + // ... and all open part files should close here. + testHarness.snapshot(1L, 1L); + + // this will create and fill out a new part file + testHarness.processElement(new StreamRecord<>(Tuple2.of("test1", 4), 4L)); + testHarness.processElement(new StreamRecord<>(Tuple2.of("test1", 5), 5L)); + TestUtils.checkLocalFs(outDir, 4, 0); + + // we take a checkpoint so we roll. + testHarness.snapshot(2L, 2L); + + // we acknowledge the first checkpoint so we should publish all but the latest in-progress file + testHarness.notifyOfCompletedCheckpoint(1L); + TestUtils.checkLocalFs(outDir, 1, 3); + } + } +} diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/functions/sink/filesystem/TestUtils.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/functions/sink/filesystem/TestUtils.java new file mode 100644 index 0000000000000..184e23ee458b4 --- /dev/null +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/functions/sink/filesystem/TestUtils.java @@ -0,0 +1,164 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.streaming.api.functions.sink.filesystem; + +import org.apache.flink.api.common.serialization.BulkWriter; +import org.apache.flink.api.common.serialization.Encoder; +import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.core.fs.Path; +import org.apache.flink.core.io.SimpleVersionedSerializer; +import org.apache.flink.streaming.api.functions.sink.filesystem.bucketers.Bucketer; +import org.apache.flink.streaming.api.functions.sink.filesystem.bucketers.SimpleVersionedStringSerializer; +import org.apache.flink.streaming.api.functions.sink.filesystem.rolling.policies.DefaultRollingPolicy; +import org.apache.flink.streaming.api.operators.StreamSink; +import org.apache.flink.streaming.util.OneInputStreamOperatorTestHarness; + +import org.apache.commons.io.FileUtils; +import org.junit.Assert; + +import java.io.File; +import java.io.IOException; +import java.nio.charset.StandardCharsets; +import java.util.Collection; +import java.util.HashMap; +import java.util.Map; + +/** + * Utilities for the {@link StreamingFileSink} tests. + */ +public class TestUtils { + + static OneInputStreamOperatorTestHarness, Object> createRescalingTestSink( + File outDir, + int totalParallelism, + int taskIdx, + long inactivityInterval, + long partMaxSize) throws Exception { + + final RollingPolicy rollingPolicy = + DefaultRollingPolicy + .create() + .withMaxPartSize(partMaxSize) + .withRolloverInterval(inactivityInterval) + .withInactivityInterval(inactivityInterval) + .build(); + + final Bucketer, String> bucketer = new TupleToStringBucketer(); + + final Encoder> encoder = (element, stream) -> { + stream.write((element.f0 + '@' + element.f1).getBytes(StandardCharsets.UTF_8)); + stream.write('\n'); + }; + + return createCustomRescalingTestSink( + outDir, + totalParallelism, + taskIdx, + 10L, + bucketer, + encoder, + rollingPolicy, + new DefaultBucketFactory<>()); + } + + static OneInputStreamOperatorTestHarness, Object> createCustomRescalingTestSink( + final File outDir, + final int totalParallelism, + final int taskIdx, + final long bucketCheckInterval, + final Bucketer, String> bucketer, + final Encoder> writer, + final RollingPolicy rollingPolicy, + final BucketFactory, String> bucketFactory) throws Exception { + + StreamingFileSink> sink = StreamingFileSink + .forRowFormat(new Path(outDir.toURI()), writer) + .withBucketer(bucketer) + .withRollingPolicy(rollingPolicy) + .withBucketCheckInterval(bucketCheckInterval) + .withBucketFactory(bucketFactory) + .build(); + + return new OneInputStreamOperatorTestHarness<>(new StreamSink<>(sink), 10, totalParallelism, taskIdx); + } + + static OneInputStreamOperatorTestHarness, Object> createTestSinkWithBulkEncoder( + final File outDir, + final int totalParallelism, + final int taskIdx, + final long bucketCheckInterval, + final Bucketer, String> bucketer, + final BulkWriter.Factory> writer, + final BucketFactory, String> bucketFactory) throws Exception { + + StreamingFileSink> sink = StreamingFileSink + .forBulkFormat(new Path(outDir.toURI()), writer) + .withBucketer(bucketer) + .withBucketCheckInterval(bucketCheckInterval) + .withBucketFactory(bucketFactory) + .build(); + + return new OneInputStreamOperatorTestHarness<>(new StreamSink<>(sink), 10, totalParallelism, taskIdx); + } + + static void checkLocalFs(File outDir, int expectedInProgress, int expectedCompleted) { + int inProgress = 0; + int finished = 0; + + for (File file: FileUtils.listFiles(outDir, null, true)) { + if (file.getAbsolutePath().endsWith("crc")) { + continue; + } + + if (file.toPath().getFileName().toString().startsWith(".")) { + inProgress++; + } else { + finished++; + } + } + + Assert.assertEquals(expectedInProgress, inProgress); + Assert.assertEquals(expectedCompleted, finished); + } + + static Map getFileContentByPath(File directory) throws IOException { + Map contents = new HashMap<>(4); + + final Collection filesInBucket = FileUtils.listFiles(directory, null, true); + for (File file : filesInBucket) { + contents.put(file, FileUtils.readFileToString(file)); + } + return contents; + } + + static class TupleToStringBucketer implements Bucketer, String> { + + private static final long serialVersionUID = 1L; + + @Override + public String getBucketId(Tuple2 element, Context context) { + return element.f0; + } + + @Override + public SimpleVersionedSerializer getSerializer() { + return SimpleVersionedStringSerializer.INSTANCE; + } + } +} From fdb11c52f1d0f39abceee02f4ad5beaf1034e05f Mon Sep 17 00:00:00 2001 From: Stephan Ewen Date: Fri, 20 Jul 2018 16:14:31 +0200 Subject: [PATCH 45/65] [FLINK-9753] [formats] Add a Parquet BulkWriter --- flink-formats/flink-parquet/pom.xml | 160 ++++++ .../flink/formats/parquet/ParquetBuilder.java | 39 ++ .../formats/parquet/ParquetBulkWriter.java | 64 +++ .../formats/parquet/ParquetWriterFactory.java | 60 ++ .../parquet/PositionOutputStreamAdapter.java | 73 +++ .../formats/parquet/StreamOutputFile.java | 82 +++ .../parquet/avro/ParquetAvroWriters.java | 94 ++++ .../avro/ParquetStreamingFileSinkITCase.java | 241 ++++++++ .../formats/parquet/generated/Address.java | 517 ++++++++++++++++++ .../parquet/testutils/FiniteTestSource.java | 79 +++ .../src/test/resources/avro/testdata.avsc | 13 + .../src/test/resources/log4j-test.properties | 23 + flink-formats/pom.xml | 1 + pom.xml | 2 + tools/maven/suppressions.xml | 1 + tools/travis_mvn_watchdog.sh | 1 + 16 files changed, 1450 insertions(+) create mode 100644 flink-formats/flink-parquet/pom.xml create mode 100644 flink-formats/flink-parquet/src/main/java/org/apache/flink/formats/parquet/ParquetBuilder.java create mode 100644 flink-formats/flink-parquet/src/main/java/org/apache/flink/formats/parquet/ParquetBulkWriter.java create mode 100644 flink-formats/flink-parquet/src/main/java/org/apache/flink/formats/parquet/ParquetWriterFactory.java create mode 100644 flink-formats/flink-parquet/src/main/java/org/apache/flink/formats/parquet/PositionOutputStreamAdapter.java create mode 100644 flink-formats/flink-parquet/src/main/java/org/apache/flink/formats/parquet/StreamOutputFile.java create mode 100644 flink-formats/flink-parquet/src/main/java/org/apache/flink/formats/parquet/avro/ParquetAvroWriters.java create mode 100644 flink-formats/flink-parquet/src/test/java/org/apache/flink/formats/parquet/avro/ParquetStreamingFileSinkITCase.java create mode 100644 flink-formats/flink-parquet/src/test/java/org/apache/flink/formats/parquet/generated/Address.java create mode 100644 flink-formats/flink-parquet/src/test/java/org/apache/flink/formats/parquet/testutils/FiniteTestSource.java create mode 100644 flink-formats/flink-parquet/src/test/resources/avro/testdata.avsc create mode 100644 flink-formats/flink-parquet/src/test/resources/log4j-test.properties diff --git a/flink-formats/flink-parquet/pom.xml b/flink-formats/flink-parquet/pom.xml new file mode 100644 index 0000000000000..14e61ea0e5be5 --- /dev/null +++ b/flink-formats/flink-parquet/pom.xml @@ -0,0 +1,160 @@ + + + + + 4.0.0 + + + org.apache.flink + flink-formats + 1.6-SNAPSHOT + .. + + + flink-parquet + flink-parquet + + jar + + + 1.10.0 + + + + + + + + org.apache.flink + flink-core + ${project.version} + provided + + + + + + org.apache.parquet + parquet-hadoop + ${flink.format.parquet.version} + + + + + org.apache.flink + flink-shaded-hadoop2 + ${project.version} + provided + + + + + it.unimi.dsi + fastutil + 8.2.1 + provided + + + + + + org.apache.parquet + parquet-avro + ${flink.format.parquet.version} + true + + + org.apache.hadoop + hadoop-client + + + it.unimi.dsi + fastutil + + + + + + + + org.apache.flink + flink-test-utils_2.11 + ${project.version} + test + + + + org.apache.flink + flink-streaming-java_2.11 + ${project.version} + test + + + + org.apache.flink + flink-avro + ${project.version} + test + + + + + + + + + + org.apache.avro + avro-maven-plugin + ${avro.version} + + + generate-sources + + schema + + + ${project.basedir}/src/test/resources/avro + ${project.basedir}/src/test/java/ + + + + + + + + org.apache.maven.plugins + maven-enforcer-plugin + + + dependency-convergence + + enforce + + + true + + + + + + + diff --git a/flink-formats/flink-parquet/src/main/java/org/apache/flink/formats/parquet/ParquetBuilder.java b/flink-formats/flink-parquet/src/main/java/org/apache/flink/formats/parquet/ParquetBuilder.java new file mode 100644 index 0000000000000..d3b1370fb8a26 --- /dev/null +++ b/flink-formats/flink-parquet/src/main/java/org/apache/flink/formats/parquet/ParquetBuilder.java @@ -0,0 +1,39 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.formats.parquet; + +import org.apache.parquet.hadoop.ParquetWriter; +import org.apache.parquet.io.OutputFile; + +import java.io.IOException; +import java.io.Serializable; + +/** + * A builder to create a {@link ParquetWriter} from a Parquet {@link OutputFile}. + * + * @param The type of elements written by the writer. + */ +@FunctionalInterface +public interface ParquetBuilder extends Serializable { + + /** + * Creates and configures a parquet writer to the given output file. + */ + ParquetWriter createWriter(OutputFile out) throws IOException; +} diff --git a/flink-formats/flink-parquet/src/main/java/org/apache/flink/formats/parquet/ParquetBulkWriter.java b/flink-formats/flink-parquet/src/main/java/org/apache/flink/formats/parquet/ParquetBulkWriter.java new file mode 100644 index 0000000000000..77bdc5baa587a --- /dev/null +++ b/flink-formats/flink-parquet/src/main/java/org/apache/flink/formats/parquet/ParquetBulkWriter.java @@ -0,0 +1,64 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.formats.parquet; + +import org.apache.flink.annotation.PublicEvolving; +import org.apache.flink.api.common.serialization.BulkWriter; + +import org.apache.parquet.hadoop.ParquetWriter; + +import java.io.IOException; + +import static org.apache.flink.util.Preconditions.checkNotNull; + +/** + * A simple {@link BulkWriter} implementation that wraps a {@link ParquetWriter}. + * + * @param The type of records written. + */ +@PublicEvolving +public class ParquetBulkWriter implements BulkWriter { + + /** The ParquetWriter to write to. */ + private final ParquetWriter parquetWriter; + + /** + * Creates a new ParquetBulkWriter wrapping the given ParquetWriter. + * + * @param parquetWriter The ParquetWriter to write to. + */ + public ParquetBulkWriter(ParquetWriter parquetWriter) { + this.parquetWriter = checkNotNull(parquetWriter, "parquetWriter"); + } + + @Override + public void addElement(T datum) throws IOException { + parquetWriter.write(datum); + } + + @Override + public void flush() { + // nothing we can do here + } + + @Override + public void finish() throws IOException { + parquetWriter.close(); + } +} diff --git a/flink-formats/flink-parquet/src/main/java/org/apache/flink/formats/parquet/ParquetWriterFactory.java b/flink-formats/flink-parquet/src/main/java/org/apache/flink/formats/parquet/ParquetWriterFactory.java new file mode 100644 index 0000000000000..46245a41b4309 --- /dev/null +++ b/flink-formats/flink-parquet/src/main/java/org/apache/flink/formats/parquet/ParquetWriterFactory.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.formats.parquet; + +import org.apache.flink.annotation.PublicEvolving; +import org.apache.flink.api.common.serialization.BulkWriter; +import org.apache.flink.core.fs.FSDataOutputStream; + +import org.apache.parquet.hadoop.ParquetWriter; +import org.apache.parquet.io.OutputFile; + +import java.io.IOException; + +/** + * A factory that creates a Parquet {@link BulkWriter}. The factory takes a user-supplied + * builder to assemble Parquet's writer and then turns it into a Flink {@code BulkWriter}. + * + * @param The type of record to write. + */ +@PublicEvolving +public class ParquetWriterFactory implements BulkWriter.Factory { + + private static final long serialVersionUID = 1L; + + /** The builder to construct the ParquetWriter. */ + private final ParquetBuilder writerBuilder; + + /** + * Creates a new ParquetWriterFactory using the given builder to assemble the + * ParquetWriter. + * + * @param writerBuilder The builder to construct the ParquetWriter. + */ + public ParquetWriterFactory(ParquetBuilder writerBuilder) { + this.writerBuilder = writerBuilder; + } + + @Override + public BulkWriter create(FSDataOutputStream stream) throws IOException { + final OutputFile out = new StreamOutputFile(stream); + final ParquetWriter writer = writerBuilder.createWriter(out); + return new ParquetBulkWriter<>(writer); + } +} diff --git a/flink-formats/flink-parquet/src/main/java/org/apache/flink/formats/parquet/PositionOutputStreamAdapter.java b/flink-formats/flink-parquet/src/main/java/org/apache/flink/formats/parquet/PositionOutputStreamAdapter.java new file mode 100644 index 0000000000000..3949ba7bb57fd --- /dev/null +++ b/flink-formats/flink-parquet/src/main/java/org/apache/flink/formats/parquet/PositionOutputStreamAdapter.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.formats.parquet; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.core.fs.FSDataOutputStream; + +import org.apache.parquet.io.PositionOutputStream; + +import java.io.IOException; + +import static org.apache.parquet.Preconditions.checkNotNull; + +/** + * An adapter to turn Flink's {@link FSDataOutputStream} into a {@link PositionOutputStream}. + */ +@Internal +class PositionOutputStreamAdapter extends PositionOutputStream { + + /** The Flink stream written to. */ + private final FSDataOutputStream out; + + /** + * Create a new PositionOutputStreamAdapter. + * + * @param out The Flink stream written to. + */ + PositionOutputStreamAdapter(FSDataOutputStream out) { + this.out = checkNotNull(out, "out"); + } + + @Override + public long getPos() throws IOException { + return out.getPos(); + } + + @Override + public void write(int b) throws IOException { + out.write(b); + } + + @Override + public void write(byte[] buffer, int off, int len) throws IOException { + out.write(buffer, off, len); + } + + @Override + public void flush() throws IOException { + out.flush(); + } + + @Override + public void close() { + // we do not actually close the internal stream here, to prevent that the finishing + // of the Parquet Writer closes the target output stream + } +} diff --git a/flink-formats/flink-parquet/src/main/java/org/apache/flink/formats/parquet/StreamOutputFile.java b/flink-formats/flink-parquet/src/main/java/org/apache/flink/formats/parquet/StreamOutputFile.java new file mode 100644 index 0000000000000..70a8557e1ecec --- /dev/null +++ b/flink-formats/flink-parquet/src/main/java/org/apache/flink/formats/parquet/StreamOutputFile.java @@ -0,0 +1,82 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.formats.parquet; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.core.fs.FSDataOutputStream; + +import org.apache.parquet.io.OutputFile; +import org.apache.parquet.io.PositionOutputStream; + +import java.util.concurrent.atomic.AtomicBoolean; + +import static org.apache.flink.util.Preconditions.checkNotNull; + +/** + * An implementation of Parquet's {@link OutputFile} interface that goes against + * a Flink {@link FSDataOutputStream}. + * + *

    Because the implementation goes against an open stream, rather than open its + * own streams against a file, instances can create one stream only. + */ +@Internal +class StreamOutputFile implements OutputFile { + + private static final long DEFAULT_BLOCK_SIZE = 64L * 1024L * 1024L; + + private final FSDataOutputStream stream; + + private final AtomicBoolean used; + + /** + * Creates a new StreamOutputFile. The first call to {@link #create(long)} + * or {@link #createOrOverwrite(long)} returns a stream that writes to the given stream. + * + * @param stream The stream to write to. + */ + StreamOutputFile(FSDataOutputStream stream) { + this.stream = checkNotNull(stream); + this.used = new AtomicBoolean(false); + } + + @Override + public PositionOutputStream create(long blockSizeHint) { + if (used.compareAndSet(false, true)) { + return new PositionOutputStreamAdapter(stream); + } + else { + throw new IllegalStateException("A stream against this file was already created."); + } + } + + @Override + public PositionOutputStream createOrOverwrite(long blockSizeHint) { + return create(blockSizeHint); + } + + @Override + public boolean supportsBlockSize() { + return false; + } + + @Override + public long defaultBlockSize() { + return DEFAULT_BLOCK_SIZE; + } +} diff --git a/flink-formats/flink-parquet/src/main/java/org/apache/flink/formats/parquet/avro/ParquetAvroWriters.java b/flink-formats/flink-parquet/src/main/java/org/apache/flink/formats/parquet/avro/ParquetAvroWriters.java new file mode 100644 index 0000000000000..87ddfdc47baa2 --- /dev/null +++ b/flink-formats/flink-parquet/src/main/java/org/apache/flink/formats/parquet/avro/ParquetAvroWriters.java @@ -0,0 +1,94 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.formats.parquet.avro; + +import org.apache.flink.formats.parquet.ParquetBuilder; +import org.apache.flink.formats.parquet.ParquetWriterFactory; + +import org.apache.avro.Schema; +import org.apache.avro.generic.GenericData; +import org.apache.avro.generic.GenericRecord; +import org.apache.avro.reflect.ReflectData; +import org.apache.avro.specific.SpecificData; +import org.apache.avro.specific.SpecificRecordBase; +import org.apache.parquet.avro.AvroParquetWriter; +import org.apache.parquet.hadoop.ParquetWriter; +import org.apache.parquet.io.OutputFile; + +import java.io.IOException; + +/** + * Convenience builder to create {@link ParquetWriterFactory} instances for the different Avro types. + */ +public class ParquetAvroWriters { + + /** + * Creates a ParquetWriterFactory for an Avro specific type. The Parquet writers will use the + * schema of that specific type to build and write the columnar data. + * + * @param type The class of the type to write. + */ + public static ParquetWriterFactory forSpecificRecord(Class type) { + final String schemaString = SpecificData.get().getSchema(type).toString(); + final ParquetBuilder builder = (out) -> createAvroParquetWriter(schemaString, SpecificData.get(), out); + return new ParquetWriterFactory<>(builder); + } + + /** + * Creates a ParquetWriterFactory that accepts and writes Avro generic types. + * The Parquet writers will use the given schema to build and write the columnar data. + * + * @param schema The schema of the generic type. + */ + public static ParquetWriterFactory forGenericRecord(Schema schema) { + final String schemaString = schema.toString(); + final ParquetBuilder builder = (out) -> createAvroParquetWriter(schemaString, GenericData.get(), out); + return new ParquetWriterFactory<>(builder); + } + + /** + * Creates a ParquetWriterFactory for the given type. The Parquet writers will use Avro + * to reflectively create a schema for the type and use that schema to write the columnar data. + * + * @param type The class of the type to write. + */ + public static ParquetWriterFactory forReflectRecord(Class type) { + final String schemaString = ReflectData.get().getSchema(type).toString(); + final ParquetBuilder builder = (out) -> createAvroParquetWriter(schemaString, ReflectData.get(), out); + return new ParquetWriterFactory<>(builder); + } + + private static ParquetWriter createAvroParquetWriter( + String schemaString, + GenericData dataModel, + OutputFile out) throws IOException { + + final Schema schema = new Schema.Parser().parse(schemaString); + + return AvroParquetWriter.builder(out) + .withSchema(schema) + .withDataModel(dataModel) + .build(); + } + + // ------------------------------------------------------------------------ + + /** Class is not meant to be instantiated. */ + private ParquetAvroWriters() {} +} diff --git a/flink-formats/flink-parquet/src/test/java/org/apache/flink/formats/parquet/avro/ParquetStreamingFileSinkITCase.java b/flink-formats/flink-parquet/src/test/java/org/apache/flink/formats/parquet/avro/ParquetStreamingFileSinkITCase.java new file mode 100644 index 0000000000000..d1f0a5f57787f --- /dev/null +++ b/flink-formats/flink-parquet/src/test/java/org/apache/flink/formats/parquet/avro/ParquetStreamingFileSinkITCase.java @@ -0,0 +1,241 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.formats.parquet.avro; + +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.core.fs.Path; +import org.apache.flink.formats.avro.typeutils.GenericRecordAvroTypeInfo; +import org.apache.flink.formats.parquet.generated.Address; +import org.apache.flink.formats.parquet.testutils.FiniteTestSource; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.streaming.api.functions.sink.filesystem.StreamingFileSink; +import org.apache.flink.test.util.AbstractTestBase; + +import org.apache.avro.Schema; +import org.apache.avro.generic.GenericData; +import org.apache.avro.generic.GenericRecord; +import org.apache.avro.reflect.ReflectData; +import org.apache.avro.specific.SpecificData; +import org.apache.hadoop.conf.Configuration; +import org.apache.parquet.avro.AvroParquetReader; +import org.apache.parquet.hadoop.ParquetReader; +import org.apache.parquet.hadoop.util.HadoopInputFile; +import org.apache.parquet.io.InputFile; +import org.junit.Test; + +import java.io.File; +import java.io.IOException; +import java.io.Serializable; +import java.util.AbstractCollection; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collection; +import java.util.Iterator; +import java.util.List; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.assertTrue; + +/** + * Simple integration test case for writing bulk encoded files with the + * {@link StreamingFileSink} with Parquet. + */ +@SuppressWarnings("serial") +public class ParquetStreamingFileSinkITCase extends AbstractTestBase { + + @Test + public void testWriteParquetAvroSpecific() throws Exception { + + final File folder = TEMPORARY_FOLDER.newFolder(); + + final List

    data = Arrays.asList( + new Address(1, "a", "b", "c", "12345"), + new Address(2, "p", "q", "r", "12345"), + new Address(3, "x", "y", "z", "12345") + ); + + final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + env.setParallelism(1); + env.enableCheckpointing(100); + + DataStream
    stream = env.addSource( + new FiniteTestSource<>(data), TypeInformation.of(Address.class)); + + stream.addSink( + StreamingFileSink.forBulkFormat( + Path.fromLocalFile(folder), + ParquetAvroWriters.forSpecificRecord(Address.class)) + .build()); + + env.execute(); + + validateResults(folder, SpecificData.get(), data); + } + + @Test + public void testWriteParquetAvroGeneric() throws Exception { + + final File folder = TEMPORARY_FOLDER.newFolder(); + + final Schema schema = Address.getClassSchema(); + + final Collection data = new GenericTestDataCollection(); + + final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + env.setParallelism(1); + env.enableCheckpointing(100); + + DataStream stream = env.addSource( + new FiniteTestSource<>(data), new GenericRecordAvroTypeInfo(schema)); + + stream.addSink( + StreamingFileSink.forBulkFormat( + Path.fromLocalFile(folder), + ParquetAvroWriters.forGenericRecord(schema)) + .build()); + + env.execute(); + + List
    expected = Arrays.asList( + new Address(1, "a", "b", "c", "12345"), + new Address(2, "x", "y", "z", "98765")); + + validateResults(folder, SpecificData.get(), expected); + } + + @Test + public void testWriteParquetAvroReflect() throws Exception { + + final File folder = TEMPORARY_FOLDER.newFolder(); + + final List data = Arrays.asList( + new Datum("a", 1), new Datum("b", 2), new Datum("c", 3)); + + final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + env.setParallelism(1); + env.enableCheckpointing(100); + + DataStream stream = env.addSource( + new FiniteTestSource<>(data), TypeInformation.of(Datum.class)); + + stream.addSink( + StreamingFileSink.forBulkFormat( + Path.fromLocalFile(folder), + ParquetAvroWriters.forReflectRecord(Datum.class)) + .build()); + + env.execute(); + + validateResults(folder, ReflectData.get(), data); + } + + // ------------------------------------------------------------------------ + + private static void validateResults(File folder, GenericData dataModel, List expected) throws Exception { + File[] buckets = folder.listFiles(); + assertNotNull(buckets); + assertEquals(1, buckets.length); + + File[] partFiles = buckets[0].listFiles(); + assertNotNull(partFiles); + assertEquals(1, partFiles.length); + assertTrue(partFiles[0].length() > 0); + + List
    results = readParquetFile(partFiles[0], dataModel); + assertEquals(expected, results); + } + + private static List readParquetFile(File file, GenericData dataModel) throws IOException { + InputFile inFile = HadoopInputFile.fromPath(new org.apache.hadoop.fs.Path(file.toURI()), new Configuration()); + ParquetReader reader = AvroParquetReader.builder(inFile).withDataModel(dataModel).build(); + + ArrayList results = new ArrayList<>(); + T next; + while ((next = reader.read()) != null) { + results.add(next); + } + + return results; + } + + private static class GenericTestDataCollection extends AbstractCollection implements Serializable { + + @Override + public Iterator iterator() { + final GenericRecord rec1 = new GenericData.Record(Address.getClassSchema()); + rec1.put(0, 1); + rec1.put(1, "a"); + rec1.put(2, "b"); + rec1.put(3, "c"); + rec1.put(4, "12345"); + + final GenericRecord rec2 = new GenericData.Record(Address.getClassSchema()); + rec2.put(0, 2); + rec2.put(1, "x"); + rec2.put(2, "y"); + rec2.put(3, "z"); + rec2.put(4, "98765"); + + return Arrays.asList(rec1, rec2).iterator(); + } + + @Override + public int size() { + return 2; + } + } + + // ------------------------------------------------------------------------ + + /** Test datum. */ + public static class Datum implements Serializable { + + public String a; + public int b; + + public Datum() {} + + public Datum(String a, int b) { + this.a = a; + this.b = b; + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + + Datum datum = (Datum) o; + return b == datum.b && (a != null ? a.equals(datum.a) : datum.a == null); + } + + @Override + public int hashCode() { + int result = a != null ? a.hashCode() : 0; + result = 31 * result + b; + return result; + } + } +} diff --git a/flink-formats/flink-parquet/src/test/java/org/apache/flink/formats/parquet/generated/Address.java b/flink-formats/flink-parquet/src/test/java/org/apache/flink/formats/parquet/generated/Address.java new file mode 100644 index 0000000000000..ca8f55faafda7 --- /dev/null +++ b/flink-formats/flink-parquet/src/test/java/org/apache/flink/formats/parquet/generated/Address.java @@ -0,0 +1,517 @@ +/** + * Autogenerated by Avro + * + * DO NOT EDIT DIRECTLY + */ +package org.apache.flink.formats.parquet.generated; + +import org.apache.avro.specific.SpecificData; +import org.apache.avro.message.BinaryMessageEncoder; +import org.apache.avro.message.BinaryMessageDecoder; +import org.apache.avro.message.SchemaStore; + +@SuppressWarnings("all") +@org.apache.avro.specific.AvroGenerated +public class Address extends org.apache.avro.specific.SpecificRecordBase implements org.apache.avro.specific.SpecificRecord { + private static final long serialVersionUID = -7342141701041388589L; + public static final org.apache.avro.Schema SCHEMA$ = new org.apache.avro.Schema.Parser().parse("{\"type\":\"record\",\"name\":\"Address\",\"namespace\":\"org.apache.flink.formats.parquet.generated\",\"fields\":[{\"name\":\"num\",\"type\":\"int\"},{\"name\":\"street\",\"type\":\"string\"},{\"name\":\"city\",\"type\":\"string\"},{\"name\":\"state\",\"type\":\"string\"},{\"name\":\"zip\",\"type\":\"string\"}]}"); + public static org.apache.avro.Schema getClassSchema() { return SCHEMA$; } + + private static SpecificData MODEL$ = new SpecificData(); + + private static final BinaryMessageEncoder
    ENCODER = + new BinaryMessageEncoder
    (MODEL$, SCHEMA$); + + private static final BinaryMessageDecoder
    DECODER = + new BinaryMessageDecoder
    (MODEL$, SCHEMA$); + + /** + * Return the BinaryMessageDecoder instance used by this class. + */ + public static BinaryMessageDecoder
    getDecoder() { + return DECODER; + } + + /** + * Create a new BinaryMessageDecoder instance for this class that uses the specified {@link SchemaStore}. + * @param resolver a {@link SchemaStore} used to find schemas by fingerprint + */ + public static BinaryMessageDecoder
    createDecoder(SchemaStore resolver) { + return new BinaryMessageDecoder
    (MODEL$, SCHEMA$, resolver); + } + + /** Serializes this Address to a ByteBuffer. */ + public java.nio.ByteBuffer toByteBuffer() throws java.io.IOException { + return ENCODER.encode(this); + } + + /** Deserializes a Address from a ByteBuffer. */ + public static Address fromByteBuffer( + java.nio.ByteBuffer b) throws java.io.IOException { + return DECODER.decode(b); + } + + @Deprecated public int num; + @Deprecated public java.lang.CharSequence street; + @Deprecated public java.lang.CharSequence city; + @Deprecated public java.lang.CharSequence state; + @Deprecated public java.lang.CharSequence zip; + + /** + * Default constructor. Note that this does not initialize fields + * to their default values from the schema. If that is desired then + * one should use newBuilder(). + */ + public Address() {} + + /** + * All-args constructor. + * @param num The new value for num + * @param street The new value for street + * @param city The new value for city + * @param state The new value for state + * @param zip The new value for zip + */ + public Address(java.lang.Integer num, java.lang.CharSequence street, java.lang.CharSequence city, java.lang.CharSequence state, java.lang.CharSequence zip) { + this.num = num; + this.street = street; + this.city = city; + this.state = state; + this.zip = zip; + } + + public org.apache.avro.Schema getSchema() { return SCHEMA$; } + // Used by DatumWriter. Applications should not call. + public java.lang.Object get(int field$) { + switch (field$) { + case 0: return num; + case 1: return street; + case 2: return city; + case 3: return state; + case 4: return zip; + default: throw new org.apache.avro.AvroRuntimeException("Bad index"); + } + } + + // Used by DatumReader. Applications should not call. + @SuppressWarnings(value="unchecked") + public void put(int field$, java.lang.Object value$) { + switch (field$) { + case 0: num = (java.lang.Integer)value$; break; + case 1: street = (java.lang.CharSequence)value$; break; + case 2: city = (java.lang.CharSequence)value$; break; + case 3: state = (java.lang.CharSequence)value$; break; + case 4: zip = (java.lang.CharSequence)value$; break; + default: throw new org.apache.avro.AvroRuntimeException("Bad index"); + } + } + + /** + * Gets the value of the 'num' field. + * @return The value of the 'num' field. + */ + public java.lang.Integer getNum() { + return num; + } + + /** + * Sets the value of the 'num' field. + * @param value the value to set. + */ + public void setNum(java.lang.Integer value) { + this.num = value; + } + + /** + * Gets the value of the 'street' field. + * @return The value of the 'street' field. + */ + public java.lang.CharSequence getStreet() { + return street; + } + + /** + * Sets the value of the 'street' field. + * @param value the value to set. + */ + public void setStreet(java.lang.CharSequence value) { + this.street = value; + } + + /** + * Gets the value of the 'city' field. + * @return The value of the 'city' field. + */ + public java.lang.CharSequence getCity() { + return city; + } + + /** + * Sets the value of the 'city' field. + * @param value the value to set. + */ + public void setCity(java.lang.CharSequence value) { + this.city = value; + } + + /** + * Gets the value of the 'state' field. + * @return The value of the 'state' field. + */ + public java.lang.CharSequence getState() { + return state; + } + + /** + * Sets the value of the 'state' field. + * @param value the value to set. + */ + public void setState(java.lang.CharSequence value) { + this.state = value; + } + + /** + * Gets the value of the 'zip' field. + * @return The value of the 'zip' field. + */ + public java.lang.CharSequence getZip() { + return zip; + } + + /** + * Sets the value of the 'zip' field. + * @param value the value to set. + */ + public void setZip(java.lang.CharSequence value) { + this.zip = value; + } + + /** + * Creates a new Address RecordBuilder. + * @return A new Address RecordBuilder + */ + public static org.apache.flink.formats.parquet.generated.Address.Builder newBuilder() { + return new org.apache.flink.formats.parquet.generated.Address.Builder(); + } + + /** + * Creates a new Address RecordBuilder by copying an existing Builder. + * @param other The existing builder to copy. + * @return A new Address RecordBuilder + */ + public static org.apache.flink.formats.parquet.generated.Address.Builder newBuilder(org.apache.flink.formats.parquet.generated.Address.Builder other) { + return new org.apache.flink.formats.parquet.generated.Address.Builder(other); + } + + /** + * Creates a new Address RecordBuilder by copying an existing Address instance. + * @param other The existing instance to copy. + * @return A new Address RecordBuilder + */ + public static org.apache.flink.formats.parquet.generated.Address.Builder newBuilder(org.apache.flink.formats.parquet.generated.Address other) { + return new org.apache.flink.formats.parquet.generated.Address.Builder(other); + } + + /** + * RecordBuilder for Address instances. + */ + public static class Builder extends org.apache.avro.specific.SpecificRecordBuilderBase
    + implements org.apache.avro.data.RecordBuilder
    { + + private int num; + private java.lang.CharSequence street; + private java.lang.CharSequence city; + private java.lang.CharSequence state; + private java.lang.CharSequence zip; + + /** Creates a new Builder */ + private Builder() { + super(SCHEMA$); + } + + /** + * Creates a Builder by copying an existing Builder. + * @param other The existing Builder to copy. + */ + private Builder(org.apache.flink.formats.parquet.generated.Address.Builder other) { + super(other); + if (isValidValue(fields()[0], other.num)) { + this.num = data().deepCopy(fields()[0].schema(), other.num); + fieldSetFlags()[0] = true; + } + if (isValidValue(fields()[1], other.street)) { + this.street = data().deepCopy(fields()[1].schema(), other.street); + fieldSetFlags()[1] = true; + } + if (isValidValue(fields()[2], other.city)) { + this.city = data().deepCopy(fields()[2].schema(), other.city); + fieldSetFlags()[2] = true; + } + if (isValidValue(fields()[3], other.state)) { + this.state = data().deepCopy(fields()[3].schema(), other.state); + fieldSetFlags()[3] = true; + } + if (isValidValue(fields()[4], other.zip)) { + this.zip = data().deepCopy(fields()[4].schema(), other.zip); + fieldSetFlags()[4] = true; + } + } + + /** + * Creates a Builder by copying an existing Address instance + * @param other The existing instance to copy. + */ + private Builder(org.apache.flink.formats.parquet.generated.Address other) { + super(SCHEMA$); + if (isValidValue(fields()[0], other.num)) { + this.num = data().deepCopy(fields()[0].schema(), other.num); + fieldSetFlags()[0] = true; + } + if (isValidValue(fields()[1], other.street)) { + this.street = data().deepCopy(fields()[1].schema(), other.street); + fieldSetFlags()[1] = true; + } + if (isValidValue(fields()[2], other.city)) { + this.city = data().deepCopy(fields()[2].schema(), other.city); + fieldSetFlags()[2] = true; + } + if (isValidValue(fields()[3], other.state)) { + this.state = data().deepCopy(fields()[3].schema(), other.state); + fieldSetFlags()[3] = true; + } + if (isValidValue(fields()[4], other.zip)) { + this.zip = data().deepCopy(fields()[4].schema(), other.zip); + fieldSetFlags()[4] = true; + } + } + + /** + * Gets the value of the 'num' field. + * @return The value. + */ + public java.lang.Integer getNum() { + return num; + } + + /** + * Sets the value of the 'num' field. + * @param value The value of 'num'. + * @return This builder. + */ + public org.apache.flink.formats.parquet.generated.Address.Builder setNum(int value) { + validate(fields()[0], value); + this.num = value; + fieldSetFlags()[0] = true; + return this; + } + + /** + * Checks whether the 'num' field has been set. + * @return True if the 'num' field has been set, false otherwise. + */ + public boolean hasNum() { + return fieldSetFlags()[0]; + } + + + /** + * Clears the value of the 'num' field. + * @return This builder. + */ + public org.apache.flink.formats.parquet.generated.Address.Builder clearNum() { + fieldSetFlags()[0] = false; + return this; + } + + /** + * Gets the value of the 'street' field. + * @return The value. + */ + public java.lang.CharSequence getStreet() { + return street; + } + + /** + * Sets the value of the 'street' field. + * @param value The value of 'street'. + * @return This builder. + */ + public org.apache.flink.formats.parquet.generated.Address.Builder setStreet(java.lang.CharSequence value) { + validate(fields()[1], value); + this.street = value; + fieldSetFlags()[1] = true; + return this; + } + + /** + * Checks whether the 'street' field has been set. + * @return True if the 'street' field has been set, false otherwise. + */ + public boolean hasStreet() { + return fieldSetFlags()[1]; + } + + + /** + * Clears the value of the 'street' field. + * @return This builder. + */ + public org.apache.flink.formats.parquet.generated.Address.Builder clearStreet() { + street = null; + fieldSetFlags()[1] = false; + return this; + } + + /** + * Gets the value of the 'city' field. + * @return The value. + */ + public java.lang.CharSequence getCity() { + return city; + } + + /** + * Sets the value of the 'city' field. + * @param value The value of 'city'. + * @return This builder. + */ + public org.apache.flink.formats.parquet.generated.Address.Builder setCity(java.lang.CharSequence value) { + validate(fields()[2], value); + this.city = value; + fieldSetFlags()[2] = true; + return this; + } + + /** + * Checks whether the 'city' field has been set. + * @return True if the 'city' field has been set, false otherwise. + */ + public boolean hasCity() { + return fieldSetFlags()[2]; + } + + + /** + * Clears the value of the 'city' field. + * @return This builder. + */ + public org.apache.flink.formats.parquet.generated.Address.Builder clearCity() { + city = null; + fieldSetFlags()[2] = false; + return this; + } + + /** + * Gets the value of the 'state' field. + * @return The value. + */ + public java.lang.CharSequence getState() { + return state; + } + + /** + * Sets the value of the 'state' field. + * @param value The value of 'state'. + * @return This builder. + */ + public org.apache.flink.formats.parquet.generated.Address.Builder setState(java.lang.CharSequence value) { + validate(fields()[3], value); + this.state = value; + fieldSetFlags()[3] = true; + return this; + } + + /** + * Checks whether the 'state' field has been set. + * @return True if the 'state' field has been set, false otherwise. + */ + public boolean hasState() { + return fieldSetFlags()[3]; + } + + + /** + * Clears the value of the 'state' field. + * @return This builder. + */ + public org.apache.flink.formats.parquet.generated.Address.Builder clearState() { + state = null; + fieldSetFlags()[3] = false; + return this; + } + + /** + * Gets the value of the 'zip' field. + * @return The value. + */ + public java.lang.CharSequence getZip() { + return zip; + } + + /** + * Sets the value of the 'zip' field. + * @param value The value of 'zip'. + * @return This builder. + */ + public org.apache.flink.formats.parquet.generated.Address.Builder setZip(java.lang.CharSequence value) { + validate(fields()[4], value); + this.zip = value; + fieldSetFlags()[4] = true; + return this; + } + + /** + * Checks whether the 'zip' field has been set. + * @return True if the 'zip' field has been set, false otherwise. + */ + public boolean hasZip() { + return fieldSetFlags()[4]; + } + + + /** + * Clears the value of the 'zip' field. + * @return This builder. + */ + public org.apache.flink.formats.parquet.generated.Address.Builder clearZip() { + zip = null; + fieldSetFlags()[4] = false; + return this; + } + + @Override + @SuppressWarnings("unchecked") + public Address build() { + try { + Address record = new Address(); + record.num = fieldSetFlags()[0] ? this.num : (java.lang.Integer) defaultValue(fields()[0]); + record.street = fieldSetFlags()[1] ? this.street : (java.lang.CharSequence) defaultValue(fields()[1]); + record.city = fieldSetFlags()[2] ? this.city : (java.lang.CharSequence) defaultValue(fields()[2]); + record.state = fieldSetFlags()[3] ? this.state : (java.lang.CharSequence) defaultValue(fields()[3]); + record.zip = fieldSetFlags()[4] ? this.zip : (java.lang.CharSequence) defaultValue(fields()[4]); + return record; + } catch (java.lang.Exception e) { + throw new org.apache.avro.AvroRuntimeException(e); + } + } + } + + @SuppressWarnings("unchecked") + private static final org.apache.avro.io.DatumWriter
    + WRITER$ = (org.apache.avro.io.DatumWriter
    )MODEL$.createDatumWriter(SCHEMA$); + + @Override public void writeExternal(java.io.ObjectOutput out) + throws java.io.IOException { + WRITER$.write(this, SpecificData.getEncoder(out)); + } + + @SuppressWarnings("unchecked") + private static final org.apache.avro.io.DatumReader
    + READER$ = (org.apache.avro.io.DatumReader
    )MODEL$.createDatumReader(SCHEMA$); + + @Override public void readExternal(java.io.ObjectInput in) + throws java.io.IOException { + READER$.read(this, SpecificData.getDecoder(in)); + } + +} diff --git a/flink-formats/flink-parquet/src/test/java/org/apache/flink/formats/parquet/testutils/FiniteTestSource.java b/flink-formats/flink-parquet/src/test/java/org/apache/flink/formats/parquet/testutils/FiniteTestSource.java new file mode 100644 index 0000000000000..03db7ff4e8161 --- /dev/null +++ b/flink-formats/flink-parquet/src/test/java/org/apache/flink/formats/parquet/testutils/FiniteTestSource.java @@ -0,0 +1,79 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.formats.parquet.testutils; + +import org.apache.flink.runtime.state.CheckpointListener; +import org.apache.flink.streaming.api.functions.source.SourceFunction; + +import java.util.Arrays; + +/** + * A stream source that emits elements without allowing checkpoints and waits + * for two more checkpoints to complete before exiting. + */ +@SuppressWarnings("SynchronizationOnLocalVariableOrMethodParameter") +public class FiniteTestSource implements SourceFunction, CheckpointListener { + + private static final long serialVersionUID = 1L; + + @SuppressWarnings("NonSerializableFieldInSerializableClass") + private final Iterable elements; + + private volatile boolean running = true; + + private transient int numCheckpointsComplete; + + @SafeVarargs + public FiniteTestSource(T... elements) { + this(Arrays.asList(elements)); + } + + public FiniteTestSource(Iterable elements) { + this.elements = elements; + } + + @Override + public void run(SourceContext ctx) throws Exception { + final Object lock = ctx.getCheckpointLock(); + final int checkpointToAwait; + + synchronized (lock) { + checkpointToAwait = numCheckpointsComplete + 2; + for (T t : elements) { + ctx.collect(t); + } + } + + synchronized (lock) { + while (running && numCheckpointsComplete < checkpointToAwait) { + lock.wait(1); + } + } + } + + @Override + public void cancel() { + running = false; + } + + @Override + public void notifyCheckpointComplete(long checkpointId) throws Exception { + numCheckpointsComplete++; + } +} diff --git a/flink-formats/flink-parquet/src/test/resources/avro/testdata.avsc b/flink-formats/flink-parquet/src/test/resources/avro/testdata.avsc new file mode 100644 index 0000000000000..c78190074e5f0 --- /dev/null +++ b/flink-formats/flink-parquet/src/test/resources/avro/testdata.avsc @@ -0,0 +1,13 @@ +[ +{"namespace": "org.apache.flink.formats.parquet.generated", + "type": "record", + "name": "Address", + "fields": [ + {"name": "num", "type": "int"}, + {"name": "street", "type": "string"}, + {"name": "city", "type": "string"}, + {"name": "state", "type": "string"}, + {"name": "zip", "type": "string"} + ] +} +] diff --git a/flink-formats/flink-parquet/src/test/resources/log4j-test.properties b/flink-formats/flink-parquet/src/test/resources/log4j-test.properties new file mode 100644 index 0000000000000..2a30ab82eaf98 --- /dev/null +++ b/flink-formats/flink-parquet/src/test/resources/log4j-test.properties @@ -0,0 +1,23 @@ +################################################################################ +# 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. +################################################################################ + +log4j.rootLogger=OFF, testlogger +log4j.appender.testlogger=org.apache.log4j.ConsoleAppender +log4j.appender.testlogger.target = System.err +log4j.appender.testlogger.layout=org.apache.log4j.PatternLayout +log4j.appender.testlogger.layout.ConversionPattern=%-4r [%t] %-5p %c %x - %m%n diff --git a/flink-formats/pom.xml b/flink-formats/pom.xml index 2fb2c677a4602..869683f627f70 100644 --- a/flink-formats/pom.xml +++ b/flink-formats/pom.xml @@ -39,6 +39,7 @@ under the License. flink-avro flink-json flink-avro-confluent-registry + flink-parquet diff --git a/tools/maven/suppressions.xml b/tools/maven/suppressions.xml index ae029d62186af..4594d7fd9f3ca 100644 --- a/tools/maven/suppressions.xml +++ b/tools/maven/suppressions.xml @@ -24,6 +24,7 @@ under the License. + diff --git a/tools/travis_mvn_watchdog.sh b/tools/travis_mvn_watchdog.sh index a47475b8c71c9..c4620c837af18 100755 --- a/tools/travis_mvn_watchdog.sh +++ b/tools/travis_mvn_watchdog.sh @@ -79,6 +79,7 @@ flink-filesystems/flink-mapr-fs,\ flink-filesystems/flink-s3-fs-hadoop,\ flink-filesystems/flink-s3-fs-presto,\ flink-formats/flink-avro,\ +flink-formats/flink-parquet,\ flink-connectors/flink-hbase,\ flink-connectors/flink-hcatalog,\ flink-connectors/flink-hadoop-compatibility,\ From 702f77355bbb9ccf560885dd5c6a717a25cafa53 Mon Sep 17 00:00:00 2001 From: kkloudas Date: Fri, 20 Jul 2018 17:03:16 +0200 Subject: [PATCH 46/65] [FLINK-9921][DataStream API] Update RollingPolicy interface --- .../functions/sink/filesystem/Buckets.java | 13 +++------ .../sink/filesystem/RollingPolicy.java | 5 ++-- .../sink/filesystem/StreamingFileSink.java | 7 +++-- .../policies/DefaultRollingPolicy.java | 27 +++++-------------- .../policies/OnCheckpointRollingPolicy.java | 4 +-- .../LocalStreamingFileSinkTest.java | 2 +- .../sink/filesystem/RollingPolicyTest.java | 8 +++--- .../functions/sink/filesystem/TestUtils.java | 4 +-- 8 files changed, 26 insertions(+), 44 deletions(-) diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/sink/filesystem/Buckets.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/sink/filesystem/Buckets.java index 6afba170c50b1..e6f8c002456aa 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/sink/filesystem/Buckets.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/sink/filesystem/Buckets.java @@ -61,7 +61,7 @@ public class Buckets { private final PartFileWriter.PartFileFactory partFileWriterFactory; - private final RollingPolicy rollingPolicy; + private final RollingPolicy rollingPolicy; // --------------------------- runtime fields ----------------------------- @@ -95,7 +95,7 @@ public class Buckets { final Bucketer bucketer, final BucketFactory bucketFactory, final PartFileWriter.PartFileFactory partFileWriterFactory, - final RollingPolicy rollingPolicy, + final RollingPolicy rollingPolicy, final int subtaskIndex) throws IOException { this.basePath = Preconditions.checkNotNull(basePath); @@ -189,7 +189,6 @@ void publishUpToCheckpoint(long checkpointId) throws IOException { void snapshotState( final long checkpointId, - final long checkpointTimestamp, final ListState bucketStates, final ListState partCounterState) throws Exception { @@ -201,11 +200,7 @@ void snapshotState( for (Bucket bucket : activeBuckets.values()) { final PartFileInfo info = bucket.getInProgressPartInfo(); - if (info != null && - (rollingPolicy.shouldRollOnCheckpoint(info) || - rollingPolicy.shouldRollOnEvent(info) || - rollingPolicy.shouldRollOnProcessingTime(info, checkpointTimestamp)) - ) { + if (info != null && rollingPolicy.shouldRollOnCheckpoint(info)) { // we also check here so that we do not have to always // wait for the "next" element to arrive. bucket.closePartFile(); @@ -249,7 +244,7 @@ void onElement(IN value, SinkFunction.Context context) throws Exception { } final PartFileInfo info = bucket.getInProgressPartInfo(); - if (info == null || rollingPolicy.shouldRollOnEvent(info)) { + if (info == null || rollingPolicy.shouldRollOnEvent(info, value)) { bucket.rollPartFile(currentProcessingTime); } bucket.write(value, currentProcessingTime); diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/sink/filesystem/RollingPolicy.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/sink/filesystem/RollingPolicy.java index 24c38aa58b7eb..b1354a9620a43 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/sink/filesystem/RollingPolicy.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/sink/filesystem/RollingPolicy.java @@ -28,7 +28,7 @@ * rolls its currently open part file and opens a new one. */ @PublicEvolving -public interface RollingPolicy extends Serializable { +public interface RollingPolicy extends Serializable { /** * Determines if the in-progress part file for a bucket should roll on every checkpoint. @@ -39,10 +39,11 @@ public interface RollingPolicy extends Serializable { /** * Determines if the in-progress part file for a bucket should roll based on its current state, e.g. its size. + * @param element the element being processed. * @param partFileState the state of the currently open part file of the bucket. * @return {@code True} if the part file should roll, {@link false} otherwise. */ - boolean shouldRollOnEvent(final PartFileInfo partFileState) throws IOException; + boolean shouldRollOnEvent(final PartFileInfo partFileState, IN element) throws IOException; /** * Determines if the in-progress part file for a bucket should roll based on a time condition. diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/sink/filesystem/StreamingFileSink.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/sink/filesystem/StreamingFileSink.java index c2080796e4b65..0ebcc4f4c0570 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/sink/filesystem/StreamingFileSink.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/sink/filesystem/StreamingFileSink.java @@ -194,7 +194,7 @@ public static class RowFormatBuilder extends StreamingFileSink.Buc private Bucketer bucketer; - private RollingPolicy rollingPolicy; + private RollingPolicy rollingPolicy; private BucketFactory bucketFactory = new DefaultBucketFactory<>(); @@ -215,12 +215,12 @@ public StreamingFileSink.RowFormatBuilder withBucketer(final Bucke return this; } - public StreamingFileSink.RowFormatBuilder withRollingPolicy(final RollingPolicy policy) { + public StreamingFileSink.RowFormatBuilder withRollingPolicy(final RollingPolicy policy) { this.rollingPolicy = Preconditions.checkNotNull(policy); return this; } - public StreamingFileSink.RowFormatBuilder withBucketerAndPolicy(final Bucketer bucketer, final RollingPolicy policy) { + public StreamingFileSink.RowFormatBuilder withBucketerAndPolicy(final Bucketer bucketer, final RollingPolicy policy) { @SuppressWarnings("unchecked") StreamingFileSink.RowFormatBuilder reInterpreted = (StreamingFileSink.RowFormatBuilder) this; reInterpreted.bucketer = Preconditions.checkNotNull(bucketer); @@ -340,7 +340,6 @@ public void snapshotState(FunctionSnapshotContext context) throws Exception { buckets.snapshotState( context.getCheckpointId(), - context.getCheckpointTimestamp(), bucketStates, maxPartCountersState); } diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/sink/filesystem/rolling/policies/DefaultRollingPolicy.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/sink/filesystem/rolling/policies/DefaultRollingPolicy.java index a9ff617bc223f..15c3b4dc3499f 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/sink/filesystem/rolling/policies/DefaultRollingPolicy.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/sink/filesystem/rolling/policies/DefaultRollingPolicy.java @@ -37,7 +37,7 @@ * */ @PublicEvolving -public final class DefaultRollingPolicy implements RollingPolicy { +public final class DefaultRollingPolicy implements RollingPolicy { private static final long serialVersionUID = 1L; @@ -67,32 +67,19 @@ private DefaultRollingPolicy(long partSize, long rolloverInterval, long inactivi } @Override - public boolean shouldRollOnCheckpoint(PartFileInfo partFileState) { - return false; + public boolean shouldRollOnCheckpoint(PartFileInfo partFileState) throws IOException { + return partFileState.getSize() > partSize; } @Override - public boolean shouldRollOnEvent(PartFileInfo partFileState) throws IOException { - if (partFileState == null) { - // this means that there is no currently open part file. - return true; - } - + public boolean shouldRollOnEvent(PartFileInfo partFileState, IN element) throws IOException { return partFileState.getSize() > partSize; } @Override public boolean shouldRollOnProcessingTime(final PartFileInfo partFileState, final long currentTime) { - if (partFileState == null) { - // this means that there is no currently open part file. - return true; - } - - if (currentTime - partFileState.getCreationTime() > rolloverInterval) { - return true; - } - - return currentTime - partFileState.getLastUpdateTime() > inactivityInterval; + return currentTime - partFileState.getCreationTime() > rolloverInterval || + currentTime - partFileState.getLastUpdateTime() > inactivityInterval; } /** @@ -150,7 +137,7 @@ public DefaultRollingPolicy.PolicyBuilder withRolloverInterval(long interval) { /** * Creates the actual policy. */ - public DefaultRollingPolicy build() { + public DefaultRollingPolicy build() { return new DefaultRollingPolicy<>(partSize, rolloverInterval, inactivityInterval); } } diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/sink/filesystem/rolling/policies/OnCheckpointRollingPolicy.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/sink/filesystem/rolling/policies/OnCheckpointRollingPolicy.java index 4361941951a3c..df15981c8a5e5 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/sink/filesystem/rolling/policies/OnCheckpointRollingPolicy.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/sink/filesystem/rolling/policies/OnCheckpointRollingPolicy.java @@ -24,7 +24,7 @@ /** * A {@link RollingPolicy} which rolls on every checkpoint. */ -public class OnCheckpointRollingPolicy implements RollingPolicy { +public class OnCheckpointRollingPolicy implements RollingPolicy { private static final long serialVersionUID = 1L; @@ -34,7 +34,7 @@ public boolean shouldRollOnCheckpoint(PartFileInfo partFileState) { } @Override - public boolean shouldRollOnEvent(PartFileInfo partFileState) { + public boolean shouldRollOnEvent(PartFileInfo partFileState, IN element) { return false; } diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/functions/sink/filesystem/LocalStreamingFileSinkTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/functions/sink/filesystem/LocalStreamingFileSinkTest.java index 6e942e95d4637..7c23918d0643a 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/functions/sink/filesystem/LocalStreamingFileSinkTest.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/functions/sink/filesystem/LocalStreamingFileSinkTest.java @@ -494,7 +494,7 @@ public void testMaxCounterUponRecovery() throws Exception { final TestBucketFactory first = new TestBucketFactory(); final TestBucketFactory second = new TestBucketFactory(); - final RollingPolicy rollingPolicy = DefaultRollingPolicy + final RollingPolicy, String> rollingPolicy = DefaultRollingPolicy .create() .withMaxPartSize(2L) .withRolloverInterval(100L) diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/functions/sink/filesystem/RollingPolicyTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/functions/sink/filesystem/RollingPolicyTest.java index 61e143350ee2f..078a46b8f5bca 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/functions/sink/filesystem/RollingPolicyTest.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/functions/sink/filesystem/RollingPolicyTest.java @@ -44,7 +44,7 @@ public class RollingPolicyTest { public void testDefaultRollingPolicy() throws Exception { final File outDir = TEMP_FOLDER.newFolder(); - final RollingPolicy rollingPolicy = DefaultRollingPolicy + final RollingPolicy, String> rollingPolicy = DefaultRollingPolicy .create() .withMaxPartSize(10L) .withInactivityInterval(4L) @@ -104,7 +104,7 @@ public void testDefaultRollingPolicy() throws Exception { public void testRollOnCheckpointPolicy() throws Exception { final File outDir = TEMP_FOLDER.newFolder(); - final RollingPolicy rollingPolicy = new OnCheckpointRollingPolicy<>(); + final RollingPolicy, String> rollingPolicy = new OnCheckpointRollingPolicy<>(); try ( OneInputStreamOperatorTestHarness, Object> testHarness = TestUtils.createCustomRescalingTestSink( @@ -159,7 +159,7 @@ public void testRollOnCheckpointPolicy() throws Exception { public void testCustomRollingPolicy() throws Exception { final File outDir = TEMP_FOLDER.newFolder(); - final RollingPolicy rollingPolicy = new RollingPolicy() { + final RollingPolicy, String> rollingPolicy = new RollingPolicy, String>() { private static final long serialVersionUID = 1L; @@ -169,7 +169,7 @@ public boolean shouldRollOnCheckpoint(PartFileInfo partFileState) { } @Override - public boolean shouldRollOnEvent(PartFileInfo partFileState) throws IOException { + public boolean shouldRollOnEvent(PartFileInfo partFileState, Tuple2 element) throws IOException { // this means that 2 elements will close the part file. return partFileState.getSize() > 12L; } diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/functions/sink/filesystem/TestUtils.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/functions/sink/filesystem/TestUtils.java index 184e23ee458b4..9589c5acb8277 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/functions/sink/filesystem/TestUtils.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/functions/sink/filesystem/TestUtils.java @@ -51,7 +51,7 @@ static OneInputStreamOperatorTestHarness, Object> create long inactivityInterval, long partMaxSize) throws Exception { - final RollingPolicy rollingPolicy = + final RollingPolicy, String> rollingPolicy = DefaultRollingPolicy .create() .withMaxPartSize(partMaxSize) @@ -84,7 +84,7 @@ static OneInputStreamOperatorTestHarness, Object> create final long bucketCheckInterval, final Bucketer, String> bucketer, final Encoder> writer, - final RollingPolicy rollingPolicy, + final RollingPolicy, String> rollingPolicy, final BucketFactory, String> bucketFactory) throws Exception { StreamingFileSink> sink = StreamingFileSink From aa25b4b324b4c025fe9e58e081677faf0ab50a7d Mon Sep 17 00:00:00 2001 From: Timo Walther Date: Mon, 23 Jul 2018 08:12:00 +0200 Subject: [PATCH 47/65] [FLINK-9846] [table] Add a Kafka table sink factory Adds a Kafka table sink factory with format discovery. Currently, this enables the SQL Client to write Avro and JSON data to Kafka. The functionality is limited due to FLINK-9870. Therefore, it is currently not possible to use time attributes in the output. Changes: - Decouple Kafka sink from formats and deprecate old classes - Add a Kafka table sink factory - Existing tests for the KafkaTableSourceFactory have been generalized to support sinks as well. This closes #6387. --- .../kafka/Kafka010JsonTableSink.java | 19 +- .../connectors/kafka/Kafka010TableSink.java | 61 +++++ ...va => Kafka010TableSourceSinkFactory.java} | 20 +- ....apache.flink.table.factories.TableFactory | 2 +- .../kafka/Kafka010JsonTableSinkTest.java | 4 + ...> Kafka010TableSourceSinkFactoryTest.java} | 29 ++- .../connectors/kafka/Kafka011TableSink.java | 64 +++++ .../connectors/kafka/Kafka011TableSource.java | 3 +- ...va => Kafka011TableSourceSinkFactory.java} | 20 +- ....apache.flink.table.factories.TableFactory | 2 +- ...> Kafka011TableSourceSinkFactoryTest.java} | 29 ++- .../kafka/Kafka08JsonTableSink.java | 19 +- .../connectors/kafka/Kafka08TableSink.java | 61 +++++ .../connectors/kafka/Kafka08TableSource.java | 3 +- ...ava => Kafka08TableSourceSinkFactory.java} | 20 +- ....apache.flink.table.factories.TableFactory | 2 +- .../kafka/Kafka08JsonTableSinkTest.java | 4 + ...=> Kafka08TableSourceSinkFactoryTest.java} | 29 ++- .../kafka/Kafka09JsonTableSink.java | 19 +- .../connectors/kafka/Kafka09TableSink.java | 61 +++++ .../connectors/kafka/Kafka09TableSource.java | 3 +- ...ava => Kafka09TableSourceSinkFactory.java} | 20 +- ....apache.flink.table.factories.TableFactory | 2 +- .../kafka/Kafka09JsonTableSinkTest.java | 4 + ...=> Kafka09TableSourceSinkFactoryTest.java} | 29 ++- .../connectors/kafka/KafkaJsonTableSink.java | 5 + .../connectors/kafka/KafkaTableSink.java | 112 ++++++++- ...a => KafkaTableSourceSinkFactoryBase.java} | 233 ++++++++++++------ .../partitioner/FlinkFixedPartitioner.java | 10 + .../kafka/KafkaTableSinkTestBase.java | 7 +- ... KafkaTableSourceSinkFactoryTestBase.java} | 143 +++++++++-- .../utils/TestSerializationSchema.scala | 16 +- .../utils/TestTableFormatFactory.scala | 18 +- 33 files changed, 930 insertions(+), 143 deletions(-) create mode 100644 flink-connectors/flink-connector-kafka-0.10/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka010TableSink.java rename flink-connectors/flink-connector-kafka-0.10/src/main/java/org/apache/flink/streaming/connectors/kafka/{Kafka010TableSourceFactory.java => Kafka010TableSourceSinkFactory.java} (80%) rename flink-connectors/flink-connector-kafka-0.10/src/test/java/org/apache/flink/streaming/connectors/kafka/{Kafka010TableSourceFactoryTest.java => Kafka010TableSourceSinkFactoryTest.java} (73%) create mode 100644 flink-connectors/flink-connector-kafka-0.11/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka011TableSink.java rename flink-connectors/flink-connector-kafka-0.11/src/main/java/org/apache/flink/streaming/connectors/kafka/{Kafka011TableSourceFactory.java => Kafka011TableSourceSinkFactory.java} (80%) rename flink-connectors/flink-connector-kafka-0.11/src/test/java/org/apache/flink/streaming/connectors/kafka/{Kafka011TableSourceFactoryTest.java => Kafka011TableSourceSinkFactoryTest.java} (73%) create mode 100644 flink-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka08TableSink.java rename flink-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/{Kafka08TableSourceFactory.java => Kafka08TableSourceSinkFactory.java} (80%) rename flink-connectors/flink-connector-kafka-0.8/src/test/java/org/apache/flink/streaming/connectors/kafka/{Kafka08TableSourceFactoryTest.java => Kafka08TableSourceSinkFactoryTest.java} (74%) create mode 100644 flink-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka09TableSink.java rename flink-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/{Kafka09TableSourceFactory.java => Kafka09TableSourceSinkFactory.java} (80%) rename flink-connectors/flink-connector-kafka-0.9/src/test/java/org/apache/flink/streaming/connectors/kafka/{Kafka09TableSourceFactoryTest.java => Kafka09TableSourceSinkFactoryTest.java} (74%) rename flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/{KafkaTableSourceFactory.java => KafkaTableSourceSinkFactoryBase.java} (64%) rename flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/{KafkaTableSourceFactoryTestBase.java => KafkaTableSourceSinkFactoryTestBase.java} (59%) diff --git a/flink-connectors/flink-connector-kafka-0.10/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka010JsonTableSink.java b/flink-connectors/flink-connector-kafka-0.10/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka010JsonTableSink.java index ef33cd56237cc..2ad31420789a4 100644 --- a/flink-connectors/flink-connector-kafka-0.10/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka010JsonTableSink.java +++ b/flink-connectors/flink-connector-kafka-0.10/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka010JsonTableSink.java @@ -18,18 +18,23 @@ package org.apache.flink.streaming.connectors.kafka; -import org.apache.flink.annotation.PublicEvolving; import org.apache.flink.api.common.serialization.SerializationSchema; import org.apache.flink.streaming.connectors.kafka.partitioner.FlinkFixedPartitioner; import org.apache.flink.streaming.connectors.kafka.partitioner.FlinkKafkaPartitioner; +import org.apache.flink.table.descriptors.ConnectorDescriptor; import org.apache.flink.types.Row; import java.util.Properties; /** * Kafka 0.10 {@link KafkaTableSink} that serializes data in JSON format. + * + * @deprecated Use the {@link org.apache.flink.table.descriptors.Kafka} descriptor together + * with descriptors for schema and format instead. Descriptors allow for + * implementation-agnostic definition of tables. See also + * {@link org.apache.flink.table.api.TableEnvironment#connect(ConnectorDescriptor)}. */ -@PublicEvolving +@Deprecated public class Kafka010JsonTableSink extends KafkaJsonTableSink { /** @@ -46,7 +51,9 @@ public class Kafka010JsonTableSink extends KafkaJsonTableSink { * * @param topic topic in Kafka to which table is written * @param properties properties to connect to Kafka + * @deprecated Use table descriptors instead of implementation-specific classes. */ + @Deprecated public Kafka010JsonTableSink(String topic, Properties properties) { super(topic, properties, new FlinkFixedPartitioner<>()); } @@ -58,14 +65,20 @@ public Kafka010JsonTableSink(String topic, Properties properties) { * @param topic topic in Kafka to which table is written * @param properties properties to connect to Kafka * @param partitioner Kafka partitioner + * @deprecated Use table descriptors instead of implementation-specific classes. */ + @Deprecated public Kafka010JsonTableSink(String topic, Properties properties, FlinkKafkaPartitioner partitioner) { super(topic, properties, partitioner); } @Override protected FlinkKafkaProducerBase createKafkaProducer(String topic, Properties properties, SerializationSchema serializationSchema, FlinkKafkaPartitioner partitioner) { - return new FlinkKafkaProducer010<>(topic, serializationSchema, properties, partitioner); + return new FlinkKafkaProducer010<>( + topic, + serializationSchema, + properties, + partitioner); } @Override diff --git a/flink-connectors/flink-connector-kafka-0.10/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka010TableSink.java b/flink-connectors/flink-connector-kafka-0.10/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka010TableSink.java new file mode 100644 index 0000000000000..a8c655398243d --- /dev/null +++ b/flink-connectors/flink-connector-kafka-0.10/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka010TableSink.java @@ -0,0 +1,61 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.streaming.connectors.kafka; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.api.common.serialization.SerializationSchema; +import org.apache.flink.streaming.connectors.kafka.partitioner.FlinkKafkaPartitioner; +import org.apache.flink.table.api.TableSchema; +import org.apache.flink.types.Row; + +import java.util.Properties; + +/** + * Kafka 0.10 table sink for writing data into Kafka. + */ +@Internal +public class Kafka010TableSink extends KafkaTableSink { + + public Kafka010TableSink( + TableSchema schema, + String topic, + Properties properties, + FlinkKafkaPartitioner partitioner, + SerializationSchema serializationSchema) { + super( + schema, + topic, + properties, + partitioner, + serializationSchema); + } + + @Override + protected FlinkKafkaProducerBase createKafkaProducer( + String topic, + Properties properties, + SerializationSchema serializationSchema, + FlinkKafkaPartitioner partitioner) { + return new FlinkKafkaProducer010<>( + topic, + serializationSchema, + properties, + partitioner); + } +} diff --git a/flink-connectors/flink-connector-kafka-0.10/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka010TableSourceFactory.java b/flink-connectors/flink-connector-kafka-0.10/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka010TableSourceSinkFactory.java similarity index 80% rename from flink-connectors/flink-connector-kafka-0.10/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka010TableSourceFactory.java rename to flink-connectors/flink-connector-kafka-0.10/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka010TableSourceSinkFactory.java index 4a860162222a7..0cf94995bdd49 100644 --- a/flink-connectors/flink-connector-kafka-0.10/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka010TableSourceFactory.java +++ b/flink-connectors/flink-connector-kafka-0.10/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka010TableSourceSinkFactory.java @@ -19,8 +19,10 @@ package org.apache.flink.streaming.connectors.kafka; import org.apache.flink.api.common.serialization.DeserializationSchema; +import org.apache.flink.api.common.serialization.SerializationSchema; import org.apache.flink.streaming.connectors.kafka.config.StartupMode; import org.apache.flink.streaming.connectors.kafka.internals.KafkaTopicPartition; +import org.apache.flink.streaming.connectors.kafka.partitioner.FlinkKafkaPartitioner; import org.apache.flink.table.api.TableSchema; import org.apache.flink.table.descriptors.KafkaValidator; import org.apache.flink.table.sources.RowtimeAttributeDescriptor; @@ -34,7 +36,7 @@ /** * Factory for creating configured instances of {@link Kafka010TableSource}. */ -public class Kafka010TableSourceFactory extends KafkaTableSourceFactory { +public class Kafka010TableSourceSinkFactory extends KafkaTableSourceSinkFactoryBase { @Override protected String kafkaVersion() { @@ -69,4 +71,20 @@ protected KafkaTableSource createKafkaTableSource( startupMode, specificStartupOffsets); } + + @Override + protected KafkaTableSink createKafkaTableSink( + TableSchema schema, + String topic, + Properties properties, + FlinkKafkaPartitioner partitioner, + SerializationSchema serializationSchema) { + + return new Kafka010TableSink( + schema, + topic, + properties, + partitioner, + serializationSchema); + } } diff --git a/flink-connectors/flink-connector-kafka-0.10/src/main/resources/META-INF/services/org.apache.flink.table.factories.TableFactory b/flink-connectors/flink-connector-kafka-0.10/src/main/resources/META-INF/services/org.apache.flink.table.factories.TableFactory index 21f57077d6348..9bb036302f8ed 100644 --- a/flink-connectors/flink-connector-kafka-0.10/src/main/resources/META-INF/services/org.apache.flink.table.factories.TableFactory +++ b/flink-connectors/flink-connector-kafka-0.10/src/main/resources/META-INF/services/org.apache.flink.table.factories.TableFactory @@ -13,4 +13,4 @@ # See the License for the specific language governing permissions and # limitations under the License. -org.apache.flink.streaming.connectors.kafka.Kafka010TableSourceFactory +org.apache.flink.streaming.connectors.kafka.Kafka010TableSourceSinkFactory diff --git a/flink-connectors/flink-connector-kafka-0.10/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka010JsonTableSinkTest.java b/flink-connectors/flink-connector-kafka-0.10/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka010JsonTableSinkTest.java index af562c6fa9e93..339420cede3d3 100644 --- a/flink-connectors/flink-connector-kafka-0.10/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka010JsonTableSinkTest.java +++ b/flink-connectors/flink-connector-kafka-0.10/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka010JsonTableSinkTest.java @@ -27,7 +27,11 @@ /** * Tests for the {@link Kafka010JsonTableSink}. + * + * @deprecated Ensures backwards compatibility with Flink 1.5. Can be removed once we + * drop support for format-specific table sinks. */ +@Deprecated public class Kafka010JsonTableSinkTest extends KafkaTableSinkTestBase { @Override diff --git a/flink-connectors/flink-connector-kafka-0.10/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka010TableSourceFactoryTest.java b/flink-connectors/flink-connector-kafka-0.10/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka010TableSourceSinkFactoryTest.java similarity index 73% rename from flink-connectors/flink-connector-kafka-0.10/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka010TableSourceFactoryTest.java rename to flink-connectors/flink-connector-kafka-0.10/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka010TableSourceSinkFactoryTest.java index ff3b0b0001f98..cc198c9159503 100644 --- a/flink-connectors/flink-connector-kafka-0.10/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka010TableSourceFactoryTest.java +++ b/flink-connectors/flink-connector-kafka-0.10/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka010TableSourceSinkFactoryTest.java @@ -19,8 +19,10 @@ package org.apache.flink.streaming.connectors.kafka; import org.apache.flink.api.common.serialization.DeserializationSchema; +import org.apache.flink.api.common.serialization.SerializationSchema; import org.apache.flink.streaming.connectors.kafka.config.StartupMode; import org.apache.flink.streaming.connectors.kafka.internals.KafkaTopicPartition; +import org.apache.flink.streaming.connectors.kafka.partitioner.FlinkKafkaPartitioner; import org.apache.flink.table.api.TableSchema; import org.apache.flink.table.descriptors.KafkaValidator; import org.apache.flink.table.sources.RowtimeAttributeDescriptor; @@ -32,9 +34,10 @@ import java.util.Properties; /** - * Test for {@link Kafka010TableSource} created by {@link Kafka010TableSourceFactory}. + * Test for {@link Kafka010TableSource} and {@link Kafka010TableSink} created + * by {@link Kafka010TableSourceSinkFactory}. */ -public class Kafka010TableSourceFactoryTest extends KafkaTableSourceFactoryTestBase { +public class Kafka010TableSourceSinkFactoryTest extends KafkaTableSourceSinkFactoryTestBase { @Override protected String getKafkaVersion() { @@ -47,6 +50,11 @@ protected Class> getExpectedFlinkKafkaConsumer() { return (Class) FlinkKafkaConsumer010.class; } + @Override + protected Class getExpectedFlinkKafkaProducer() { + return FlinkKafkaProducer010.class; + } + @Override protected KafkaTableSource getExpectedKafkaTableSource( TableSchema schema, @@ -71,4 +79,21 @@ protected KafkaTableSource getExpectedKafkaTableSource( specificStartupOffsets ); } + + @Override + protected KafkaTableSink getExpectedKafkaTableSink( + TableSchema schema, + String topic, + Properties properties, + FlinkKafkaPartitioner partitioner, + SerializationSchema serializationSchema) { + + return new Kafka010TableSink( + schema, + topic, + properties, + partitioner, + serializationSchema + ); + } } diff --git a/flink-connectors/flink-connector-kafka-0.11/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka011TableSink.java b/flink-connectors/flink-connector-kafka-0.11/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka011TableSink.java new file mode 100644 index 0000000000000..22c6da13b05a8 --- /dev/null +++ b/flink-connectors/flink-connector-kafka-0.11/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka011TableSink.java @@ -0,0 +1,64 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.streaming.connectors.kafka; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.api.common.serialization.SerializationSchema; +import org.apache.flink.streaming.api.functions.sink.SinkFunction; +import org.apache.flink.streaming.connectors.kafka.partitioner.FlinkKafkaPartitioner; +import org.apache.flink.streaming.util.serialization.KeyedSerializationSchemaWrapper; +import org.apache.flink.table.api.TableSchema; +import org.apache.flink.types.Row; + +import java.util.Optional; +import java.util.Properties; + +/** + * Kafka 0.11 table sink for writing data into Kafka. + */ +@Internal +public class Kafka011TableSink extends KafkaTableSink { + + public Kafka011TableSink( + TableSchema schema, + String topic, + Properties properties, + FlinkKafkaPartitioner partitioner, + SerializationSchema serializationSchema) { + super( + schema, + topic, + properties, + partitioner, + serializationSchema); + } + + @Override + protected SinkFunction createKafkaProducer( + String topic, + Properties properties, + SerializationSchema serializationSchema, + FlinkKafkaPartitioner partitioner) { + return new FlinkKafkaProducer011<>( + topic, + new KeyedSerializationSchemaWrapper<>(serializationSchema), + properties, + Optional.of(partitioner)); + } +} diff --git a/flink-connectors/flink-connector-kafka-0.11/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka011TableSource.java b/flink-connectors/flink-connector-kafka-0.11/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka011TableSource.java index 85f566925bb60..a646317fe0a10 100644 --- a/flink-connectors/flink-connector-kafka-0.11/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka011TableSource.java +++ b/flink-connectors/flink-connector-kafka-0.11/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka011TableSource.java @@ -58,7 +58,8 @@ public Kafka011TableSource( Optional proctimeAttribute, List rowtimeAttributeDescriptors, Optional> fieldMapping, - String topic, Properties properties, + String topic, + Properties properties, DeserializationSchema deserializationSchema, StartupMode startupMode, Map specificStartupOffsets) { diff --git a/flink-connectors/flink-connector-kafka-0.11/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka011TableSourceFactory.java b/flink-connectors/flink-connector-kafka-0.11/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka011TableSourceSinkFactory.java similarity index 80% rename from flink-connectors/flink-connector-kafka-0.11/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka011TableSourceFactory.java rename to flink-connectors/flink-connector-kafka-0.11/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka011TableSourceSinkFactory.java index b1e392921a3f9..c26df42ed4ac4 100644 --- a/flink-connectors/flink-connector-kafka-0.11/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka011TableSourceFactory.java +++ b/flink-connectors/flink-connector-kafka-0.11/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka011TableSourceSinkFactory.java @@ -19,8 +19,10 @@ package org.apache.flink.streaming.connectors.kafka; import org.apache.flink.api.common.serialization.DeserializationSchema; +import org.apache.flink.api.common.serialization.SerializationSchema; import org.apache.flink.streaming.connectors.kafka.config.StartupMode; import org.apache.flink.streaming.connectors.kafka.internals.KafkaTopicPartition; +import org.apache.flink.streaming.connectors.kafka.partitioner.FlinkKafkaPartitioner; import org.apache.flink.table.api.TableSchema; import org.apache.flink.table.descriptors.KafkaValidator; import org.apache.flink.table.sources.RowtimeAttributeDescriptor; @@ -34,7 +36,7 @@ /** * Factory for creating configured instances of {@link Kafka011TableSource}. */ -public class Kafka011TableSourceFactory extends KafkaTableSourceFactory { +public class Kafka011TableSourceSinkFactory extends KafkaTableSourceSinkFactoryBase { @Override protected String kafkaVersion() { @@ -69,4 +71,20 @@ protected KafkaTableSource createKafkaTableSource( startupMode, specificStartupOffsets); } + + @Override + protected KafkaTableSink createKafkaTableSink( + TableSchema schema, + String topic, + Properties properties, + FlinkKafkaPartitioner partitioner, + SerializationSchema serializationSchema) { + + return new Kafka011TableSink( + schema, + topic, + properties, + partitioner, + serializationSchema); + } } diff --git a/flink-connectors/flink-connector-kafka-0.11/src/main/resources/META-INF/services/org.apache.flink.table.factories.TableFactory b/flink-connectors/flink-connector-kafka-0.11/src/main/resources/META-INF/services/org.apache.flink.table.factories.TableFactory index c056097ef8c12..b59b4a77ef5e3 100644 --- a/flink-connectors/flink-connector-kafka-0.11/src/main/resources/META-INF/services/org.apache.flink.table.factories.TableFactory +++ b/flink-connectors/flink-connector-kafka-0.11/src/main/resources/META-INF/services/org.apache.flink.table.factories.TableFactory @@ -13,4 +13,4 @@ # See the License for the specific language governing permissions and # limitations under the License. -org.apache.flink.streaming.connectors.kafka.Kafka011TableSourceFactory +org.apache.flink.streaming.connectors.kafka.Kafka011TableSourceSinkFactory diff --git a/flink-connectors/flink-connector-kafka-0.11/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka011TableSourceFactoryTest.java b/flink-connectors/flink-connector-kafka-0.11/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka011TableSourceSinkFactoryTest.java similarity index 73% rename from flink-connectors/flink-connector-kafka-0.11/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka011TableSourceFactoryTest.java rename to flink-connectors/flink-connector-kafka-0.11/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka011TableSourceSinkFactoryTest.java index abaa49081f069..996c50838603f 100644 --- a/flink-connectors/flink-connector-kafka-0.11/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka011TableSourceFactoryTest.java +++ b/flink-connectors/flink-connector-kafka-0.11/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka011TableSourceSinkFactoryTest.java @@ -19,8 +19,10 @@ package org.apache.flink.streaming.connectors.kafka; import org.apache.flink.api.common.serialization.DeserializationSchema; +import org.apache.flink.api.common.serialization.SerializationSchema; import org.apache.flink.streaming.connectors.kafka.config.StartupMode; import org.apache.flink.streaming.connectors.kafka.internals.KafkaTopicPartition; +import org.apache.flink.streaming.connectors.kafka.partitioner.FlinkKafkaPartitioner; import org.apache.flink.table.api.TableSchema; import org.apache.flink.table.descriptors.KafkaValidator; import org.apache.flink.table.sources.RowtimeAttributeDescriptor; @@ -32,9 +34,10 @@ import java.util.Properties; /** - * Test for {@link Kafka011TableSource} created by {@link Kafka011TableSourceFactory}. + * Test for {@link Kafka011TableSource} and {@link Kafka011TableSink} created + * by {@link Kafka011TableSourceSinkFactory}. */ -public class Kafka011TableSourceFactoryTest extends KafkaTableSourceFactoryTestBase { +public class Kafka011TableSourceSinkFactoryTest extends KafkaTableSourceSinkFactoryTestBase { @Override protected String getKafkaVersion() { @@ -47,6 +50,11 @@ protected Class> getExpectedFlinkKafkaConsumer() { return (Class) FlinkKafkaConsumer011.class; } + @Override + protected Class getExpectedFlinkKafkaProducer() { + return FlinkKafkaProducer011.class; + } + @Override protected KafkaTableSource getExpectedKafkaTableSource( TableSchema schema, @@ -71,4 +79,21 @@ protected KafkaTableSource getExpectedKafkaTableSource( specificStartupOffsets ); } + + @Override + protected KafkaTableSink getExpectedKafkaTableSink( + TableSchema schema, + String topic, + Properties properties, + FlinkKafkaPartitioner partitioner, + SerializationSchema serializationSchema) { + + return new Kafka011TableSink( + schema, + topic, + properties, + partitioner, + serializationSchema + ); + } } diff --git a/flink-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka08JsonTableSink.java b/flink-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka08JsonTableSink.java index c60288d3f011e..45588cdb14127 100644 --- a/flink-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka08JsonTableSink.java +++ b/flink-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka08JsonTableSink.java @@ -18,20 +18,25 @@ package org.apache.flink.streaming.connectors.kafka; -import org.apache.flink.annotation.PublicEvolving; import org.apache.flink.api.common.serialization.SerializationSchema; import org.apache.flink.streaming.connectors.kafka.partitioner.FlinkFixedPartitioner; import org.apache.flink.streaming.connectors.kafka.partitioner.FlinkKafkaDelegatePartitioner; import org.apache.flink.streaming.connectors.kafka.partitioner.FlinkKafkaPartitioner; import org.apache.flink.streaming.connectors.kafka.partitioner.KafkaPartitioner; +import org.apache.flink.table.descriptors.ConnectorDescriptor; import org.apache.flink.types.Row; import java.util.Properties; /** * Kafka 0.8 {@link KafkaTableSink} that serializes data in JSON format. + * + * @deprecated Use the {@link org.apache.flink.table.descriptors.Kafka} descriptor together + * with descriptors for schema and format instead. Descriptors allow for + * implementation-agnostic definition of tables. See also + * {@link org.apache.flink.table.api.TableEnvironment#connect(ConnectorDescriptor)}. */ -@PublicEvolving +@Deprecated public class Kafka08JsonTableSink extends KafkaJsonTableSink { /** @@ -48,7 +53,9 @@ public class Kafka08JsonTableSink extends KafkaJsonTableSink { * * @param topic topic in Kafka to which table is written * @param properties properties to connect to Kafka + * @deprecated Use table descriptors instead of implementation-specific classes. */ + @Deprecated public Kafka08JsonTableSink(String topic, Properties properties) { super(topic, properties, new FlinkFixedPartitioner<>()); } @@ -60,7 +67,9 @@ public Kafka08JsonTableSink(String topic, Properties properties) { * @param topic topic in Kafka to which table is written * @param properties properties to connect to Kafka * @param partitioner Kafka partitioner + * @deprecated Use table descriptors instead of implementation-specific classes. */ + @Deprecated public Kafka08JsonTableSink(String topic, Properties properties, FlinkKafkaPartitioner partitioner) { super(topic, properties, partitioner); } @@ -84,7 +93,11 @@ public Kafka08JsonTableSink(String topic, Properties properties, KafkaPartitione @Override protected FlinkKafkaProducerBase createKafkaProducer(String topic, Properties properties, SerializationSchema serializationSchema, FlinkKafkaPartitioner partitioner) { - return new FlinkKafkaProducer08<>(topic, serializationSchema, properties, partitioner); + return new FlinkKafkaProducer08<>( + topic, + serializationSchema, + properties, + partitioner); } @Override diff --git a/flink-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka08TableSink.java b/flink-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka08TableSink.java new file mode 100644 index 0000000000000..c34de13efde33 --- /dev/null +++ b/flink-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka08TableSink.java @@ -0,0 +1,61 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.streaming.connectors.kafka; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.api.common.serialization.SerializationSchema; +import org.apache.flink.streaming.connectors.kafka.partitioner.FlinkKafkaPartitioner; +import org.apache.flink.table.api.TableSchema; +import org.apache.flink.types.Row; + +import java.util.Properties; + +/** + * Kafka 0.8 table sink for writing data into Kafka. + */ +@Internal +public class Kafka08TableSink extends KafkaTableSink { + + public Kafka08TableSink( + TableSchema schema, + String topic, + Properties properties, + FlinkKafkaPartitioner partitioner, + SerializationSchema serializationSchema) { + super( + schema, + topic, + properties, + partitioner, + serializationSchema); + } + + @Override + protected FlinkKafkaProducerBase createKafkaProducer( + String topic, + Properties properties, + SerializationSchema serializationSchema, + FlinkKafkaPartitioner partitioner) { + return new FlinkKafkaProducer08<>( + topic, + serializationSchema, + properties, + partitioner); + } +} diff --git a/flink-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka08TableSource.java b/flink-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka08TableSource.java index 1a025b8a24457..97c293e0690b3 100644 --- a/flink-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka08TableSource.java +++ b/flink-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka08TableSource.java @@ -58,7 +58,8 @@ public Kafka08TableSource( Optional proctimeAttribute, List rowtimeAttributeDescriptors, Optional> fieldMapping, - String topic, Properties properties, + String topic, + Properties properties, DeserializationSchema deserializationSchema, StartupMode startupMode, Map specificStartupOffsets) { diff --git a/flink-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka08TableSourceFactory.java b/flink-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka08TableSourceSinkFactory.java similarity index 80% rename from flink-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka08TableSourceFactory.java rename to flink-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka08TableSourceSinkFactory.java index cd33751561463..3e93b6fdeacc0 100644 --- a/flink-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka08TableSourceFactory.java +++ b/flink-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka08TableSourceSinkFactory.java @@ -19,8 +19,10 @@ package org.apache.flink.streaming.connectors.kafka; import org.apache.flink.api.common.serialization.DeserializationSchema; +import org.apache.flink.api.common.serialization.SerializationSchema; import org.apache.flink.streaming.connectors.kafka.config.StartupMode; import org.apache.flink.streaming.connectors.kafka.internals.KafkaTopicPartition; +import org.apache.flink.streaming.connectors.kafka.partitioner.FlinkKafkaPartitioner; import org.apache.flink.table.api.TableSchema; import org.apache.flink.table.descriptors.KafkaValidator; import org.apache.flink.table.sources.RowtimeAttributeDescriptor; @@ -34,7 +36,7 @@ /** * Factory for creating configured instances of {@link Kafka08TableSource}. */ -public class Kafka08TableSourceFactory extends KafkaTableSourceFactory { +public class Kafka08TableSourceSinkFactory extends KafkaTableSourceSinkFactoryBase { @Override protected String kafkaVersion() { @@ -69,4 +71,20 @@ protected KafkaTableSource createKafkaTableSource( startupMode, specificStartupOffsets); } + + @Override + protected KafkaTableSink createKafkaTableSink( + TableSchema schema, + String topic, + Properties properties, + FlinkKafkaPartitioner partitioner, + SerializationSchema serializationSchema) { + + return new Kafka08TableSink( + schema, + topic, + properties, + partitioner, + serializationSchema); + } } diff --git a/flink-connectors/flink-connector-kafka-0.8/src/main/resources/META-INF/services/org.apache.flink.table.factories.TableFactory b/flink-connectors/flink-connector-kafka-0.8/src/main/resources/META-INF/services/org.apache.flink.table.factories.TableFactory index b83bb3ffc5aee..f2e1c3f5e605a 100644 --- a/flink-connectors/flink-connector-kafka-0.8/src/main/resources/META-INF/services/org.apache.flink.table.factories.TableFactory +++ b/flink-connectors/flink-connector-kafka-0.8/src/main/resources/META-INF/services/org.apache.flink.table.factories.TableFactory @@ -13,4 +13,4 @@ # See the License for the specific language governing permissions and # limitations under the License. -org.apache.flink.streaming.connectors.kafka.Kafka08TableSourceFactory +org.apache.flink.streaming.connectors.kafka.Kafka08TableSourceSinkFactory diff --git a/flink-connectors/flink-connector-kafka-0.8/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka08JsonTableSinkTest.java b/flink-connectors/flink-connector-kafka-0.8/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka08JsonTableSinkTest.java index 53da9f6ba4951..32bd3b69c06ee 100644 --- a/flink-connectors/flink-connector-kafka-0.8/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka08JsonTableSinkTest.java +++ b/flink-connectors/flink-connector-kafka-0.8/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka08JsonTableSinkTest.java @@ -27,7 +27,11 @@ /** * Tests for the {@link Kafka08JsonTableSink}. + * + * @deprecated Ensures backwards compatibility with Flink 1.5. Can be removed once we + * drop support for format-specific table sinks. */ +@Deprecated public class Kafka08JsonTableSinkTest extends KafkaTableSinkTestBase { @Override diff --git a/flink-connectors/flink-connector-kafka-0.8/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka08TableSourceFactoryTest.java b/flink-connectors/flink-connector-kafka-0.8/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka08TableSourceSinkFactoryTest.java similarity index 74% rename from flink-connectors/flink-connector-kafka-0.8/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka08TableSourceFactoryTest.java rename to flink-connectors/flink-connector-kafka-0.8/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka08TableSourceSinkFactoryTest.java index d939d885242fa..b67501e449ec9 100644 --- a/flink-connectors/flink-connector-kafka-0.8/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka08TableSourceFactoryTest.java +++ b/flink-connectors/flink-connector-kafka-0.8/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka08TableSourceSinkFactoryTest.java @@ -19,8 +19,10 @@ package org.apache.flink.streaming.connectors.kafka; import org.apache.flink.api.common.serialization.DeserializationSchema; +import org.apache.flink.api.common.serialization.SerializationSchema; import org.apache.flink.streaming.connectors.kafka.config.StartupMode; import org.apache.flink.streaming.connectors.kafka.internals.KafkaTopicPartition; +import org.apache.flink.streaming.connectors.kafka.partitioner.FlinkKafkaPartitioner; import org.apache.flink.table.api.TableSchema; import org.apache.flink.table.descriptors.KafkaValidator; import org.apache.flink.table.sources.RowtimeAttributeDescriptor; @@ -32,9 +34,10 @@ import java.util.Properties; /** - * Test for {@link Kafka08TableSource} created by {@link Kafka08TableSourceFactory}. + * Test for {@link Kafka08TableSource} and {@link Kafka08TableSink} created + * by {@link Kafka08TableSourceSinkFactory}. */ -public class Kafka08TableSourceFactoryTest extends KafkaTableSourceFactoryTestBase { +public class Kafka08TableSourceSinkFactoryTest extends KafkaTableSourceSinkFactoryTestBase { @Override protected String getKafkaVersion() { @@ -47,6 +50,11 @@ protected Class> getExpectedFlinkKafkaConsumer() { return (Class) FlinkKafkaConsumer08.class; } + @Override + protected Class getExpectedFlinkKafkaProducer() { + return FlinkKafkaProducer08.class; + } + @Override protected KafkaTableSource getExpectedKafkaTableSource( TableSchema schema, @@ -71,4 +79,21 @@ protected KafkaTableSource getExpectedKafkaTableSource( specificStartupOffsets ); } + + @Override + protected KafkaTableSink getExpectedKafkaTableSink( + TableSchema schema, + String topic, + Properties properties, + FlinkKafkaPartitioner partitioner, + SerializationSchema serializationSchema) { + + return new Kafka08TableSink( + schema, + topic, + properties, + partitioner, + serializationSchema + ); + } } diff --git a/flink-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka09JsonTableSink.java b/flink-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka09JsonTableSink.java index 95ce4e6eff3c7..b3cc0aa77cad4 100644 --- a/flink-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka09JsonTableSink.java +++ b/flink-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka09JsonTableSink.java @@ -18,20 +18,25 @@ package org.apache.flink.streaming.connectors.kafka; -import org.apache.flink.annotation.PublicEvolving; import org.apache.flink.api.common.serialization.SerializationSchema; import org.apache.flink.streaming.connectors.kafka.partitioner.FlinkFixedPartitioner; import org.apache.flink.streaming.connectors.kafka.partitioner.FlinkKafkaDelegatePartitioner; import org.apache.flink.streaming.connectors.kafka.partitioner.FlinkKafkaPartitioner; import org.apache.flink.streaming.connectors.kafka.partitioner.KafkaPartitioner; +import org.apache.flink.table.descriptors.ConnectorDescriptor; import org.apache.flink.types.Row; import java.util.Properties; /** * Kafka 0.9 {@link KafkaTableSink} that serializes data in JSON format. + * + * @deprecated Use the {@link org.apache.flink.table.descriptors.Kafka} descriptor together + * with descriptors for schema and format instead. Descriptors allow for + * implementation-agnostic definition of tables. See also + * {@link org.apache.flink.table.api.TableEnvironment#connect(ConnectorDescriptor)}. */ -@PublicEvolving +@Deprecated public class Kafka09JsonTableSink extends KafkaJsonTableSink { /** @@ -48,7 +53,9 @@ public class Kafka09JsonTableSink extends KafkaJsonTableSink { * * @param topic topic in Kafka to which table is written * @param properties properties to connect to Kafka + * @deprecated Use table descriptors instead of implementation-specific classes. */ + @Deprecated public Kafka09JsonTableSink(String topic, Properties properties) { super(topic, properties, new FlinkFixedPartitioner<>()); } @@ -60,7 +67,9 @@ public Kafka09JsonTableSink(String topic, Properties properties) { * @param topic topic in Kafka to which table is written * @param properties properties to connect to Kafka * @param partitioner Kafka partitioner + * @deprecated Use table descriptors instead of implementation-specific classes. */ + @Deprecated public Kafka09JsonTableSink(String topic, Properties properties, FlinkKafkaPartitioner partitioner) { super(topic, properties, partitioner); } @@ -84,7 +93,11 @@ public Kafka09JsonTableSink(String topic, Properties properties, KafkaPartitione @Override protected FlinkKafkaProducerBase createKafkaProducer(String topic, Properties properties, SerializationSchema serializationSchema, FlinkKafkaPartitioner partitioner) { - return new FlinkKafkaProducer09<>(topic, serializationSchema, properties, partitioner); + return new FlinkKafkaProducer09<>( + topic, + serializationSchema, + properties, + partitioner); } @Override diff --git a/flink-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka09TableSink.java b/flink-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka09TableSink.java new file mode 100644 index 0000000000000..8c349d7a0b23c --- /dev/null +++ b/flink-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka09TableSink.java @@ -0,0 +1,61 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.streaming.connectors.kafka; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.api.common.serialization.SerializationSchema; +import org.apache.flink.streaming.connectors.kafka.partitioner.FlinkKafkaPartitioner; +import org.apache.flink.table.api.TableSchema; +import org.apache.flink.types.Row; + +import java.util.Properties; + +/** + * Kafka 0.9 table sink for writing data into Kafka. + */ +@Internal +public class Kafka09TableSink extends KafkaTableSink { + + public Kafka09TableSink( + TableSchema schema, + String topic, + Properties properties, + FlinkKafkaPartitioner partitioner, + SerializationSchema serializationSchema) { + super( + schema, + topic, + properties, + partitioner, + serializationSchema); + } + + @Override + protected FlinkKafkaProducerBase createKafkaProducer( + String topic, + Properties properties, + SerializationSchema serializationSchema, + FlinkKafkaPartitioner partitioner) { + return new FlinkKafkaProducer09<>( + topic, + serializationSchema, + properties, + partitioner); + } +} diff --git a/flink-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka09TableSource.java b/flink-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka09TableSource.java index 18bc1c4351363..8f9e799ec96b2 100644 --- a/flink-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka09TableSource.java +++ b/flink-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka09TableSource.java @@ -58,7 +58,8 @@ public Kafka09TableSource( Optional proctimeAttribute, List rowtimeAttributeDescriptors, Optional> fieldMapping, - String topic, Properties properties, + String topic, + Properties properties, DeserializationSchema deserializationSchema, StartupMode startupMode, Map specificStartupOffsets) { diff --git a/flink-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka09TableSourceFactory.java b/flink-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka09TableSourceSinkFactory.java similarity index 80% rename from flink-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka09TableSourceFactory.java rename to flink-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka09TableSourceSinkFactory.java index 14c52fd0f8300..9958b4ef3161e 100644 --- a/flink-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka09TableSourceFactory.java +++ b/flink-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka09TableSourceSinkFactory.java @@ -19,8 +19,10 @@ package org.apache.flink.streaming.connectors.kafka; import org.apache.flink.api.common.serialization.DeserializationSchema; +import org.apache.flink.api.common.serialization.SerializationSchema; import org.apache.flink.streaming.connectors.kafka.config.StartupMode; import org.apache.flink.streaming.connectors.kafka.internals.KafkaTopicPartition; +import org.apache.flink.streaming.connectors.kafka.partitioner.FlinkKafkaPartitioner; import org.apache.flink.table.api.TableSchema; import org.apache.flink.table.descriptors.KafkaValidator; import org.apache.flink.table.sources.RowtimeAttributeDescriptor; @@ -34,7 +36,7 @@ /** * Factory for creating configured instances of {@link Kafka09TableSource}. */ -public class Kafka09TableSourceFactory extends KafkaTableSourceFactory { +public class Kafka09TableSourceSinkFactory extends KafkaTableSourceSinkFactoryBase { @Override protected String kafkaVersion() { @@ -69,4 +71,20 @@ protected KafkaTableSource createKafkaTableSource( startupMode, specificStartupOffsets); } + + @Override + protected KafkaTableSink createKafkaTableSink( + TableSchema schema, + String topic, + Properties properties, + FlinkKafkaPartitioner partitioner, + SerializationSchema serializationSchema) { + + return new Kafka09TableSink( + schema, + topic, + properties, + partitioner, + serializationSchema); + } } diff --git a/flink-connectors/flink-connector-kafka-0.9/src/main/resources/META-INF/services/org.apache.flink.table.factories.TableFactory b/flink-connectors/flink-connector-kafka-0.9/src/main/resources/META-INF/services/org.apache.flink.table.factories.TableFactory index fb14ddb505507..262587369377a 100644 --- a/flink-connectors/flink-connector-kafka-0.9/src/main/resources/META-INF/services/org.apache.flink.table.factories.TableFactory +++ b/flink-connectors/flink-connector-kafka-0.9/src/main/resources/META-INF/services/org.apache.flink.table.factories.TableFactory @@ -13,4 +13,4 @@ # See the License for the specific language governing permissions and # limitations under the License. -org.apache.flink.streaming.connectors.kafka.Kafka09TableSourceFactory +org.apache.flink.streaming.connectors.kafka.Kafka09TableSourceSinkFactory diff --git a/flink-connectors/flink-connector-kafka-0.9/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka09JsonTableSinkTest.java b/flink-connectors/flink-connector-kafka-0.9/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka09JsonTableSinkTest.java index 610e048af2a49..79f251b83023b 100644 --- a/flink-connectors/flink-connector-kafka-0.9/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka09JsonTableSinkTest.java +++ b/flink-connectors/flink-connector-kafka-0.9/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka09JsonTableSinkTest.java @@ -27,7 +27,11 @@ /** * Tests for the {@link Kafka09JsonTableSink}. + * + * @deprecated Ensures backwards compatibility with Flink 1.5. Can be removed once we + * drop support for format-specific table sinks. */ +@Deprecated public class Kafka09JsonTableSinkTest extends KafkaTableSinkTestBase { @Override diff --git a/flink-connectors/flink-connector-kafka-0.9/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka09TableSourceFactoryTest.java b/flink-connectors/flink-connector-kafka-0.9/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka09TableSourceSinkFactoryTest.java similarity index 74% rename from flink-connectors/flink-connector-kafka-0.9/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka09TableSourceFactoryTest.java rename to flink-connectors/flink-connector-kafka-0.9/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka09TableSourceSinkFactoryTest.java index b976e14e8cdce..a6c8bd4b27980 100644 --- a/flink-connectors/flink-connector-kafka-0.9/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka09TableSourceFactoryTest.java +++ b/flink-connectors/flink-connector-kafka-0.9/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka09TableSourceSinkFactoryTest.java @@ -19,8 +19,10 @@ package org.apache.flink.streaming.connectors.kafka; import org.apache.flink.api.common.serialization.DeserializationSchema; +import org.apache.flink.api.common.serialization.SerializationSchema; import org.apache.flink.streaming.connectors.kafka.config.StartupMode; import org.apache.flink.streaming.connectors.kafka.internals.KafkaTopicPartition; +import org.apache.flink.streaming.connectors.kafka.partitioner.FlinkKafkaPartitioner; import org.apache.flink.table.api.TableSchema; import org.apache.flink.table.descriptors.KafkaValidator; import org.apache.flink.table.sources.RowtimeAttributeDescriptor; @@ -32,9 +34,10 @@ import java.util.Properties; /** - * Test for {@link Kafka09TableSource} created by {@link Kafka09TableSourceFactory}. + * Test for {@link Kafka09TableSource} and {@link Kafka09TableSink} created + * by {@link Kafka09TableSourceSinkFactory}. */ -public class Kafka09TableSourceFactoryTest extends KafkaTableSourceFactoryTestBase { +public class Kafka09TableSourceSinkFactoryTest extends KafkaTableSourceSinkFactoryTestBase { @Override protected String getKafkaVersion() { @@ -47,6 +50,11 @@ protected Class> getExpectedFlinkKafkaConsumer() { return (Class) FlinkKafkaConsumer09.class; } + @Override + protected Class getExpectedFlinkKafkaProducer() { + return FlinkKafkaProducer09.class; + } + @Override protected KafkaTableSource getExpectedKafkaTableSource( TableSchema schema, @@ -71,4 +79,21 @@ protected KafkaTableSource getExpectedKafkaTableSource( specificStartupOffsets ); } + + @Override + protected KafkaTableSink getExpectedKafkaTableSink( + TableSchema schema, + String topic, + Properties properties, + FlinkKafkaPartitioner partitioner, + SerializationSchema serializationSchema) { + + return new Kafka09TableSink( + schema, + topic, + properties, + partitioner, + serializationSchema + ); + } } diff --git a/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaJsonTableSink.java b/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaJsonTableSink.java index ec273989a55f3..231edddb311cb 100644 --- a/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaJsonTableSink.java +++ b/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaJsonTableSink.java @@ -29,7 +29,10 @@ /** * Base class for {@link KafkaTableSink} that serializes data in JSON format. + * + * @deprecated Use table descriptors instead of implementation-specific classes. */ +@Deprecated @Internal public abstract class KafkaJsonTableSink extends KafkaTableSink { @@ -39,7 +42,9 @@ public abstract class KafkaJsonTableSink extends KafkaTableSink { * @param topic topic in Kafka to which table is written * @param properties properties to connect to Kafka * @param partitioner Kafka partitioner + * @deprecated Use table descriptors instead of implementation-specific classes. */ + @Deprecated public KafkaJsonTableSink(String topic, Properties properties, FlinkKafkaPartitioner partitioner) { super(topic, properties, partitioner); } diff --git a/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaTableSink.java b/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaTableSink.java index 687df589cd0c9..7853bb702a5dd 100644 --- a/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaTableSink.java +++ b/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaTableSink.java @@ -23,12 +23,17 @@ import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.java.typeutils.RowTypeInfo; import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.functions.sink.SinkFunction; import org.apache.flink.streaming.connectors.kafka.partitioner.FlinkKafkaPartitioner; +import org.apache.flink.table.api.TableSchema; import org.apache.flink.table.sinks.AppendStreamTableSink; import org.apache.flink.table.util.TableConnectorUtil; import org.apache.flink.types.Row; import org.apache.flink.util.Preconditions; +import java.util.Arrays; +import java.util.Objects; +import java.util.Optional; import java.util.Properties; /** @@ -40,27 +45,59 @@ @Internal public abstract class KafkaTableSink implements AppendStreamTableSink { + // TODO make all attributes final and mandatory once we drop support for format-specific table sinks + + /** The schema of the table. */ + private final Optional schema; + + /** The Kafka topic to write to. */ protected final String topic; + + /** Properties for the Kafka producer. */ protected final Properties properties; - protected SerializationSchema serializationSchema; + + /** Serialization schema for encoding records to Kafka. */ + protected Optional> serializationSchema; + + /** Partitioner to select Kafka partition for each item. */ protected final FlinkKafkaPartitioner partitioner; + + // legacy variables protected String[] fieldNames; protected TypeInformation[] fieldTypes; + protected KafkaTableSink( + TableSchema schema, + String topic, + Properties properties, + FlinkKafkaPartitioner partitioner, + SerializationSchema serializationSchema) { + this.schema = Optional.of(Preconditions.checkNotNull(schema, "Schema must not be null.")); + this.topic = Preconditions.checkNotNull(topic, "Topic must not be null."); + this.properties = Preconditions.checkNotNull(properties, "Properties must not be null."); + this.partitioner = Preconditions.checkNotNull(partitioner, "Partitioner must not be null."); + this.serializationSchema = Optional.of(Preconditions.checkNotNull( + serializationSchema, "Serialization schema must not be null.")); + } + /** * Creates KafkaTableSink. * * @param topic Kafka topic to write to. - * @param properties Properties for the Kafka consumer. + * @param properties Properties for the Kafka producer. * @param partitioner Partitioner to select Kafka partition for each item + * @deprecated Use table descriptors instead of implementation-specific classes. */ + @Deprecated public KafkaTableSink( String topic, Properties properties, FlinkKafkaPartitioner partitioner) { + this.schema = Optional.empty(); this.topic = Preconditions.checkNotNull(topic, "topic"); this.properties = Preconditions.checkNotNull(properties, "properties"); this.partitioner = Preconditions.checkNotNull(partitioner, "partitioner"); + this.serializationSchema = Optional.empty(); } /** @@ -72,8 +109,9 @@ public KafkaTableSink( * @param partitioner Partitioner to select Kafka partition. * @return The version-specific Kafka producer */ - protected abstract FlinkKafkaProducerBase createKafkaProducer( - String topic, Properties properties, + protected abstract SinkFunction createKafkaProducer( + String topic, + Properties properties, SerializationSchema serializationSchema, FlinkKafkaPartitioner partitioner); @@ -82,40 +120,57 @@ protected abstract FlinkKafkaProducerBase createKafkaProducer( * * @param rowSchema the schema of the row to serialize. * @return Instance of serialization schema + * @deprecated Use the constructor to pass a serialization schema instead. */ - protected abstract SerializationSchema createSerializationSchema(RowTypeInfo rowSchema); + @Deprecated + protected SerializationSchema createSerializationSchema(RowTypeInfo rowSchema) { + throw new UnsupportedOperationException("This method only exists for backwards compatibility."); + } /** * Create a deep copy of this sink. * * @return Deep copy of this sink */ - protected abstract KafkaTableSink createCopy(); + @Deprecated + protected KafkaTableSink createCopy() { + throw new UnsupportedOperationException("This method only exists for backwards compatibility."); + } @Override public void emitDataStream(DataStream dataStream) { - FlinkKafkaProducerBase kafkaProducer = createKafkaProducer(topic, properties, serializationSchema, partitioner); - // always enable flush on checkpoint to achieve at-least-once if query runs with checkpointing enabled. - kafkaProducer.setFlushOnCheckpoint(true); + SinkFunction kafkaProducer = createKafkaProducer( + topic, + properties, + serializationSchema.orElseThrow(() -> new IllegalStateException("No serialization schema defined.")), + partitioner); dataStream.addSink(kafkaProducer).name(TableConnectorUtil.generateRuntimeName(this.getClass(), fieldNames)); } @Override public TypeInformation getOutputType() { - return new RowTypeInfo(getFieldTypes()); + return schema + .map(TableSchema::toRowType) + .orElseGet(() -> new RowTypeInfo(getFieldTypes())); } public String[] getFieldNames() { - return fieldNames; + return schema.map(TableSchema::getColumnNames).orElse(fieldNames); } @Override public TypeInformation[] getFieldTypes() { - return fieldTypes; + return schema.map(TableSchema::getTypes).orElse(fieldTypes); } @Override public KafkaTableSink configure(String[] fieldNames, TypeInformation[] fieldTypes) { + if (schema.isPresent()) { + // a fixed schema is defined so reconfiguration is not supported + throw new UnsupportedOperationException("Reconfiguration of this sink is not supported."); + } + + // legacy code KafkaTableSink copy = createCopy(); copy.fieldNames = Preconditions.checkNotNull(fieldNames, "fieldNames"); copy.fieldTypes = Preconditions.checkNotNull(fieldTypes, "fieldTypes"); @@ -123,8 +178,39 @@ public KafkaTableSink configure(String[] fieldNames, TypeInformation[] fieldT "Number of provided field names and types does not match."); RowTypeInfo rowSchema = new RowTypeInfo(fieldTypes, fieldNames); - copy.serializationSchema = createSerializationSchema(rowSchema); + copy.serializationSchema = Optional.of(createSerializationSchema(rowSchema)); return copy; } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + KafkaTableSink that = (KafkaTableSink) o; + return Objects.equals(schema, that.schema) && + Objects.equals(topic, that.topic) && + Objects.equals(properties, that.properties) && + Objects.equals(serializationSchema, that.serializationSchema) && + Objects.equals(partitioner, that.partitioner) && + Arrays.equals(fieldNames, that.fieldNames) && + Arrays.equals(fieldTypes, that.fieldTypes); + } + + @Override + public int hashCode() { + int result = Objects.hash( + schema, + topic, + properties, + serializationSchema, + partitioner); + result = 31 * result + Arrays.hashCode(fieldNames); + result = 31 * result + Arrays.hashCode(fieldTypes); + return result; + } } diff --git a/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaTableSourceFactory.java b/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaTableSourceSinkFactoryBase.java similarity index 64% rename from flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaTableSourceFactory.java rename to flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaTableSourceSinkFactoryBase.java index d7e42f543b391..3307994bb0545 100644 --- a/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaTableSourceFactory.java +++ b/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaTableSourceSinkFactoryBase.java @@ -19,16 +19,22 @@ package org.apache.flink.streaming.connectors.kafka; import org.apache.flink.api.common.serialization.DeserializationSchema; +import org.apache.flink.api.common.serialization.SerializationSchema; import org.apache.flink.streaming.connectors.kafka.config.StartupMode; import org.apache.flink.streaming.connectors.kafka.internals.KafkaTopicPartition; +import org.apache.flink.streaming.connectors.kafka.partitioner.FlinkFixedPartitioner; +import org.apache.flink.streaming.connectors.kafka.partitioner.FlinkKafkaPartitioner; import org.apache.flink.table.api.TableException; import org.apache.flink.table.api.TableSchema; import org.apache.flink.table.descriptors.DescriptorProperties; import org.apache.flink.table.descriptors.KafkaValidator; import org.apache.flink.table.descriptors.SchemaValidator; import org.apache.flink.table.factories.DeserializationSchemaFactory; +import org.apache.flink.table.factories.SerializationSchemaFactory; +import org.apache.flink.table.factories.StreamTableSinkFactory; import org.apache.flink.table.factories.StreamTableSourceFactory; import org.apache.flink.table.factories.TableFactoryService; +import org.apache.flink.table.sinks.StreamTableSink; import org.apache.flink.table.sources.RowtimeAttributeDescriptor; import org.apache.flink.table.sources.StreamTableSource; import org.apache.flink.types.Row; @@ -73,7 +79,9 @@ /** * Factory for creating configured instances of {@link KafkaTableSource}. */ -public abstract class KafkaTableSourceFactory implements StreamTableSourceFactory { +public abstract class KafkaTableSourceSinkFactoryBase implements + StreamTableSourceFactory, + StreamTableSinkFactory { @Override public Map requiredContext() { @@ -122,89 +130,46 @@ public List supportedProperties() { @Override public StreamTableSource createStreamTableSource(Map properties) { - final DescriptorProperties params = new DescriptorProperties(true); - params.putProperties(properties); + final DescriptorProperties descriptorProperties = getValidatedProperties(properties); - // validate - // allow Kafka timestamps to be used, watermarks can not be received from source - new SchemaValidator(true, supportsKafkaTimestamps(), false).validate(params); - new KafkaValidator().validate(params); - - // deserialization schema using format discovery - final DeserializationSchemaFactory formatFactory = TableFactoryService.find( - DeserializationSchemaFactory.class, - properties, - this.getClass().getClassLoader()); - @SuppressWarnings("unchecked") - final DeserializationSchema deserializationSchema = (DeserializationSchema) formatFactory - .createDeserializationSchema(properties); - - // schema - final TableSchema schema = params.getTableSchema(SCHEMA()); - - // proctime - final Optional proctimeAttribute = SchemaValidator.deriveProctimeAttribute(params); - - // rowtime - final List rowtimeAttributes = SchemaValidator.deriveRowtimeAttributes(params); - - // field mapping - final Map fieldMapping = SchemaValidator.deriveFieldMapping(params, Optional.of(schema)); - - // properties - final Properties kafkaProperties = new Properties(); - final List> propsList = params.getFixedIndexedProperties( - CONNECTOR_PROPERTIES, - Arrays.asList(CONNECTOR_PROPERTIES_KEY, CONNECTOR_PROPERTIES_VALUE)); - propsList.forEach(kv -> kafkaProperties.put( - params.getString(kv.get(CONNECTOR_PROPERTIES_KEY)), - params.getString(kv.get(CONNECTOR_PROPERTIES_VALUE)) - )); - - // topic - final String topic = params.getString(CONNECTOR_TOPIC); + final TableSchema schema = descriptorProperties.getTableSchema(SCHEMA()); + final String topic = descriptorProperties.getString(CONNECTOR_TOPIC); + final StartupOptions startupOptions = getStartupOptions(descriptorProperties, topic); - // startup mode - final Map specificOffsets = new HashMap<>(); - final StartupMode startupMode = params - .getOptionalString(CONNECTOR_STARTUP_MODE) - .map(modeString -> { - switch (modeString) { - case KafkaValidator.CONNECTOR_STARTUP_MODE_VALUE_EARLIEST: - return StartupMode.EARLIEST; + return createKafkaTableSource( + schema, + SchemaValidator.deriveProctimeAttribute(descriptorProperties), + SchemaValidator.deriveRowtimeAttributes(descriptorProperties), + SchemaValidator.deriveFieldMapping(descriptorProperties, Optional.of(schema)), + topic, + getKafkaProperties(descriptorProperties), + getDeserializationSchema(properties), + startupOptions.startupMode, + startupOptions.specificOffsets); + } - case KafkaValidator.CONNECTOR_STARTUP_MODE_VALUE_LATEST: - return StartupMode.LATEST; + @Override + public StreamTableSink createStreamTableSink(Map properties) { + final DescriptorProperties descriptorProperties = getValidatedProperties(properties); - case KafkaValidator.CONNECTOR_STARTUP_MODE_VALUE_GROUP_OFFSETS: - return StartupMode.GROUP_OFFSETS; + final TableSchema schema = descriptorProperties.getTableSchema(SCHEMA()); + final String topic = descriptorProperties.getString(CONNECTOR_TOPIC); + final Optional proctime = SchemaValidator.deriveProctimeAttribute(descriptorProperties); + final List rowtimeAttributeDescriptors = + SchemaValidator.deriveRowtimeAttributes(descriptorProperties); - case KafkaValidator.CONNECTOR_STARTUP_MODE_VALUE_SPECIFIC_OFFSETS: - final List> offsetList = params.getFixedIndexedProperties( - CONNECTOR_SPECIFIC_OFFSETS, - Arrays.asList(CONNECTOR_SPECIFIC_OFFSETS_PARTITION, CONNECTOR_SPECIFIC_OFFSETS_OFFSET)); - offsetList.forEach(kv -> { - final int partition = params.getInt(kv.get(CONNECTOR_SPECIFIC_OFFSETS_PARTITION)); - final long offset = params.getLong(kv.get(CONNECTOR_SPECIFIC_OFFSETS_OFFSET)); - final KafkaTopicPartition topicPartition = new KafkaTopicPartition(topic, partition); - specificOffsets.put(topicPartition, offset); - }); - return StartupMode.SPECIFIC_OFFSETS; - default: - throw new TableException("Unsupported startup mode. Validator should have checked that."); - } - }).orElse(StartupMode.GROUP_OFFSETS); + // see also FLINK-9870 + if (proctime.isPresent() || !rowtimeAttributeDescriptors.isEmpty() || + checkForCustomFieldMapping(descriptorProperties, schema)) { + throw new TableException("Time attributes and custom field mappings are not supported yet."); + } - return createKafkaTableSource( + return createKafkaTableSink( schema, - proctimeAttribute, - rowtimeAttributes, - fieldMapping, topic, - kafkaProperties, - deserializationSchema, - startupMode, - specificOffsets); + getKafkaProperties(descriptorProperties), + getFlinkKafkaPartitioner(), + getSerializationSchema(properties)); } // -------------------------------------------------------------------------------------------- @@ -243,9 +208,123 @@ protected abstract KafkaTableSource createKafkaTableSource( Optional proctimeAttribute, List rowtimeAttributeDescriptors, Map fieldMapping, - String topic, Properties properties, + String topic, + Properties properties, DeserializationSchema deserializationSchema, StartupMode startupMode, Map specificStartupOffsets); + /** + * Constructs the version-specific Kafka table sink. + * + * @param schema Schema of the produced table. + * @param topic Kafka topic to consume. + * @param properties Properties for the Kafka consumer. + * @param partitioner Partitioner to select Kafka partition for each item. + */ + protected abstract KafkaTableSink createKafkaTableSink( + TableSchema schema, + String topic, + Properties properties, + FlinkKafkaPartitioner partitioner, + SerializationSchema serializationSchema); + + // -------------------------------------------------------------------------------------------- + // Helper methods + // -------------------------------------------------------------------------------------------- + + private DescriptorProperties getValidatedProperties(Map properties) { + final DescriptorProperties descriptorProperties = new DescriptorProperties(true); + descriptorProperties.putProperties(properties); + + // allow Kafka timestamps to be used, watermarks can not be received from source + new SchemaValidator(true, supportsKafkaTimestamps(), false).validate(descriptorProperties); + new KafkaValidator().validate(descriptorProperties); + + return descriptorProperties; + } + + private DeserializationSchema getDeserializationSchema(Map properties) { + @SuppressWarnings("unchecked") + final DeserializationSchemaFactory formatFactory = TableFactoryService.find( + DeserializationSchemaFactory.class, + properties, + this.getClass().getClassLoader()); + return formatFactory.createDeserializationSchema(properties); + } + + private SerializationSchema getSerializationSchema(Map properties) { + @SuppressWarnings("unchecked") + final SerializationSchemaFactory formatFactory = TableFactoryService.find( + SerializationSchemaFactory.class, + properties, + this.getClass().getClassLoader()); + return formatFactory.createSerializationSchema(properties); + } + + private Properties getKafkaProperties(DescriptorProperties descriptorProperties) { + final Properties kafkaProperties = new Properties(); + final List> propsList = descriptorProperties.getFixedIndexedProperties( + CONNECTOR_PROPERTIES, + Arrays.asList(CONNECTOR_PROPERTIES_KEY, CONNECTOR_PROPERTIES_VALUE)); + propsList.forEach(kv -> kafkaProperties.put( + descriptorProperties.getString(kv.get(CONNECTOR_PROPERTIES_KEY)), + descriptorProperties.getString(kv.get(CONNECTOR_PROPERTIES_VALUE)) + )); + return kafkaProperties; + } + + private StartupOptions getStartupOptions( + DescriptorProperties descriptorProperties, + String topic) { + final Map specificOffsets = new HashMap<>(); + final StartupMode startupMode = descriptorProperties + .getOptionalString(CONNECTOR_STARTUP_MODE) + .map(modeString -> { + switch (modeString) { + case KafkaValidator.CONNECTOR_STARTUP_MODE_VALUE_EARLIEST: + return StartupMode.EARLIEST; + + case KafkaValidator.CONNECTOR_STARTUP_MODE_VALUE_LATEST: + return StartupMode.LATEST; + + case KafkaValidator.CONNECTOR_STARTUP_MODE_VALUE_GROUP_OFFSETS: + return StartupMode.GROUP_OFFSETS; + + case KafkaValidator.CONNECTOR_STARTUP_MODE_VALUE_SPECIFIC_OFFSETS: + final List> offsetList = descriptorProperties.getFixedIndexedProperties( + CONNECTOR_SPECIFIC_OFFSETS, + Arrays.asList(CONNECTOR_SPECIFIC_OFFSETS_PARTITION, CONNECTOR_SPECIFIC_OFFSETS_OFFSET)); + offsetList.forEach(kv -> { + final int partition = descriptorProperties.getInt(kv.get(CONNECTOR_SPECIFIC_OFFSETS_PARTITION)); + final long offset = descriptorProperties.getLong(kv.get(CONNECTOR_SPECIFIC_OFFSETS_OFFSET)); + final KafkaTopicPartition topicPartition = new KafkaTopicPartition(topic, partition); + specificOffsets.put(topicPartition, offset); + }); + return StartupMode.SPECIFIC_OFFSETS; + default: + throw new TableException("Unsupported startup mode. Validator should have checked that."); + } + }).orElse(StartupMode.GROUP_OFFSETS); + final StartupOptions options = new StartupOptions(); + options.startupMode = startupMode; + options.specificOffsets = specificOffsets; + return options; + } + + private FlinkKafkaPartitioner getFlinkKafkaPartitioner() { + // we don't support custom partitioner so far + return new FlinkFixedPartitioner<>(); + } + + private boolean checkForCustomFieldMapping(DescriptorProperties descriptorProperties, TableSchema schema) { + final Map fieldMapping = SchemaValidator.deriveFieldMapping(descriptorProperties, Optional.of(schema)); + return fieldMapping.size() != schema.getColumnNames().length || + !fieldMapping.entrySet().stream().allMatch(mapping -> mapping.getKey().equals(mapping.getValue())); + } + + private static class StartupOptions { + private StartupMode startupMode; + private Map specificOffsets; + } } diff --git a/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/partitioner/FlinkFixedPartitioner.java b/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/partitioner/FlinkFixedPartitioner.java index 6e83ddd32eaf1..7e0d1fbe45622 100644 --- a/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/partitioner/FlinkFixedPartitioner.java +++ b/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/partitioner/FlinkFixedPartitioner.java @@ -74,4 +74,14 @@ public int partition(T record, byte[] key, byte[] value, String targetTopic, int return partitions[parallelInstanceId % partitions.length]; } + + @Override + public boolean equals(Object o) { + return this == o || o instanceof FlinkFixedPartitioner; + } + + @Override + public int hashCode() { + return FlinkFixedPartitioner.class.hashCode(); + } } diff --git a/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTableSinkTestBase.java b/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTableSinkTestBase.java index a87c622859841..946b6eb589508 100644 --- a/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTableSinkTestBase.java +++ b/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTableSinkTestBase.java @@ -44,7 +44,11 @@ /** * Abstract test base for all Kafka table sink tests. + * + * @deprecated Ensures backwards compatibility with Flink 1.5. Can be removed once we + * drop support for format-specific table sinks. */ +@Deprecated public abstract class KafkaTableSinkTestBase { private static final String TOPIC = "testTopic"; @@ -94,7 +98,8 @@ protected abstract KafkaTableSink createTableSink( protected abstract Class getProducerClass(); private KafkaTableSink createTableSink() { - return createTableSink(TOPIC, PROPERTIES, PARTITIONER); + KafkaTableSink sink = createTableSink(TOPIC, PROPERTIES, PARTITIONER); + return sink.configure(FIELD_NAMES, FIELD_TYPES); } private static Properties createSinkProperties() { diff --git a/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTableSourceFactoryTestBase.java b/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTableSourceSinkFactoryTestBase.java similarity index 59% rename from flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTableSourceFactoryTestBase.java rename to flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTableSourceSinkFactoryTestBase.java index 96f160763413d..d8e8f7d37c3c7 100644 --- a/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTableSourceFactoryTestBase.java +++ b/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTableSourceSinkFactoryTestBase.java @@ -20,11 +20,20 @@ import org.apache.flink.api.common.JobExecutionResult; import org.apache.flink.api.common.serialization.DeserializationSchema; +import org.apache.flink.api.common.serialization.SerializationSchema; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.datastream.DataStreamSink; import org.apache.flink.streaming.api.datastream.DataStreamSource; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.streaming.api.functions.sink.SinkFunction; import org.apache.flink.streaming.api.functions.source.SourceFunction; +import org.apache.flink.streaming.api.operators.ChainingStrategy; +import org.apache.flink.streaming.api.transformations.StreamTransformation; import org.apache.flink.streaming.connectors.kafka.config.StartupMode; import org.apache.flink.streaming.connectors.kafka.internals.KafkaTopicPartition; +import org.apache.flink.streaming.connectors.kafka.partitioner.FlinkFixedPartitioner; +import org.apache.flink.streaming.connectors.kafka.partitioner.FlinkKafkaPartitioner; import org.apache.flink.table.api.TableSchema; import org.apache.flink.table.api.Types; import org.apache.flink.table.descriptors.DescriptorProperties; @@ -32,10 +41,13 @@ import org.apache.flink.table.descriptors.Rowtime; import org.apache.flink.table.descriptors.Schema; import org.apache.flink.table.descriptors.TestTableDescriptor; +import org.apache.flink.table.factories.StreamTableSinkFactory; import org.apache.flink.table.factories.StreamTableSourceFactory; import org.apache.flink.table.factories.TableFactoryService; import org.apache.flink.table.factories.utils.TestDeserializationSchema; +import org.apache.flink.table.factories.utils.TestSerializationSchema; import org.apache.flink.table.factories.utils.TestTableFormat; +import org.apache.flink.table.sinks.TableSink; import org.apache.flink.table.sources.RowtimeAttributeDescriptor; import org.apache.flink.table.sources.TableSource; import org.apache.flink.table.sources.tsextractors.ExistingField; @@ -45,6 +57,7 @@ import org.junit.Test; +import java.util.Collection; import java.util.Collections; import java.util.HashMap; import java.util.List; @@ -56,9 +69,9 @@ import static org.junit.Assert.assertTrue; /** - * Abstract test base for {@link KafkaTableSourceFactory}. + * Abstract test base for {@link KafkaTableSourceSinkFactoryBase}. */ -public abstract class KafkaTableSourceFactoryTestBase extends TestLogger { +public abstract class KafkaTableSourceSinkFactoryTestBase extends TestLogger { private static final String TOPIC = "myTopic"; private static final int PARTITION_0 = 0; @@ -75,6 +88,13 @@ public abstract class KafkaTableSourceFactoryTestBase extends TestLogger { static { KAFKA_PROPERTIES.setProperty("zookeeper.connect", "dummy"); KAFKA_PROPERTIES.setProperty("group.id", "dummy"); + KAFKA_PROPERTIES.setProperty("bootstrap.servers", "dummy"); + } + + private static final Map OFFSETS = new HashMap<>(); + static { + OFFSETS.put(PARTITION_0, OFFSET_0); + OFFSETS.put(PARTITION_1, OFFSET_1); } @Test @@ -110,8 +130,6 @@ public void testTableSource() { .toRowType() ); - final StartupMode startupMode = StartupMode.SPECIFIC_OFFSETS; - final KafkaTableSource expected = getExpectedKafkaTableSource( schema, Optional.of(PROC_TIME), @@ -120,21 +138,17 @@ public void testTableSource() { TOPIC, KAFKA_PROPERTIES, deserializationSchema, - startupMode, + StartupMode.SPECIFIC_OFFSETS, specificOffsets); // construct table source using descriptors and table source factory - final Map offsets = new HashMap<>(); - offsets.put(PARTITION_0, OFFSET_0); - offsets.put(PARTITION_1, OFFSET_1); - final TestTableDescriptor testDesc = new TestTableDescriptor( new Kafka() .version(getKafkaVersion()) .topic(TOPIC) .properties(KAFKA_PROPERTIES) - .startFromSpecificOffsets(offsets)) + .startFromSpecificOffsets(OFFSETS)) .withFormat(new TestTableFormat()) .withSchema( new Schema() @@ -144,11 +158,9 @@ public void testTableSource() { new Rowtime().timestampsFromField(TIME).watermarksPeriodicAscending()) .field(PROC_TIME, Types.SQL_TIMESTAMP()).proctime()) .inAppendMode(); - final DescriptorProperties descriptorProperties = new DescriptorProperties(true); - testDesc.addProperties(descriptorProperties); - final Map propertiesMap = descriptorProperties.asMap(); - final TableSource actualSource = TableFactoryService.find(StreamTableSourceFactory.class, testDesc) + final Map propertiesMap = DescriptorProperties.toJavaMap(testDesc); + final TableSource actualSource = TableFactoryService.find(StreamTableSourceFactory.class, propertiesMap) .createStreamTableSource(propertiesMap); assertEquals(expected, actualSource); @@ -157,17 +169,66 @@ public void testTableSource() { final KafkaTableSource actualKafkaSource = (KafkaTableSource) actualSource; final StreamExecutionEnvironmentMock mock = new StreamExecutionEnvironmentMock(); actualKafkaSource.getDataStream(mock); - assertTrue(getExpectedFlinkKafkaConsumer().isAssignableFrom(mock.function.getClass())); + assertTrue(getExpectedFlinkKafkaConsumer().isAssignableFrom(mock.sourceFunction.getClass())); + } + + /** + * This test can be unified with the corresponding source test once we have fixed FLINK-9870. + */ + @Test + public void testTableSink() { + // prepare parameters for Kafka table sink + + final TableSchema schema = TableSchema.builder() + .field(FRUIT_NAME, Types.STRING()) + .field(COUNT, Types.DECIMAL()) + .field(EVENT_TIME, Types.SQL_TIMESTAMP()) + .build(); + + final KafkaTableSink expected = getExpectedKafkaTableSink( + schema, + TOPIC, + KAFKA_PROPERTIES, + new FlinkFixedPartitioner<>(), // a custom partitioner is not support yet + new TestSerializationSchema(schema.toRowType())); + + // construct table sink using descriptors and table sink factory + + final TestTableDescriptor testDesc = new TestTableDescriptor( + new Kafka() + .version(getKafkaVersion()) + .topic(TOPIC) + .properties(KAFKA_PROPERTIES) + .startFromSpecificOffsets(OFFSETS)) // test if they accepted although not needed + .withFormat(new TestTableFormat()) + .withSchema( + new Schema() + .field(FRUIT_NAME, Types.STRING()) + .field(COUNT, Types.DECIMAL()) + .field(EVENT_TIME, Types.SQL_TIMESTAMP())) + .inAppendMode(); + + final Map propertiesMap = DescriptorProperties.toJavaMap(testDesc); + final TableSink actualSink = TableFactoryService.find(StreamTableSinkFactory.class, propertiesMap) + .createStreamTableSink(propertiesMap); + + assertEquals(expected, actualSink); + + // test Kafka producer + final KafkaTableSink actualKafkaSink = (KafkaTableSink) actualSink; + final DataStreamMock streamMock = new DataStreamMock(new StreamExecutionEnvironmentMock(), schema.toRowType()); + actualKafkaSink.emitDataStream(streamMock); + assertTrue(getExpectedFlinkKafkaProducer().isAssignableFrom(streamMock.sinkFunction.getClass())); } private static class StreamExecutionEnvironmentMock extends StreamExecutionEnvironment { - public SourceFunction function; + public SourceFunction sourceFunction; @Override - public DataStreamSource addSource(SourceFunction function) { - this.function = function; - return super.addSource(function); + public DataStreamSource addSource(SourceFunction sourceFunction) { + this.sourceFunction = sourceFunction; + return super.addSource(sourceFunction); } @Override @@ -176,6 +237,38 @@ public JobExecutionResult execute(String jobName) { } } + private static class DataStreamMock extends DataStream { + + public SinkFunction sinkFunction; + + public DataStreamMock(StreamExecutionEnvironment environment, TypeInformation outType) { + super(environment, new StreamTransformationMock("name", outType, 1)); + } + + @Override + public DataStreamSink addSink(SinkFunction sinkFunction) { + this.sinkFunction = sinkFunction; + return super.addSink(sinkFunction); + } + } + + private static class StreamTransformationMock extends StreamTransformation { + + public StreamTransformationMock(String name, TypeInformation outputType, int parallelism) { + super(name, outputType, parallelism); + } + + @Override + public void setChainingStrategy(ChainingStrategy strategy) { + // do nothing + } + + @Override + public Collection> getTransitivePredecessors() { + return null; + } + } + // -------------------------------------------------------------------------------------------- // For version-specific tests // -------------------------------------------------------------------------------------------- @@ -184,13 +277,23 @@ public JobExecutionResult execute(String jobName) { protected abstract Class> getExpectedFlinkKafkaConsumer(); + protected abstract Class getExpectedFlinkKafkaProducer(); + protected abstract KafkaTableSource getExpectedKafkaTableSource( TableSchema schema, Optional proctimeAttribute, List rowtimeAttributeDescriptors, Map fieldMapping, - String topic, Properties properties, + String topic, + Properties properties, DeserializationSchema deserializationSchema, StartupMode startupMode, Map specificStartupOffsets); + + protected abstract KafkaTableSink getExpectedKafkaTableSink( + TableSchema schema, + String topic, + Properties properties, + FlinkKafkaPartitioner partitioner, + SerializationSchema serializationSchema); } diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/factories/utils/TestSerializationSchema.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/factories/utils/TestSerializationSchema.scala index ab613a9187e3a..a7eaa48174b3f 100644 --- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/factories/utils/TestSerializationSchema.scala +++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/factories/utils/TestSerializationSchema.scala @@ -19,12 +19,26 @@ package org.apache.flink.table.factories.utils import org.apache.flink.api.common.serialization.SerializationSchema +import org.apache.flink.api.common.typeinfo.TypeInformation import org.apache.flink.types.Row /** * Serialization schema for testing purposes. */ -class TestSerializationSchema extends SerializationSchema[Row] { +class TestSerializationSchema(val typeInfo: TypeInformation[Row]) extends SerializationSchema[Row] { override def serialize(element: Row): Array[Byte] = throw new UnsupportedOperationException() + + def canEqual(other: Any): Boolean = other.isInstanceOf[TestSerializationSchema] + + override def equals(other: Any): Boolean = other match { + case that: TestSerializationSchema => + (that canEqual this) && + typeInfo == that.typeInfo + case _ => false + } + + override def hashCode(): Int = { + 31 * typeInfo.hashCode() + } } diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/factories/utils/TestTableFormatFactory.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/factories/utils/TestTableFormatFactory.scala index 475cff951bd5d..39c268e3e36f6 100644 --- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/factories/utils/TestTableFormatFactory.scala +++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/factories/utils/TestTableFormatFactory.scala @@ -20,9 +20,9 @@ package org.apache.flink.table.factories.utils import java.util -import org.apache.flink.api.common.serialization.DeserializationSchema +import org.apache.flink.api.common.serialization.{DeserializationSchema, SerializationSchema} import org.apache.flink.table.descriptors.{DescriptorProperties, FormatDescriptorValidator, SchemaValidator} -import org.apache.flink.table.factories.{DeserializationSchemaFactory, TableFormatFactoryServiceTest} +import org.apache.flink.table.factories.{DeserializationSchemaFactory, SerializationSchemaFactory, TableFormatFactoryServiceTest} import org.apache.flink.types.Row /** @@ -31,7 +31,9 @@ import org.apache.flink.types.Row * It has the same context as [[TestAmbiguousTableFormatFactory]] and both support COMMON_PATH. * This format does not support SPECIAL_PATH but supports schema derivation. */ -class TestTableFormatFactory extends DeserializationSchemaFactory[Row] { +class TestTableFormatFactory + extends DeserializationSchemaFactory[Row] + with SerializationSchemaFactory[Row] { override def requiredContext(): util.Map[String, String] = { val context = new util.HashMap[String, String]() @@ -62,4 +64,14 @@ class TestTableFormatFactory extends DeserializationSchemaFactory[Row] { val schema = SchemaValidator.deriveFormatFields(props) new TestDeserializationSchema(schema.toRowType) } + + override def createSerializationSchema( + properties: util.Map[String, String]) + : SerializationSchema[Row] = { + + val props = new DescriptorProperties(true) + props.putProperties(properties) + val schema = SchemaValidator.deriveFormatFields(props) + new TestSerializationSchema(schema.toRowType) + } } From 37b82ba1ba051f0bda006157e3fee6eccccd3521 Mon Sep 17 00:00:00 2001 From: Till Rohrmann Date: Thu, 19 Jul 2018 13:07:44 +0200 Subject: [PATCH 48/65] [FLINK-9838][logging] Don't log slot request failures on the ResourceManager This closes #6373. --- .../org/apache/flink/runtime/jobmaster/slotpool/SlotPool.java | 4 ++-- .../apache/flink/runtime/resourcemanager/ResourceManager.java | 1 - 2 files changed, 2 insertions(+), 3 deletions(-) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/slotpool/SlotPool.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/slotpool/SlotPool.java index 27440a33d19db..829c82ef9dff6 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/slotpool/SlotPool.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/slotpool/SlotPool.java @@ -1011,7 +1011,7 @@ public void failAllocation(final AllocationID allocationID, final Exception caus failPendingRequest(pendingRequest, cause); } else if (availableSlots.tryRemove(allocationID)) { - log.debug("Failed available slot [{}] with ", allocationID, cause); + log.debug("Failed available slot with allocation id {}.", allocationID, cause); } else { AllocatedSlot allocatedSlot = allocatedSlots.remove(allocationID); @@ -1021,7 +1021,7 @@ else if (availableSlots.tryRemove(allocationID)) { allocatedSlot.releasePayload(cause); } else { - log.trace("Outdated request to fail slot [{}] with ", allocationID, cause); + log.trace("Outdated request to fail slot with allocation id {}.", allocationID, cause); } } // TODO: add some unit tests when the previous two are ready, the allocation may failed at any phase 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 6b104ede50a4b..fb1b888a5c76a 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 @@ -1012,7 +1012,6 @@ public void allocateResource(ResourceProfile resourceProfile) throws ResourceMan @Override public void notifyAllocationFailure(JobID jobId, AllocationID allocationId, Exception cause) { validateRunsInMainThread(); - log.info("Slot request with allocation id {} for job {} failed.", allocationId, jobId, cause); JobManagerRegistration jobManagerRegistration = jobManagerRegistrations.get(jobId); if (jobManagerRegistration != null) { From 957b647ed8d5ed047b0d09c8951dc7c25f492082 Mon Sep 17 00:00:00 2001 From: Till Rohrmann Date: Thu, 19 Jul 2018 13:41:03 +0200 Subject: [PATCH 49/65] [hotfix] Improve logging of SlotPool and SlotSharingManager --- .../clusterframework/types/AllocationID.java | 5 +++ .../runtime/jobmaster/SlotRequestId.java | 5 +++ .../runtime/jobmaster/slotpool/SlotPool.java | 36 +++++++++---------- .../slotpool/SlotSharingManager.java | 21 +++++++++-- 4 files changed, 46 insertions(+), 21 deletions(-) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/types/AllocationID.java b/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/types/AllocationID.java index e722e9fbb8d8e..7004eff51255f 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/types/AllocationID.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/types/AllocationID.java @@ -52,4 +52,9 @@ public AllocationID() { public AllocationID(long lowerPart, long upperPart) { super(lowerPart, upperPart); } + + @Override + public String toString() { + return "AllocationID{" + super.toString() + '}'; + } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/SlotRequestId.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/SlotRequestId.java index 203139c0f71db..5ac200dcb7493 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/SlotRequestId.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/SlotRequestId.java @@ -40,4 +40,9 @@ public SlotRequestId(long lowerPart, long upperPart) { } public SlotRequestId() {} + + @Override + public String toString() { + return "SlotRequestId{" + super.toString() + '}'; + } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/slotpool/SlotPool.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/slotpool/SlotPool.java index 829c82ef9dff6..13f0462455c4a 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/slotpool/SlotPool.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/slotpool/SlotPool.java @@ -323,7 +323,7 @@ public CompletableFuture allocateSlot( boolean allowQueuedScheduling, Time allocationTimeout) { - log.debug("Allocating slot with request {} for task execution {}", slotRequestId, task.getTaskToExecute()); + log.debug("Received slot request [{}] for task: {}", slotRequestId, task.getTaskToExecute()); final SlotSharingGroupId slotSharingGroupId = task.getSlotSharingGroupId(); @@ -686,7 +686,7 @@ private void requestSlotFromResourceManager( checkNotNull(resourceManagerGateway); checkNotNull(pendingRequest); - log.info("Requesting slot with profile {} from resource manager (request = {}).", pendingRequest.getResourceProfile(), pendingRequest.getSlotRequestId()); + log.info("Requesting new slot [{}] and profile {} from resource manager.", pendingRequest.getSlotRequestId(), pendingRequest.getResourceProfile()); final AllocationID allocationId = new AllocationID(); @@ -723,7 +723,7 @@ private void slotRequestToResourceManagerFailed(SlotRequestId slotRequestID, Thr "No pooled slot available and request to ResourceManager for new slot failed", failure)); } else { if (log.isDebugEnabled()) { - log.debug("Unregistered slot request {} failed.", slotRequestID, failure); + log.debug("Unregistered slot request [{}] failed.", slotRequestID, failure); } } } @@ -731,7 +731,7 @@ private void slotRequestToResourceManagerFailed(SlotRequestId slotRequestID, Thr private void stashRequestWaitingForResourceManager(final PendingRequest pendingRequest) { log.info("Cannot serve slot request, no ResourceManager connected. " + - "Adding as pending request {}", pendingRequest.getSlotRequestId()); + "Adding as pending request [{}]", pendingRequest.getSlotRequestId()); waitingForResourceManager.put(pendingRequest.getSlotRequestId(), pendingRequest); } @@ -742,7 +742,7 @@ private void stashRequestWaitingForResourceManager(final PendingRequest pendingR @Override public CompletableFuture releaseSlot(SlotRequestId slotRequestId, @Nullable SlotSharingGroupId slotSharingGroupId, Throwable cause) { - log.debug("Releasing slot with slot request id {} because of {}.", slotRequestId, cause != null ? cause.getMessage() : "null"); + log.debug("Releasing slot [{}] because: {}", slotRequestId, cause != null ? cause.getMessage() : "null"); if (slotSharingGroupId != null) { final SlotSharingManager multiTaskSlotManager = slotSharingManagers.get(slotSharingGroupId); @@ -753,7 +753,7 @@ public CompletableFuture releaseSlot(SlotRequestId slotRequestId, @ if (taskSlot != null) { taskSlot.release(cause); } else { - log.debug("Could not find slot {} in slot sharing group {}. Ignoring release slot request.", slotRequestId, slotSharingGroupId); + log.debug("Could not find slot [{}] in slot sharing group {}. Ignoring release slot request.", slotRequestId, slotSharingGroupId); } } else { log.debug("Could not find slot sharing group {}. Ignoring release slot request.", slotSharingGroupId); @@ -770,7 +770,7 @@ public CompletableFuture releaseSlot(SlotRequestId slotRequestId, @ allocatedSlot.releasePayload(cause); tryFulfillSlotRequestOrMakeAvailable(allocatedSlot); } else { - log.debug("There is no allocated slot with slot request id {}. Ignoring the release slot request.", slotRequestId); + log.debug("There is no allocated slot [{}]. Ignoring the release slot request.", slotRequestId); } } } @@ -801,11 +801,11 @@ private PendingRequest removePendingRequest(SlotRequestId requestId) { } private void failPendingRequest(PendingRequest pendingRequest, Exception e) { - Preconditions.checkNotNull(pendingRequest); - Preconditions.checkNotNull(e); + checkNotNull(pendingRequest); + checkNotNull(e); if (!pendingRequest.getAllocatedSlotFuture().isDone()) { - log.info("Failing pending request {}.", pendingRequest.getSlotRequestId()); + log.info("Failing pending slot request [{}]: {}", pendingRequest.getSlotRequestId(), e.getMessage()); pendingRequest.getAllocatedSlotFuture().completeExceptionally(e); } } @@ -833,7 +833,7 @@ private void tryFulfillSlotRequestOrMakeAvailable(AllocatedSlot allocatedSlot) { final PendingRequest pendingRequest = pollMatchingPendingRequest(allocatedSlot); if (pendingRequest != null) { - log.debug("Fulfilling pending request [{}] early with returned slot [{}]", + log.debug("Fulfilling pending slot request [{}] early with returned slot [{}]", pendingRequest.getSlotRequestId(), allocatedSlot.getAllocationId()); allocatedSlots.add(pendingRequest.getSlotRequestId(), allocatedSlot); @@ -970,7 +970,7 @@ public CompletableFuture offerSlot( allocatedSlots.remove(pendingRequest.getSlotRequestId()); tryFulfillSlotRequestOrMakeAvailable(allocatedSlot); } else { - log.debug("Fulfilled slot request {} with allocated slot {}.", pendingRequest.getSlotRequestId(), allocationID); + log.debug("Fulfilled slot request [{}] with allocated slot [{}].", pendingRequest.getSlotRequestId(), allocationID); } } else { @@ -1011,7 +1011,7 @@ public void failAllocation(final AllocationID allocationID, final Exception caus failPendingRequest(pendingRequest, cause); } else if (availableSlots.tryRemove(allocationID)) { - log.debug("Failed available slot with allocation id {}.", allocationID, cause); + log.debug("Failed available slot [{}].", allocationID, cause); } else { AllocatedSlot allocatedSlot = allocatedSlots.remove(allocationID); @@ -1021,7 +1021,7 @@ else if (availableSlots.tryRemove(allocationID)) { allocatedSlot.releasePayload(cause); } else { - log.trace("Outdated request to fail slot with allocation id {}.", allocationID, cause); + log.trace("Outdated request to fail slot [{}].", allocationID, cause); } } // TODO: add some unit tests when the previous two are ready, the allocation may failed at any phase @@ -1068,7 +1068,7 @@ public CompletableFuture releaseTaskManager(final ResourceID resour @VisibleForTesting protected void timeoutPendingSlotRequest(SlotRequestId slotRequestId) { - log.info("Pending slot request {} timed out.", slotRequestId); + log.info("Pending slot request [{}] timed out.", slotRequestId); removePendingRequest(slotRequestId); } @@ -1109,7 +1109,7 @@ private void checkIdleSlot() { final AllocationID allocationID = expiredSlot.getAllocationId(); if (availableSlots.tryRemove(allocationID)) { - log.info("Releasing idle slot {}.", allocationID); + log.info("Releasing idle slot [{}].", allocationID); final CompletableFuture freeSlotFuture = expiredSlot.getTaskManagerGateway().freeSlot( allocationID, cause, @@ -1119,12 +1119,12 @@ private void checkIdleSlot() { (Acknowledge ignored, Throwable throwable) -> { if (throwable != null) { if (registeredTaskManagers.contains(expiredSlot.getTaskManagerId())) { - log.debug("Releasing slot {} of registered TaskExecutor {} failed. " + + log.debug("Releasing slot [{}] of registered TaskExecutor {} failed. " + "Trying to fulfill a different slot request.", allocationID, expiredSlot.getTaskManagerId(), throwable); tryFulfillSlotRequestOrMakeAvailable(expiredSlot); } else { - log.debug("Releasing slot {} failed and owning TaskExecutor {} is no " + + log.debug("Releasing slot [{}] failed and owning TaskExecutor {} is no " + "longer registered. Discarding slot.", allocationID, expiredSlot.getTaskManagerId()); } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/slotpool/SlotSharingManager.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/slotpool/SlotSharingManager.java index eaa5787394833..afcd24f1064e4 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/slotpool/SlotSharingManager.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/slotpool/SlotSharingManager.java @@ -32,6 +32,9 @@ import org.apache.flink.util.FlinkException; import org.apache.flink.util.Preconditions; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + import javax.annotation.Nullable; import javax.annotation.concurrent.GuardedBy; @@ -77,6 +80,8 @@ */ public class SlotSharingManager { + private static final Logger LOG = LoggerFactory.getLogger(SlotSharingManager.class); + /** Lock for the internal data structures. */ private final Object lock = new Object(); @@ -143,6 +148,8 @@ MultiTaskSlot createRootSlot( slotContextFuture, allocatedSlotRequestId); + LOG.debug("Create multi task slot [{}] in slot [{}].", slotRequestId, allocatedSlotRequestId); + allTaskSlots.put(slotRequestId, rootMultiTaskSlot); synchronized (lock) { @@ -158,6 +165,8 @@ MultiTaskSlot createRootSlot( final MultiTaskSlot resolvedRootNode = unresolvedRootSlots.remove(slotRequestId); if (resolvedRootNode != null) { + LOG.trace("Fulfill multi task slot [{}] with slot [{}].", slotRequestId, slotContext.getAllocationId()); + final Set innerCollection = resolvedRootSlots.computeIfAbsent( slotContext.getTaskManagerLocation(), taskManagerLocation -> new HashSet<>(4)); @@ -384,6 +393,8 @@ CompletableFuture getSlotContextFuture() { MultiTaskSlot allocateMultiTaskSlot(SlotRequestId slotRequestId, AbstractID groupId) { Preconditions.checkState(!super.contains(groupId)); + LOG.debug("Create nested multi task slot [{}] in parent multi task slot [{}] for group [{}].", slotRequestId, getSlotRequestId(), groupId); + final MultiTaskSlot inner = new MultiTaskSlot( slotRequestId, groupId, @@ -412,6 +423,8 @@ SingleTaskSlot allocateSingleTaskSlot( Locality locality) { Preconditions.checkState(!super.contains(groupId)); + LOG.debug("Create single task slot [{}] in multi task slot [{}] for group {}.", slotRequestId, getSlotRequestId(), groupId); + final SingleTaskSlot leaf = new SingleTaskSlot( slotRequestId, groupId, @@ -557,13 +570,15 @@ private SingleTaskSlot( Preconditions.checkNotNull(locality); singleLogicalSlotFuture = parent.getSlotContextFuture() .thenApply( - (SlotContext slotContext) -> - new SingleLogicalSlot( + (SlotContext slotContext) -> { + LOG.trace("Fulfill single task slot [{}] with slot [{}].", slotRequestId, slotContext.getAllocationId()); + return new SingleLogicalSlot( slotRequestId, slotContext, slotSharingGroupId, locality, - slotOwner)); + slotOwner); + }); } CompletableFuture getLogicalSlotFuture() { From 7345cff2360a18ee98e9b7e0edc66e52f3b7a8ee Mon Sep 17 00:00:00 2001 From: Till Rohrmann Date: Sun, 22 Jul 2018 20:05:05 +0200 Subject: [PATCH 50/65] [FLINK-9908][scheduling] Do not cancel individual scheduling future Since the individual scheduling futures contain logic to release the slot if it cannot be assigned to the Execution, we must not cancel them. Otherwise we might risk that slots are not returned to the SlotPool leaving it in an inconsistent state. This closes #6383. --- .../executiongraph/ExecutionGraph.java | 8 -- .../ExecutionGraphSchedulingTest.java | 111 +++++++++++++++++- .../executiongraph/TestingSlotProvider.java | 82 +++++++++++++ .../jobmanager/slots/TestingSlotOwner.java | 2 +- 4 files changed, 193 insertions(+), 10 deletions(-) create mode 100644 flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/TestingSlotProvider.java 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 22c11efae873b..acb1e16fe713e 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 @@ -989,14 +989,6 @@ private CompletableFuture scheduleEager(SlotProvider slotProvider, final T throw new CompletionException(resultThrowable); }); - currentSchedulingFuture.whenComplete( - (Void ignored, Throwable throwable) -> { - if (throwable instanceof CancellationException) { - // cancel the individual allocation futures - allAllocationsFuture.cancel(false); - } - }); - return currentSchedulingFuture; } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphSchedulingTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphSchedulingTest.java index e43137b3b2e71..6092f5249a853 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphSchedulingTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphSchedulingTest.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.core.testutils.OneShotLatch; import org.apache.flink.metrics.groups.UnregisteredMetricsGroup; import org.apache.flink.runtime.blob.VoidBlobWriter; import org.apache.flink.runtime.checkpoint.StandaloneCheckpointRecoveryFactory; @@ -39,6 +40,7 @@ import org.apache.flink.runtime.jobgraph.JobStatus; import org.apache.flink.runtime.jobgraph.JobVertex; import org.apache.flink.runtime.jobgraph.ScheduleMode; +import org.apache.flink.runtime.jobmanager.scheduler.Locality; import org.apache.flink.runtime.jobmanager.slots.DummySlotOwner; import org.apache.flink.runtime.jobmanager.slots.TaskManagerGateway; import org.apache.flink.runtime.jobmanager.slots.TestingSlotOwner; @@ -46,6 +48,7 @@ import org.apache.flink.runtime.jobmaster.SlotOwner; import org.apache.flink.runtime.jobmaster.SlotRequestId; import org.apache.flink.runtime.jobmaster.TestingLogicalSlot; +import org.apache.flink.runtime.jobmaster.slotpool.SingleLogicalSlot; import org.apache.flink.runtime.jobmaster.slotpool.SlotProvider; import org.apache.flink.runtime.messages.Acknowledge; import org.apache.flink.runtime.taskmanager.LocalTaskManagerLocation; @@ -62,14 +65,19 @@ import javax.annotation.Nullable; import java.net.InetAddress; +import java.util.Set; import java.util.concurrent.ArrayBlockingQueue; import java.util.concurrent.BlockingQueue; import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ConcurrentMap; +import java.util.concurrent.CountDownLatch; import java.util.concurrent.Executors; import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeoutException; +import static org.hamcrest.Matchers.empty; import static org.hamcrest.Matchers.is; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertThat; @@ -504,6 +512,89 @@ public void testSlotReleasingFailsSchedulingOperation() throws Exception { assertThat(executionGraph.getTerminationFuture().get(), is(JobStatus.FAILED)); } + /** + * Tests that all slots are being returned to the {@link SlotOwner} if the + * {@link ExecutionGraph} is being cancelled. See FLINK-9908 + */ + @Test + public void testCancellationOfIncompleteScheduling() throws Exception { + final int parallelism = 10; + + final JobVertex jobVertex = new JobVertex("Test job vertex"); + jobVertex.setInvokableClass(NoOpInvokable.class); + jobVertex.setParallelism(parallelism); + + final JobGraph jobGraph = new JobGraph(jobVertex); + jobGraph.setAllowQueuedScheduling(true); + jobGraph.setScheduleMode(ScheduleMode.EAGER); + + final TestingSlotOwner slotOwner = new TestingSlotOwner(); + final SimpleAckingTaskManagerGateway taskManagerGateway = new SimpleAckingTaskManagerGateway(); + + final ConcurrentMap slotRequestIds = new ConcurrentHashMap<>(parallelism); + final CountDownLatch requestedSlotsLatch = new CountDownLatch(parallelism); + + final TestingSlotProvider slotProvider = new TestingSlotProvider( + (SlotRequestId slotRequestId) -> { + slotRequestIds.put(slotRequestId, 1); + requestedSlotsLatch.countDown(); + return new CompletableFuture<>(); + }); + + + final ExecutionGraph executionGraph = createExecutionGraph(jobGraph, slotProvider); + + executionGraph.scheduleForExecution(); + + // wait until we have requested all slots + requestedSlotsLatch.await(); + + final Set slotRequestIdsToReturn = ConcurrentHashMap.newKeySet(slotRequestIds.size()); + slotRequestIdsToReturn.addAll(slotRequestIds.keySet()); + final CountDownLatch countDownLatch = new CountDownLatch(slotRequestIds.size()); + + slotOwner.setReturnAllocatedSlotConsumer(logicalSlot -> { + slotRequestIdsToReturn.remove(logicalSlot.getSlotRequestId()); + countDownLatch.countDown(); + }); + slotProvider.setSlotCanceller(slotRequestId -> { + slotRequestIdsToReturn.remove(slotRequestId); + countDownLatch.countDown(); + }); + + final OneShotLatch slotRequestsBeingFulfilled = new OneShotLatch(); + + // start completing the slot requests asynchronously + executor.execute( + () -> { + slotRequestsBeingFulfilled.trigger(); + + for (SlotRequestId slotRequestId : slotRequestIds.keySet()) { + final SingleLogicalSlot singleLogicalSlot = createSingleLogicalSlot(slotOwner, taskManagerGateway, slotRequestId); + slotProvider.complete(slotRequestId, singleLogicalSlot); + } + }); + + // make sure that we complete cancellations of deployed tasks + taskManagerGateway.setCancelConsumer( + (ExecutionAttemptID executionAttemptId) -> { + final Execution execution = executionGraph.getRegisteredExecutions().get(executionAttemptId); + + // if the execution was cancelled in state SCHEDULING, then it might already have been removed + if (execution != null) { + execution.cancelingComplete(); + } + } + ); + + slotRequestsBeingFulfilled.await(); + + executionGraph.cancel(); + + countDownLatch.await(); + assertThat(slotRequestIdsToReturn, is(empty())); + } + // ------------------------------------------------------------------------ // Utilities // ------------------------------------------------------------------------ @@ -548,11 +639,29 @@ private SimpleSlot createSlot(TaskManagerGateway taskManager, JobID jobId, SlotO return new SimpleSlot(slot, slotOwner, 0); } + @Nonnull + private SingleLogicalSlot createSingleLogicalSlot(TestingSlotOwner slotOwner, SimpleAckingTaskManagerGateway taskManagerGateway, SlotRequestId slotRequestId) { + TaskManagerLocation location = new TaskManagerLocation( + ResourceID.generate(), InetAddress.getLoopbackAddress(), 12345); + + SimpleSlotContext slotContext = new SimpleSlotContext( + new AllocationID(), + location, + 0, + taskManagerGateway); + + return new SingleLogicalSlot( + slotRequestId, + slotContext, + null, + Locality.LOCAL, + slotOwner); + } + private static TaskManagerGateway createTaskManager() { TaskManagerGateway tm = mock(TaskManagerGateway.class); when(tm.submitTask(any(TaskDeploymentDescriptor.class), any(Time.class))) .thenReturn(CompletableFuture.completedFuture(Acknowledge.get())); - return tm; } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/TestingSlotProvider.java b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/TestingSlotProvider.java new file mode 100644 index 0000000000000..ed8fe130de6be --- /dev/null +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/TestingSlotProvider.java @@ -0,0 +1,82 @@ +/* + * 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.executiongraph; + +import org.apache.flink.api.common.time.Time; +import org.apache.flink.runtime.clusterframework.types.SlotProfile; +import org.apache.flink.runtime.instance.SlotSharingGroupId; +import org.apache.flink.runtime.jobmanager.scheduler.ScheduledUnit; +import org.apache.flink.runtime.jobmaster.LogicalSlot; +import org.apache.flink.runtime.jobmaster.SlotRequestId; +import org.apache.flink.runtime.jobmaster.slotpool.SlotProvider; +import org.apache.flink.runtime.messages.Acknowledge; +import org.apache.flink.util.Preconditions; + +import javax.annotation.Nullable; + +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ConcurrentMap; +import java.util.function.Consumer; +import java.util.function.Function; + +/** + * {@link SlotProvider} implementation for testing purposes. + */ +final class TestingSlotProvider implements SlotProvider { + + private final ConcurrentMap> slotFutures; + + private final Function> slotFutureCreator; + + private volatile Consumer slotCanceller = ignored -> {}; + + TestingSlotProvider(Function> slotFutureCreator) { + this.slotFutureCreator = slotFutureCreator; + this.slotFutures = new ConcurrentHashMap<>(4); + } + + public void setSlotCanceller(Consumer slotCanceller) { + this.slotCanceller = slotCanceller; + } + + @Override + public CompletableFuture allocateSlot(SlotRequestId slotRequestId, ScheduledUnit task, boolean allowQueued, SlotProfile slotProfile, Time timeout) { + Preconditions.checkState(!slotFutures.containsKey(slotRequestId)); + final CompletableFuture slotFuture = slotFutureCreator.apply(slotRequestId); + + slotFutures.put(slotRequestId, slotFuture); + + return slotFuture; + } + + @Override + public CompletableFuture cancelSlotRequest(SlotRequestId slotRequestId, @Nullable SlotSharingGroupId slotSharingGroupId, Throwable cause) { + final CompletableFuture slotFuture = slotFutures.remove(slotRequestId); + slotFuture.cancel(false); + + slotCanceller.accept(slotRequestId); + + return CompletableFuture.completedFuture(Acknowledge.get()); + } + + public void complete(SlotRequestId slotRequestId, LogicalSlot logicalSlot) { + slotFutures.get(slotRequestId).complete(logicalSlot); + } +} diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/slots/TestingSlotOwner.java b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/slots/TestingSlotOwner.java index 727c0b5723af4..b922204f0e075 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/slots/TestingSlotOwner.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/slots/TestingSlotOwner.java @@ -37,7 +37,7 @@ public void setReturnAllocatedSlotConsumer(Consumer returnAllocated @Override public CompletableFuture returnAllocatedSlot(LogicalSlot logicalSlot) { - final Consumer currentReturnAllocatedSlotConsumer = this.returnAllocatedSlotConsumer; + final Consumer currentReturnAllocatedSlotConsumer = returnAllocatedSlotConsumer; if (currentReturnAllocatedSlotConsumer != null) { currentReturnAllocatedSlotConsumer.accept(logicalSlot); From 234356772ee480e2a331f7aaf0f7f4654a3423d5 Mon Sep 17 00:00:00 2001 From: Till Rohrmann Date: Sun, 22 Jul 2018 20:17:11 +0200 Subject: [PATCH 51/65] [FLINK-9909][core] ConjunctFuture does not cancel input futures If a ConjunctFuture is cancelled, then it won't cancel all of its input futures automatically. If the users needs this behaviour then he has to implement it explicitly. The reason for this change is that an implicit cancellation can have unwanted side effects, because all of the cancelled input futures' producers won't be executed. This closes #6384. --- .../flink/runtime/concurrent/FutureUtils.java | 40 ------------------- .../runtime/concurrent/FutureUtilsTest.java | 34 ---------------- 2 files changed, 74 deletions(-) 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 1cffaab112f90..3a7e800eb401d 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 @@ -508,22 +508,6 @@ public abstract static class ConjunctFuture extends CompletableFuture { * @return The number of Futures in the conjunction that are already complete */ public abstract int getNumFuturesCompleted(); - - /** - * Gets the individual futures which make up the {@link ConjunctFuture}. - * - * @return Collection of futures which make up the {@link ConjunctFuture} - */ - protected abstract Collection> getConjunctFutures(); - - @Override - public boolean cancel(boolean mayInterruptIfRunning) { - for (CompletableFuture completableFuture : getConjunctFutures()) { - completableFuture.cancel(mayInterruptIfRunning); - } - - return super.cancel(mayInterruptIfRunning); - } } /** @@ -531,8 +515,6 @@ public boolean cancel(boolean mayInterruptIfRunning) { */ private static class ResultConjunctFuture extends ConjunctFuture> { - private final Collection> resultFutures; - /** The total number of futures in the conjunction. */ private final int numTotal; @@ -564,7 +546,6 @@ private void handleCompletedFuture(T value, Throwable throwable) { @SuppressWarnings("unchecked") ResultConjunctFuture(Collection> resultFutures) { - this.resultFutures = checkNotNull(resultFutures); this.numTotal = resultFutures.size(); results = (T[]) new Object[numTotal]; @@ -587,11 +568,6 @@ public int getNumFuturesTotal() { public int getNumFuturesCompleted() { return numCompleted.get(); } - - @Override - protected Collection> getConjunctFutures() { - return resultFutures; - } } /** @@ -600,8 +576,6 @@ protected Collection> getConjunctFutures() { */ private static final class WaitingConjunctFuture extends ConjunctFuture { - private final Collection> futures; - /** Number of completed futures. */ private final AtomicInteger numCompleted = new AtomicInteger(0); @@ -620,7 +594,6 @@ private void handleCompletedFuture(Object ignored, Throwable throwable) { } private WaitingConjunctFuture(Collection> futures) { - this.futures = checkNotNull(futures); this.numTotal = futures.size(); if (futures.isEmpty()) { @@ -641,11 +614,6 @@ public int getNumFuturesTotal() { public int getNumFuturesCompleted() { return numCompleted.get(); } - - @Override - protected Collection> getConjunctFutures() { - return futures; - } } /** @@ -673,14 +641,11 @@ private static final class CompletionConjunctFuture extends ConjunctFuture private final int numFuturesTotal; - private final Collection> futuresToComplete; - private int futuresCompleted; private Throwable globalThrowable; private CompletionConjunctFuture(Collection> futuresToComplete) { - this.futuresToComplete = checkNotNull(futuresToComplete); numFuturesTotal = futuresToComplete.size(); futuresCompleted = 0; @@ -725,11 +690,6 @@ public int getNumFuturesCompleted() { return futuresCompleted; } } - - @Override - protected Collection> getConjunctFutures() { - return futuresToComplete; - } } // ------------------------------------------------------------------------ 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 07bc4c19be403..1639c9195695d 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 @@ -28,7 +28,6 @@ import org.junit.Test; -import java.util.ArrayList; import java.util.Arrays; import java.util.Collection; import java.util.List; @@ -42,7 +41,6 @@ import java.util.concurrent.TimeoutException; import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicReference; -import java.util.function.Function; import java.util.function.Supplier; import static org.hamcrest.CoreMatchers.containsString; @@ -547,38 +545,6 @@ public void testCompleteAllExceptional() throws Exception { } } - @Test - public void testCancelWaitingConjunctFuture() { - cancelConjunctFuture(inputFutures -> FutureUtils.waitForAll(inputFutures)); - } - - @Test - public void testCancelResultConjunctFuture() { - cancelConjunctFuture(inputFutures -> FutureUtils.combineAll(inputFutures)); - } - - @Test - public void testCancelCompleteConjunctFuture() { - cancelConjunctFuture(inputFutures -> FutureUtils.completeAll(inputFutures)); - } - - private void cancelConjunctFuture(Function>, FutureUtils.ConjunctFuture> conjunctFutureFactory) { - final int numInputFutures = 10; - final Collection> inputFutures = new ArrayList<>(numInputFutures); - - for (int i = 0; i < numInputFutures; i++) { - inputFutures.add(new CompletableFuture<>()); - } - - final FutureUtils.ConjunctFuture conjunctFuture = conjunctFutureFactory.apply(inputFutures); - - conjunctFuture.cancel(false); - - for (CompletableFuture inputFuture : inputFutures) { - assertThat(inputFuture.isCancelled(), is(true)); - } - } - @Test public void testSupplyAsyncFailure() throws Exception { final String exceptionMessage = "Test exception"; From 17fde3386de8f05f331f77f213148774e46f6617 Mon Sep 17 00:00:00 2001 From: Till Rohrmann Date: Sun, 22 Jul 2018 20:20:53 +0200 Subject: [PATCH 52/65] [hotfix] Fix checkstyle violations in FutureUtils --- .../flink/runtime/concurrent/FutureUtils.java | 14 +++++++------- 1 file changed, 7 insertions(+), 7 deletions(-) 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 3a7e800eb401d..d4a65dec8f636 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 @@ -22,7 +22,6 @@ import org.apache.flink.api.common.time.Time; import org.apache.flink.runtime.util.ExecutorThreadFactory; import org.apache.flink.util.ExceptionUtils; -import org.apache.flink.util.Preconditions; import org.apache.flink.util.function.RunnableWithException; import org.apache.flink.util.function.SupplierWithException; @@ -786,7 +785,7 @@ private static final class Timeout implements Runnable { private final CompletableFuture future; private Timeout(CompletableFuture future) { - this.future = Preconditions.checkNotNull(future); + this.future = checkNotNull(future); } @Override @@ -800,8 +799,9 @@ public void run() { * *

    This class creates a singleton scheduler used to run the provided actions. */ - private static final class Delayer { - static final ScheduledThreadPoolExecutor delayer = new ScheduledThreadPoolExecutor( + private enum Delayer { + ; + static final ScheduledThreadPoolExecutor DELAYER = new ScheduledThreadPoolExecutor( 1, new ExecutorThreadFactory("FlinkCompletableFutureDelayScheduler")); @@ -814,10 +814,10 @@ private static final class Delayer { * @return Future of the scheduled action */ private static ScheduledFuture delay(Runnable runnable, long delay, TimeUnit timeUnit) { - Preconditions.checkNotNull(runnable); - Preconditions.checkNotNull(timeUnit); + checkNotNull(runnable); + checkNotNull(timeUnit); - return delayer.schedule(runnable, delay, timeUnit); + return DELAYER.schedule(runnable, delay, timeUnit); } } } From a678f45d9713d2dccb0cd2459e94af75f0649803 Mon Sep 17 00:00:00 2001 From: Till Rohrmann Date: Sun, 22 Jul 2018 20:34:33 +0200 Subject: [PATCH 53/65] [hotfix] Replace check state condition in Execution#tryAssignResource with if check Instead of risking an IllegalStateException it is better to check that the taskManagerLocationFuture has not been completed yet. If, then we also reject the assignment of the LogicalSlot to the Execution. That way, we don't risk that we don't release the slot in case of an exception in Execution#allocateAndAssignSlotForExecution. --- .../runtime/executiongraph/Execution.java | 20 +++++++++++-------- 1 file changed, 12 insertions(+), 8 deletions(-) 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 853732f82a7f9..57aa0d5b3e0dc 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 @@ -280,15 +280,19 @@ boolean tryAssignResource(final LogicalSlot logicalSlot) { // only allow to set the assigned resource in state SCHEDULED or CREATED // note: we also accept resource assignment when being in state CREATED for testing purposes if (state == SCHEDULED || state == CREATED) { - if (ASSIGNED_SLOT_UPDATER.compareAndSet(this, null, logicalSlot) && logicalSlot.tryAssignPayload(this)) { - // check for concurrent modification (e.g. cancelling call) - if (state == SCHEDULED || state == CREATED) { - checkState(!taskManagerLocationFuture.isDone(), "The TaskManagerLocationFuture should not be set if we haven't assigned a resource yet."); - taskManagerLocationFuture.complete(logicalSlot.getTaskManagerLocation()); - assignedAllocationID = logicalSlot.getAllocationId(); - return true; + if (ASSIGNED_SLOT_UPDATER.compareAndSet(this, null, logicalSlot)) { + if (logicalSlot.tryAssignPayload(this)) { + // check for concurrent modification (e.g. cancelling call) + if ((state == SCHEDULED || state == CREATED) && !taskManagerLocationFuture.isDone()) { + taskManagerLocationFuture.complete(logicalSlot.getTaskManagerLocation()); + assignedAllocationID = logicalSlot.getAllocationId(); + return true; + } else { + // free assigned resource and return false + ASSIGNED_SLOT_UPDATER.set(this, null); + return false; + } } else { - // free assigned resource and return false ASSIGNED_SLOT_UPDATER.set(this, null); return false; } From 2d38d158824f0249d646db761826463eafb6b942 Mon Sep 17 00:00:00 2001 From: Till Rohrmann Date: Sun, 22 Jul 2018 20:43:44 +0200 Subject: [PATCH 54/65] [hotfix] Fix checkstyle violations in ExecutionVertex --- .../executiongraph/ExecutionVertex.java | 80 +++++++++---------- 1 file changed, 39 insertions(+), 41 deletions(-) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionVertex.java b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionVertex.java index 8b57a7a6a9377..e385318b810b5 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionVertex.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionVertex.java @@ -33,9 +33,7 @@ import org.apache.flink.runtime.deployment.ResultPartitionDeploymentDescriptor; import org.apache.flink.runtime.deployment.TaskDeploymentDescriptor; import org.apache.flink.runtime.execution.ExecutionState; -import org.apache.flink.runtime.jobmaster.LogicalSlot; import org.apache.flink.runtime.instance.SimpleSlot; -import org.apache.flink.runtime.jobmaster.slotpool.SlotProvider; import org.apache.flink.runtime.io.network.partition.ResultPartitionID; import org.apache.flink.runtime.io.network.partition.ResultPartitionType; import org.apache.flink.runtime.jobgraph.DistributionPattern; @@ -46,6 +44,8 @@ import org.apache.flink.runtime.jobmanager.scheduler.CoLocationConstraint; import org.apache.flink.runtime.jobmanager.scheduler.CoLocationGroup; import org.apache.flink.runtime.jobmanager.scheduler.LocationPreferenceConstraint; +import org.apache.flink.runtime.jobmaster.LogicalSlot; +import org.apache.flink.runtime.jobmaster.slotpool.SlotProvider; import org.apache.flink.runtime.state.KeyGroupRangeAssignment; import org.apache.flink.runtime.taskmanager.TaskManagerLocation; import org.apache.flink.runtime.util.EvictingBoundedList; @@ -96,12 +96,12 @@ public class ExecutionVertex implements AccessExecutionVertex, Archiveable *

  • If the task execution has state to load (from a checkpoint), then the location preference * is the location of the previous execution (if there is a previous execution attempt). *
  • If the task execution has no state or no previous location, then the location preference * is based on the task's inputs. * - * - * These rules should result in the following behavior: - * + * + *

    These rules should result in the following behavior: + * *

      *
    • Stateless tasks are always scheduled based on co-location with inputs. *
    • Stateful tasks are on their initial attempt executed based on co-location with inputs. *
    • Repeated executions of stateful tasks try to co-locate the execution with its state. *
    - * - * @return The preferred execution locations for the execution attempt. - * + * * @see #getPreferredLocationsBasedOnState() - * @see #getPreferredLocationsBasedOnInputs() + * @see #getPreferredLocationsBasedOnInputs() + * + * @return The preferred execution locations for the execution attempt. */ public Collection> getPreferredLocations() { Collection> basedOnState = getPreferredLocationsBasedOnState(); return basedOnState != null ? basedOnState : getPreferredLocationsBasedOnInputs(); } - + /** * Gets the preferred location to execute the current task execution attempt, based on the state * that the execution attempt will resume. - * + * * @return A size-one collection with the location preference, or null, if there is no * location preference based on the state. */ @@ -542,27 +543,25 @@ public Collection> getPreferredLocationsB /** * Archives the current Execution and creates a new Execution for this vertex. - * + * *

    This method atomically checks if the ExecutionGraph is still of an expected * global mod. version and replaces the execution if that is the case. If the ExecutionGraph * has increased its global mod. version in the meantime, this operation fails. - * + * *

    This mechanism can be used to prevent conflicts between various concurrent recovery and * reconfiguration actions in a similar way as "optimistic concurrency control". - * + * * @param timestamp * The creation timestamp for the new Execution * @param originatingGlobalModVersion - * The - * - * @return Returns the new created Execution. - * + * + * @return Returns the new created Execution. + * * @throws GlobalModVersionMismatch Thrown, if the execution graph has a new global mod * version than the one passed to this message. */ public Execution resetForNewExecution(final long timestamp, final long originatingGlobalModVersion) - throws GlobalModVersionMismatch - { + throws GlobalModVersionMismatch { LOG.debug("Resetting execution vertex {} for new execution.", getTaskNameWithSubtaskIndex()); synchronized (priorExecutions) { @@ -642,12 +641,13 @@ public void deployToSlot(SimpleSlot slot) throws JobException { } /** - * + * Cancels this ExecutionVertex. + * * @return A future that completes once the execution has reached its final state. */ public CompletableFuture cancel() { // to avoid any case of mixup in the presence of concurrent calls, - // we copy a reference to the stack to make sure both calls go to the same Execution + // we copy a reference to the stack to make sure both calls go to the same Execution final Execution exec = this.currentExecution; exec.cancel(); return exec.getReleaseFuture(); @@ -742,7 +742,7 @@ void executionFailed(Execution execution, Throwable cause) { // -------------------------------------------------------------------------------------------- /** - * Simply forward this notification + * Simply forward this notification. */ void notifyStateTransition(Execution execution, ExecutionState newState, Throwable error) { // only forward this notification if the execution is still the current execution @@ -754,7 +754,6 @@ void notifyStateTransition(Execution execution, ExecutionState newState, Throwab /** * Creates a task deployment descriptor to deploy a subtask to the given target slot. - * * TODO: This should actually be in the EXECUTION */ TaskDeploymentDescriptor createDeploymentDescriptor( @@ -762,13 +761,13 @@ TaskDeploymentDescriptor createDeploymentDescriptor( LogicalSlot targetSlot, @Nullable JobManagerTaskRestore taskRestore, int attemptNumber) throws ExecutionGraphException { - + // Produced intermediate results List producedPartitions = new ArrayList<>(resultPartitions.size()); - + // Consumed intermediate results List consumedPartitions = new ArrayList<>(inputEdges.length); - + boolean lazyScheduling = getExecutionGraph().getScheduleMode().allowLazyDeployment(); for (IntermediateResultPartition partition : resultPartitions.values()) { @@ -791,8 +790,7 @@ TaskDeploymentDescriptor createDeploymentDescriptor( producedPartitions.add(ResultPartitionDeploymentDescriptor.from(partition, maxParallelism, lazyScheduling)); } } - - + for (ExecutionEdge[] edges : inputEdges) { InputChannelDeploymentDescriptor[] partitions = InputChannelDeploymentDescriptor.fromEdges( edges, From 43592e2e925421876194e93740b3f06d29cf9efa Mon Sep 17 00:00:00 2001 From: Till Rohrmann Date: Sun, 22 Jul 2018 20:46:37 +0200 Subject: [PATCH 55/65] [hotfix] Fix checkstyle violations in ExecutionJobVertex --- .../executiongraph/ExecutionJobVertex.java | 44 +++++++++---------- 1 file changed, 22 insertions(+), 22 deletions(-) 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 0691cc76e698a..6da1e0db8927e 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 @@ -317,7 +317,7 @@ private void setMaxParallelismInternal(int maxParallelism) { public ExecutionGraph getGraph() { return graph; } - + public JobVertex getJobVertex() { return jobVertex; } @@ -344,33 +344,33 @@ public boolean isMaxParallelismConfigured() { public JobID getJobId() { return graph.getJobID(); } - + @Override public JobVertexID getJobVertexId() { return jobVertex.getID(); } - + @Override public ExecutionVertex[] getTaskVertices() { return taskVertices; } - + public IntermediateResult[] getProducedDataSets() { return producedDataSets; } - + public InputSplitAssigner getSplitAssigner() { return splitAssigner; } - + public SlotSharingGroup getSlotSharingGroup() { return slotSharingGroup; } - + public CoLocationGroup getCoLocationGroup() { return coLocationGroup; } - + public List getInputs() { return inputs; } @@ -423,28 +423,28 @@ private String generateDebugString() { //--------------------------------------------------------------------------------------------- - + public void connectToPredecessors(Map intermediateDataSets) throws JobException { - + List inputs = jobVertex.getInputs(); - + if (LOG.isDebugEnabled()) { LOG.debug(String.format("Connecting ExecutionJobVertex %s (%s) to %d predecessors.", jobVertex.getID(), jobVertex.getName(), inputs.size())); } - + for (int num = 0; num < inputs.size(); num++) { JobEdge edge = inputs.get(num); - + if (LOG.isDebugEnabled()) { if (edge.getSource() == null) { - LOG.debug(String.format("Connecting input %d of vertex %s (%s) to intermediate result referenced via ID %s.", + LOG.debug(String.format("Connecting input %d of vertex %s (%s) to intermediate result referenced via ID %s.", num, jobVertex.getID(), jobVertex.getName(), edge.getSourceId())); } else { LOG.debug(String.format("Connecting input %d of vertex %s (%s) to intermediate result referenced via predecessor %s (%s).", num, jobVertex.getID(), jobVertex.getName(), edge.getSource().getProducer().getID(), edge.getSource().getProducer().getName())); } } - + // fetch the intermediate result via ID. if it does not exist, then it either has not been created, or the order // in which this method is called for the job vertices is not a topological order IntermediateResult ires = intermediateDataSets.get(edge.getSourceId()); @@ -452,18 +452,18 @@ public void connectToPredecessors(Map throw new JobException("Cannot connect this job graph to the previous graph. No previous intermediate result found for ID " + edge.getSourceId()); } - + this.inputs.add(ires); - + int consumerIndex = ires.registerConsumer(); - + for (int i = 0; i < parallelism; i++) { ExecutionVertex ev = taskVertices[i]; ev.connectSource(num, ires, edge, consumerIndex); } } } - + //--------------------------------------------------------------------------------------------- // Actions //--------------------------------------------------------------------------------------------- @@ -480,7 +480,7 @@ public CompletableFuture scheduleAll( SlotProvider slotProvider, boolean queued, LocationPreferenceConstraint locationPreferenceConstraint) { - + final ExecutionVertex[] vertices = this.taskVertices; final ArrayList> scheduleFutures = new ArrayList<>(vertices.length); @@ -497,9 +497,9 @@ public CompletableFuture scheduleAll( * Acquires a slot for all the execution vertices of this ExecutionJobVertex. The method returns * pairs of the slots and execution attempts, to ease correlation between vertices and execution * attempts. - * + * *

    If this method throws an exception, it makes sure to release all so far requested slots. - * + * * @param resourceProvider The resource provider from whom the slots are requested. * @param queued if the allocation can be queued * @param locationPreferenceConstraint constraint for the location preferences From b06d2411c1d5bbd2f3df2470d558bfa3ba0a3b06 Mon Sep 17 00:00:00 2001 From: Till Rohrmann Date: Sun, 22 Jul 2018 20:48:53 +0200 Subject: [PATCH 56/65] [hotfix] Fix checkstyle violations in Execution --- .../runtime/executiongraph/Execution.java | 27 +++++++++---------- 1 file changed, 13 insertions(+), 14 deletions(-) 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 57aa0d5b3e0dc..f8419d33cc647 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 @@ -179,7 +179,7 @@ public class Execution implements AccessExecution, ArchiveableThis version is bumped in the ExecutionGraph whenever a global failover happens. It is used * to resolve conflicts between concurrent modification by global and local failover actions. */ @@ -391,7 +391,7 @@ public CompletableFuture scheduleForExecution() { * NOTE: This method only throws exceptions if it is in an illegal state to be scheduled, or if the tasks needs * to be scheduled immediately and no resource is available. If the task is accepted by the schedule, any * error sets the vertex state to failed and triggers the recovery logic. - * + * * @param slotProvider The slot provider to use to allocate slot for this execution attempt. * @param queued Flag to indicate whether the scheduler may queue this task if it cannot * immediately deploy it. @@ -773,11 +773,10 @@ else if (numConsumers == 0) { final TaskManagerLocation partitionTaskManagerLocation = partition.getProducer() .getCurrentAssignedResource().getTaskManagerLocation(); final ResourceID partitionTaskManager = partitionTaskManagerLocation.getResourceID(); - + final ResourceID consumerTaskManager = consumerSlot.getTaskManagerLocation().getResourceID(); final ResultPartitionID partitionId = new ResultPartitionID(partition.getPartitionId(), attemptId); - final ResultPartitionLocation partitionLocation; @@ -986,7 +985,7 @@ else if (current == CANCELED || current == FAILED) { void cancelingComplete() { cancelingComplete(null, null); } - + void cancelingComplete(Map> userAccumulators, IOMetrics metrics) { // the taskmanagers can themselves cancel tasks without an external trigger, if they find that the @@ -1148,7 +1147,7 @@ boolean switchToRunning() { else if (currentState == CANCELING || currentState == FAILED) { if (LOG.isDebugEnabled()) { // this log statement is guarded because the 'getVertexWithAttempt()' method - // performs string concatenations + // performs string concatenations LOG.debug("Concurrent canceling/failing of {} while deployment was in progress.", getVertexWithAttempt()); } sendCancelRpcCall(); @@ -1175,7 +1174,7 @@ else if (currentState == CANCELING || currentState == FAILED) { /** * This method sends a CancelTask message to the instance of the assigned slot. * - * The sending is tried up to NUM_CANCEL_CALL_TRIES times. + *

    The sending is tried up to NUM_CANCEL_CALL_TRIES times. */ private void sendCancelRpcCall() { final LogicalSlot slot = assignedResource; @@ -1238,9 +1237,9 @@ private void sendUpdatePartitionInfoRpcCall( /** * Releases the assigned resource and completes the release future - * once the assigned resource has been successfully released + * once the assigned resource has been successfully released. * - * @param cause for the resource release, null if none + * @param cause for the resource release, null if none */ private void releaseAssignedResource(@Nullable Throwable cause) { final LogicalSlot slot = assignedResource; @@ -1357,7 +1356,7 @@ public String getVertexWithAttempt() { // ------------------------------------------------------------------------ // Accumulators // ------------------------------------------------------------------------ - + /** * Update accumulators (discarded when the Execution has already been terminated). * @param userAccumulators the user accumulators @@ -1369,7 +1368,7 @@ public void setAccumulators(Map> userAccumulators) { } } } - + public Map> getUserAccumulators() { return userAccumulators; } @@ -1389,7 +1388,7 @@ public StringifiedAccumulatorResult[] getUserAccumulatorsStringified() { public int getParallelSubtaskIndex() { return getVertex().getParallelSubtaskIndex(); } - + @Override public IOMetrics getIOMetrics() { return ioMetrics; @@ -1409,7 +1408,7 @@ private void updateAccumulatorsAndMetrics(Map> userAcc // ------------------------------------------------------------------------ // Standard utilities // ------------------------------------------------------------------------ - + @Override public String toString() { final LogicalSlot slot = assignedResource; From 4608f67df61d58569e229db04590b161c014c8a0 Mon Sep 17 00:00:00 2001 From: Till Rohrmann Date: Sun, 22 Jul 2018 21:38:42 +0200 Subject: [PATCH 57/65] [FLINK-9910][scheduling] Execution#scheduleForeExecution does not cancel slot future In order to properly give back an allocated slot to the SlotPool, one must not complete the result future of Execution#allocateAndAssignSlotForExecution. This commit changes the behaviour in Execution#scheduleForExecution accordingly. This closes #6385. --- .../FutureConsumerWithException.java | 43 +++++++++++ .../runtime/executiongraph/Execution.java | 24 +++--- .../ExecutionGraphSchedulingTest.java | 2 +- .../runtime/executiongraph/ExecutionTest.java | 75 +++++++++++++++++++ 4 files changed, 129 insertions(+), 15 deletions(-) create mode 100644 flink-core/src/main/java/org/apache/flink/util/concurrent/FutureConsumerWithException.java diff --git a/flink-core/src/main/java/org/apache/flink/util/concurrent/FutureConsumerWithException.java b/flink-core/src/main/java/org/apache/flink/util/concurrent/FutureConsumerWithException.java new file mode 100644 index 0000000000000..c49d7dc08d49c --- /dev/null +++ b/flink-core/src/main/java/org/apache/flink/util/concurrent/FutureConsumerWithException.java @@ -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.util.concurrent; + +import java.util.concurrent.CompletionException; +import java.util.function.Consumer; + +/** + * A checked extension of the {@link Consumer} interface which rethrows + * exceptions wrapped in a {@link CompletionException}. + * + * @param type of the first argument + * @param type of the thrown exception + */ +public interface FutureConsumerWithException extends Consumer { + + void acceptWithException(T value) throws E; + + @Override + default void accept(T value) { + try { + acceptWithException(value); + } catch (Throwable t) { + throw new CompletionException(t); + } + } +} 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 f8419d33cc647..801f35a41dcc9 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 @@ -54,6 +54,7 @@ import org.apache.flink.util.FlinkException; import org.apache.flink.util.FlinkRuntimeException; import org.apache.flink.util.OptionalFailure; +import org.apache.flink.util.concurrent.FutureConsumerWithException; import org.slf4j.Logger; @@ -413,24 +414,19 @@ public CompletableFuture scheduleForExecution( // IMPORTANT: We have to use the synchronous handle operation (direct executor) here so // that we directly deploy the tasks if the slot allocation future is completed. This is // necessary for immediate deployment. - final CompletableFuture deploymentFuture = allocationFuture.handle( - (Execution ignored, Throwable throwable) -> { - if (throwable != null) { - markFailed(ExceptionUtils.stripCompletionException(throwable)); - } else { - try { - deploy(); - } catch (Throwable t) { - markFailed(ExceptionUtils.stripCompletionException(t)); - } + final CompletableFuture deploymentFuture = allocationFuture.thenAccept( + (FutureConsumerWithException) value -> deploy()); + + deploymentFuture.whenComplete( + (Void ignored, Throwable failure) -> { + if (failure != null) { + markFailed(ExceptionUtils.stripCompletionException(failure)); } - return null; - } - ); + }); // if tasks have to scheduled immediately check that the task has been deployed if (!queued && !deploymentFuture.isDone()) { - allocationFuture.completeExceptionally(new IllegalArgumentException("The slot allocation future has not been completed yet.")); + deploymentFuture.completeExceptionally(new IllegalArgumentException("The slot allocation future has not been completed yet.")); } return deploymentFuture; diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphSchedulingTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphSchedulingTest.java index 6092f5249a853..6680c9e741e9d 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphSchedulingTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphSchedulingTest.java @@ -640,7 +640,7 @@ private SimpleSlot createSlot(TaskManagerGateway taskManager, JobID jobId, SlotO } @Nonnull - private SingleLogicalSlot createSingleLogicalSlot(TestingSlotOwner slotOwner, SimpleAckingTaskManagerGateway taskManagerGateway, SlotRequestId slotRequestId) { + static SingleLogicalSlot createSingleLogicalSlot(SlotOwner slotOwner, TaskManagerGateway taskManagerGateway, SlotRequestId slotRequestId) { TaskManagerLocation location = new TaskManagerLocation( ResourceID.generate(), InetAddress.getLoopbackAddress(), 12345); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionTest.java index d3e88e1832187..56fd7e12369b0 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionTest.java @@ -33,6 +33,7 @@ import org.apache.flink.runtime.jobmaster.SlotOwner; import org.apache.flink.runtime.jobmaster.SlotRequestId; import org.apache.flink.runtime.jobmaster.TestingLogicalSlot; +import org.apache.flink.runtime.jobmaster.slotpool.SingleLogicalSlot; import org.apache.flink.runtime.jobmaster.slotpool.SlotProvider; import org.apache.flink.runtime.taskmanager.LocalTaskManagerLocation; import org.apache.flink.runtime.taskmanager.TaskManagerLocation; @@ -50,6 +51,8 @@ import java.util.Set; import java.util.concurrent.CompletableFuture; import java.util.concurrent.ExecutionException; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; import static org.hamcrest.Matchers.containsInAnyOrder; import static org.hamcrest.Matchers.equalTo; @@ -419,6 +422,78 @@ public void testTaskRestoreStateIsNulledAfterDeployment() throws Exception { assertThat(execution.getTaskRestore(), is(nullValue())); } + @Test + public void testEagerSchedulingFailureReturnsSlot() throws Exception { + final JobVertex jobVertex = createNoOpJobVertex(); + final JobVertexID jobVertexId = jobVertex.getID(); + + final SimpleAckingTaskManagerGateway taskManagerGateway = new SimpleAckingTaskManagerGateway(); + final SingleSlotTestingSlotOwner slotOwner = new SingleSlotTestingSlotOwner(); + + final CompletableFuture slotRequestIdFuture = new CompletableFuture<>(); + final CompletableFuture returnedSlotFuture = new CompletableFuture<>(); + + final TestingSlotProvider slotProvider = new TestingSlotProvider( + (SlotRequestId slotRequestId) -> { + slotRequestIdFuture.complete(slotRequestId); + return new CompletableFuture<>(); + }); + + slotProvider.setSlotCanceller(returnedSlotFuture::complete); + slotOwner.getReturnedSlotFuture().thenAccept( + (LogicalSlot logicalSlot) -> returnedSlotFuture.complete(logicalSlot.getSlotRequestId())); + + ExecutionGraph executionGraph = ExecutionGraphTestUtils.createSimpleTestGraph( + new JobID(), + slotProvider, + new NoRestartStrategy(), + jobVertex); + + ExecutionJobVertex executionJobVertex = executionGraph.getJobVertex(jobVertexId); + + ExecutionVertex executionVertex = executionJobVertex.getTaskVertices()[0]; + + final Execution execution = executionVertex.getCurrentExecutionAttempt(); + + taskManagerGateway.setCancelConsumer( + executionAttemptID -> { + if (execution.getAttemptId().equals(executionAttemptID)) { + execution.cancelingComplete(); + } + } + ); + + final ExecutorService executorService = Executors.newFixedThreadPool(1); + + try { + slotRequestIdFuture.thenAcceptAsync( + (SlotRequestId slotRequestId) -> { + final SingleLogicalSlot singleLogicalSlot = ExecutionGraphSchedulingTest.createSingleLogicalSlot( + slotOwner, + taskManagerGateway, + slotRequestId); + slotProvider.complete(slotRequestId, singleLogicalSlot); + }, + executorService); + + final CompletableFuture schedulingFuture = execution.scheduleForExecution( + slotProvider, + false, + LocationPreferenceConstraint.ANY); + + try { + schedulingFuture.get(); + // cancel the execution in case we could schedule the execution + execution.cancel(); + } catch (ExecutionException ignored) { + } + + assertThat(returnedSlotFuture.get(), is(equalTo(slotRequestIdFuture.get()))); + } finally { + executorService.shutdownNow(); + } + } + @Nonnull private JobVertex createNoOpJobVertex() { final JobVertex jobVertex = new JobVertex("Test vertex", new JobVertexID()); From 5f5b0fb96e6ec6f665701f2894a2704893537594 Mon Sep 17 00:00:00 2001 From: Till Rohrmann Date: Sun, 22 Jul 2018 21:57:59 +0200 Subject: [PATCH 58/65] [FLINK-9911][JM] Use SlotPoolGateway to call failAllocation Since the SlotPool is an actor, we must use the SlotPoolGateway to interact with the SlotPool. Otherwise, we might risk an inconsistent state since there are multiple threads modifying the component. This closes #6386. --- .../main/java/org/apache/flink/runtime/jobmaster/JobMaster.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) 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 1660f95d26703..c47f4fd19ffe7 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 @@ -983,7 +983,7 @@ public CompletableFuture requestOperatorBackP @Override public void notifyAllocationFailure(AllocationID allocationID, Exception cause) { - slotPool.failAllocation(allocationID, cause); + slotPoolGateway.failAllocation(allocationID, cause); } //---------------------------------------------------------------------------------------------- From 195d53e3daa13503e130dfcae70b5d659c429d25 Mon Sep 17 00:00:00 2001 From: gyao Date: Thu, 19 Jul 2018 09:35:44 +0200 Subject: [PATCH 59/65] [FLINK-9892][tests] Disable local recovery in Jepsen tests. This closes #6369. --- flink-jepsen/src/jepsen/flink/db.clj | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/flink-jepsen/src/jepsen/flink/db.clj b/flink-jepsen/src/jepsen/flink/db.clj index ff934e1de6e34..857e1d3778744 100644 --- a/flink-jepsen/src/jepsen/flink/db.clj +++ b/flink-jepsen/src/jepsen/flink/db.clj @@ -56,7 +56,7 @@ :taskmanager.numberOfTaskSlots taskmanager-slots :yarn.application-attempts 99999 :slotmanager.taskmanager-timeout 10000 - :state.backend.local-recovery "true" + :state.backend.local-recovery "false" :taskmanager.registration.timeout "30 s"}) (defn master-nodes From 004d1162c4de415fd2b2c6d88c7724546f8e07c2 Mon Sep 17 00:00:00 2001 From: "Tzu-Li (Gordon) Tai" Date: Tue, 17 Jul 2018 18:00:40 +0800 Subject: [PATCH 60/65] [FLINK-9862] [test] Extend general puropose DataStream test to have a tumbling window This closes #6351. --- .../DataStreamAllroundTestJobFactory.java | 28 +++++++++++++++++- .../tests/DataStreamAllroundTestProgram.java | 21 ++++++++++++-- .../streaming/tests/SemanticsCheckMapper.java | 5 ++-- .../tests/SequenceGeneratorSource.java | 29 +++++++++++++++++-- flink-end-to-end-tests/run-nightly-tests.sh | 9 ++++-- .../test-scripts/test_resume_savepoint.sh | 6 ++++ 6 files changed, 87 insertions(+), 11 deletions(-) diff --git a/flink-end-to-end-tests/flink-datastream-allround-test/src/main/java/org/apache/flink/streaming/tests/DataStreamAllroundTestJobFactory.java b/flink-end-to-end-tests/flink-datastream-allround-test/src/main/java/org/apache/flink/streaming/tests/DataStreamAllroundTestJobFactory.java index 4710100e7e0f6..123807297efa5 100644 --- a/flink-end-to-end-tests/flink-datastream-allround-test/src/main/java/org/apache/flink/streaming/tests/DataStreamAllroundTestJobFactory.java +++ b/flink-end-to-end-tests/flink-datastream-allround-test/src/main/java/org/apache/flink/streaming/tests/DataStreamAllroundTestJobFactory.java @@ -30,11 +30,14 @@ import org.apache.flink.runtime.state.filesystem.FsStateBackend; import org.apache.flink.streaming.api.CheckpointingMode; import org.apache.flink.streaming.api.TimeCharacteristic; +import org.apache.flink.streaming.api.datastream.KeyedStream; +import org.apache.flink.streaming.api.datastream.WindowedStream; import org.apache.flink.streaming.api.environment.CheckpointConfig; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; import org.apache.flink.streaming.api.functions.source.SourceFunction; import org.apache.flink.streaming.api.functions.timestamps.BoundedOutOfOrdernessTimestampExtractor; import org.apache.flink.streaming.api.windowing.time.Time; +import org.apache.flink.streaming.api.windowing.windows.TimeWindow; import org.apache.flink.streaming.tests.artificialstate.ArtificalOperatorStateMapper; import org.apache.flink.streaming.tests.artificialstate.ArtificialKeyedStateMapper; import org.apache.flink.streaming.tests.artificialstate.builder.ArtificialListStateBuilder; @@ -79,6 +82,8 @@ *

  • sequence_generator_source.sleep_after_elements (long, default - 0): Number of elements to emit before sleeping in the sequence generator. Set to 0 to disable sleeping.
  • *
  • sequence_generator_source.event_time.max_out_of_order (long, default - 500): Max event time out-of-orderness for events emitted by the sequence generator.
  • *
  • sequence_generator_source.event_time.clock_progress (long, default - 100): The amount of event time to progress per event generated by the sequence generator.
  • + *
  • tumbling_window_operator.num_events (long, default - 20L): The duration of the window, indirectly determined by the target number of events in each window. + * Total duration is (sliding_window_operator.num_events) * (sequence_generator_source.event_time.clock_progress).
  • * */ class DataStreamAllroundTestJobFactory { @@ -184,12 +189,16 @@ class DataStreamAllroundTestJobFactory { private static final ConfigOption SEQUENCE_GENERATOR_SRC_EVENT_TIME_MAX_OUT_OF_ORDERNESS = ConfigOptions .key("sequence_generator_source.event_time.max_out_of_order") - .defaultValue(500L); + .defaultValue(0L); private static final ConfigOption SEQUENCE_GENERATOR_SRC_EVENT_TIME_CLOCK_PROGRESS = ConfigOptions .key("sequence_generator_source.event_time.clock_progress") .defaultValue(100L); + private static final ConfigOption TUMBLING_WINDOW_OPERATOR_NUM_EVENTS = ConfigOptions + .key("tumbling_window_operator.num_events") + .defaultValue(20L); + static void setupEnvironment(StreamExecutionEnvironment env, ParameterTool pt) throws Exception { // set checkpointing semantics @@ -318,6 +327,23 @@ public long extractTimestamp(Event element) { }; } + static WindowedStream applyTumblingWindows( + KeyedStream keyedStream, ParameterTool pt) { + + long eventTimeProgressPerEvent = pt.getLong( + SEQUENCE_GENERATOR_SRC_EVENT_TIME_CLOCK_PROGRESS.key(), + SEQUENCE_GENERATOR_SRC_EVENT_TIME_CLOCK_PROGRESS.defaultValue()); + + return keyedStream.timeWindow( + Time.milliseconds( + pt.getLong( + TUMBLING_WINDOW_OPERATOR_NUM_EVENTS.key(), + TUMBLING_WINDOW_OPERATOR_NUM_EVENTS.defaultValue() + ) * eventTimeProgressPerEvent + ) + ); + } + static FlatMapFunction createSemanticsCheckMapper(ParameterTool pt) { String semantics = pt.get(TEST_SEMANTICS.key(), TEST_SEMANTICS.defaultValue()); diff --git a/flink-end-to-end-tests/flink-datastream-allround-test/src/main/java/org/apache/flink/streaming/tests/DataStreamAllroundTestProgram.java b/flink-end-to-end-tests/flink-datastream-allround-test/src/main/java/org/apache/flink/streaming/tests/DataStreamAllroundTestProgram.java index afbc01ae26f90..ea90e6551038e 100644 --- a/flink-end-to-end-tests/flink-datastream-allround-test/src/main/java/org/apache/flink/streaming/tests/DataStreamAllroundTestProgram.java +++ b/flink-end-to-end-tests/flink-datastream-allround-test/src/main/java/org/apache/flink/streaming/tests/DataStreamAllroundTestProgram.java @@ -25,10 +25,14 @@ import org.apache.flink.streaming.api.datastream.DataStream; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; import org.apache.flink.streaming.api.functions.sink.PrintSinkFunction; +import org.apache.flink.streaming.api.functions.windowing.WindowFunction; +import org.apache.flink.streaming.api.windowing.windows.TimeWindow; import org.apache.flink.streaming.tests.artificialstate.ComplexPayload; +import org.apache.flink.util.Collector; import java.util.Collections; +import static org.apache.flink.streaming.tests.DataStreamAllroundTestJobFactory.applyTumblingWindows; import static org.apache.flink.streaming.tests.DataStreamAllroundTestJobFactory.createArtificialKeyedStateMapper; import static org.apache.flink.streaming.tests.DataStreamAllroundTestJobFactory.createArtificialOperatorStateMapper; import static org.apache.flink.streaming.tests.DataStreamAllroundTestJobFactory.createEventSource; @@ -57,6 +61,7 @@ public class DataStreamAllroundTestProgram { private static final String KEYED_STATE_OPER_NAME = "ArtificalKeyedStateMapper"; private static final String OPERATOR_STATE_OPER_NAME = "ArtificalOperatorStateMapper"; + private static final String TIME_WINDOW_OPER_NAME = "TumblingWindowOperator"; private static final String SEMANTICS_CHECK_MAPPER_NAME = "SemanticsCheckMapper"; private static final String FAILURE_MAPPER_NAME = "ExceptionThrowingFailureMapper"; @@ -92,14 +97,26 @@ public static void main(String[] args) throws Exception { .name(OPERATOR_STATE_OPER_NAME) .returns(Event.class); + // apply a tumbling window that simply passes forward window elements; + // this allows the job to cover timers state + DataStream eventStream3 = applyTumblingWindows(eventStream2.keyBy(Event::getKey), pt) + .apply(new WindowFunction() { + @Override + public void apply(Integer integer, TimeWindow window, Iterable input, Collector out) throws Exception { + for (Event e : input) { + out.collect(e); + } + } + }).name(TIME_WINDOW_OPER_NAME); + if (isSimulateFailures(pt)) { - eventStream2 = eventStream2 + eventStream3 = eventStream3 .map(createExceptionThrowingFailureMapper(pt)) .setParallelism(1) .name(FAILURE_MAPPER_NAME); } - eventStream2.keyBy(Event::getKey) + eventStream3.keyBy(Event::getKey) .flatMap(createSemanticsCheckMapper(pt)) .name(SEMANTICS_CHECK_MAPPER_NAME) .addSink(new PrintSinkFunction<>()); diff --git a/flink-end-to-end-tests/flink-datastream-allround-test/src/main/java/org/apache/flink/streaming/tests/SemanticsCheckMapper.java b/flink-end-to-end-tests/flink-datastream-allround-test/src/main/java/org/apache/flink/streaming/tests/SemanticsCheckMapper.java index 53c3414250490..b67e282334288 100644 --- a/flink-end-to-end-tests/flink-datastream-allround-test/src/main/java/org/apache/flink/streaming/tests/SemanticsCheckMapper.java +++ b/flink-end-to-end-tests/flink-datastream-allround-test/src/main/java/org/apache/flink/streaming/tests/SemanticsCheckMapper.java @@ -53,9 +53,8 @@ public void flatMap(Event event, Collector out) throws Exception { long nextValue = event.getSequenceNumber(); - if (validator.check(currentValue, nextValue)) { - sequenceValue.update(nextValue); - } else { + sequenceValue.update(nextValue); + if (!validator.check(currentValue, nextValue)) { out.collect("Alert: " + currentValue + " -> " + nextValue + " (" + event.getKey() + ")"); } } diff --git a/flink-end-to-end-tests/flink-datastream-allround-test/src/main/java/org/apache/flink/streaming/tests/SequenceGeneratorSource.java b/flink-end-to-end-tests/flink-datastream-allround-test/src/main/java/org/apache/flink/streaming/tests/SequenceGeneratorSource.java index 40c0db5d77692..c6ecb3f768099 100644 --- a/flink-end-to-end-tests/flink-datastream-allround-test/src/main/java/org/apache/flink/streaming/tests/SequenceGeneratorSource.java +++ b/flink-end-to-end-tests/flink-datastream-allround-test/src/main/java/org/apache/flink/streaming/tests/SequenceGeneratorSource.java @@ -63,12 +63,18 @@ public class SequenceGeneratorSource extends RichParallelSourceFunction i /** This determines after how many generated events we sleep. A value < 1 deactivates sleeping. */ private final long sleepAfterElements; + /** The current event time progress of this source; will start from 0. */ + private long monotonousEventTime; + /** This holds the key ranges for which this source generates events. */ private transient List keyRanges; /** This is used to snapshot the state of this source, one entry per key range. */ private transient ListState snapshotKeyRanges; + /** This is used to snapshot the event time progress of the sources. */ + private transient ListState lastEventTimes; + /** Flag that determines if this source is running, i.e. generating events. */ private volatile boolean running; @@ -102,7 +108,6 @@ private void runActive(SourceContext ctx) throws Exception { Random random = new Random(); // this holds the current event time, from which generated events can up to +/- (maxOutOfOrder). - long monotonousEventTime = 0L; long elementsBeforeSleep = sleepAfterElements; while (running) { @@ -162,7 +167,11 @@ private void runIdle(SourceContext ctx) throws Exception { } private long generateEventTimeWithOutOfOrderness(Random random, long correctTime) { - return correctTime - maxOutOfOrder + ((random.nextLong() & Long.MAX_VALUE) % (2 * maxOutOfOrder)); + if (maxOutOfOrder > 0) { + return correctTime - maxOutOfOrder + ((random.nextLong() & Long.MAX_VALUE) % (2 * maxOutOfOrder)); + } else { + return correctTime; + } } @Override @@ -173,6 +182,9 @@ public void cancel() { @Override public void snapshotState(FunctionSnapshotContext context) throws Exception { snapshotKeyRanges.update(keyRanges); + + lastEventTimes.clear(); + lastEventTimes.add(monotonousEventTime); } @Override @@ -182,6 +194,11 @@ public void initializeState(FunctionInitializationContext context) throws Except final int parallelism = runtimeContext.getNumberOfParallelSubtasks(); final int maxParallelism = runtimeContext.getMaxNumberOfParallelSubtasks(); + ListStateDescriptor unionWatermarksStateDescriptor = + new ListStateDescriptor<>("watermarks", Long.class); + + lastEventTimes = context.getOperatorStateStore().getUnionListState(unionWatermarksStateDescriptor); + ListStateDescriptor stateDescriptor = new ListStateDescriptor<>("keyRanges", KeyRangeStates.class); @@ -193,6 +210,11 @@ public void initializeState(FunctionInitializationContext context) throws Except for (KeyRangeStates keyRange : snapshotKeyRanges.get()) { keyRanges.add(keyRange); } + + // let event time start from the max of all event time progress across subtasks in the last execution + for (Long lastEventTime : lastEventTimes.get()) { + monotonousEventTime = Math.max(monotonousEventTime, lastEventTime); + } } else { // determine the key ranges that belong to the subtask int rangeStartIdx = (subtaskIdx * maxParallelism) / parallelism; @@ -207,6 +229,9 @@ public void initializeState(FunctionInitializationContext context) throws Except keyRanges.add(new KeyRangeStates(start, end)); } } + + // fresh run; start from event time o + monotonousEventTime = 0L; } } diff --git a/flink-end-to-end-tests/run-nightly-tests.sh b/flink-end-to-end-tests/run-nightly-tests.sh index dc8424f25eccf..c3acd99182dda 100755 --- a/flink-end-to-end-tests/run-nightly-tests.sh +++ b/flink-end-to-end-tests/run-nightly-tests.sh @@ -59,9 +59,12 @@ run_test "Resuming Savepoint (file, async, scale up) end-to-end test" "$END_TO_E run_test "Resuming Savepoint (file, sync, scale up) end-to-end test" "$END_TO_END_DIR/test-scripts/test_resume_savepoint.sh 2 4 file false" run_test "Resuming Savepoint (file, async, scale down) end-to-end test" "$END_TO_END_DIR/test-scripts/test_resume_savepoint.sh 4 2 file true" run_test "Resuming Savepoint (file, sync, scale down) end-to-end test" "$END_TO_END_DIR/test-scripts/test_resume_savepoint.sh 4 2 file false" -run_test "Resuming Savepoint (rocks, no parallelism change) end-to-end test" "$END_TO_END_DIR/test-scripts/test_resume_savepoint.sh 2 2 rocks" -run_test "Resuming Savepoint (rocks, scale up) end-to-end test" "$END_TO_END_DIR/test-scripts/test_resume_savepoint.sh 2 4 rocks" -run_test "Resuming Savepoint (rocks, scale down) end-to-end test" "$END_TO_END_DIR/test-scripts/test_resume_savepoint.sh 4 2 rocks" +run_test "Resuming Savepoint (rocks, no parallelism change, heap timers) end-to-end test" "$END_TO_END_DIR/test-scripts/test_resume_savepoint.sh 2 2 rocks false heap" +run_test "Resuming Savepoint (rocks, scale up, heap timers) end-to-end test" "$END_TO_END_DIR/test-scripts/test_resume_savepoint.sh 2 4 rocks false heap" +run_test "Resuming Savepoint (rocks, scale down, heap timers) end-to-end test" "$END_TO_END_DIR/test-scripts/test_resume_savepoint.sh 4 2 rocks false heap" +run_test "Resuming Savepoint (rocks, no parallelism change, rocks timers) end-to-end test" "$END_TO_END_DIR/test-scripts/test_resume_savepoint.sh 2 2 rocks false rocks" +run_test "Resuming Savepoint (rocks, scale up, rocks timers) end-to-end test" "$END_TO_END_DIR/test-scripts/test_resume_savepoint.sh 2 4 rocks false rocks" +run_test "Resuming Savepoint (rocks, scale down, rocks timers) end-to-end test" "$END_TO_END_DIR/test-scripts/test_resume_savepoint.sh 4 2 rocks false rocks" run_test "Resuming Externalized Checkpoint (file, async, no parallelism change) end-to-end test" "$END_TO_END_DIR/test-scripts/test_resume_externalized_checkpoints.sh 2 2 file true true" run_test "Resuming Externalized Checkpoint (file, sync, no parallelism change) end-to-end test" "$END_TO_END_DIR/test-scripts/test_resume_externalized_checkpoints.sh 2 2 file false true" diff --git a/flink-end-to-end-tests/test-scripts/test_resume_savepoint.sh b/flink-end-to-end-tests/test-scripts/test_resume_savepoint.sh index 1651ab04df873..fb8c51e312843 100755 --- a/flink-end-to-end-tests/test-scripts/test_resume_savepoint.sh +++ b/flink-end-to-end-tests/test-scripts/test_resume_savepoint.sh @@ -47,6 +47,7 @@ ORIGINAL_DOP=$1 NEW_DOP=$2 STATE_BACKEND_TYPE=${3:-file} STATE_BACKEND_FILE_ASYNC=${4:-true} +STATE_BACKEND_ROCKS_TIMER_SERVICE_TYPE=${5:-heap} if (( $ORIGINAL_DOP >= $NEW_DOP )); then NUM_SLOTS=$ORIGINAL_DOP @@ -56,6 +57,11 @@ fi backup_config change_conf "taskmanager.numberOfTaskSlots" "1" "${NUM_SLOTS}" + +if [ $STATE_BACKEND_ROCKS_TIMER_SERVICE_TYPE == 'rocks' ]; then + set_conf "state.backend.rocksdb.timer-service.impl" "rocksdb" +fi + setup_flink_slf4j_metric_reporter start_cluster From 02d8865ce7880a8fc4d98d38640222dddb155956 Mon Sep 17 00:00:00 2001 From: Timo Walther Date: Thu, 5 Jul 2018 13:25:28 +0200 Subject: [PATCH 61/65] [FLINK-9765] [sql-client] Improve CLI responsiveness when cluster is not reachable Moves the job cancellation into the final phase of the refresh thread in order to keep the CLI responsive. This closes #6265. --- .../flink/table/client/cli/CliResultView.java | 17 +++++++++-------- .../flink/table/client/gateway/Executor.java | 3 ++- 2 files changed, 11 insertions(+), 9 deletions(-) diff --git a/flink-libraries/flink-sql-client/src/main/java/org/apache/flink/table/client/cli/CliResultView.java b/flink-libraries/flink-sql-client/src/main/java/org/apache/flink/table/client/cli/CliResultView.java index 9f893bb1f58ca..df42edd63b3a2 100644 --- a/flink-libraries/flink-sql-client/src/main/java/org/apache/flink/table/client/cli/CliResultView.java +++ b/flink-libraries/flink-sql-client/src/main/java/org/apache/flink/table/client/cli/CliResultView.java @@ -217,15 +217,7 @@ protected List computeMainLines() { @Override protected void cleanUp() { - // stop retrieval stopRetrieval(); - - // cancel table program - try { - client.getExecutor().cancelQuery(client.getContext(), resultDescriptor.getResultId()); - } catch (SqlExecutionException e) { - // ignore further exceptions - } } // -------------------------------------------------------------------------------------------- @@ -285,6 +277,15 @@ public void run() { display(); } } + + // cancel table program + try { + // the cancellation happens in the refresh thread in order to keep the main thread + // responsive at all times; esp. if the cluster is not available + client.getExecutor().cancelQuery(client.getContext(), resultDescriptor.getResultId()); + } catch (SqlExecutionException e) { + // ignore further exceptions + } } } } diff --git a/flink-libraries/flink-sql-client/src/main/java/org/apache/flink/table/client/gateway/Executor.java b/flink-libraries/flink-sql-client/src/main/java/org/apache/flink/table/client/gateway/Executor.java index 7f903a4657ded..3a4dd81bfb817 100644 --- a/flink-libraries/flink-sql-client/src/main/java/org/apache/flink/table/client/gateway/Executor.java +++ b/flink-libraries/flink-sql-client/src/main/java/org/apache/flink/table/client/gateway/Executor.java @@ -83,7 +83,8 @@ public interface Executor { List retrieveResultPage(String resultId, int page) throws SqlExecutionException; /** - * Cancels a table program and stops the result retrieval. + * Cancels a table program and stops the result retrieval. Blocking until cancellation command has + * been sent to cluster. */ void cancelQuery(SessionContext session, String resultId) throws SqlExecutionException; From c6a3efef15a1ad9ebda7496481bb3a0967be7235 Mon Sep 17 00:00:00 2001 From: Alexander Koltsov Date: Tue, 10 Jul 2018 16:45:12 +0300 Subject: [PATCH 62/65] [FLINK-5750] [table] Fix incorrect translation of n-ary Union. In certain cases, Calcite produces union operators with more than two input relations. However, Flink's translation rules only considered the first two relations This closes #6341. --- .../table/calcite/FlinkPlannerImpl.scala | 2 - .../plan/nodes/dataset/DataSetUnion.scala | 34 +++++++++------- .../nodes/datastream/DataStreamUnion.scala | 34 +++++++++------- .../plan/rules/dataSet/DataSetUnionRule.scala | 13 +++++-- .../datastream/DataStreamUnionRule.scala | 13 +++++-- .../flink/table/api/ExternalCatalogTest.scala | 5 +++ .../flink/table/api/TableSourceTest.scala | 1 + .../api/batch/sql/GroupingSetsTest.scala | 6 +++ .../api/batch/sql/SetOperatorsTest.scala | 39 ++++++++++++++++++- .../api/batch/table/SetOperatorsTest.scala | 4 ++ .../stream/StreamTableEnvironmentTest.scala | 1 + .../api/stream/sql/SetOperatorsTest.scala | 33 ++++++++++++++++ .../table/api/stream/sql/UnionTest.scala | 2 + .../api/stream/table/SetOperatorsTest.scala | 2 + .../plan/TimeIndicatorConversionTest.scala | 1 + .../batch/sql/SetOperatorsITCase.scala | 16 ++++++++ .../flink/table/utils/TableTestBase.scala | 7 ++++ .../src/test/scala/resources/testUnion0.out | 2 +- .../src/test/scala/resources/testUnion1.out | 2 +- .../test/scala/resources/testUnionStream0.out | 2 +- 20 files changed, 176 insertions(+), 43 deletions(-) diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/calcite/FlinkPlannerImpl.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/calcite/FlinkPlannerImpl.scala index d5730863deb30..de5b8993c28c7 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/calcite/FlinkPlannerImpl.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/calcite/FlinkPlannerImpl.scala @@ -19,10 +19,8 @@ package org.apache.flink.table.calcite import java.util -import java.util.Properties import com.google.common.collect.ImmutableList -import org.apache.calcite.config.{CalciteConnectionConfig, CalciteConnectionConfigImpl, CalciteConnectionProperty} import org.apache.calcite.jdbc.CalciteSchema import org.apache.calcite.plan.RelOptTable.ViewExpander import org.apache.calcite.plan._ diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/dataset/DataSetUnion.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/dataset/DataSetUnion.scala index 1cdd198d627f3..bb94f2819a0ee 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/dataset/DataSetUnion.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/dataset/DataSetUnion.scala @@ -18,12 +18,15 @@ package org.apache.flink.table.plan.nodes.dataset +import java.util.{List => JList} + import org.apache.calcite.plan.{RelOptCluster, RelOptCost, RelOptPlanner, RelTraitSet} import org.apache.calcite.rel.`type`.RelDataType +import org.apache.calcite.rel.core.{SetOp, Union} import org.apache.calcite.rel.metadata.RelMetadataQuery -import org.apache.calcite.rel.{BiRel, RelNode, RelWriter} +import org.apache.calcite.rel.{RelNode, RelWriter} import org.apache.flink.api.java.DataSet -import org.apache.flink.table.api.{BatchQueryConfig, BatchTableEnvironment} +import org.apache.flink.table.api.{BatchQueryConfig, BatchTableEnvironment, TableException} import org.apache.flink.types.Row import scala.collection.JavaConversions._ @@ -36,22 +39,24 @@ import scala.collection.JavaConverters._ class DataSetUnion( cluster: RelOptCluster, traitSet: RelTraitSet, - leftNode: RelNode, - rightNode: RelNode, + inputs: JList[RelNode], rowRelDataType: RelDataType) - extends BiRel(cluster, traitSet, leftNode, rightNode) + extends Union(cluster, traitSet, inputs, true) with DataSetRel { override def deriveRowType() = rowRelDataType - override def copy(traitSet: RelTraitSet, inputs: java.util.List[RelNode]): RelNode = { + override def copy(traitSet: RelTraitSet, inputs: JList[RelNode], all: Boolean): SetOp = { + + if (!all) { + throw new TableException("DataSetUnion only supports UNION ALL.") + } + new DataSetUnion( cluster, traitSet, - inputs.get(0), - inputs.get(1), - rowRelDataType - ) + inputs, + rowRelDataType) } override def toString: String = { @@ -81,14 +86,13 @@ class DataSetUnion( tableEnv: BatchTableEnvironment, queryConfig: BatchQueryConfig): DataSet[Row] = { - val leftDataSet = left.asInstanceOf[DataSetRel].translateToPlan(tableEnv, queryConfig) - val rightDataSet = right.asInstanceOf[DataSetRel].translateToPlan(tableEnv, queryConfig) - - leftDataSet.union(rightDataSet) + getInputs + .asScala + .map(_.asInstanceOf[DataSetRel].translateToPlan(tableEnv, queryConfig)) + .reduce((dataSetLeft, dataSetRight) => dataSetLeft.union(dataSetRight)) } private def unionSelectionToString: String = { rowRelDataType.getFieldNames.asScala.toList.mkString(", ") } - } diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamUnion.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamUnion.scala index 7258ec88e4b93..3355971ee4859 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamUnion.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamUnion.scala @@ -18,13 +18,18 @@ 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.{BiRel, RelNode, RelWriter} +import org.apache.calcite.rel.core.{SetOp, Union} +import org.apache.calcite.rel.{RelNode, RelWriter} import org.apache.flink.streaming.api.datastream.DataStream -import org.apache.flink.table.api.{StreamQueryConfig, StreamTableEnvironment} +import org.apache.flink.table.api.{StreamQueryConfig, StreamTableEnvironment, TableException} import org.apache.flink.table.plan.schema.RowSchema import org.apache.flink.table.runtime.types.CRow +import scala.collection.JavaConverters._ + /** * Flink RelNode which matches along with Union. * @@ -32,22 +37,24 @@ import org.apache.flink.table.runtime.types.CRow class DataStreamUnion( cluster: RelOptCluster, traitSet: RelTraitSet, - leftNode: RelNode, - rightNode: RelNode, + inputs: JList[RelNode], schema: RowSchema) - extends BiRel(cluster, traitSet, leftNode, rightNode) + extends Union(cluster, traitSet, inputs, true) with DataStreamRel { override def deriveRowType() = schema.relDataType - override def copy(traitSet: RelTraitSet, inputs: java.util.List[RelNode]): RelNode = { + override def copy(traitSet: RelTraitSet, inputs: JList[RelNode], all: Boolean): SetOp = { + + if (!all) { + throw new TableException("DataStreamUnion only supports UNION ALL.") + } + new DataStreamUnion( cluster, traitSet, - inputs.get(0), - inputs.get(1), - schema - ) + inputs, + schema) } override def explainTerms(pw: RelWriter): RelWriter = { @@ -62,9 +69,10 @@ class DataStreamUnion( tableEnv: StreamTableEnvironment, queryConfig: StreamQueryConfig): DataStream[CRow] = { - val leftDataSet = left.asInstanceOf[DataStreamRel].translateToPlan(tableEnv, queryConfig) - val rightDataSet = right.asInstanceOf[DataStreamRel].translateToPlan(tableEnv, queryConfig) - leftDataSet.union(rightDataSet) + getInputs + .asScala + .map(_.asInstanceOf[DataStreamRel].translateToPlan(tableEnv, queryConfig)) + .reduce((dataSetLeft, dataSetRight) => dataSetLeft.union(dataSetRight)) } private def unionSelectionToString: String = { diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/dataSet/DataSetUnionRule.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/dataSet/DataSetUnionRule.scala index d4bbb6b176b4d..137e59d2f83df 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/dataSet/DataSetUnionRule.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/dataSet/DataSetUnionRule.scala @@ -26,6 +26,8 @@ import org.apache.flink.table.plan.nodes.FlinkConventions import org.apache.flink.table.plan.nodes.dataset.DataSetUnion import org.apache.flink.table.plan.nodes.logical.FlinkLogicalUnion +import scala.collection.JavaConverters._ + class DataSetUnionRule extends ConverterRule( classOf[FlinkLogicalUnion], @@ -46,14 +48,17 @@ class DataSetUnionRule def convert(rel: RelNode): RelNode = { val union: FlinkLogicalUnion = rel.asInstanceOf[FlinkLogicalUnion] val traitSet: RelTraitSet = rel.getTraitSet.replace(FlinkConventions.DATASET) - val convLeft: RelNode = RelOptRule.convert(union.getInput(0), FlinkConventions.DATASET) - val convRight: RelNode = RelOptRule.convert(union.getInput(1), FlinkConventions.DATASET) + + val newInputs = union + .getInputs + .asScala + .map(RelOptRule.convert(_, FlinkConventions.DATASET)) + .asJava new DataSetUnion( rel.getCluster, traitSet, - convLeft, - convRight, + newInputs, rel.getRowType) } } diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/datastream/DataStreamUnionRule.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/datastream/DataStreamUnionRule.scala index 8402f6df0351a..62236529b62d8 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/datastream/DataStreamUnionRule.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/datastream/DataStreamUnionRule.scala @@ -26,6 +26,8 @@ import org.apache.flink.table.plan.nodes.datastream.DataStreamUnion import org.apache.flink.table.plan.nodes.logical.FlinkLogicalUnion import org.apache.flink.table.plan.schema.RowSchema +import scala.collection.JavaConverters._ + class DataStreamUnionRule extends ConverterRule( classOf[FlinkLogicalUnion], @@ -37,14 +39,17 @@ class DataStreamUnionRule def convert(rel: RelNode): RelNode = { val union: FlinkLogicalUnion = rel.asInstanceOf[FlinkLogicalUnion] val traitSet: RelTraitSet = rel.getTraitSet.replace(FlinkConventions.DATASTREAM) - val convLeft: RelNode = RelOptRule.convert(union.getInput(0), FlinkConventions.DATASTREAM) - val convRight: RelNode = RelOptRule.convert(union.getInput(1), FlinkConventions.DATASTREAM) + + val newInputs = union + .getInputs + .asScala + .map(RelOptRule.convert(_, FlinkConventions.DATASTREAM)) + .asJava new DataStreamUnion( rel.getCluster, traitSet, - convLeft, - convRight, + newInputs, new RowSchema(rel.getRowType)) } } diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/ExternalCatalogTest.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/ExternalCatalogTest.scala index 7f567f9d2d749..17f51f3cdd8d1 100644 --- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/ExternalCatalogTest.scala +++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/ExternalCatalogTest.scala @@ -61,6 +61,7 @@ class ExternalCatalogTest extends TableTestBase { sourceBatchTableNode(table1Path, table1ProjectedFields), term("select", "*(a, 2) AS _c0", "b", "UPPER(c) AS _c2") ), + term("all", "true"), term("union", "_c0", "e", "_c2") ) @@ -90,6 +91,7 @@ class ExternalCatalogTest extends TableTestBase { sourceBatchTableNode(table1Path, table1ProjectedFields), term("select", "*(a, 2) AS EXPR$0", "b", "c") ), + term("all", "true"), term("union", "EXPR$0", "e", "g")) util.verifySql(sqlQuery, expected) @@ -124,6 +126,7 @@ class ExternalCatalogTest extends TableTestBase { sourceStreamTableNode(table1Path, table1ProjectedFields), term("select", "*(a, 2) AS _c0", "b", "UPPER(c) AS _c2") ), + term("all", "true"), term("union all", "_c0", "e", "_c2") ) @@ -153,6 +156,7 @@ class ExternalCatalogTest extends TableTestBase { sourceStreamTableNode(table1Path, table1ProjectedFields), term("select", "*(a, 2) AS EXPR$0", "b", "c") ), + term("all", "true"), term("union all", "EXPR$0", "e", "g")) util.verifySql(sqlQuery, expected) @@ -185,6 +189,7 @@ class ExternalCatalogTest extends TableTestBase { sourceBatchTableNode(table1TopLevelPath, table1ProjectedFields), term("select", "*(a, 2) AS _c0", "b", "UPPER(c) AS _c2") ), + term("all", "true"), term("union", "_c0", "e", "_c2") ) diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/TableSourceTest.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/TableSourceTest.scala index 050f1a12476a6..ee2e749c2461c 100644 --- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/TableSourceTest.scala +++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/TableSourceTest.scala @@ -60,6 +60,7 @@ class TableSourceTest extends TableTestBase { "table2", Array("name", "id", "amount", "price"), "'amount > 2"), + term("all", "true"), term("union", "name, id, amount, price") ) util.verifyTable(result, expected) diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/batch/sql/GroupingSetsTest.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/batch/sql/GroupingSetsTest.scala index 57a4c5a03f0cd..0bc88c3ef21fb 100644 --- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/batch/sql/GroupingSetsTest.scala +++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/batch/sql/GroupingSetsTest.scala @@ -56,6 +56,7 @@ class GroupingSetsTest extends TableTestBase { ), term("select", "null AS b", "c", "a", "2 AS g") ), + term("all", "true"), term("union", "b", "c", "a", "g") ) @@ -130,12 +131,15 @@ class GroupingSetsTest extends TableTestBase { "DataSetUnion", group1, group2, + term("all", "true"), term("union", "b", "c", "a", "g", "gb", "gc", "gib", "gic", "gid") ), group3, + term("all", "true"), term("union", "b", "c", "a", "g", "gb", "gc", "gib", "gic", "gid") ), group4, + term("all", "true"), term("union", "b", "c", "a", "g", "gb", "gc", "gib", "gic", "gid") ) @@ -195,9 +199,11 @@ class GroupingSetsTest extends TableTestBase { "DataSetUnion", group1, group2, + term("all", "true"), term("union", "b", "c", "a", "g", "gb", "gc", "gib", "gic", "gid") ), group3, + term("all", "true"), term("union", "b", "c", "a", "g", "gb", "gc", "gib", "gic", "gid") ) diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/batch/sql/SetOperatorsTest.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/batch/sql/SetOperatorsTest.scala index e6f4a462b589f..a6e47caa6c7a4 100644 --- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/batch/sql/SetOperatorsTest.scala +++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/batch/sql/SetOperatorsTest.scala @@ -18,13 +18,13 @@ package org.apache.flink.table.api.batch.sql -import org.apache.flink.api.java.typeutils.{GenericTypeInfo, RowTypeInfo} +import org.apache.flink.api.java.typeutils.GenericTypeInfo import org.apache.flink.api.scala._ import org.apache.flink.table.api.Types import org.apache.flink.table.api.scala._ import org.apache.flink.table.runtime.utils.CommonTestData.NonPojo -import org.apache.flink.table.utils.TableTestUtil._ import org.apache.flink.table.utils.TableTestBase +import org.apache.flink.table.utils.TableTestUtil._ import org.junit.{Ignore, Test} class SetOperatorsTest extends TableTestBase { @@ -190,6 +190,7 @@ class SetOperatorsTest extends TableTestBase { batchTableNode(0), term("select", "CASE(>(c, 0), b, null) AS EXPR$0") ), + term("all", "true"), term("union", "a") ) @@ -219,9 +220,43 @@ class SetOperatorsTest extends TableTestBase { batchTableNode(0), term("select", "b") ), + term("all", "true"), term("union", "a") ) util.verifyJavaSql("SELECT a FROM A UNION ALL SELECT b FROM A", expected) } + + @Test + def testValuesWithCast(): Unit = { + val util = batchTestUtil() + + val expected = naryNode( + "DataSetUnion", + List( + unaryNode("DataSetCalc", + values("DataSetValues", + tuples(List("0")), + "values=[ZERO]"), + term("select", "1 AS EXPR$0, 1 AS EXPR$1")), + unaryNode("DataSetCalc", + values("DataSetValues", + tuples(List("0")), + "values=[ZERO]"), + term("select", "2 AS EXPR$0, 2 AS EXPR$1")), + unaryNode("DataSetCalc", + values("DataSetValues", + tuples(List("0")), + "values=[ZERO]"), + term("select", "3 AS EXPR$0, 3 AS EXPR$1")) + ), + term("all", "true"), + term("union", "EXPR$0, EXPR$1") + ) + + util.verifySql( + "VALUES (1, cast(1 as BIGINT) ),(2, cast(2 as BIGINT)),(3, cast(3 as BIGINT))", + expected + ) + } } diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/batch/table/SetOperatorsTest.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/batch/table/SetOperatorsTest.scala index 929ce9c656d9e..9226200f68c29 100644 --- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/batch/table/SetOperatorsTest.scala +++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/batch/table/SetOperatorsTest.scala @@ -102,6 +102,7 @@ class SetOperatorsTest extends TableTestBase { batchTableNode(0), term("select", "CASE(>(c, 0), b, null) AS _c0") ), + term("all", "true"), term("union", "a") ) @@ -130,6 +131,7 @@ class SetOperatorsTest extends TableTestBase { batchTableNode(0), term("select", "b") ), + term("all", "true"), term("union", "a") ) @@ -165,6 +167,7 @@ class SetOperatorsTest extends TableTestBase { term("select", "a", "b", "c"), term("where", ">(a, 0)") ), + term("all", "true"), term("union", "a", "b", "c") ), term("groupBy", "b"), @@ -238,6 +241,7 @@ class SetOperatorsTest extends TableTestBase { batchTableNode(1), term("select", "b", "c") ), + term("all", "true"), term("union", "b", "c") ) diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/stream/StreamTableEnvironmentTest.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/stream/StreamTableEnvironmentTest.scala index 863d07b4c4938..30116cd884dba 100644 --- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/stream/StreamTableEnvironmentTest.scala +++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/stream/StreamTableEnvironmentTest.scala @@ -60,6 +60,7 @@ class StreamTableEnvironmentTest extends TableTestBase { "DataStreamUnion", streamTableNode(1), streamTableNode(0), + term("all", "true"), term("union all", "d, e, f")) util.verifyTable(sqlTable2, expected2) diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/stream/sql/SetOperatorsTest.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/stream/sql/SetOperatorsTest.scala index 266e3ffce6e3a..97dbe0dad6674 100644 --- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/stream/sql/SetOperatorsTest.scala +++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/stream/sql/SetOperatorsTest.scala @@ -170,4 +170,37 @@ class SetOperatorsTest extends TableTestBase { streamUtil.verifySql(sqlQuery, expected) } + + @Test + def testValuesWithCast(): Unit = { + val util = batchTestUtil() + + val expected = naryNode( + "DataSetUnion", + List( + unaryNode("DataSetCalc", + values("DataSetValues", + tuples(List("0")), + "values=[ZERO]"), + term("select", "1 AS EXPR$0, 1 AS EXPR$1")), + unaryNode("DataSetCalc", + values("DataSetValues", + tuples(List("0")), + "values=[ZERO]"), + term("select", "2 AS EXPR$0, 2 AS EXPR$1")), + unaryNode("DataSetCalc", + values("DataSetValues", + tuples(List("0")), + "values=[ZERO]"), + term("select", "3 AS EXPR$0, 3 AS EXPR$1")) + ), + term("all", "true"), + term("union", "EXPR$0, EXPR$1") + ) + + util.verifySql( + "VALUES (1, cast(1 as BIGINT) ),(2, cast(2 as BIGINT)),(3, cast(3 as BIGINT))", + expected + ) + } } diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/stream/sql/UnionTest.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/stream/sql/UnionTest.scala index 7e807f6d73a0c..b8bd2223f621b 100644 --- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/stream/sql/UnionTest.scala +++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/stream/sql/UnionTest.scala @@ -46,6 +46,7 @@ class UnionTest extends TableTestBase { streamTableNode(0), term("select", "CASE(>(c, 0), b, null) AS EXPR$0") ), + term("all", "true"), term("union all", "a") ) @@ -75,6 +76,7 @@ class UnionTest extends TableTestBase { streamTableNode(0), term("select", "b") ), + term("all", "true"), term("union all", "a") ) diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/stream/table/SetOperatorsTest.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/stream/table/SetOperatorsTest.scala index e84c630f59df8..dfbaf40d524f8 100644 --- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/stream/table/SetOperatorsTest.scala +++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/stream/table/SetOperatorsTest.scala @@ -55,6 +55,7 @@ class SetOperatorsTest extends TableTestBase { term("select", "a", "b", "c"), term("where", ">(a, 0)") ), + term("all", "true"), term("union all", "a", "b", "c") ), term("groupBy", "b"), @@ -88,6 +89,7 @@ class SetOperatorsTest extends TableTestBase { streamTableNode(1), term("select", "b", "c") ), + term("all", "true"), term("union all", "b", "c") ) diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/plan/TimeIndicatorConversionTest.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/plan/TimeIndicatorConversionTest.scala index faca7f995bd51..6a77f125ea5b3 100644 --- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/plan/TimeIndicatorConversionTest.scala +++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/plan/TimeIndicatorConversionTest.scala @@ -221,6 +221,7 @@ class TimeIndicatorConversionTest extends TableTestBase { streamTableNode(0), term("select", "rowtime") ), + term("all", "true"), term("union all", "rowtime") ) diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/batch/sql/SetOperatorsITCase.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/batch/sql/SetOperatorsITCase.scala index d965e0cc09a14..396310f36dc29 100644 --- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/batch/sql/SetOperatorsITCase.scala +++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/batch/sql/SetOperatorsITCase.scala @@ -122,6 +122,22 @@ class SetOperatorsITCase( TestBaseUtils.compareResultAsText(results.asJava, expected) } + @Test + def testValuesWithCast(): Unit = { + val env = ExecutionEnvironment.getExecutionEnvironment + val tEnv = TableEnvironment.getTableEnvironment(env, config) + + val sqlQuery = "VALUES (1, cast(1 as BIGINT) )," + + "(2, cast(2 as BIGINT))," + + "(3, cast(3 as BIGINT))" + + val result = tEnv.sqlQuery(sqlQuery) + val results = result.toDataSet[Row].collect() + + val expected = "1,1\n2,2\n3,3" + TestBaseUtils.compareResultAsText(results.asJava, expected) + } + @Test def testExcept(): Unit = { diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/utils/TableTestBase.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/utils/TableTestBase.scala index f414dd4f96b5b..5e1aabe8a9790 100644 --- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/utils/TableTestBase.scala +++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/utils/TableTestBase.scala @@ -122,6 +122,13 @@ object TableTestUtil { |""".stripMargin.stripLineEnd } + def naryNode(node: String, inputs: List[AnyRef], term: String*): String = { + val strInputs = inputs.mkString("\n") + s"""$node(${term.mkString(", ")}) + |$strInputs + |""".stripMargin.stripLineEnd + } + def values(node: String, term: String*): String = { s"$node(${term.mkString(", ")})" } diff --git a/flink-libraries/flink-table/src/test/scala/resources/testUnion0.out b/flink-libraries/flink-table/src/test/scala/resources/testUnion0.out index 5fbd1b559bf43..b4d12b61292ae 100644 --- a/flink-libraries/flink-table/src/test/scala/resources/testUnion0.out +++ b/flink-libraries/flink-table/src/test/scala/resources/testUnion0.out @@ -4,7 +4,7 @@ LogicalUnion(all=[true]) LogicalTableScan(table=[[_DataSetTable_1]]) == Optimized Logical Plan == -DataSetUnion(union=[count, word]) +DataSetUnion(all=[true], union=[count, word]) DataSetScan(table=[[_DataSetTable_0]]) DataSetScan(table=[[_DataSetTable_1]]) diff --git a/flink-libraries/flink-table/src/test/scala/resources/testUnion1.out b/flink-libraries/flink-table/src/test/scala/resources/testUnion1.out index d7d343beb0aa2..4822772b279db 100644 --- a/flink-libraries/flink-table/src/test/scala/resources/testUnion1.out +++ b/flink-libraries/flink-table/src/test/scala/resources/testUnion1.out @@ -4,7 +4,7 @@ LogicalUnion(all=[true]) LogicalTableScan(table=[[_DataSetTable_1]]) == Optimized Logical Plan == -DataSetUnion(union=[count, word]) +DataSetUnion(all=[true], union=[count, word]) DataSetScan(table=[[_DataSetTable_0]]) DataSetScan(table=[[_DataSetTable_1]]) diff --git a/flink-libraries/flink-table/src/test/scala/resources/testUnionStream0.out b/flink-libraries/flink-table/src/test/scala/resources/testUnionStream0.out index 2d19bdce65d3e..af6c691552569 100644 --- a/flink-libraries/flink-table/src/test/scala/resources/testUnionStream0.out +++ b/flink-libraries/flink-table/src/test/scala/resources/testUnionStream0.out @@ -4,7 +4,7 @@ LogicalUnion(all=[true]) LogicalTableScan(table=[[_DataStreamTable_1]]) == Optimized Logical Plan == -DataStreamUnion(union all=[count, word]) +DataStreamUnion(all=[true], union all=[count, word]) DataStreamScan(table=[[_DataStreamTable_0]]) DataStreamScan(table=[[_DataStreamTable_1]]) From 8450dab8b586b8b4aaf06e09a3842ab2a972cd18 Mon Sep 17 00:00:00 2001 From: Fabian Hueske Date: Mon, 23 Jul 2018 16:02:58 +0200 Subject: [PATCH 63/65] [FLINK-9296] [table] Add support for non-windowed DISTINCT aggregates. This closes #6393. --- docs/dev/table/sql.md | 1 - .../DataStreamGroupAggregateRule.scala | 8 +----- .../stream/sql/DistinctAggregateTest.scala | 22 ++++++++++++++++ .../table/runtime/stream/sql/SqlITCase.scala | 26 +++++++++++++++++++ 4 files changed, 49 insertions(+), 8 deletions(-) diff --git a/docs/dev/table/sql.md b/docs/dev/table/sql.md index 57e0ba5b781e4..b968c43788dd7 100644 --- a/docs/dev/table/sql.md +++ b/docs/dev/table/sql.md @@ -2645,7 +2645,6 @@ The following functions are not supported yet: - Binary string operators and functions - System functions -- Distinct aggregate functions like COUNT DISTINCT {% top %} diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/datastream/DataStreamGroupAggregateRule.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/datastream/DataStreamGroupAggregateRule.scala index 0b8e411b0f076..e99a86ed3ff5a 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/datastream/DataStreamGroupAggregateRule.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/datastream/DataStreamGroupAggregateRule.scala @@ -43,19 +43,13 @@ class DataStreamGroupAggregateRule override def matches(call: RelOptRuleCall): Boolean = { val agg: FlinkLogicalAggregate = call.rel(0).asInstanceOf[FlinkLogicalAggregate] - // check if we have distinct aggregates - val distinctAggs = agg.getAggCallList.exists(_.isDistinct) - if (distinctAggs) { - throw TableException("DISTINCT aggregates are currently not supported.") - } - // check if we have grouping sets val groupSets = agg.getGroupSets.size() != 1 || agg.getGroupSets.get(0) != agg.getGroupSet if (groupSets || agg.indicator) { throw TableException("GROUPING SETS are currently not supported.") } - !distinctAggs && !groupSets && !agg.indicator + !groupSets && !agg.indicator } override def convert(rel: RelNode): RelNode = { diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/stream/sql/DistinctAggregateTest.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/stream/sql/DistinctAggregateTest.scala index 1ce63c69c954c..95aa3ea4db57d 100644 --- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/stream/sql/DistinctAggregateTest.scala +++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/stream/sql/DistinctAggregateTest.scala @@ -71,6 +71,28 @@ class DistinctAggregateTest extends TableTestBase { streamUtil.verifySql(sql, expected) } + @Test + def testDistinctAggregate(): Unit = { + val sqlQuery = "SELECT " + + " c, SUM(DISTINCT a), SUM(a), COUNT(DISTINCT b) " + + "FROM MyTable " + + "GROUP BY c " + + val expected = + unaryNode( + "DataStreamGroupAggregate", + unaryNode( + "DataStreamCalc", + streamTableNode(0), + term("select", "c", "a", "b") + ), + term("groupBy", "c"), + term("select", "c", + "SUM(DISTINCT a) AS EXPR$1", "SUM(a) AS EXPR$2", "COUNT(DISTINCT b) AS EXPR$3") + ) + streamUtil.verifySql(sqlQuery, expected) + } + @Test def testDistinctAggregateOnTumbleWindow(): Unit = { val sqlQuery = "SELECT COUNT(DISTINCT a), " + diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/stream/sql/SqlITCase.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/stream/sql/SqlITCase.scala index 15b8699aeed86..a2d9bb26c4cc1 100644 --- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/stream/sql/SqlITCase.scala +++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/stream/sql/SqlITCase.scala @@ -238,6 +238,32 @@ class SqlITCase extends StreamingWithStateTestBase { assertEquals(expected.sorted, StreamITCase.retractedResults.sorted) } + @Test + def testDistinctGroupBy(): Unit = { + + val env = StreamExecutionEnvironment.getExecutionEnvironment + val tEnv = TableEnvironment.getTableEnvironment(env) + StreamITCase.clear + + val sqlQuery = + "SELECT b, " + + " SUM(DISTINCT (a / 3)), " + + " COUNT(DISTINCT SUBSTRING(c FROM 1 FOR 2))," + + " COUNT(DISTINCT c) " + + "FROM MyTable " + + "GROUP BY b" + + val t = StreamTestData.get3TupleDataStream(env).toTable(tEnv).as('a, 'b, 'c) + tEnv.registerTable("MyTable", t) + + val result = tEnv.sqlQuery(sqlQuery).toRetractStream[Row] + result.addSink(new StreamITCase.RetractingSink).setParallelism(1) + env.execute() + + val expected = List("1,0,1,1", "2,1,1,2", "3,3,3,3", "4,5,1,4", "5,12,1,5", "6,18,1,6") + assertEquals(expected.sorted, StreamITCase.retractedResults.sorted) + } + @Test def testUnboundedGroupByCollect(): Unit = { From ae80215c866678450b1a4c5d904b8b8e1b86df21 Mon Sep 17 00:00:00 2001 From: Timo Walther Date: Tue, 24 Jul 2018 14:38:39 +0200 Subject: [PATCH 64/65] [FLINK-9296] [table] Add documentation for DISTINCT aggregates --- docs/dev/table/sql.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/dev/table/sql.md b/docs/dev/table/sql.md index b968c43788dd7..5b83e9d23cbe2 100644 --- a/docs/dev/table/sql.md +++ b/docs/dev/table/sql.md @@ -2222,7 +2222,7 @@ COUNT(value [, value]* ) {% endhighlight %} -

    Returns the number of input rows for which value is not null.

    +

    Returns the number of input rows for which value is not null. Use COUNT(DISTINCT value) for the number of unique values in the column or expression.

    @@ -2255,7 +2255,7 @@ SUM(numeric) {% endhighlight %} -

    Returns the sum of numeric across all input values.

    +

    Returns the sum of numeric across all input values. Use SUM(DISTINCT value) for the sum of unique values in the column or expression.

    From d180d599a94456ef286c74dd97425b5495a69a3f Mon Sep 17 00:00:00 2001 From: Till Rohrmann Date: Mon, 23 Jul 2018 22:25:25 +0200 Subject: [PATCH 65/65] [FLINK-9923][tests] Harden OneInputStreamTaskTest#testWatermarkMetrics Make WatermarkGauge's currentWatermark field volatile because writes and reads can happen from different threads. This closes #6398. --- .../flink/streaming/runtime/metrics/WatermarkGauge.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/metrics/WatermarkGauge.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/metrics/WatermarkGauge.java index 42c70008bdb6b..462a7b4cbfa1c 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/metrics/WatermarkGauge.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/metrics/WatermarkGauge.java @@ -25,10 +25,10 @@ */ public class WatermarkGauge implements Gauge { - private long currentWatermark = Long.MIN_VALUE; + private volatile long currentWatermark = Long.MIN_VALUE; public void setCurrentWatermark(long watermark) { - this.currentWatermark = watermark; + currentWatermark = watermark; } @Override