outputFormat, Schema schema) throws IOException {
+ outputFormat.configure(new Configuration());
+ outputFormat.open(1, 1);
+ for (int i = 0; i < 100; i++) {
+ GenericRecord record = new GenericData.Record(schema);
+ record.put("user_name", "testUser");
+ record.put("favorite_number", 1);
+ record.put("favorite_color", "blue");
+ outputFormat.writeRecord(record);
+ }
+ outputFormat.close();
+ }
}
diff --git a/flink-connectors/flink-connector-cassandra/src/test/java/org/apache/flink/batch/connectors/cassandra/example/BatchExample.java b/flink-connectors/flink-connector-cassandra/src/test/java/org/apache/flink/batch/connectors/cassandra/example/BatchExample.java
index af21f2d4a5eab..20a020802630a 100644
--- a/flink-connectors/flink-connector-cassandra/src/test/java/org/apache/flink/batch/connectors/cassandra/example/BatchExample.java
+++ b/flink-connectors/flink-connector-cassandra/src/test/java/org/apache/flink/batch/connectors/cassandra/example/BatchExample.java
@@ -34,8 +34,9 @@
/**
* This is an example showing the to use the Cassandra Input-/OutputFormats in the Batch API.
*
- * The example assumes that a table exists in a local cassandra database, according to the following query:
- * CREATE TABLE test.batches (number int, strings text, PRIMARY KEY(number, strings));
+ *
The example assumes that a table exists in a local cassandra database, according to the following queries:
+ * CREATE KEYSPACE IF NOT EXISTS test WITH replication = {'class': 'SimpleStrategy', 'replication_factor': ‘1’};
+ * CREATE TABLE IF NOT EXISTS test.batches (number int, strings text, PRIMARY KEY(number, strings));
*/
public class BatchExample {
private static final String INSERT_QUERY = "INSERT INTO test.batches (number, strings) VALUES (?,?);";
diff --git a/flink-connectors/flink-connector-cassandra/src/test/java/org/apache/flink/streaming/connectors/cassandra/example/CassandraPojoSinkExample.java b/flink-connectors/flink-connector-cassandra/src/test/java/org/apache/flink/streaming/connectors/cassandra/example/CassandraPojoSinkExample.java
index a38b73b72c991..01cd6e8048f27 100644
--- a/flink-connectors/flink-connector-cassandra/src/test/java/org/apache/flink/streaming/connectors/cassandra/example/CassandraPojoSinkExample.java
+++ b/flink-connectors/flink-connector-cassandra/src/test/java/org/apache/flink/streaming/connectors/cassandra/example/CassandraPojoSinkExample.java
@@ -32,7 +32,8 @@
*
*
Pojo's have to be annotated with datastax annotations to work with this sink.
*
- *
The example assumes that a table exists in a local cassandra database, according to the following query:
+ *
The example assumes that a table exists in a local cassandra database, according to the following queries:
+ * CREATE KEYSPACE IF NOT EXISTS test WITH replication = {'class': 'SimpleStrategy', 'replication_factor': ‘1’};
* CREATE TABLE IF NOT EXISTS test.message(body txt PRIMARY KEY)
*/
public class CassandraPojoSinkExample {
diff --git a/flink-connectors/flink-connector-cassandra/src/test/java/org/apache/flink/streaming/connectors/cassandra/example/CassandraTupleSinkExample.java b/flink-connectors/flink-connector-cassandra/src/test/java/org/apache/flink/streaming/connectors/cassandra/example/CassandraTupleSinkExample.java
index ce2326f4e569c..72013d5141a3f 100644
--- a/flink-connectors/flink-connector-cassandra/src/test/java/org/apache/flink/streaming/connectors/cassandra/example/CassandraTupleSinkExample.java
+++ b/flink-connectors/flink-connector-cassandra/src/test/java/org/apache/flink/streaming/connectors/cassandra/example/CassandraTupleSinkExample.java
@@ -31,7 +31,8 @@
/**
* This is an example showing the to use the Tuple Cassandra Sink in the Streaming API.
*
- *
The example assumes that a table exists in a local cassandra database, according to the following query:
+ *
The example assumes that a table exists in a local cassandra database, according to the following queries:
+ * CREATE KEYSPACE IF NOT EXISTS test WITH replication = {'class': 'SimpleStrategy', 'replication_factor': ‘1’};
* CREATE TABLE IF NOT EXISTS test.writetuple(element1 text PRIMARY KEY, element2 int)
*/
public class CassandraTupleSinkExample {
diff --git a/flink-connectors/flink-connector-cassandra/src/test/java/org/apache/flink/streaming/connectors/cassandra/example/CassandraTupleWriteAheadSinkExample.java b/flink-connectors/flink-connector-cassandra/src/test/java/org/apache/flink/streaming/connectors/cassandra/example/CassandraTupleWriteAheadSinkExample.java
index 38618feaf7771..8cab311be7f90 100644
--- a/flink-connectors/flink-connector-cassandra/src/test/java/org/apache/flink/streaming/connectors/cassandra/example/CassandraTupleWriteAheadSinkExample.java
+++ b/flink-connectors/flink-connector-cassandra/src/test/java/org/apache/flink/streaming/connectors/cassandra/example/CassandraTupleWriteAheadSinkExample.java
@@ -36,7 +36,8 @@
/**
* This is an example showing the to use the Cassandra Sink (with write-ahead log) in the Streaming API.
*
- *
The example assumes that a table exists in a local cassandra database, according to the following query:
+ *
The example assumes that a table exists in a local cassandra database, according to the following queries:
+ * CREATE KEYSPACE IF NOT EXISTS example WITH replication = {'class': 'SimpleStrategy', 'replication_factor': ‘1’};
* CREATE TABLE example.values (id text, count int, PRIMARY KEY(id));
*
*
Important things to note are that checkpointing is enabled, a StateBackend is set and the enableWriteAheadLog() call
diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/util/RetryRejectedExecutionFailureHandler.java b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/util/RetryRejectedExecutionFailureHandler.java
index 9380959934cbb..370625714e96a 100644
--- a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/util/RetryRejectedExecutionFailureHandler.java
+++ b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/util/RetryRejectedExecutionFailureHandler.java
@@ -36,7 +36,7 @@ public class RetryRejectedExecutionFailureHandler implements ActionRequestFailur
@Override
public void onFailure(ActionRequest action, Throwable failure, int restStatusCode, RequestIndexer indexer) throws Throwable {
- if (ExceptionUtils.containsThrowable(failure, EsRejectedExecutionException.class)) {
+ if (ExceptionUtils.findThrowable(failure, EsRejectedExecutionException.class).isPresent()) {
indexer.add(action);
} else {
// rethrow all other failures
diff --git a/flink-connectors/flink-connector-elasticsearch2/pom.xml b/flink-connectors/flink-connector-elasticsearch2/pom.xml
index 1f342bc2b2044..1dbe1144cfae2 100644
--- a/flink-connectors/flink-connector-elasticsearch2/pom.xml
+++ b/flink-connectors/flink-connector-elasticsearch2/pom.xml
@@ -91,4 +91,35 @@ under the License.
+
+
+
+ org.apache.maven.plugins
+ maven-shade-plugin
+
+
+ shade-flink
+
+
+
+ com.google.guava:guava
+
+
+
+
+ com.google
+ org.apache.flink.elasticsearch.shaded.com.google
+
+ com.google.protobuf.**
+ com.google.inject.**
+
+
+
+
+
+
+
+
+
+
diff --git a/flink-connectors/flink-connector-eventhubs/pom.xml b/flink-connectors/flink-connector-eventhubs/pom.xml
new file mode 100644
index 0000000000000..8ee8bb45ed427
--- /dev/null
+++ b/flink-connectors/flink-connector-eventhubs/pom.xml
@@ -0,0 +1,158 @@
+
+
+
+
+
+ flink-connectors
+ org.apache.flink
+ 1.4-SNAPSHOT
+
+ 4.0.0
+
+ flink-connector-azureeventhubs_${scala.binary.version}
+ flink-connector-azureeventhubs
+
+ jar
+
+
+
+
+
+ com.microsoft.azure
+ azure-eventhubs
+ 0.14.0
+ provided
+
+
+
+ org.apache.flink
+ flink-connector-kafka-base_${scala.binary.version}
+ ${project.version}
+
+
+
+
+ org.apache.flink
+ flink-streaming-java_${scala.binary.version}
+ ${project.version}
+ provided
+
+
+
+ org.apache.flink
+ flink-table_${scala.binary.version}
+ ${project.version}
+ provided
+
+ true
+
+
+
+
+
+ org.apache.flink
+ flink-streaming-java_${scala.binary.version}
+ ${project.version}
+ test
+ test-jar
+
+
+
+ org.apache.flink
+ flink-tests_${scala.binary.version}
+ ${project.version}
+ test-jar
+ test
+
+
+
+ org.apache.flink
+ flink-test-utils_${scala.binary.version}
+ ${project.version}
+ test
+
+
+
+ org.apache.flink
+ flink-runtime_${scala.binary.version}
+ ${project.version}
+ test-jar
+ test
+
+
+
+ org.apache.flink
+ flink-metrics-jmx
+ ${project.version}
+ test
+
+
+
+
+
+
+
+ org.apache.maven.plugins
+ maven-jar-plugin
+
+
+
+ test-jar
+
+
+
+ **/KafkaTestEnvironmentImpl*
+
+
+
+
+
+
+ org.apache.maven.plugins
+ maven-source-plugin
+
+
+ attach-test-sources
+
+ test-jar-no-fork
+
+
+
+ **/KafkaTestEnvironmentImpl*
+
+
+
+
+
+
+ org.apache.maven.plugins
+ maven-surefire-plugin
+
+
+ 1
+ -Xms256m -Xmx1000m -Dlog4j.configuration=${log4j.configuration} -Dmvn.forkNumber=${surefire.forkNumber} -XX:-UseGCOverheadLimit
+
+
+
+
+
+
+
diff --git a/flink-connectors/flink-connector-eventhubs/src/main/java/org/apache/flink/streaming/connectors/eventhubs/FlinkEventHubConsumer.java b/flink-connectors/flink-connector-eventhubs/src/main/java/org/apache/flink/streaming/connectors/eventhubs/FlinkEventHubConsumer.java
new file mode 100644
index 0000000000000..e5598597de73f
--- /dev/null
+++ b/flink-connectors/flink-connector-eventhubs/src/main/java/org/apache/flink/streaming/connectors/eventhubs/FlinkEventHubConsumer.java
@@ -0,0 +1,325 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.streaming.connectors.eventhubs;
+
+import org.apache.flink.api.common.state.ListState;
+import org.apache.flink.api.common.state.OperatorStateStore;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.api.java.ClosureCleaner;
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.api.java.typeutils.ResultTypeQueryable;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.metrics.Counter;
+import org.apache.flink.runtime.state.DefaultOperatorStateBackend;
+import org.apache.flink.runtime.state.FunctionInitializationContext;
+import org.apache.flink.runtime.state.FunctionSnapshotContext;
+import org.apache.flink.streaming.api.checkpoint.CheckpointedFunction;
+import org.apache.flink.streaming.api.functions.AssignerWithPeriodicWatermarks;
+import org.apache.flink.streaming.api.functions.AssignerWithPunctuatedWatermarks;
+import org.apache.flink.streaming.api.functions.source.RichParallelSourceFunction;
+import org.apache.flink.streaming.api.operators.StreamingRuntimeContext;
+import org.apache.flink.streaming.api.watermark.Watermark;
+import org.apache.flink.streaming.connectors.eventhubs.internals.EventFetcher;
+import org.apache.flink.streaming.connectors.eventhubs.internals.EventhubPartition;
+import org.apache.flink.streaming.util.serialization.DeserializationSchema;
+import org.apache.flink.streaming.util.serialization.KeyedDeserializationSchema;
+import org.apache.flink.streaming.util.serialization.KeyedDeserializationSchemaWrapper;
+import org.apache.flink.util.Preconditions;
+import org.apache.flink.util.SerializedValue;
+
+import com.microsoft.azure.eventhubs.PartitionReceiver;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Properties;
+/**
+ * Created by jozh on 5/22/2017.
+ * Flink eventhub connnector has implemented with same design of flink kafka connector
+ * This class is used to create datastream from event hub
+ */
+
+public class FlinkEventHubConsumer extends RichParallelSourceFunction implements
+ CheckpointedFunction,
+ ResultTypeQueryable {
+ private static final long serialVersionUID = -3247976470793561346L;
+ protected static final Logger LOGGER = LoggerFactory.getLogger(FlinkEventHubConsumer.class);
+ protected static final String DEFAULTOFFSETSTATENAME = "flink.eventhub.offset";
+
+ protected final KeyedDeserializationSchema deserializer;
+ protected final Properties eventhubsProps;
+ protected final int partitionCount;
+ protected List> subscribedPartitions;
+ protected final String defaultEventhubInitOffset;
+
+ private Map subscribedPartitionsToStartOffsets;
+ private SerializedValue> periodicWatermarkAssigner;
+ private SerializedValue> punctuatedWatermarkAssigner;
+ private transient ListState> offsetsStateForCheckpoint;
+ private transient volatile EventFetcher eventhubFetcher;
+ private transient volatile HashMap restoreToOffset;
+ private volatile boolean running = true;
+
+ private Counter receivedCount;
+
+ public FlinkEventHubConsumer(Properties eventhubsProps, DeserializationSchema deserializer){
+ this(eventhubsProps, new KeyedDeserializationSchemaWrapper(deserializer));
+ }
+
+ public FlinkEventHubConsumer(Properties eventhubsProps, KeyedDeserializationSchema deserializer){
+ Preconditions.checkNotNull(eventhubsProps);
+ Preconditions.checkNotNull(deserializer);
+ Preconditions.checkNotNull(eventhubsProps.getProperty("eventhubs.policyname"));
+ Preconditions.checkNotNull(eventhubsProps.getProperty("eventhubs.policykey"));
+ Preconditions.checkNotNull(eventhubsProps.getProperty("eventhubs.namespace"));
+ Preconditions.checkNotNull(eventhubsProps.getProperty("eventhubs.name"));
+ Preconditions.checkNotNull(eventhubsProps.getProperty("eventhubs.partition.count"));
+
+ this.eventhubsProps = eventhubsProps;
+ this.partitionCount = Integer.parseInt(eventhubsProps.getProperty("eventhubs.partition.count"));
+ this.deserializer = deserializer;
+
+ String userDefinedOffset = eventhubsProps.getProperty("eventhubs.auto.offset");
+ if (userDefinedOffset != null && userDefinedOffset.toLowerCase().compareTo("lastest") == 0){
+ this.defaultEventhubInitOffset = PartitionReceiver.END_OF_STREAM;
+ }
+ else {
+ this.defaultEventhubInitOffset = PartitionReceiver.START_OF_STREAM;
+ }
+
+ if (this.partitionCount <= 0){
+ throw new IllegalArgumentException("eventhubs.partition.count must greater than 0");
+ }
+ }
+
+ @Override
+ public void open(Configuration parameters) throws Exception {
+ super.open(parameters);
+ receivedCount = getRuntimeContext().getMetricGroup().addGroup(this.getClass().getName()).counter("received_event_count");
+
+ List eventhubPartitions = this.getAllEventhubPartitions();
+ this.subscribedPartitionsToStartOffsets = new HashMap<>(eventhubPartitions.size());
+
+ if (this.restoreToOffset != null){
+ for (EventhubPartition partition : eventhubPartitions){
+ if (this.restoreToOffset.containsKey(partition)){
+ this.subscribedPartitionsToStartOffsets.put(partition, restoreToOffset.get(partition));
+ }
+ }
+
+ LOGGER.info("Consumer subtask {} will start reading {} partitions with offsets in restored state: {}",
+ getRuntimeContext().getIndexOfThisSubtask(),
+ this.subscribedPartitionsToStartOffsets.size(),
+ this.subscribedPartitionsToStartOffsets);
+ }
+ else {
+ //If there is no restored state. Then all partitions to read from start, the offset is "-1". In the
+ //future eventhub supports specify offset, we modify here
+ //We assign partition to each subTask in round robin mode
+ int numParallelSubtasks = getRuntimeContext().getNumberOfParallelSubtasks();
+ int indexofThisSubtask = getRuntimeContext().getIndexOfThisSubtask();
+ for (int i = 0; i < eventhubPartitions.size(); i++) {
+ if (i % numParallelSubtasks == indexofThisSubtask) {
+ this.subscribedPartitionsToStartOffsets.put(eventhubPartitions.get(i), defaultEventhubInitOffset);
+ }
+ }
+
+ LOGGER.info("Consumer subtask {} will start reading {} partitions with offsets: {}",
+ getRuntimeContext().getIndexOfThisSubtask(),
+ this.subscribedPartitionsToStartOffsets.size(),
+ this.subscribedPartitionsToStartOffsets);
+ }
+ }
+
+ @Override
+ public void run(SourceContext sourceContext) throws Exception {
+ if (this.subscribedPartitionsToStartOffsets == null || this.subscribedPartitionsToStartOffsets.size() == 0){
+ throw new Exception("The partitions were not set for the consumer");
+ }
+
+ StreamingRuntimeContext runtimeContext = (StreamingRuntimeContext) getRuntimeContext();
+
+ if (!this.subscribedPartitionsToStartOffsets.isEmpty()){
+ final EventFetcher fetcher = new EventFetcher(sourceContext,
+ subscribedPartitionsToStartOffsets,
+ deserializer,
+ periodicWatermarkAssigner,
+ punctuatedWatermarkAssigner,
+ runtimeContext.getProcessingTimeService(),
+ runtimeContext.getExecutionConfig().getAutoWatermarkInterval(),
+ runtimeContext.getUserCodeClassLoader(),
+ runtimeContext.getTaskNameWithSubtasks(),
+ eventhubsProps,
+ false,
+ receivedCount);
+
+ this.eventhubFetcher = fetcher;
+ if (!this.running){
+ return;
+ }
+
+ this.eventhubFetcher.runFetchLoop();
+ }
+ else {
+ sourceContext.emitWatermark(new Watermark(Long.MAX_VALUE));
+
+ final Object waitObj = new Object();
+ while (this.running){
+ try {
+ synchronized (waitObj){
+ waitObj.wait();
+ }
+ }
+ catch (InterruptedException ex){
+ if (this.running){
+ Thread.currentThread().interrupt();
+ }
+ }
+ }
+ }
+ }
+
+ @Override
+ public void close() throws Exception {
+ try {
+ this.cancel();
+ }
+ finally {
+ super.close();
+ }
+ }
+
+ @Override
+ public void cancel() {
+ this.running = false;
+
+ if (this.eventhubFetcher != null){
+ this.eventhubFetcher.cancel();
+ }
+ }
+
+ @Override
+ public void snapshotState(FunctionSnapshotContext context) throws Exception {
+ if (!this.running){
+ LOGGER.info("Consumer subtask {}: snapshotState() is called on the closed source", getRuntimeContext().getIndexOfThisSubtask());
+ return;
+ }
+
+ this.offsetsStateForCheckpoint.clear();
+ final EventFetcher> fetcher = this.eventhubFetcher;
+ if (fetcher == null){
+ for (Map.Entry subscribedPartition : this.subscribedPartitionsToStartOffsets.entrySet()){
+ this.offsetsStateForCheckpoint.add(Tuple2.of(subscribedPartition.getKey(), subscribedPartition.getValue()));
+ }
+ }
+ else {
+ HashMap currentOffsets = fetcher.snapshotCurrentState();
+ for (Map.Entry subscribedPartition : currentOffsets.entrySet()){
+ this.offsetsStateForCheckpoint.add(Tuple2.of(subscribedPartition.getKey(), subscribedPartition.getValue()));
+ }
+ }
+ }
+
+ @Override
+ public void initializeState(FunctionInitializationContext context) throws Exception {
+ LOGGER.info("Consumer subtask {}:Start init eventhub offset state", getRuntimeContext().getIndexOfThisSubtask());
+ OperatorStateStore stateStore = context.getOperatorStateStore();
+ /* this.offsetsStateForCheckpoint = stateStore
+ .getListState(new ListStateDescriptor>(DEFAULT_OFFSET_STATE_NAME, TypeInformation.of(new TypeHint>(){})));
+*/
+ this.offsetsStateForCheckpoint = stateStore.getSerializableListState(DefaultOperatorStateBackend.DEFAULT_OPERATOR_STATE_NAME);
+ if (context.isRestored()){
+ if (this.restoreToOffset == null){
+ this.restoreToOffset = new HashMap<>();
+ for (Tuple2 offsetState : this.offsetsStateForCheckpoint.get()){
+ this.restoreToOffset.put(offsetState.f0, offsetState.f1);
+ }
+
+ LOGGER.info("Consumer subtask {}:Eventhub offset state is restored from checkpoint", getRuntimeContext().getIndexOfThisSubtask());
+ }
+ else if (this.restoreToOffset.isEmpty()){
+ this.restoreToOffset = null;
+ }
+ }
+ else {
+ LOGGER.info("Consumer subtask {}:No restore state for flink-eventhub-consumer", getRuntimeContext().getIndexOfThisSubtask());
+ }
+ }
+
+ //deprecated for CheckpointedRestoring
+ public void restoreState(HashMap eventhubPartitionOffsets) throws Exception {
+ LOGGER.info("{} (taskIdx={}) restoring offsets from an older version.",
+ getClass().getSimpleName(), getRuntimeContext().getIndexOfThisSubtask());
+
+ this.restoreToOffset = eventhubPartitionOffsets;
+
+ if (LOGGER.isDebugEnabled()) {
+ LOGGER.debug("{} (taskIdx={}) restored offsets from an older Flink version: {}",
+ getClass().getSimpleName(), getRuntimeContext().getIndexOfThisSubtask(), eventhubPartitionOffsets);
+ }
+ }
+
+ @Override
+ public TypeInformation getProducedType() {
+ return this.deserializer.getProducedType();
+ }
+
+ public FlinkEventHubConsumer assignTimestampsAndWatermarks(AssignerWithPunctuatedWatermarks assigner) {
+ Preconditions.checkNotNull(assigner);
+
+ if (this.periodicWatermarkAssigner != null) {
+ throw new IllegalStateException("A periodic watermark emitter has already been set.");
+ }
+ try {
+ ClosureCleaner.clean(assigner, true);
+ this.punctuatedWatermarkAssigner = new SerializedValue<>(assigner);
+ return this;
+ } catch (Exception e) {
+ throw new IllegalArgumentException("The given assigner is not serializable", e);
+ }
+ }
+
+ public FlinkEventHubConsumer assignTimestampsAndWatermarks(AssignerWithPeriodicWatermarks assigner) {
+ Preconditions.checkNotNull(assigner);
+
+ if (this.punctuatedWatermarkAssigner != null) {
+ throw new IllegalStateException("A punctuated watermark emitter has already been set.");
+ }
+ try {
+ ClosureCleaner.clean(assigner, true);
+ this.periodicWatermarkAssigner = new SerializedValue<>(assigner);
+ return this;
+ } catch (Exception e) {
+ throw new IllegalArgumentException("The given assigner is not serializable", e);
+ }
+ }
+
+ private List getAllEventhubPartitions() {
+ List partitions = new ArrayList<>();
+ for (int i = 0; i < this.partitionCount; i++){
+ partitions.add(new EventhubPartition(this.eventhubsProps, i));
+ }
+
+ LOGGER.info("Consumer subtask {}:Create {} eventhub partitions info", getRuntimeContext().getIndexOfThisSubtask(), this.partitionCount);
+ return partitions;
+ }
+}
diff --git a/flink-connectors/flink-connector-eventhubs/src/main/java/org/apache/flink/streaming/connectors/eventhubs/FlinkEventHubProducer.java b/flink-connectors/flink-connector-eventhubs/src/main/java/org/apache/flink/streaming/connectors/eventhubs/FlinkEventHubProducer.java
new file mode 100644
index 0000000000000..344e1f1a5ad59
--- /dev/null
+++ b/flink-connectors/flink-connector-eventhubs/src/main/java/org/apache/flink/streaming/connectors/eventhubs/FlinkEventHubProducer.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.streaming.connectors.eventhubs;
+
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.metrics.Counter;
+import org.apache.flink.runtime.state.FunctionInitializationContext;
+import org.apache.flink.runtime.state.FunctionSnapshotContext;
+import org.apache.flink.streaming.api.checkpoint.CheckpointedFunction;
+import org.apache.flink.streaming.api.functions.sink.RichSinkFunction;
+import org.apache.flink.streaming.connectors.eventhubs.internals.EventhubProducerThread;
+import org.apache.flink.streaming.connectors.eventhubs.internals.ProducerCache;
+import org.apache.flink.streaming.util.serialization.SerializationSchema;
+import org.apache.flink.util.Preconditions;
+
+import com.microsoft.azure.eventhubs.EventData;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.Properties;
+
+/**
+ * Created by jozh on 6/20/2017.
+ * Will support customize parttion in next version
+ */
+public class FlinkEventHubProducer extends RichSinkFunction implements CheckpointedFunction {
+
+ private static final Logger logger = LoggerFactory.getLogger(FlinkEventHubProducer.class);
+ private static final long serialVersionUID = -7486455932880508035L;
+ private final SerializationSchema schema;
+ private final ProducerCache cache;
+ private final Properties eventhubsProps;
+ private EventhubProducerThread producerThread;
+ private Counter prepareSendCount;
+ private Counter commitSendCount;
+
+ public FlinkEventHubProducer(SerializationSchema serializationSchema, Properties eventhubsProps){
+ Preconditions.checkNotNull(serializationSchema);
+ Preconditions.checkNotNull(eventhubsProps);
+ Preconditions.checkNotNull(eventhubsProps.getProperty("eventhubs.policyname"));
+ Preconditions.checkNotNull(eventhubsProps.getProperty("eventhubs.policykey"));
+ Preconditions.checkNotNull(eventhubsProps.getProperty("eventhubs.namespace"));
+ Preconditions.checkNotNull(eventhubsProps.getProperty("eventhubs.name"));
+
+ this.schema = serializationSchema;
+ this.eventhubsProps = eventhubsProps;
+
+ int capacity = eventhubsProps.getProperty("eventhubs.cache.capacity") == null
+ ? ProducerCache.DEFAULTCAPACITY : Integer.parseInt(eventhubsProps.getProperty("eventhubs.cache.capacity"));
+
+ long timeout = eventhubsProps.getProperty("eventhubs.cache.timeout") == null
+ ? ProducerCache.DEFAULTTIMEOUTMILLISECOND : Long.parseLong(eventhubsProps.getProperty("eventhubs.cache.timeout"));
+
+ this.cache = new ProducerCache(capacity, timeout);
+
+ logger.info("Created eventhub producer for namespace: {}, name: {}",
+ eventhubsProps.getProperty("eventhubs.namespace"),
+ eventhubsProps.getProperty("eventhubs.name"));
+ }
+
+ @Override
+ public void snapshotState(FunctionSnapshotContext context) throws Exception {
+ return;
+ }
+
+ @Override
+ public void initializeState(FunctionInitializationContext context) throws Exception {
+ return;
+ }
+
+ @Override
+ public void invoke(OUT value) throws Exception {
+ cache.checkErr();
+ EventData event = new EventData(this.schema.serialize(value));
+ cache.put(event);
+ prepareSendCount.inc();
+ logger.debug("Insert a event input output cache");
+ cache.checkErr();
+ }
+
+ @Override
+ public void open(Configuration parameters) throws Exception {
+ super.open(parameters);
+ prepareSendCount = getRuntimeContext().getMetricGroup().addGroup(this.getClass().getName()).counter("prepare_send_event_count");
+ commitSendCount = getRuntimeContext().getMetricGroup().addGroup(this.getClass().getName()).counter("commit_send_event_count");
+ String threadName = getEventhubProducerName();
+
+ logger.info("Eventhub producer thread {} starting", threadName);
+ producerThread = new EventhubProducerThread(
+ logger,
+ threadName,
+ cache,
+ eventhubsProps,
+ commitSendCount);
+ producerThread.start();
+ logger.info("Eventhub producer thread {} started", threadName);
+ cache.checkErr();
+ }
+
+ @Override
+ public void close() throws Exception {
+ super.close();
+ logger.info("Eventhub producer thread close on demand");
+ producerThread.shutdown();
+ cache.close();
+ cache.checkErr();
+ }
+
+ protected String getEventhubProducerName(){
+ return "Eventhub producer " + getRuntimeContext().getTaskNameWithSubtasks();
+ }
+}
diff --git a/flink-connectors/flink-connector-eventhubs/src/main/java/org/apache/flink/streaming/connectors/eventhubs/internals/EventFetcher.java b/flink-connectors/flink-connector-eventhubs/src/main/java/org/apache/flink/streaming/connectors/eventhubs/internals/EventFetcher.java
new file mode 100644
index 0000000000000..3212702da1be1
--- /dev/null
+++ b/flink-connectors/flink-connector-eventhubs/src/main/java/org/apache/flink/streaming/connectors/eventhubs/internals/EventFetcher.java
@@ -0,0 +1,388 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.streaming.connectors.eventhubs.internals;
+
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.metrics.Counter;
+import org.apache.flink.streaming.api.functions.AssignerWithPeriodicWatermarks;
+import org.apache.flink.streaming.api.functions.AssignerWithPunctuatedWatermarks;
+import org.apache.flink.streaming.api.functions.source.SourceFunction;
+import org.apache.flink.streaming.api.watermark.Watermark;
+import org.apache.flink.streaming.runtime.tasks.ProcessingTimeCallback;
+import org.apache.flink.streaming.runtime.tasks.ProcessingTimeService;
+import org.apache.flink.streaming.util.serialization.KeyedDeserializationSchema;
+import org.apache.flink.util.SerializedValue;
+
+import com.microsoft.azure.eventhubs.EventData;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Properties;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * Created by jozh on 6/14/2017.
+ * Flink eventhub connnector has implemented with same design of flink kafka connector.
+ * A fetcher that fetches data from Eventhub via the EventhubUtil.
+ * Eventhub offset is stored at flink checkpoint backend
+ * @param The type of elements produced by the fetcher.
+ */
+public class EventFetcher {
+ protected static final int NO_TIMESTAMPS_WATERMARKS = 0;
+ protected static final int PERIODIC_WATERMARKS = 1;
+ protected static final int PUNCTUATED_WATERMARKS = 2;
+ private static final Logger logger = LoggerFactory.getLogger(EventFetcher.class);
+ private volatile boolean running = true;
+
+ private final KeyedDeserializationSchema deserializer;
+ private final Handover handover;
+ private final Properties eventhubProps;
+ private final EventhubConsumerThread consumerThread;
+ private final String taskNameWithSubtasks;
+
+
+ protected final SourceFunction.SourceContext sourceContext;
+ protected final Object checkpointLock;
+ private final Map subscribedPartitionStates;
+ protected final int timestampWatermarkMode;
+ protected final boolean useMetrics;
+ private volatile long maxWatermarkSoFar = Long.MIN_VALUE;
+ private Counter receivedCount;
+
+ public EventFetcher(
+ SourceFunction.SourceContext sourceContext,
+ Map assignedPartitionsWithInitialOffsets,
+ KeyedDeserializationSchema deserializer,
+ SerializedValue> watermarksPeriodic,
+ SerializedValue> watermarksPunctuated,
+ ProcessingTimeService processTimerProvider,
+ long autoWatermarkInterval,
+ ClassLoader userCodeClassLoader,
+ String taskNameWithSubtasks,
+ Properties eventhubProps,
+ boolean useMetrics,
+ Counter receivedCount) throws Exception {
+
+ this.sourceContext = checkNotNull(sourceContext);
+ this.deserializer = checkNotNull(deserializer);
+ this.eventhubProps = eventhubProps;
+ this.checkpointLock = sourceContext.getCheckpointLock();
+ this.useMetrics = useMetrics;
+ this.receivedCount = receivedCount;
+ this.taskNameWithSubtasks = taskNameWithSubtasks;
+ this.timestampWatermarkMode = getTimestampWatermarkMode(watermarksPeriodic, watermarksPunctuated);
+
+ this.subscribedPartitionStates = initializeSubscribedPartitionStates(
+ assignedPartitionsWithInitialOffsets,
+ timestampWatermarkMode,
+ watermarksPeriodic, watermarksPunctuated,
+ userCodeClassLoader);
+
+ this.handover = new Handover();
+ this.consumerThread = new EventhubConsumerThread(logger,
+ handover,
+ this.eventhubProps,
+ getFetcherName() + " for " + taskNameWithSubtasks,
+ this.subscribedPartitionStates.values().toArray(new EventhubPartitionState[this.subscribedPartitionStates.size()]));
+
+ if (this.timestampWatermarkMode == PERIODIC_WATERMARKS) {
+ PeriodicWatermarkEmitter periodicEmitter =
+ new PeriodicWatermarkEmitter(this.subscribedPartitionStates, sourceContext, processTimerProvider, autoWatermarkInterval);
+ periodicEmitter.start();
+ }
+
+ }
+
+ public HashMap snapshotCurrentState() {
+ // this method assumes that the checkpoint lock is held
+ logger.debug("snapshot current offset state for subtask {}", taskNameWithSubtasks);
+ assert Thread.holdsLock(checkpointLock);
+
+ HashMap state = new HashMap<>(subscribedPartitionStates.size());
+ for (Map.Entry partition : subscribedPartitionStates.entrySet()){
+ state.put(partition.getKey(), partition.getValue().getOffset());
+ }
+
+ return state;
+ }
+
+ public void runFetchLoop() throws Exception{
+ try {
+ final Handover handover = this.handover;
+ consumerThread.start();
+ logger.info("Eventhub consumer thread started for substask {}", taskNameWithSubtasks);
+
+ logger.info("Start fetcher loop to get data from eventhub and emit to flink for subtask {}", taskNameWithSubtasks);
+ while (running){
+ final Tuple2> eventsTuple = handover.pollNext();
+ for (EventData event : eventsTuple.f1){
+ final T value = deserializer.deserialize(null,
+ event.getBytes(),
+ event.getSystemProperties().getPartitionKey(),
+ eventsTuple.f0.getParitionId(),
+ event.getSystemProperties().getSequenceNumber());
+
+ if (deserializer.isEndOfStream(value)){
+ running = false;
+ break;
+ }
+ emitRecord(value, subscribedPartitionStates.get(eventsTuple.f0), event.getSystemProperties().getOffset());
+ receivedCount.inc();
+ }
+ }
+ }
+ finally {
+ logger.warn("Stopping eventhub consumer thread of subtask {}, because something wrong when deserializing received event "
+ , taskNameWithSubtasks);
+ consumerThread.shutdown();
+ }
+
+ try {
+ consumerThread.join();
+ logger.warn("Waiting eventhub consumer thread of subtask {} stopped", taskNameWithSubtasks);
+ }
+ catch (InterruptedException ex){
+ Thread.currentThread().interrupt();
+ }
+
+ logger.info("EventFetcher of subtask {} stopped", taskNameWithSubtasks);
+ }
+
+ public void cancel(){
+ logger.info("EventFetcher of subtask {} canceled on demand", taskNameWithSubtasks);
+ running = false;
+ handover.close();
+ consumerThread.shutdown();
+ }
+
+ protected void emitRecord(T record, EventhubPartitionState partitionState, String offset) throws Exception{
+ if (record == null){
+ synchronized (this.checkpointLock){
+ partitionState.setOffset(offset);
+ }
+ return;
+ }
+
+ if (timestampWatermarkMode == NO_TIMESTAMPS_WATERMARKS){
+ synchronized (this.checkpointLock){
+ sourceContext.collect(record);
+ partitionState.setOffset(offset);
+ }
+ }
+ else if (timestampWatermarkMode == PERIODIC_WATERMARKS){
+ emitRecordWithTimestampAndPeriodicWatermark(record, partitionState, offset, Long.MIN_VALUE);
+ }
+ else {
+ emitRecordWithTimestampAndPunctuatedWatermark(record, partitionState, offset, Long.MIN_VALUE);
+ }
+ }
+
+ protected void emitRecordWithTimestampAndPunctuatedWatermark(
+ T record,
+ EventhubPartitionState partitionState,
+ String offset,
+ long eventTimestamp) {
+
+ final EventhubPartitionStateWithPeriodicWatermarks withWatermarksState =
+ (EventhubPartitionStateWithPeriodicWatermarks) partitionState;
+
+ final long timestamp;
+ synchronized (withWatermarksState) {
+ timestamp = withWatermarksState.getTimestampForRecord(record, eventTimestamp);
+ }
+
+ synchronized (checkpointLock) {
+ sourceContext.collectWithTimestamp(record, timestamp);
+ partitionState.setOffset(offset);
+ }
+ }
+
+ protected void emitRecordWithTimestampAndPeriodicWatermark(
+ T record,
+ EventhubPartitionState partitionState,
+ String offset,
+ long eventTimestamp) {
+
+ final EventhubPartitionStateWithPunctuatedWatermarks withWatermarksState =
+ (EventhubPartitionStateWithPunctuatedWatermarks) partitionState;
+
+ final long timestamp = withWatermarksState.getTimestampForRecord(record, eventTimestamp);
+ final Watermark newWatermark = withWatermarksState.checkAndGetNewWatermark(record, timestamp);
+
+ synchronized (checkpointLock) {
+ sourceContext.collectWithTimestamp(record, timestamp);
+ partitionState.setOffset(offset);
+ }
+
+ if (newWatermark != null) {
+ updateMinPunctuatedWatermark(newWatermark);
+ }
+ }
+
+ protected String getFetcherName() {
+ return "Eventhubs Fetcher";
+ }
+
+ private int getTimestampWatermarkMode(SerializedValue> watermarksPeriodic,
+ SerializedValue> watermarksPunctuated)
+ throws IllegalArgumentException {
+ if (watermarksPeriodic == null){
+ if (watermarksPunctuated == null){
+ return NO_TIMESTAMPS_WATERMARKS;
+ }
+ else {
+ return PUNCTUATED_WATERMARKS;
+ }
+ }
+ else {
+ if (watermarksPunctuated == null){
+ return PERIODIC_WATERMARKS;
+ }
+ else {
+ throw new IllegalArgumentException("Cannot have both periodic and punctuated watermarks");
+ }
+ }
+ }
+
+ private Map initializeSubscribedPartitionStates(
+ Map assignedPartitionsWithInitialOffsets,
+ int timestampWatermarkMode,
+ SerializedValue> watermarksPeriodic,
+ SerializedValue> watermarksPunctuated,
+ ClassLoader userCodeClassLoader) throws IOException, ClassNotFoundException {
+
+ if (timestampWatermarkMode != NO_TIMESTAMPS_WATERMARKS
+ && timestampWatermarkMode != PERIODIC_WATERMARKS
+ && timestampWatermarkMode != PUNCTUATED_WATERMARKS) {
+ throw new RuntimeException();
+ }
+
+ Map partitionsState = new HashMap<>(assignedPartitionsWithInitialOffsets.size());
+ for (Map.Entry partition : assignedPartitionsWithInitialOffsets.entrySet()){
+ switch (timestampWatermarkMode){
+ case NO_TIMESTAMPS_WATERMARKS:{
+ partitionsState.put(partition.getKey(), new EventhubPartitionState(partition.getKey(), partition.getValue()));
+ logger.info("NO_TIMESTAMPS_WATERMARKS: Assigned partition {}, offset is {}", partition.getKey(), partition.getValue());
+ break;
+ }
+
+ case PERIODIC_WATERMARKS:{
+ AssignerWithPeriodicWatermarks assignerInstance =
+ watermarksPeriodic.deserializeValue(userCodeClassLoader);
+ partitionsState.put(partition.getKey(),
+ new EventhubPartitionStateWithPeriodicWatermarks(partition.getKey(), partition.getValue(), assignerInstance));
+ logger.info("PERIODIC_WATERMARKS: Assigned partition {}, offset is {}", partition.getKey(), partition.getValue());
+ break;
+ }
+
+ case PUNCTUATED_WATERMARKS: {
+ AssignerWithPunctuatedWatermarks assignerInstance =
+ watermarksPunctuated.deserializeValue(userCodeClassLoader);
+ partitionsState.put(partition.getKey(),
+ new EventhubPartitionStateWithPunctuatedWatermarks(partition.getKey(), partition.getValue(), assignerInstance));
+ logger.info("PUNCTUATED_WATERMARKS: Assigned partition {}, offset is {}", partition.getKey(), partition.getValue());
+ break;
+ }
+ }
+ }
+ return partitionsState;
+ }
+
+ private void updateMinPunctuatedWatermark(Watermark nextWatermark) {
+ if (nextWatermark.getTimestamp() > maxWatermarkSoFar) {
+ long newMin = Long.MAX_VALUE;
+
+ for (Map.Entry partition : subscribedPartitionStates.entrySet()){
+ final EventhubPartitionStateWithPunctuatedWatermarks withWatermarksState =
+ (EventhubPartitionStateWithPunctuatedWatermarks) partition.getValue();
+
+ newMin = Math.min(newMin, withWatermarksState.getCurrentPartitionWatermark());
+ }
+
+ // double-check locking pattern
+ if (newMin > maxWatermarkSoFar) {
+ synchronized (checkpointLock) {
+ if (newMin > maxWatermarkSoFar) {
+ maxWatermarkSoFar = newMin;
+ sourceContext.emitWatermark(new Watermark(newMin));
+ }
+ }
+ }
+ }
+ }
+
+ private static class PeriodicWatermarkEmitter implements ProcessingTimeCallback {
+
+ private final Map allPartitions;
+
+ private final SourceFunction.SourceContext> emitter;
+
+ private final ProcessingTimeService timerService;
+
+ private final long interval;
+
+ private long lastWatermarkTimestamp;
+
+ //-------------------------------------------------
+
+ PeriodicWatermarkEmitter(
+ Map allPartitions,
+ SourceFunction.SourceContext> emitter,
+ ProcessingTimeService timerService,
+ long autoWatermarkInterval) {
+ this.allPartitions = checkNotNull(allPartitions);
+ this.emitter = checkNotNull(emitter);
+ this.timerService = checkNotNull(timerService);
+ this.interval = autoWatermarkInterval;
+ this.lastWatermarkTimestamp = Long.MIN_VALUE;
+ }
+
+ public void start() {
+ timerService.registerTimer(timerService.getCurrentProcessingTime() + interval, this);
+ }
+
+ @Override
+ public void onProcessingTime(long timestamp) throws Exception {
+
+ long minAcrossAll = Long.MAX_VALUE;
+ for (Map.Entry partition : allPartitions.entrySet()){
+ final long curr;
+ EventhubPartitionStateWithPeriodicWatermarks state =
+ (EventhubPartitionStateWithPeriodicWatermarks) partition.getValue();
+
+ synchronized (state) {
+ curr = state.getCurrentWatermarkTimestamp();
+ }
+
+ minAcrossAll = Math.min(minAcrossAll, curr);
+ }
+
+ if (minAcrossAll > lastWatermarkTimestamp) {
+ lastWatermarkTimestamp = minAcrossAll;
+ emitter.emitWatermark(new Watermark(minAcrossAll));
+ }
+
+ timerService.registerTimer(timerService.getCurrentProcessingTime() + interval, this);
+ }
+ }
+}
diff --git a/flink-connectors/flink-connector-eventhubs/src/main/java/org/apache/flink/streaming/connectors/eventhubs/internals/EventhubClientWrapper.java b/flink-connectors/flink-connector-eventhubs/src/main/java/org/apache/flink/streaming/connectors/eventhubs/internals/EventhubClientWrapper.java
new file mode 100644
index 0000000000000..39f22a263a9e7
--- /dev/null
+++ b/flink-connectors/flink-connector-eventhubs/src/main/java/org/apache/flink/streaming/connectors/eventhubs/internals/EventhubClientWrapper.java
@@ -0,0 +1,218 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.streaming.connectors.eventhubs.internals;
+
+import org.apache.flink.util.Preconditions;
+
+import com.microsoft.azure.eventhubs.EventData;
+import com.microsoft.azure.eventhubs.EventHubClient;
+import com.microsoft.azure.eventhubs.PartitionReceiver;
+import com.microsoft.azure.servicebus.ConnectionStringBuilder;
+import com.microsoft.azure.servicebus.ServiceBusException;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.io.Serializable;
+import java.net.URI;
+import java.net.URISyntaxException;
+import java.time.Duration;
+import java.time.Instant;
+import java.util.Properties;
+import java.util.concurrent.ExecutionException;
+
+/**
+ * Created by jozh on 6/14/2017.
+ * Flink eventhub connnector has implemented with same design of flink kafka connector
+ */
+public class EventhubClientWrapper implements Serializable {
+ private static final long serialVersionUID = -5319150387753930840L;
+ private static final Logger logger = LoggerFactory.getLogger(EventhubClientWrapper.class);
+ private EventHubClient eventHubClient;
+ private PartitionReceiver eventhubReceiver;
+ private ConnectionStringBuilder connectionString;
+ private String consumerGroup;
+ private Long receiverEpoch;
+
+ private Duration receiverTimeout;
+ private EventhubOffsetType offsetType;
+ private String currentOffset;
+ private String partitionId;
+
+ private final int minPrefetchCount = 10;
+ private int maxPrefetchCount = 999;
+ private int maxEventRate = 0;
+ private final Long defaultReceiverEpoch = -1L;
+ private final String defaultReceiverTimeout = "60000";
+
+ public void createReveiver(Properties eventhubParams, String partitionId)
+ throws IllegalArgumentException, URISyntaxException, IOException, ServiceBusException{
+ int maxEventRate = Integer.parseInt(eventhubParams.getProperty("eventhubs.maxRate", "10"));
+ this.createReveiver(eventhubParams, partitionId, maxEventRate, PartitionReceiver.START_OF_STREAM);
+ }
+
+ public void createReveiver(Properties eventhubParams, String partitionId, String offset)
+ throws IllegalArgumentException, URISyntaxException, IOException, ServiceBusException{
+ int maxEventRate = Integer.parseInt(eventhubParams.getProperty("eventhubs.maxRate", "10"));
+ this.createReveiver(eventhubParams, partitionId, maxEventRate, offset);
+ }
+
+ public void createReveiver(Properties eventhubParams, String partitionId, int maxEventRate)
+ throws IllegalArgumentException, URISyntaxException, IOException, ServiceBusException{
+ this.createReveiver(eventhubParams, partitionId, maxEventRate, PartitionReceiver.START_OF_STREAM);
+ }
+
+ /*Will not implement a standalone offset store here, will leverage flink state to save the offset of eventhub*/
+ public void createReveiver(Properties eventhubParams, String partitionId, int maxEventRate, String offset)
+ throws IllegalArgumentException, URISyntaxException, IOException, ServiceBusException{
+ if (eventhubParams.containsKey("eventhubs.uri") && eventhubParams.containsKey("eventhubs.namespace")) {
+ throw new IllegalArgumentException("Eventhubs URI and namespace cannot both be specified at the same time.");
+ }
+
+ if (eventhubParams.containsKey("eventhubs.namespace")){
+ this.connectionString = new ConnectionStringBuilder(
+ eventhubParams.getProperty("eventhubs.namespace"),
+ eventhubParams.getProperty("eventhubs.name"),
+ eventhubParams.getProperty("eventhubs.policyname"),
+ eventhubParams.getProperty("eventhubs.policykey"));
+ }
+ else if (eventhubParams.containsKey("eventhubs.uri")){
+ this.connectionString = new ConnectionStringBuilder(new URI(
+ eventhubParams.getProperty("eventhubs.uri")),
+ eventhubParams.getProperty("eventhubs.name"),
+ eventhubParams.getProperty("eventhubs.policyname"),
+ eventhubParams.getProperty("eventhubs.policykey"));
+ }
+ else {
+ throw new IllegalArgumentException("Either Eventhubs URI or namespace nust be specified.");
+ }
+
+ this.partitionId = Preconditions.checkNotNull(partitionId, "partitionId is no valid, cannot be null or empty");
+ this.consumerGroup = eventhubParams.getProperty("eventhubs.consumergroup", EventHubClient.DEFAULT_CONSUMER_GROUP_NAME);
+ this.receiverEpoch = Long.parseLong(eventhubParams.getProperty("eventhubs.epoch", defaultReceiverEpoch.toString()));
+ this.receiverTimeout = Duration.ofMillis(Long.parseLong(eventhubParams.getProperty("eventhubs.receiver.timeout", defaultReceiverTimeout)));
+ this.offsetType = EventhubOffsetType.None;
+ this.currentOffset = PartitionReceiver.START_OF_STREAM;
+
+ String previousOffset = offset;
+
+ if (previousOffset.compareTo(PartitionReceiver.START_OF_STREAM) != 0 && previousOffset != null) {
+
+ offsetType = EventhubOffsetType.PreviousCheckpoint;
+ currentOffset = previousOffset;
+
+ } else if (eventhubParams.containsKey("eventhubs.filter.offset")) {
+
+ offsetType = EventhubOffsetType.InputByteOffset;
+ currentOffset = eventhubParams.getProperty("eventhubs.filter.offset");
+
+ } else if (eventhubParams.containsKey("eventhubs.filter.enqueuetime")) {
+
+ offsetType = EventhubOffsetType.InputTimeOffset;
+ currentOffset = eventhubParams.getProperty("eventhubs.filter.enqueuetime");
+ }
+
+ this.maxEventRate = maxEventRate;
+
+ if (maxEventRate > 0 && maxEventRate < minPrefetchCount) {
+ maxPrefetchCount = minPrefetchCount;
+ }
+ else if (maxEventRate >= minPrefetchCount && maxEventRate < maxPrefetchCount) {
+ maxPrefetchCount = maxEventRate + 1;
+ }
+ else {
+ this.maxEventRate = maxPrefetchCount - 1;
+ }
+
+ this.createReceiverInternal();
+ }
+
+ public Iterable receive () throws ExecutionException, InterruptedException {
+ return this.eventhubReceiver.receive(maxEventRate).get();
+ }
+
+ public void close(){
+ logger.info("Close eventhub client on demand of partition {}", this.partitionId);
+ if (this.eventhubReceiver != null){
+ try {
+ this.eventhubReceiver.closeSync();
+ }
+ catch (ServiceBusException ex){
+ logger.error("Close eventhub client of partition {} failed, reason: {}", this.partitionId, ex.getMessage());
+ }
+ }
+ }
+
+ private void createReceiverInternal() throws IOException, ServiceBusException{
+ this.eventHubClient = EventHubClient.createFromConnectionStringSync(this.connectionString.toString());
+
+ switch (this.offsetType){
+ case None: {
+ if (this.receiverEpoch > defaultReceiverEpoch){
+ this.eventhubReceiver = this.eventHubClient.createEpochReceiverSync(consumerGroup, partitionId, currentOffset, receiverEpoch);
+ }
+ else {
+ this.eventhubReceiver = this.eventHubClient.createReceiverSync(consumerGroup, partitionId, currentOffset, false);
+ }
+ break;
+ }
+ case PreviousCheckpoint: {
+ if (this.receiverEpoch > defaultReceiverEpoch){
+ this.eventhubReceiver = this.eventHubClient.createEpochReceiverSync(consumerGroup, partitionId, currentOffset, false, receiverEpoch);
+ }
+ else {
+ this.eventhubReceiver = this.eventHubClient.createReceiverSync(consumerGroup, partitionId, currentOffset, false);
+ }
+ break;
+ }
+ case InputByteOffset: {
+ if (this.receiverEpoch > defaultReceiverEpoch){
+ this.eventhubReceiver = this.eventHubClient.createEpochReceiverSync(consumerGroup, partitionId, currentOffset, false, receiverEpoch);
+ }
+ else {
+ this.eventhubReceiver = this.eventHubClient.createReceiverSync(consumerGroup, partitionId, currentOffset, false);
+ }
+ break;
+ }
+ case InputTimeOffset: {
+ if (this.receiverEpoch > defaultReceiverEpoch){
+ this.eventhubReceiver = this.eventHubClient.createEpochReceiverSync(consumerGroup, partitionId, Instant.ofEpochSecond(Long.parseLong(currentOffset)), receiverEpoch);
+ }
+ else {
+ this.eventhubReceiver = this.eventHubClient.createReceiverSync(consumerGroup, partitionId, Instant.ofEpochSecond(Long.parseLong(currentOffset)));
+ }
+ break;
+ }
+ }
+
+ this.eventhubReceiver.setPrefetchCount(maxPrefetchCount);
+ this.eventhubReceiver.setReceiveTimeout(this.receiverTimeout);
+ logger.info("Successfully create eventhub receiver for partition {}, max_event_rate {}, max_prefetch_rate {}, receive_timeout {}, offset {}, ",
+ this.partitionId,
+ this.maxEventRate,
+ this.maxPrefetchCount,
+ this.receiverTimeout,
+ this.currentOffset);
+ }
+
+ public Duration getReceiverTimeout() {
+ return receiverTimeout;
+ }
+}
diff --git a/flink-connectors/flink-connector-eventhubs/src/main/java/org/apache/flink/streaming/connectors/eventhubs/internals/EventhubConsumerThread.java b/flink-connectors/flink-connector-eventhubs/src/main/java/org/apache/flink/streaming/connectors/eventhubs/internals/EventhubConsumerThread.java
new file mode 100644
index 0000000000000..a0f9f261d512e
--- /dev/null
+++ b/flink-connectors/flink-connector-eventhubs/src/main/java/org/apache/flink/streaming/connectors/eventhubs/internals/EventhubConsumerThread.java
@@ -0,0 +1,146 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.streaming.connectors.eventhubs.internals;
+
+import org.apache.flink.api.java.tuple.Tuple2;
+
+import com.microsoft.azure.eventhubs.EventData;
+import org.slf4j.Logger;
+
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Properties;
+
+/**
+ * Created by jozh on 5/24/2017.
+ * Flink eventhub connnector has implemented with same design of flink kafka connector
+ * Cause eventhub client can only access one partition at one time, so here we should have multiple eventhub clients
+ * In this worker thread, it will receive event from each partition in round robin mode, any partition failed to retrive
+ * events will lead thread exception, and leverage flink HA framework to start from begining again
+ */
+public class EventhubConsumerThread extends Thread {
+ private final Logger logger;
+ private final Handover handover;
+ private final Properties eventhubProps;
+ private final EventhubPartitionState[] subscribedPartitionStates;
+ private final Map clients;
+ private volatile boolean running;
+
+ public EventhubConsumerThread(
+ Logger logger,
+ Handover handover,
+ Properties eventhubProps,
+ String threadName,
+ EventhubPartitionState[] subscribedPartitionStates) throws Exception{
+
+ super(threadName);
+ setDaemon(true);
+
+ this.logger = logger;
+ this.handover = handover;
+ this.eventhubProps = eventhubProps;
+ this.subscribedPartitionStates = subscribedPartitionStates;
+ this.running = true;
+
+ this.clients = new HashMap<>(this.subscribedPartitionStates.length);
+ for (int i = 0; i < this.subscribedPartitionStates.length; i++){
+ EventhubClientWrapper client = new EventhubClientWrapper();
+ this.clients.put(this.subscribedPartitionStates[i], client);
+ }
+ }
+
+ public void shutdown(){
+ logger.info("Shutdown eventhub consumer thread {} on demand", this.getName());
+ running = false;
+ handover.wakeupProducer();
+ }
+
+ @Override
+ public void run() {
+ if (!running){
+ logger.info("Eventhub consumer thread is set to STOP, thread {} exit", this.getName());
+ return;
+ }
+
+ try {
+ logger.info("Starting create {} eventhub clients on {}", this.subscribedPartitionStates.length, this.getName());
+ for (Map.Entry client : clients.entrySet()){
+ EventhubPartitionState state = client.getKey();
+ client.getValue().createReveiver(this.eventhubProps, Integer.toString(state.getPartition().getParitionId()), state.getOffset());
+ }
+ }
+ catch (Throwable t){
+ logger.error("Create eventhub client of {}, error: {}", this.getName(), t);
+ handover.reportError(t);
+ clearReceiveClients();
+ return;
+ }
+
+ try {
+ int currentClientIndex = 0;
+ while (running){
+ EventhubPartitionState partitionState = subscribedPartitionStates[currentClientIndex];
+ EventhubClientWrapper client = clients.get(partitionState);
+ Iterable events = client.receive();
+ if (events != null){
+ handover.produce(Tuple2.of(partitionState.getPartition(), events));
+ logger.debug("Received event from {} on {}", partitionState.getPartition().toString(), this.getName());
+ }
+ else {
+ logger.warn("Receive events from {} timeout, timeout set to {}, thread {}",
+ partitionState.getPartition().toString(),
+ client.getReceiverTimeout(),
+ this.getName());
+ }
+
+ currentClientIndex++;
+ currentClientIndex = currentClientIndex % subscribedPartitionStates.length;
+ }
+ }
+ catch (Throwable t){
+ logger.error("Receving events error, {}", t);
+ handover.reportError(t);
+ }
+ finally {
+ logger.info("Exit from eventhub consumer thread, {}", this.getName());
+ handover.close();
+ clearReceiveClients();
+ }
+
+ logger.info("EventhubConsumerThread {} quit", this.getName());
+ }
+
+ private void clearReceiveClients(){
+ if (clients == null){
+ return;
+ }
+
+ for (Map.Entry client : clients.entrySet()){
+ try {
+ client.getValue().close();
+ logger.info("Eventhub client for partition {} closed", client.getKey().getPartition().getParitionId());
+ }
+ catch (Throwable t){
+ logger.warn("Error while close eventhub client for partition {}, error is {}",
+ client.getKey().getPartition().getParitionId(),
+ t.getMessage());
+ }
+ }
+ }
+}
diff --git a/flink-connectors/flink-connector-eventhubs/src/main/java/org/apache/flink/streaming/connectors/eventhubs/internals/EventhubOffsetType.java b/flink-connectors/flink-connector-eventhubs/src/main/java/org/apache/flink/streaming/connectors/eventhubs/internals/EventhubOffsetType.java
new file mode 100644
index 0000000000000..e745ded06afeb
--- /dev/null
+++ b/flink-connectors/flink-connector-eventhubs/src/main/java/org/apache/flink/streaming/connectors/eventhubs/internals/EventhubOffsetType.java
@@ -0,0 +1,30 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.streaming.connectors.eventhubs.internals;
+
+/**
+ * Created by jozh on 5/22/2017.
+ * Flink eventhub connnector has implemented with same design of flink kafka connector
+ */
+public enum EventhubOffsetType {
+ None,
+ PreviousCheckpoint,
+ InputByteOffset,
+ InputTimeOffset
+}
diff --git a/flink-connectors/flink-connector-eventhubs/src/main/java/org/apache/flink/streaming/connectors/eventhubs/internals/EventhubPartition.java b/flink-connectors/flink-connector-eventhubs/src/main/java/org/apache/flink/streaming/connectors/eventhubs/internals/EventhubPartition.java
new file mode 100644
index 0000000000000..9ae168ccf664c
--- /dev/null
+++ b/flink-connectors/flink-connector-eventhubs/src/main/java/org/apache/flink/streaming/connectors/eventhubs/internals/EventhubPartition.java
@@ -0,0 +1,88 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.streaming.connectors.eventhubs.internals;
+
+import org.apache.flink.util.Preconditions;
+
+import java.io.Serializable;
+import java.util.Properties;
+
+/**
+ * Created by jozh on 5/23/2017.
+ * Flink eventhub connnector has implemented with same design of flink kafka connector
+ */
+
+public class EventhubPartition implements Serializable {
+ private static final long serialVersionUID = 134878919919793479L;
+ private final int cachedHash;
+ private final String policyName;
+ private final String policyKey;
+ private final String namespace;
+ private final String name;
+
+ public int getParitionId() {
+ return paritionId;
+ }
+
+ public String getPartitionName(){
+ return namespace + "-" + name;
+ }
+
+ private final int paritionId;
+
+ public EventhubPartition(Properties props, int parition){
+ this(props.getProperty("eventhubs.policyname"),
+ props.getProperty("eventhubs.policykey"),
+ props.getProperty("eventhubs.namespace"),
+ props.getProperty("eventhubs.name"),
+ parition);
+ }
+
+ public EventhubPartition(String policyName, String policyKey, String namespace, String name, int paritionId){
+ Preconditions.checkArgument(paritionId >= 0);
+
+ this.policyName = Preconditions.checkNotNull(policyName);
+ this.policyKey = Preconditions.checkNotNull(policyKey);
+ this.name = Preconditions.checkNotNull(name);
+ this.namespace = Preconditions.checkNotNull(namespace);
+ this.paritionId = paritionId;
+ this.cachedHash = 31 * (this.namespace + this.name).hashCode() + paritionId;
+ }
+
+ @Override
+ public String toString() {
+ return "EventhubPartition, namespace: " + this.namespace +
+ " name: " + this.name +
+ " partition: " + this.paritionId;
+ }
+
+ @Override
+ public boolean equals(Object obj) {
+ if (obj instanceof EventhubPartition){
+ return this.hashCode() == ((EventhubPartition) obj).hashCode();
+ }
+
+ return false;
+ }
+
+ @Override
+ public int hashCode() {
+ return this.cachedHash;
+ }
+}
diff --git a/flink-connectors/flink-connector-eventhubs/src/main/java/org/apache/flink/streaming/connectors/eventhubs/internals/EventhubPartitionState.java b/flink-connectors/flink-connector-eventhubs/src/main/java/org/apache/flink/streaming/connectors/eventhubs/internals/EventhubPartitionState.java
new file mode 100644
index 0000000000000..d1547a539bde6
--- /dev/null
+++ b/flink-connectors/flink-connector-eventhubs/src/main/java/org/apache/flink/streaming/connectors/eventhubs/internals/EventhubPartitionState.java
@@ -0,0 +1,47 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.streaming.connectors.eventhubs.internals;
+
+/**
+ * Created by jozh on 5/23/2017.
+ * Flink eventhub connnector has implemented with same design of flink kafka connector
+ */
+
+public class EventhubPartitionState {
+ private final EventhubPartition partition;
+ private volatile String offset;
+
+ public EventhubPartitionState(EventhubPartition partition, String offset){
+ this.partition = partition;
+ this.offset = offset;
+ }
+
+ public final String getOffset() {
+ return this.offset;
+ }
+
+ public final void setOffset(String offset) {
+ this.offset = offset;
+ }
+
+ public EventhubPartition getPartition() {
+ return this.partition;
+ }
+}
+
diff --git a/flink-connectors/flink-connector-eventhubs/src/main/java/org/apache/flink/streaming/connectors/eventhubs/internals/EventhubPartitionStateWithPeriodicWatermarks.java b/flink-connectors/flink-connector-eventhubs/src/main/java/org/apache/flink/streaming/connectors/eventhubs/internals/EventhubPartitionStateWithPeriodicWatermarks.java
new file mode 100644
index 0000000000000..1fab7ff074633
--- /dev/null
+++ b/flink-connectors/flink-connector-eventhubs/src/main/java/org/apache/flink/streaming/connectors/eventhubs/internals/EventhubPartitionStateWithPeriodicWatermarks.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.streaming.connectors.eventhubs.internals;
+
+import org.apache.flink.streaming.api.functions.AssignerWithPeriodicWatermarks;
+import org.apache.flink.streaming.api.watermark.Watermark;
+
+/**
+ * Created by jozh on 6/16/2017.
+ * Flink eventhub connnector has implemented with same design of flink kafka connector
+ */
+
+public class EventhubPartitionStateWithPeriodicWatermarks extends EventhubPartitionState {
+ private final AssignerWithPeriodicWatermarks timestampsAndWatermarks;
+ private long partitionWatermark;
+
+ public EventhubPartitionStateWithPeriodicWatermarks(EventhubPartition key, String value, AssignerWithPeriodicWatermarks timestampsAndWatermarks) {
+ super(key, value);
+ this.timestampsAndWatermarks = timestampsAndWatermarks;
+ this.partitionWatermark = Long.MIN_VALUE;
+ }
+
+ public long getTimestampForRecord(T record, long kafkaEventTimestamp) {
+ return timestampsAndWatermarks.extractTimestamp(record, kafkaEventTimestamp);
+ }
+
+ public long getCurrentWatermarkTimestamp() {
+ Watermark wm = timestampsAndWatermarks.getCurrentWatermark();
+ if (wm != null) {
+ partitionWatermark = Math.max(partitionWatermark, wm.getTimestamp());
+ }
+ return partitionWatermark;
+ }
+
+ @Override
+ public String toString() {
+ return "EventhubPartitionStateWithPeriodicWatermarks: partition=" + getPartition()
+ + ", offset=" + getOffset() + ", watermark=" + partitionWatermark;
+ }
+}
diff --git a/flink-connectors/flink-connector-eventhubs/src/main/java/org/apache/flink/streaming/connectors/eventhubs/internals/EventhubPartitionStateWithPunctuatedWatermarks.java b/flink-connectors/flink-connector-eventhubs/src/main/java/org/apache/flink/streaming/connectors/eventhubs/internals/EventhubPartitionStateWithPunctuatedWatermarks.java
new file mode 100644
index 0000000000000..ae3d07ea3cd91
--- /dev/null
+++ b/flink-connectors/flink-connector-eventhubs/src/main/java/org/apache/flink/streaming/connectors/eventhubs/internals/EventhubPartitionStateWithPunctuatedWatermarks.java
@@ -0,0 +1,66 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.streaming.connectors.eventhubs.internals;
+
+import org.apache.flink.streaming.api.functions.AssignerWithPunctuatedWatermarks;
+import org.apache.flink.streaming.api.watermark.Watermark;
+
+import javax.annotation.Nullable;
+
+/**
+ * Created by jozh on 6/16/2017.
+ * Flink eventhub connnector has implemented with same design of flink kafka connector
+ */
+
+public class EventhubPartitionStateWithPunctuatedWatermarks extends EventhubPartitionState {
+ private final AssignerWithPunctuatedWatermarks timestampsAndWatermarks;
+ private long partitionWatermark;
+
+ public EventhubPartitionStateWithPunctuatedWatermarks(EventhubPartition key, String value, AssignerWithPunctuatedWatermarks timestampsAndWatermarks) {
+ super(key, value);
+ this.timestampsAndWatermarks = timestampsAndWatermarks;
+ this.partitionWatermark = Long.MIN_VALUE;
+ }
+
+ public long getTimestampForRecord(T record, long kafkaEventTimestamp) {
+ return timestampsAndWatermarks.extractTimestamp(record, kafkaEventTimestamp);
+ }
+
+ @Nullable
+ public Watermark checkAndGetNewWatermark(T record, long timestamp) {
+ Watermark mark = timestampsAndWatermarks.checkAndGetNextWatermark(record, timestamp);
+ if (mark != null && mark.getTimestamp() > partitionWatermark) {
+ partitionWatermark = mark.getTimestamp();
+ return mark;
+ }
+ else {
+ return null;
+ }
+ }
+
+ public long getCurrentPartitionWatermark() {
+ return partitionWatermark;
+ }
+
+ @Override
+ public String toString() {
+ return "EventhubPartitionStateWithPunctuatedWatermarks: partition=" + getPartition()
+ + ", offset=" + getOffset() + ", watermark=" + partitionWatermark;
+ }
+}
diff --git a/flink-connectors/flink-connector-eventhubs/src/main/java/org/apache/flink/streaming/connectors/eventhubs/internals/EventhubProducerThread.java b/flink-connectors/flink-connector-eventhubs/src/main/java/org/apache/flink/streaming/connectors/eventhubs/internals/EventhubProducerThread.java
new file mode 100644
index 0000000000000..b51236749386f
--- /dev/null
+++ b/flink-connectors/flink-connector-eventhubs/src/main/java/org/apache/flink/streaming/connectors/eventhubs/internals/EventhubProducerThread.java
@@ -0,0 +1,113 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.streaming.connectors.eventhubs.internals;
+
+import org.apache.flink.metrics.Counter;
+
+import com.microsoft.azure.eventhubs.EventData;
+import com.microsoft.azure.eventhubs.EventHubClient;
+import com.microsoft.azure.servicebus.ConnectionStringBuilder;
+import com.microsoft.azure.servicebus.ServiceBusException;
+import org.slf4j.Logger;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Properties;
+/**
+ * Created by jozh on 6/20/2017.
+ */
+
+public class EventhubProducerThread extends Thread {
+ private final Logger logger;
+ private final ProducerCache producerCache;
+ private final Properties eventhubProps;
+ private final EventHubClient producer;
+ private volatile boolean running;
+ private Counter commitSendCount;
+
+ public EventhubProducerThread(
+ Logger logger,
+ String threadName,
+ ProducerCache producerCache,
+ Properties eventhubProps,
+ Counter commitSendCount) throws IOException, ServiceBusException{
+
+ super(threadName);
+ setDaemon(true);
+
+ this.logger = logger;
+ this.producerCache = producerCache;
+ this.eventhubProps = eventhubProps;
+ this.commitSendCount = commitSendCount;
+
+ ConnectionStringBuilder connectionStringBuilder = new ConnectionStringBuilder(
+ eventhubProps.getProperty("eventhubs.namespace"),
+ eventhubProps.getProperty("eventhubs.name"),
+ eventhubProps.getProperty("eventhubs.policyname"),
+ eventhubProps.getProperty("eventhubs.policykey"));
+ this.producer = EventHubClient.createFromConnectionStringSync(connectionStringBuilder.toString());
+ this.running = true;
+ }
+
+ public void shutdown(){
+ logger.info("Shutdown eventhub producer thread {} on demand", this.getName());
+ running = false;
+ }
+
+ @Override
+ public void run() {
+ if (!running){
+ logger.info("Eventhub producer thread is set to STOP, thread {} exit", this.getName());
+ return;
+ }
+
+ try {
+ logger.info("Eventhub producer thread {} started", this.getName());
+ while (running){
+ final ArrayList events = producerCache.pollNextBatch();
+ if (events != null && events.size() > 0){
+ producer.sendSync(events);
+ commitSendCount.inc(events.size());
+ logger.info("Eventhub producer thread send {} events success", events.size());
+ }
+ else {
+ logger.debug("Eventhub producer thread received a null eventdata from producer cache");
+ }
+ }
+ }
+ catch (Throwable t){
+ logger.error("Sending events error, {}", t.toString());
+ producerCache.reportError(t);
+ }
+ finally {
+ logger.info("Exit from eventhub producer thread, {}", this.getName());
+ if (producer != null){
+ try {
+ producer.closeSync();
+ }
+ catch (Exception ex) {
+ logger.error("Close eventhubclient {} error {}", eventhubProps.getProperty("eventhubs.name"), ex.getMessage());
+ producerCache.reportError(ex);
+ }
+ }
+ }
+
+ logger.info("EventhubProducerThread {} quit", this.getName());
+ }
+}
diff --git a/flink-connectors/flink-connector-eventhubs/src/main/java/org/apache/flink/streaming/connectors/eventhubs/internals/Handover.java b/flink-connectors/flink-connector-eventhubs/src/main/java/org/apache/flink/streaming/connectors/eventhubs/internals/Handover.java
new file mode 100644
index 0000000000000..277b4bb7a1380
--- /dev/null
+++ b/flink-connectors/flink-connector-eventhubs/src/main/java/org/apache/flink/streaming/connectors/eventhubs/internals/Handover.java
@@ -0,0 +1,126 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.streaming.connectors.eventhubs.internals;
+
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.util.ExceptionUtils;
+
+import com.microsoft.azure.eventhubs.EventData;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.concurrent.ThreadSafe;
+
+import java.io.Closeable;
+import java.util.concurrent.ConcurrentLinkedQueue;
+
+
+/**
+ * Created by jozh on 5/23/2017.
+ * Flink eventhub connnector has implemented with same design of flink kafka connector
+ */
+@ThreadSafe
+public final class Handover implements Closeable {
+ private static final int MAX_EVENTS_BLOCK_IN_QUEUE = 1000;
+ private static final Logger logger = LoggerFactory.getLogger(Handover.class);
+ private ConcurrentLinkedQueue>> eventQueue = new ConcurrentLinkedQueue();
+
+ private volatile boolean allProducerWakeup = true;
+
+ private Throwable error;
+
+ public Tuple2> pollNext() throws Exception{
+ logger.debug("###Begin to poll data from event cache queue");
+ synchronized (eventQueue){
+ while (eventQueue.isEmpty() && error == null){
+ logger.debug("### No data in the msg queue, waiting... ");
+ eventQueue.wait();
+ }
+
+ logger.debug("### Get notified from consummer thread");
+ Tuple2> events = eventQueue.poll();
+ if (events != null && events.f0 != null && events.f1 != null){
+ logger.debug("### Get event data from {}", events.f0.toString());
+ int queueSize = eventQueue.size();
+ if (queueSize < MAX_EVENTS_BLOCK_IN_QUEUE / 2){
+ eventQueue.notifyAll();
+ }
+ return events;
+ }
+ else {
+ ExceptionUtils.rethrowException(error, error.getMessage());
+ return null;
+ }
+ }
+ }
+
+ public void produce(final Tuple2> events) throws InterruptedException{
+ if (events == null || events.f0 == null || events.f1 == null){
+ logger.error("Received empty events from event producer");
+ return;
+ }
+
+ synchronized (eventQueue){
+ while (eventQueue.size() > MAX_EVENTS_BLOCK_IN_QUEUE){
+ logger.warn("Event queue is full, current size is {}", eventQueue.size());
+ eventQueue.wait();
+ }
+
+ eventQueue.add(events);
+ eventQueue.notifyAll();
+ logger.debug("Add received events into queue");
+ }
+ }
+
+ @Override
+ public void close() {
+ synchronized (eventQueue){
+ logger.info("Close handover on demand");
+ eventQueue.clear();
+ if (error == null){
+ error = new Throwable("Handover closed on command");
+ }
+
+ eventQueue.notifyAll();
+ }
+ }
+
+ public void reportError(Throwable t) {
+ if (t == null){
+ return;
+ }
+
+ synchronized (eventQueue){
+ if (error == null){
+ error = t;
+ }
+ eventQueue.clear();
+ eventQueue.notifyAll();
+ logger.info("Consumer thread report a error: {}", error.getMessage());
+ }
+ }
+
+ public void wakeupProducer() {
+ synchronized (eventQueue){
+ logger.info("Wakeup producer on demand");
+ eventQueue.clear();
+ eventQueue.notifyAll();
+ }
+ }
+}
diff --git a/flink-connectors/flink-connector-eventhubs/src/main/java/org/apache/flink/streaming/connectors/eventhubs/internals/ProducerCache.java b/flink-connectors/flink-connector-eventhubs/src/main/java/org/apache/flink/streaming/connectors/eventhubs/internals/ProducerCache.java
new file mode 100644
index 0000000000000..5b29c4ff95f71
--- /dev/null
+++ b/flink-connectors/flink-connector-eventhubs/src/main/java/org/apache/flink/streaming/connectors/eventhubs/internals/ProducerCache.java
@@ -0,0 +1,143 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.streaming.connectors.eventhubs.internals;
+
+import org.apache.flink.util.ExceptionUtils;
+
+import com.microsoft.azure.eventhubs.EventData;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.Closeable;
+import java.io.Serializable;
+import java.util.ArrayList;
+import java.util.Date;
+import java.util.concurrent.ArrayBlockingQueue;
+
+/**
+ * Created by jozh on 6/20/2017.
+ */
+public final class ProducerCache implements Closeable, Serializable {
+ private static final Logger logger = LoggerFactory.getLogger(ProducerCache.class);
+ private static final long defaultCheckQueueStatusInterval = 50;
+ public static final int DEFAULTCAPACITY = 100;
+ public static final long DEFAULTTIMEOUTMILLISECOND = 100;
+ private final ArrayBlockingQueue cacheQueue;
+ private final int queueCapacity;
+ private final long pollTimeout;
+ private Date lastPollTime;
+ private Throwable error;
+ private volatile boolean closed;
+
+ public ProducerCache(){
+ this(DEFAULTCAPACITY, DEFAULTTIMEOUTMILLISECOND);
+ }
+
+ public ProducerCache(int capacity){
+ this(capacity, DEFAULTTIMEOUTMILLISECOND);
+ }
+
+ public ProducerCache(int capacity, long timeout){
+ this.queueCapacity = capacity;
+ this.pollTimeout = timeout;
+ this.cacheQueue = new ArrayBlockingQueue(this.queueCapacity);
+ this.lastPollTime = new Date();
+ this.closed = false;
+ }
+
+ public void put(EventData value) throws Exception{
+ if (value == null){
+ logger.error("Received empty events from event producer");
+ return;
+ }
+
+ synchronized (cacheQueue){
+ while (cacheQueue.remainingCapacity() <= 0 && !closed){
+ checkErr();
+ logger.warn("Event queue is full, current size is {}", cacheQueue.size());
+ cacheQueue.wait(defaultCheckQueueStatusInterval);
+ }
+
+ if (closed){
+ logger.info("Cache is closed, event is dropped.");
+ return;
+ }
+
+ cacheQueue.add(value);
+ cacheQueue.notifyAll();
+
+ logger.debug("Add event into queue");
+ }
+ }
+
+ public ArrayList pollNextBatch() throws InterruptedException{
+ logger.debug("###Begin to poll all data from event cache queue");
+
+ synchronized (cacheQueue){
+ while (!isPollTimeout() && !closed && cacheQueue.remainingCapacity() > 0){
+ cacheQueue.wait(defaultCheckQueueStatusInterval);
+ }
+
+ final ArrayList result = new ArrayList<>(cacheQueue.size());
+ for (EventData item : cacheQueue){
+ result.add(item);
+ }
+ cacheQueue.clear();
+ cacheQueue.notifyAll();
+
+ lastPollTime = new Date();
+ return result;
+ }
+ }
+
+ public void reportError(Throwable t) {
+ if (t == null){
+ return;
+ }
+
+ synchronized (cacheQueue){
+ if (error == null){
+ error = t;
+ }
+ logger.info("Producer thread report a error: {}", t.toString());
+ }
+ }
+
+ @Override
+ public void close() {
+ synchronized (cacheQueue){
+ logger.info("Close cache on demand");
+ closed = true;
+ cacheQueue.notifyAll();
+ }
+ }
+
+ public void checkErr() throws Exception {
+ synchronized (cacheQueue){
+ if (error != null){
+ ExceptionUtils.rethrowException(error, error.getMessage());
+ }
+ }
+ }
+
+ private boolean isPollTimeout(){
+ long pollInterval = (new Date()).getTime() - lastPollTime.getTime();
+ return pollInterval > pollTimeout;
+ }
+}
diff --git a/flink-connectors/flink-connector-filesystem/src/main/java/org/apache/flink/streaming/connectors/fs/RollingSink.java b/flink-connectors/flink-connector-filesystem/src/main/java/org/apache/flink/streaming/connectors/fs/RollingSink.java
index 3d3ea05cf3093..e5758e8920fdb 100644
--- a/flink-connectors/flink-connector-filesystem/src/main/java/org/apache/flink/streaming/connectors/fs/RollingSink.java
+++ b/flink-connectors/flink-connector-filesystem/src/main/java/org/apache/flink/streaming/connectors/fs/RollingSink.java
@@ -29,7 +29,6 @@
import org.apache.flink.runtime.state.FunctionInitializationContext;
import org.apache.flink.runtime.state.FunctionSnapshotContext;
import org.apache.flink.streaming.api.checkpoint.CheckpointedFunction;
-import org.apache.flink.streaming.api.checkpoint.CheckpointedRestoring;
import org.apache.flink.streaming.api.functions.sink.RichSinkFunction;
import org.apache.flink.streaming.connectors.fs.bucketing.BucketingSink;
import org.apache.flink.util.Preconditions;
@@ -132,7 +131,7 @@
@Deprecated
public class RollingSink extends RichSinkFunction
implements InputTypeConfigurable, CheckpointedFunction,
- CheckpointListener, CheckpointedRestoring {
+ CheckpointListener {
private static final long serialVersionUID = 1L;
@@ -758,25 +757,6 @@ private void handleRestoredBucketState(BucketState bucketState) {
}
}
- // --------------------------------------------------------------------------------------------
- // Backwards compatibility with Flink 1.1
- // --------------------------------------------------------------------------------------------
-
- @Override
- public void restoreState(BucketState state) throws Exception {
- LOG.info("{} (taskIdx={}) restored bucket state from an older Flink version: {}",
- getClass().getSimpleName(), getRuntimeContext().getIndexOfThisSubtask(), state);
-
- try {
- initFileSystem();
- } catch (IOException e) {
- LOG.error("Error while creating FileSystem when restoring the state of the RollingSink.", e);
- throw new RuntimeException("Error while creating FileSystem when restoring the state of the RollingSink.", e);
- }
-
- handleRestoredBucketState(state);
- }
-
// --------------------------------------------------------------------------------------------
// Setters for User configuration values
// --------------------------------------------------------------------------------------------
diff --git a/flink-connectors/flink-connector-filesystem/src/main/java/org/apache/flink/streaming/connectors/fs/bucketing/BucketingSink.java b/flink-connectors/flink-connector-filesystem/src/main/java/org/apache/flink/streaming/connectors/fs/bucketing/BucketingSink.java
index 70168b55164bb..cc924a4a2056f 100644
--- a/flink-connectors/flink-connector-filesystem/src/main/java/org/apache/flink/streaming/connectors/fs/bucketing/BucketingSink.java
+++ b/flink-connectors/flink-connector-filesystem/src/main/java/org/apache/flink/streaming/connectors/fs/bucketing/BucketingSink.java
@@ -30,7 +30,6 @@
import org.apache.flink.runtime.state.FunctionInitializationContext;
import org.apache.flink.runtime.state.FunctionSnapshotContext;
import org.apache.flink.streaming.api.checkpoint.CheckpointedFunction;
-import org.apache.flink.streaming.api.checkpoint.CheckpointedRestoring;
import org.apache.flink.streaming.api.functions.sink.RichSinkFunction;
import org.apache.flink.streaming.api.operators.StreamingRuntimeContext;
import org.apache.flink.streaming.connectors.fs.Clock;
@@ -154,8 +153,7 @@
*/
public class BucketingSink
extends RichSinkFunction
- implements InputTypeConfigurable, CheckpointedFunction, CheckpointListener,
- CheckpointedRestoring, ProcessingTimeCallback {
+ implements InputTypeConfigurable, CheckpointedFunction, CheckpointListener, ProcessingTimeCallback {
private static final long serialVersionUID = 1L;
@@ -872,25 +870,6 @@ private void handlePendingFilesForPreviousCheckpoints(Map> pe
}
}
- // --------------------------------------------------------------------------------------------
- // Backwards compatibility with Flink 1.1
- // --------------------------------------------------------------------------------------------
-
- @Override
- public void restoreState(RollingSink.BucketState state) throws Exception {
- LOG.info("{} (taskIdx={}) restored bucket state from the RollingSink an older Flink version: {}",
- getClass().getSimpleName(), getRuntimeContext().getIndexOfThisSubtask(), state);
-
- try {
- initFileSystem();
- } catch (IOException e) {
- LOG.error("Error while creating FileSystem when restoring the state of the BucketingSink.", e);
- throw new RuntimeException("Error while creating FileSystem when restoring the state of the BucketingSink.", e);
- }
-
- handleRestoredRollingSinkState(state);
- }
-
// --------------------------------------------------------------------------------------------
// Setters for User configuration values
// --------------------------------------------------------------------------------------------
diff --git a/flink-connectors/flink-connector-filesystem/src/test/java/org/apache/flink/streaming/connectors/fs/bucketing/RollingSinkMigrationTest.java b/flink-connectors/flink-connector-filesystem/src/test/java/org/apache/flink/streaming/connectors/fs/bucketing/RollingSinkMigrationTest.java
deleted file mode 100644
index e0413795b2aa7..0000000000000
--- a/flink-connectors/flink-connector-filesystem/src/test/java/org/apache/flink/streaming/connectors/fs/bucketing/RollingSinkMigrationTest.java
+++ /dev/null
@@ -1,213 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.connectors.fs.bucketing;
-
-import org.apache.flink.streaming.api.operators.StreamSink;
-import org.apache.flink.streaming.connectors.fs.RollingSink;
-import org.apache.flink.streaming.connectors.fs.StringWriter;
-import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
-import org.apache.flink.streaming.util.OneInputStreamOperatorTestHarness;
-import org.apache.flink.util.OperatingSystem;
-
-import org.apache.commons.io.FileUtils;
-import org.junit.Assert;
-import org.junit.Assume;
-import org.junit.BeforeClass;
-import org.junit.ClassRule;
-import org.junit.Test;
-import org.junit.rules.TemporaryFolder;
-
-import java.io.File;
-import java.io.IOException;
-import java.net.URL;
-import java.util.List;
-import java.util.Map;
-
-/**
- * Tests the migration from 1.1 snapshots.
- */
-@Deprecated
-public class RollingSinkMigrationTest {
-
- @ClassRule
- public static TemporaryFolder tempFolder = new TemporaryFolder();
-
- private static final String PART_PREFIX = "part";
- private static final String PENDING_SUFFIX = ".pending";
- private static final String IN_PROGRESS_SUFFIX = ".in-progress";
- private static final String VALID_LENGTH_SUFFIX = ".valid";
-
- @BeforeClass
- public static void verifyOS() {
- Assume.assumeTrue("HDFS cluster cannot be started on Windows without extensions.", !OperatingSystem.isWindows());
- }
-
- @Test
- public void testMigration() throws Exception {
-
- /*
- * Code ran to get the snapshot:
- *
- * final File outDir = tempFolder.newFolder();
-
- RollingSink sink = new RollingSink(outDir.getAbsolutePath())
- .setWriter(new StringWriter())
- .setBatchSize(5)
- .setPartPrefix(PART_PREFIX)
- .setInProgressPrefix("")
- .setPendingPrefix("")
- .setValidLengthPrefix("")
- .setInProgressSuffix(IN_PROGRESS_SUFFIX)
- .setPendingSuffix(PENDING_SUFFIX)
- .setValidLengthSuffix(VALID_LENGTH_SUFFIX);
-
- OneInputStreamOperatorTestHarness testHarness1 =
- new OneInputStreamOperatorTestHarness<>(new StreamSink<>(sink));
-
- testHarness1.setup();
- testHarness1.open();
-
- testHarness1.processElement(new StreamRecord<>("test1", 0L));
- testHarness1.processElement(new StreamRecord<>("test2", 0L));
-
- checkFs(outDir, 1, 1, 0, 0);
-
- testHarness1.processElement(new StreamRecord<>("test3", 0L));
- testHarness1.processElement(new StreamRecord<>("test4", 0L));
- testHarness1.processElement(new StreamRecord<>("test5", 0L));
-
- checkFs(outDir, 1, 4, 0, 0);
-
- StreamTaskState taskState = testHarness1.snapshot(0, 0);
- testHarness1.snaphotToFile(taskState, "src/test/resources/rolling-sink-migration-test-flink1.1-snapshot");
- testHarness1.close();
- * */
-
- final File outDir = tempFolder.newFolder();
-
- RollingSink sink = new ValidatingRollingSink(outDir.getAbsolutePath())
- .setWriter(new StringWriter())
- .setBatchSize(5)
- .setPartPrefix(PART_PREFIX)
- .setInProgressPrefix("")
- .setPendingPrefix("")
- .setValidLengthPrefix("")
- .setInProgressSuffix(IN_PROGRESS_SUFFIX)
- .setPendingSuffix(PENDING_SUFFIX)
- .setValidLengthSuffix(VALID_LENGTH_SUFFIX);
-
- OneInputStreamOperatorTestHarness testHarness1 = new OneInputStreamOperatorTestHarness<>(
- new StreamSink<>(sink), 10, 1, 0);
- testHarness1.setup();
- testHarness1.initializeStateFromLegacyCheckpoint(getResourceFilename("rolling-sink-migration-test-flink1.1-snapshot"));
- testHarness1.open();
-
- testHarness1.processElement(new StreamRecord<>("test1", 0L));
- testHarness1.processElement(new StreamRecord<>("test2", 0L));
-
- checkFs(outDir, 1, 1, 0, 0);
-
- testHarness1.close();
- }
-
- private void checkFs(File outDir, int inprogress, int pending, int completed, int valid) throws IOException {
- int inProg = 0;
- int pend = 0;
- int compl = 0;
- int val = 0;
-
- for (File file: FileUtils.listFiles(outDir, null, true)) {
- if (file.getAbsolutePath().endsWith("crc")) {
- continue;
- }
- String path = file.getPath();
- if (path.endsWith(IN_PROGRESS_SUFFIX)) {
- inProg++;
- } else if (path.endsWith(PENDING_SUFFIX)) {
- pend++;
- } else if (path.endsWith(VALID_LENGTH_SUFFIX)) {
- val++;
- } else if (path.contains(PART_PREFIX)) {
- compl++;
- }
- }
-
- Assert.assertEquals(inprogress, inProg);
- Assert.assertEquals(pending, pend);
- Assert.assertEquals(completed, compl);
- Assert.assertEquals(valid, val);
- }
-
- private static String getResourceFilename(String filename) {
- ClassLoader cl = RollingSinkMigrationTest.class.getClassLoader();
- URL resource = cl.getResource(filename);
- return resource.getFile();
- }
-
- static class ValidatingRollingSink extends RollingSink {
-
- private static final long serialVersionUID = -4263974081712009141L;
-
- ValidatingRollingSink(String basePath) {
- super(basePath);
- }
-
- @Override
- public void restoreState(BucketState state) throws Exception {
-
- /**
- * this validates that we read the state that was checkpointed by the previous version. We expect it to be:
- * In-progress=/var/folders/z5/fxvg1j6s6mn94nsf8b1yc8s80000gn/T/junit2927527303216950257/junit5645682027227039270/2017-01-09--18/part-0-4
- * validLength=6
- * pendingForNextCheckpoint=[]
- * pendingForPrevCheckpoints={0=[ /var/folders/z5/fxvg1j6s6mn94nsf8b1yc8s80000gn/T/junit2927527303216950257/junit5645682027227039270/2017-01-09--18/part-0-0,
- * /var/folders/z5/fxvg1j6s6mn94nsf8b1yc8s80000gn/T/junit2927527303216950257/junit5645682027227039270/2017-01-09--18/part-0-1,
- * /var/folders/z5/fxvg1j6s6mn94nsf8b1yc8s80000gn/T/junit2927527303216950257/junit5645682027227039270/2017-01-09--18/part-0-2,
- * /var/folders/z5/fxvg1j6s6mn94nsf8b1yc8s80000gn/T/junit2927527303216950257/junit5645682027227039270/2017-01-09--18/part-0-3]}
- * */
-
- String current = state.currentFile;
- long validLength = state.currentFileValidLength;
-
- Assert.assertEquals("/var/folders/z5/fxvg1j6s6mn94nsf8b1yc8s80000gn/T/junit2927527303216950257/junit5645682027227039270/2017-01-09--18/part-0-4", current);
- Assert.assertEquals(6, validLength);
-
- List pendingFiles = state.pendingFiles;
- Assert.assertTrue(pendingFiles.isEmpty());
-
- final Map> pendingFilesPerCheckpoint = state.pendingFilesPerCheckpoint;
- Assert.assertEquals(1, pendingFilesPerCheckpoint.size());
-
- for (Map.Entry> entry: pendingFilesPerCheckpoint.entrySet()) {
- long checkpoint = entry.getKey();
- List files = entry.getValue();
-
- Assert.assertEquals(0L, checkpoint);
- Assert.assertEquals(4, files.size());
-
- for (int i = 0; i < 4; i++) {
- Assert.assertEquals(
- "/var/folders/z5/fxvg1j6s6mn94nsf8b1yc8s80000gn/T/junit2927527303216950257/junit5645682027227039270/2017-01-09--18/part-0-" + i,
- files.get(i));
- }
- }
- super.restoreState(state);
- }
- }
-}
diff --git a/flink-connectors/flink-connector-filesystem/src/test/java/org/apache/flink/streaming/connectors/fs/bucketing/RollingToBucketingMigrationTest.java b/flink-connectors/flink-connector-filesystem/src/test/java/org/apache/flink/streaming/connectors/fs/bucketing/RollingToBucketingMigrationTest.java
deleted file mode 100644
index 8a8dbd6bc9301..0000000000000
--- a/flink-connectors/flink-connector-filesystem/src/test/java/org/apache/flink/streaming/connectors/fs/bucketing/RollingToBucketingMigrationTest.java
+++ /dev/null
@@ -1,174 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.connectors.fs.bucketing;
-
-import org.apache.flink.streaming.api.operators.StreamSink;
-import org.apache.flink.streaming.connectors.fs.RollingSink;
-import org.apache.flink.streaming.connectors.fs.StringWriter;
-import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
-import org.apache.flink.streaming.util.OneInputStreamOperatorTestHarness;
-import org.apache.flink.util.OperatingSystem;
-
-import org.apache.commons.io.FileUtils;
-import org.junit.Assert;
-import org.junit.Assume;
-import org.junit.BeforeClass;
-import org.junit.ClassRule;
-import org.junit.Test;
-import org.junit.rules.TemporaryFolder;
-
-import java.io.File;
-import java.io.IOException;
-import java.net.URL;
-import java.util.List;
-import java.util.Map;
-
-/**
- * Tests the migration from {@link RollingSink} to {@link BucketingSink}.
- */
-public class RollingToBucketingMigrationTest {
-
- @ClassRule
- public static TemporaryFolder tempFolder = new TemporaryFolder();
-
- private static final String PART_PREFIX = "part";
- private static final String PENDING_SUFFIX = ".pending";
- private static final String IN_PROGRESS_SUFFIX = ".in-progress";
- private static final String VALID_LENGTH_SUFFIX = ".valid";
-
- @BeforeClass
- public static void verifyOS() {
- Assume.assumeTrue("HDFS cluster cannot be started on Windows without extensions.", !OperatingSystem.isWindows());
- }
-
- @Test
- public void testMigration() throws Exception {
- final File outDir = tempFolder.newFolder();
-
- BucketingSink sink = new ValidatingBucketingSink(outDir.getAbsolutePath())
- .setWriter(new StringWriter())
- .setBatchSize(5)
- .setPartPrefix(PART_PREFIX)
- .setInProgressPrefix("")
- .setPendingPrefix("")
- .setValidLengthPrefix("")
- .setInProgressSuffix(IN_PROGRESS_SUFFIX)
- .setPendingSuffix(PENDING_SUFFIX)
- .setValidLengthSuffix(VALID_LENGTH_SUFFIX);
-
- OneInputStreamOperatorTestHarness testHarness1 = new OneInputStreamOperatorTestHarness<>(
- new StreamSink<>(sink), 10, 1, 0);
- testHarness1.setup();
- testHarness1.initializeStateFromLegacyCheckpoint(getResourceFilename("rolling-sink-migration-test-flink1.1-snapshot"));
- testHarness1.open();
-
- testHarness1.processElement(new StreamRecord<>("test1", 0L));
- testHarness1.processElement(new StreamRecord<>("test2", 0L));
-
- checkFs(outDir, 1, 1, 0, 0);
-
- testHarness1.close();
- }
-
- private static String getResourceFilename(String filename) {
- ClassLoader cl = RollingToBucketingMigrationTest.class.getClassLoader();
- URL resource = cl.getResource(filename);
- return resource.getFile();
- }
-
- private void checkFs(File outDir, int inprogress, int pending, int completed, int valid) throws IOException {
- int inProg = 0;
- int pend = 0;
- int compl = 0;
- int val = 0;
-
- for (File file: FileUtils.listFiles(outDir, null, true)) {
- if (file.getAbsolutePath().endsWith("crc")) {
- continue;
- }
- String path = file.getPath();
- if (path.endsWith(IN_PROGRESS_SUFFIX)) {
- inProg++;
- } else if (path.endsWith(PENDING_SUFFIX)) {
- pend++;
- } else if (path.endsWith(VALID_LENGTH_SUFFIX)) {
- val++;
- } else if (path.contains(PART_PREFIX)) {
- compl++;
- }
- }
-
- Assert.assertEquals(inprogress, inProg);
- Assert.assertEquals(pending, pend);
- Assert.assertEquals(completed, compl);
- Assert.assertEquals(valid, val);
- }
-
- static class ValidatingBucketingSink extends BucketingSink {
-
- private static final long serialVersionUID = -4263974081712009141L;
-
- ValidatingBucketingSink(String basePath) {
- super(basePath);
- }
-
- @Override
- public void restoreState(RollingSink.BucketState state) throws Exception {
-
- /**
- * this validates that we read the state that was checkpointed by the previous version. We expect it to be:
- * In-progress=/var/folders/z5/fxvg1j6s6mn94nsf8b1yc8s80000gn/T/junit2927527303216950257/junit5645682027227039270/2017-01-09--18/part-0-4
- * validLength=6
- * pendingForNextCheckpoint=[]
- * pendingForPrevCheckpoints={0=[ /var/folders/z5/fxvg1j6s6mn94nsf8b1yc8s80000gn/T/junit2927527303216950257/junit5645682027227039270/2017-01-09--18/part-0-0,
- * /var/folders/z5/fxvg1j6s6mn94nsf8b1yc8s80000gn/T/junit2927527303216950257/junit5645682027227039270/2017-01-09--18/part-0-1,
- * /var/folders/z5/fxvg1j6s6mn94nsf8b1yc8s80000gn/T/junit2927527303216950257/junit5645682027227039270/2017-01-09--18/part-0-2,
- * /var/folders/z5/fxvg1j6s6mn94nsf8b1yc8s80000gn/T/junit2927527303216950257/junit5645682027227039270/2017-01-09--18/part-0-3]}
- * */
-
- String current = state.currentFile;
- long validLength = state.currentFileValidLength;
-
- Assert.assertEquals("/var/folders/z5/fxvg1j6s6mn94nsf8b1yc8s80000gn/T/junit2927527303216950257/junit5645682027227039270/2017-01-09--18/part-0-4", current);
- Assert.assertEquals(6, validLength);
-
- List pendingFiles = state.pendingFiles;
- Assert.assertTrue(pendingFiles.isEmpty());
-
- final Map> pendingFilesPerCheckpoint = state.pendingFilesPerCheckpoint;
- Assert.assertEquals(1, pendingFilesPerCheckpoint.size());
-
- for (Map.Entry> entry: pendingFilesPerCheckpoint.entrySet()) {
- long checkpoint = entry.getKey();
- List files = entry.getValue();
-
- Assert.assertEquals(0L, checkpoint);
- Assert.assertEquals(4, files.size());
-
- for (int i = 0; i < 4; i++) {
- Assert.assertEquals(
- "/var/folders/z5/fxvg1j6s6mn94nsf8b1yc8s80000gn/T/junit2927527303216950257/junit5645682027227039270/2017-01-09--18/part-0-" + i,
- files.get(i));
- }
- }
-
- super.restoreState(state);
- }
- }
-}
diff --git a/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumerBase.java b/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumerBase.java
index f3c9e5e342044..3088b1552a65d 100644
--- a/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumerBase.java
+++ b/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumerBase.java
@@ -33,7 +33,6 @@
import org.apache.flink.runtime.state.FunctionInitializationContext;
import org.apache.flink.runtime.state.FunctionSnapshotContext;
import org.apache.flink.streaming.api.checkpoint.CheckpointedFunction;
-import org.apache.flink.streaming.api.checkpoint.CheckpointedRestoring;
import org.apache.flink.streaming.api.functions.AssignerWithPeriodicWatermarks;
import org.apache.flink.streaming.api.functions.AssignerWithPunctuatedWatermarks;
import org.apache.flink.streaming.api.functions.source.RichParallelSourceFunction;
@@ -78,8 +77,7 @@
public abstract class FlinkKafkaConsumerBase extends RichParallelSourceFunction implements
CheckpointListener,
ResultTypeQueryable,
- CheckpointedFunction,
- CheckpointedRestoring> {
+ CheckpointedFunction {
private static final long serialVersionUID = -6272159445203409112L;
@@ -766,22 +764,6 @@ public final void snapshotState(FunctionSnapshotContext context) throws Exceptio
}
}
- @Override
- public final void restoreState(HashMap restoredOffsets) {
- LOG.info("{} (taskIdx={}) restoring offsets from an older version: {}",
- getClass().getSimpleName(), getRuntimeContext().getIndexOfThisSubtask(), restoredOffsets);
-
- restoredFromOldState = true;
-
- if (restoredOffsets.size() > 0 && discoveryIntervalMillis != PARTITION_DISCOVERY_DISABLED) {
- throw new IllegalArgumentException(
- "Topic / partition discovery cannot be enabled if the job is restored from a savepoint from Flink 1.1.x.");
- }
-
- restoredState = new TreeMap<>(new KafkaTopicPartition.Comparator());
- restoredState.putAll(restoredOffsets);
- }
-
@Override
public final void notifyCheckpointComplete(long checkpointId) throws Exception {
if (!running) {
diff --git a/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/AbstractPartitionDiscoverer.java b/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/AbstractPartitionDiscoverer.java
index 39645be129f3d..b336fdc57061b 100644
--- a/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/AbstractPartitionDiscoverer.java
+++ b/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/AbstractPartitionDiscoverer.java
@@ -17,10 +17,10 @@
package org.apache.flink.streaming.connectors.kafka.internals;
-import java.util.HashMap;
+import java.util.HashSet;
import java.util.Iterator;
import java.util.List;
-import java.util.Map;
+import java.util.Set;
import static org.apache.flink.util.Preconditions.checkNotNull;
@@ -68,7 +68,7 @@ public abstract class AbstractPartitionDiscoverer {
* to keep track of only the largest partition id because Kafka partition numbers are only
* allowed to be increased and has incremental ids.
*/
- private Map topicsToLargestDiscoveredPartitionId;
+ private Set discoveredPartitions;
public AbstractPartitionDiscoverer(
KafkaTopicsDescriptor topicsDescriptor,
@@ -78,7 +78,7 @@ public AbstractPartitionDiscoverer(
this.topicsDescriptor = checkNotNull(topicsDescriptor);
this.indexOfThisSubtask = indexOfThisSubtask;
this.numParallelSubtasks = numParallelSubtasks;
- this.topicsToLargestDiscoveredPartitionId = new HashMap<>();
+ this.discoveredPartitions = new HashSet<>();
}
/**
@@ -149,10 +149,6 @@ public List discoverPartitions() throws WakeupException, Cl
if (newDiscoveredPartitions == null || newDiscoveredPartitions.isEmpty()) {
throw new RuntimeException("Unable to retrieve any partitions with KafkaTopicsDescriptor: " + topicsDescriptor);
} else {
- // sort so that we make sure the topicsToLargestDiscoveredPartitionId state is updated
- // with incremental partition ids of the same topics (otherwise some partition ids may be skipped)
- KafkaTopicPartition.sort(newDiscoveredPartitions);
-
Iterator iter = newDiscoveredPartitions.iterator();
KafkaTopicPartition nextPartition;
while (iter.hasNext()) {
@@ -196,7 +192,7 @@ public List discoverPartitions() throws WakeupException, Cl
*/
public boolean setAndCheckDiscoveredPartition(KafkaTopicPartition partition) {
if (isUndiscoveredPartition(partition)) {
- topicsToLargestDiscoveredPartitionId.put(partition.getTopic(), partition.getPartition());
+ discoveredPartitions.add(partition);
return KafkaTopicPartitionAssigner.assign(partition, numParallelSubtasks) == indexOfThisSubtask;
}
@@ -246,11 +242,6 @@ public static final class ClosedException extends Exception {
}
private boolean isUndiscoveredPartition(KafkaTopicPartition partition) {
- return !topicsToLargestDiscoveredPartitionId.containsKey(partition.getTopic())
- || partition.getPartition() > topicsToLargestDiscoveredPartitionId.get(partition.getTopic());
- }
-
- public static boolean shouldAssignToThisSubtask(KafkaTopicPartition partition, int indexOfThisSubtask, int numParallelSubtasks) {
- return Math.abs(partition.hashCode() % numParallelSubtasks) == indexOfThisSubtask;
+ return !discoveredPartitions.contains(partition);
}
}
diff --git a/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/KafkaTopicPartition.java b/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/KafkaTopicPartition.java
index 3500cd81edc3f..d35d5856f15ef 100644
--- a/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/KafkaTopicPartition.java
+++ b/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/KafkaTopicPartition.java
@@ -19,7 +19,6 @@
import java.io.Serializable;
import java.util.ArrayList;
-import java.util.Collections;
import java.util.List;
import java.util.Map;
@@ -131,8 +130,4 @@ public int compare(KafkaTopicPartition p1, KafkaTopicPartition p2) {
}
}
}
-
- public static void sort(List partitions) {
- Collections.sort(partitions, new Comparator());
- }
}
diff --git a/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumerBaseMigrationTest.java b/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumerBaseMigrationTest.java
index e3f337ec671dd..84f0e388e1969 100644
--- a/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumerBaseMigrationTest.java
+++ b/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumerBaseMigrationTest.java
@@ -92,7 +92,7 @@ public class FlinkKafkaConsumerBaseMigrationTest {
@Parameterized.Parameters(name = "Migration Savepoint: {0}")
public static Collection parameters () {
- return Arrays.asList(MigrationVersion.v1_1, MigrationVersion.v1_2, MigrationVersion.v1_3);
+ return Arrays.asList(MigrationVersion.v1_2, MigrationVersion.v1_3);
}
public FlinkKafkaConsumerBaseMigrationTest(MigrationVersion testMigrateVersion) {
@@ -322,7 +322,7 @@ public void testRestore() throws Exception {
*/
@Test
public void testRestoreFailsWithNonEmptyPreFlink13StatesIfDiscoveryEnabled() throws Exception {
- assumeTrue(testMigrateVersion == MigrationVersion.v1_1 || testMigrateVersion == MigrationVersion.v1_2);
+ assumeTrue(testMigrateVersion == MigrationVersion.v1_3 || testMigrateVersion == MigrationVersion.v1_2);
final List partitions = new ArrayList<>(PARTITION_STATE.keySet());
diff --git a/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaProducerTestBase.java b/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaProducerTestBase.java
index 4a611039af4bf..000de5268fe22 100644
--- a/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaProducerTestBase.java
+++ b/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaProducerTestBase.java
@@ -45,13 +45,13 @@
import org.apache.flink.test.util.SuccessException;
import org.apache.flink.util.Preconditions;
-import com.google.common.collect.ImmutableSet;
import org.apache.kafka.clients.consumer.ConsumerRecord;
import org.junit.Test;
import java.io.Serializable;
import java.util.ArrayList;
import java.util.Collection;
+import java.util.Collections;
import java.util.HashMap;
import java.util.HashSet;
import java.util.List;
@@ -285,7 +285,7 @@ public int partition(Integer record, byte[] key, byte[] value, String targetTopi
properties,
topic,
partition,
- ImmutableSet.copyOf(getIntegersSequence(BrokerRestartingMapper.numElementsBeforeSnapshot)),
+ Collections.unmodifiableSet(new HashSet<>(getIntegersSequence(BrokerRestartingMapper.numElementsBeforeSnapshot))),
30000L);
deleteTestTopic(topic);
diff --git a/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/internals/AbstractPartitionDiscovererTest.java b/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/internals/AbstractPartitionDiscovererTest.java
index 2633b951ae1f8..e9f1537ed1be1 100644
--- a/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/internals/AbstractPartitionDiscovererTest.java
+++ b/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/internals/AbstractPartitionDiscovererTest.java
@@ -394,6 +394,41 @@ public void testDeterministicAssignmentWithDifferentFetchedPartitionOrdering() t
}
}
+ @Test
+ public void testNonContiguousPartitionIdDiscovery() throws Exception {
+ List mockGetAllPartitionsForTopicsReturn1 = Arrays.asList(
+ new KafkaTopicPartition("test-topic", 1),
+ new KafkaTopicPartition("test-topic", 4));
+
+ List mockGetAllPartitionsForTopicsReturn2 = Arrays.asList(
+ new KafkaTopicPartition("test-topic", 0),
+ new KafkaTopicPartition("test-topic", 1),
+ new KafkaTopicPartition("test-topic", 2),
+ new KafkaTopicPartition("test-topic", 3),
+ new KafkaTopicPartition("test-topic", 4));
+
+ TestPartitionDiscoverer partitionDiscoverer = new TestPartitionDiscoverer(
+ topicsDescriptor,
+ 0,
+ 1,
+ TestPartitionDiscoverer.createMockGetAllTopicsSequenceFromFixedReturn(Collections.singletonList("test-topic")),
+ // first metadata fetch has missing partitions that appears only in the second fetch;
+ // need to create new modifiable lists for each fetch, since internally Iterable.remove() is used.
+ Arrays.asList(new ArrayList<>(mockGetAllPartitionsForTopicsReturn1), new ArrayList<>(mockGetAllPartitionsForTopicsReturn2)));
+ partitionDiscoverer.open();
+
+ List discoveredPartitions1 = partitionDiscoverer.discoverPartitions();
+ assertEquals(2, discoveredPartitions1.size());
+ assertTrue(discoveredPartitions1.contains(new KafkaTopicPartition("test-topic", 1)));
+ assertTrue(discoveredPartitions1.contains(new KafkaTopicPartition("test-topic", 4)));
+
+ List discoveredPartitions2 = partitionDiscoverer.discoverPartitions();
+ assertEquals(3, discoveredPartitions2.size());
+ assertTrue(discoveredPartitions2.contains(new KafkaTopicPartition("test-topic", 0)));
+ assertTrue(discoveredPartitions2.contains(new KafkaTopicPartition("test-topic", 2)));
+ assertTrue(discoveredPartitions2.contains(new KafkaTopicPartition("test-topic", 3)));
+ }
+
private boolean contains(List partitions, int partition) {
for (KafkaTopicPartition ktp : partitions) {
if (ktp.getPartition() == partition) {
diff --git a/flink-connectors/flink-connector-kinesis/pom.xml b/flink-connectors/flink-connector-kinesis/pom.xml
index 41daaa7f6ca95..83934f64608be 100644
--- a/flink-connectors/flink-connector-kinesis/pom.xml
+++ b/flink-connectors/flink-connector-kinesis/pom.xml
@@ -33,9 +33,9 @@ under the License.
flink-connector-kinesis_${scala.binary.version}
flink-connector-kinesis
- 1.10.71
- 1.6.2
- 0.10.2
+ 1.11.171
+ 1.8.1
+ 0.12.5
jar
diff --git a/flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/FlinkKinesisConsumer.java b/flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/FlinkKinesisConsumer.java
index d127f2b6fb710..a3681eca52fee 100644
--- a/flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/FlinkKinesisConsumer.java
+++ b/flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/FlinkKinesisConsumer.java
@@ -28,13 +28,11 @@
import org.apache.flink.runtime.state.FunctionInitializationContext;
import org.apache.flink.runtime.state.FunctionSnapshotContext;
import org.apache.flink.streaming.api.checkpoint.CheckpointedFunction;
-import org.apache.flink.streaming.api.checkpoint.CheckpointedRestoring;
import org.apache.flink.streaming.api.functions.source.RichParallelSourceFunction;
import org.apache.flink.streaming.api.functions.source.SourceFunction;
import org.apache.flink.streaming.connectors.kinesis.config.ConsumerConfigConstants;
import org.apache.flink.streaming.connectors.kinesis.config.ConsumerConfigConstants.InitialPosition;
import org.apache.flink.streaming.connectors.kinesis.internals.KinesisDataFetcher;
-import org.apache.flink.streaming.connectors.kinesis.model.KinesisStreamShard;
import org.apache.flink.streaming.connectors.kinesis.model.KinesisStreamShardState;
import org.apache.flink.streaming.connectors.kinesis.model.SentinelSequenceNumber;
import org.apache.flink.streaming.connectors.kinesis.model.SequenceNumber;
@@ -44,6 +42,7 @@
import org.apache.flink.streaming.connectors.kinesis.serialization.KinesisDeserializationSchemaWrapper;
import org.apache.flink.streaming.connectors.kinesis.util.KinesisConfigUtil;
import org.apache.flink.streaming.util.serialization.DeserializationSchema;
+import org.apache.flink.util.InstantiationUtil;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@@ -72,8 +71,7 @@
*/
public class FlinkKinesisConsumer extends RichParallelSourceFunction implements
ResultTypeQueryable,
- CheckpointedFunction,
- CheckpointedRestoring> {
+ CheckpointedFunction {
private static final long serialVersionUID = 4724006128720664870L;
@@ -176,7 +174,12 @@ public FlinkKinesisConsumer(List streams, KinesisDeserializationSchema entry : lastStateSnapshot.entrySet()) {
@@ -362,23 +365,6 @@ public void snapshotState(FunctionSnapshotContext context) throws Exception {
}
}
- @Override
- public void restoreState(HashMap restoredState) throws Exception {
- LOG.info("Subtask {} restoring offsets from an older Flink version: {}",
- getRuntimeContext().getIndexOfThisSubtask(), sequenceNumsToRestore);
-
- if (restoredState.isEmpty()) {
- sequenceNumsToRestore = null;
- } else {
- sequenceNumsToRestore = new HashMap<>();
- for (Map.Entry stateEntry : restoredState.entrySet()) {
- sequenceNumsToRestore.put(
- KinesisStreamShard.convertToStreamShardMetadata(stateEntry.getKey()),
- stateEntry.getValue());
- }
- }
- }
-
/** This method is exposed for tests that need to mock the KinesisDataFetcher in the consumer. */
protected KinesisDataFetcher createFetcher(
List streams,
diff --git a/flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/FlinkKinesisProducer.java b/flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/FlinkKinesisProducer.java
index 04d7055ca8230..1f5e64c1fade3 100644
--- a/flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/FlinkKinesisProducer.java
+++ b/flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/FlinkKinesisProducer.java
@@ -17,15 +17,13 @@
package org.apache.flink.streaming.connectors.kinesis;
-import org.apache.flink.api.java.ClosureCleaner;
import org.apache.flink.configuration.Configuration;
import org.apache.flink.streaming.api.functions.sink.RichSinkFunction;
-import org.apache.flink.streaming.connectors.kinesis.config.ProducerConfigConstants;
import org.apache.flink.streaming.connectors.kinesis.serialization.KinesisSerializationSchema;
import org.apache.flink.streaming.connectors.kinesis.util.AWSUtil;
import org.apache.flink.streaming.connectors.kinesis.util.KinesisConfigUtil;
import org.apache.flink.streaming.util.serialization.SerializationSchema;
-import org.apache.flink.util.PropertiesUtil;
+import org.apache.flink.util.InstantiationUtil;
import com.amazonaws.services.kinesis.producer.Attempt;
import com.amazonaws.services.kinesis.producer.KinesisProducer;
@@ -35,14 +33,15 @@
import com.google.common.util.concurrent.FutureCallback;
import com.google.common.util.concurrent.Futures;
import com.google.common.util.concurrent.ListenableFuture;
+
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import java.nio.ByteBuffer;
import java.util.List;
-import java.util.Objects;
import java.util.Properties;
+import static org.apache.flink.util.Preconditions.checkArgument;
import static org.apache.flink.util.Preconditions.checkNotNull;
/**
@@ -90,7 +89,7 @@ public class FlinkKinesisProducer extends RichSinkFunction {
* This is a constructor supporting Flink's {@see SerializationSchema}.
*
* @param schema Serialization schema for the data type
- * @param configProps The properties used to configure AWS credentials and AWS region
+ * @param configProps The properties used to configure KinesisProducer, including AWS credentials and AWS region
*/
public FlinkKinesisProducer(final SerializationSchema schema, Properties configProps) {
@@ -115,15 +114,17 @@ public String getTargetStream(OUT element) {
* This is a constructor supporting {@see KinesisSerializationSchema}.
*
* @param schema Kinesis serialization schema for the data type
- * @param configProps The properties used to configure AWS credentials and AWS region
+ * @param configProps The properties used to configure KinesisProducer, including AWS credentials and AWS region
*/
public FlinkKinesisProducer(KinesisSerializationSchema schema, Properties configProps) {
- this.configProps = checkNotNull(configProps, "configProps can not be null");
-
- // check the configuration properties for any conflicting settings
- KinesisConfigUtil.validateProducerConfiguration(this.configProps);
-
- ClosureCleaner.ensureSerializable(Objects.requireNonNull(schema));
+ checkNotNull(configProps, "configProps can not be null");
+ this.configProps = KinesisConfigUtil.replaceDeprecatedProducerKeys(configProps);
+
+ checkNotNull(schema, "serialization schema cannot be null");
+ checkArgument(
+ InstantiationUtil.isSerializable(schema),
+ "The provided serialization schema is not serializable: " + schema.getClass().getName() + ". " +
+ "Please check that it does not contain references to non-serializable instances.");
this.schema = schema;
}
@@ -154,8 +155,12 @@ public void setDefaultPartition(String defaultPartition) {
}
public void setCustomPartitioner(KinesisPartitioner partitioner) {
- Objects.requireNonNull(partitioner);
- ClosureCleaner.ensureSerializable(partitioner);
+ checkNotNull(partitioner, "partitioner cannot be null");
+ checkArgument(
+ InstantiationUtil.isSerializable(partitioner),
+ "The provided custom partitioner is not serializable: " + partitioner.getClass().getName() + ". " +
+ "Please check that it does not contain references to non-serializable instances.");
+
this.customPartitioner = partitioner;
}
@@ -165,18 +170,9 @@ public void setCustomPartitioner(KinesisPartitioner partitioner) {
public void open(Configuration parameters) throws Exception {
super.open(parameters);
- KinesisProducerConfiguration producerConfig = new KinesisProducerConfiguration();
-
- producerConfig.setRegion(configProps.getProperty(ProducerConfigConstants.AWS_REGION));
+ // check and pass the configuration properties
+ KinesisProducerConfiguration producerConfig = KinesisConfigUtil.validateProducerConfiguration(configProps);
producerConfig.setCredentialsProvider(AWSUtil.getCredentialsProvider(configProps));
- if (configProps.containsKey(ProducerConfigConstants.COLLECTION_MAX_COUNT)) {
- producerConfig.setCollectionMaxCount(PropertiesUtil.getLong(configProps,
- ProducerConfigConstants.COLLECTION_MAX_COUNT, producerConfig.getCollectionMaxCount(), LOG));
- }
- if (configProps.containsKey(ProducerConfigConstants.AGGREGATION_MAX_COUNT)) {
- producerConfig.setAggregationMaxCount(PropertiesUtil.getLong(configProps,
- ProducerConfigConstants.AGGREGATION_MAX_COUNT, producerConfig.getAggregationMaxCount(), LOG));
- }
producer = new KinesisProducer(producerConfig);
callback = new FutureCallback() {
diff --git a/flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/config/ProducerConfigConstants.java b/flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/config/ProducerConfigConstants.java
index d131150b1697b..d66bb90f9c8a1 100644
--- a/flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/config/ProducerConfigConstants.java
+++ b/flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/config/ProducerConfigConstants.java
@@ -21,13 +21,30 @@
/**
* Optional producer specific configuration keys for {@link FlinkKinesisProducer}.
+ *
+ * @deprecated This class is deprecated in favor of the official AWS Kinesis producer configuration keys.
+ * See
+ * here for the full list of available configs.
+ * For configuring the region and credentials, please use the keys in {@link AWSConfigConstants}.
*/
+@Deprecated
public class ProducerConfigConstants extends AWSConfigConstants {
- /** Maximum number of items to pack into an PutRecords request. **/
+ /**
+ * Deprecated key.
+ *
+ * @deprecated This is deprecated in favor of the official AWS Kinesis producer configuration keys.
+ * Please use {@code CollectionMaxCount} instead.
+ **/
+ @Deprecated
public static final String COLLECTION_MAX_COUNT = "aws.producer.collectionMaxCount";
- /** Maximum number of items to pack into an aggregated record. **/
+ /**
+ * Deprecated key.
+ *
+ * @deprecated This is deprecated in favor of the official AWS Kinesis producer configuration keys.
+ * Please use {@code AggregationMaxCount} instead.
+ **/
+ @Deprecated
public static final String AGGREGATION_MAX_COUNT = "aws.producer.aggregationMaxCount";
-
}
diff --git a/flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/examples/ProduceIntoKinesis.java b/flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/examples/ProduceIntoKinesis.java
index ee031eb80b1b9..8d21c2caa1f91 100644
--- a/flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/examples/ProduceIntoKinesis.java
+++ b/flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/examples/ProduceIntoKinesis.java
@@ -22,7 +22,7 @@
import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
import org.apache.flink.streaming.api.functions.source.SourceFunction;
import org.apache.flink.streaming.connectors.kinesis.FlinkKinesisProducer;
-import org.apache.flink.streaming.connectors.kinesis.config.ProducerConfigConstants;
+import org.apache.flink.streaming.connectors.kinesis.config.AWSConfigConstants;
import org.apache.flink.streaming.util.serialization.SimpleStringSchema;
import org.apache.commons.lang3.RandomStringUtils;
@@ -43,9 +43,9 @@ public static void main(String[] args) throws Exception {
DataStream simpleStringStream = see.addSource(new EventsGenerator());
Properties kinesisProducerConfig = new Properties();
- kinesisProducerConfig.setProperty(ProducerConfigConstants.AWS_REGION, pt.getRequired("region"));
- kinesisProducerConfig.setProperty(ProducerConfigConstants.AWS_ACCESS_KEY_ID, pt.getRequired("accessKey"));
- kinesisProducerConfig.setProperty(ProducerConfigConstants.AWS_SECRET_ACCESS_KEY, pt.getRequired("secretKey"));
+ kinesisProducerConfig.setProperty(AWSConfigConstants.AWS_REGION, pt.getRequired("region"));
+ kinesisProducerConfig.setProperty(AWSConfigConstants.AWS_ACCESS_KEY_ID, pt.getRequired("accessKey"));
+ kinesisProducerConfig.setProperty(AWSConfigConstants.AWS_SECRET_ACCESS_KEY, pt.getRequired("secretKey"));
FlinkKinesisProducer kinesis = new FlinkKinesisProducer<>(
new SimpleStringSchema(), kinesisProducerConfig);
diff --git a/flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/util/KinesisConfigUtil.java b/flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/util/KinesisConfigUtil.java
index 42f1af055ad82..997191c464f68 100644
--- a/flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/util/KinesisConfigUtil.java
+++ b/flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/util/KinesisConfigUtil.java
@@ -26,6 +26,7 @@
import org.apache.flink.streaming.connectors.kinesis.config.ProducerConfigConstants;
import com.amazonaws.regions.Regions;
+import com.amazonaws.services.kinesis.producer.KinesisProducerConfiguration;
import java.text.ParseException;
import java.text.SimpleDateFormat;
@@ -38,6 +39,22 @@
* Utilities for Flink Kinesis connector configuration.
*/
public class KinesisConfigUtil {
+
+ /** Maximum number of items to pack into an PutRecords request. **/
+ protected static final String COLLECTION_MAX_COUNT = "CollectionMaxCount";
+
+ /** Maximum number of items to pack into an aggregated record. **/
+ protected static final String AGGREGATION_MAX_COUNT = "AggregationMaxCount";
+
+ /** Limits the maximum allowed put rate for a shard, as a percentage of the backend limits.
+ * The default value is set as 100% in Flink. KPL's default value is 150% but it makes KPL throw
+ * RateLimitExceededException too frequently and breaks Flink sink as a result.
+ **/
+ private static final String RATE_LIMIT = "RateLimit";
+
+ /** Default values for RateLimit. **/
+ private static final String DEFAULT_RATE_LIMIT = "100";
+
/**
* Validate configuration properties for {@link FlinkKinesisConsumer}.
*/
@@ -126,19 +143,40 @@ public static void validateConsumerConfiguration(Properties config) {
}
}
+ /**
+ * Replace deprecated configuration properties for {@link FlinkKinesisProducer}.
+ * This should be remove along with deprecated keys
+ */
+ public static Properties replaceDeprecatedProducerKeys(Properties configProps) {
+ // Replace deprecated key
+ if (configProps.containsKey(ProducerConfigConstants.COLLECTION_MAX_COUNT)) {
+ configProps.setProperty(COLLECTION_MAX_COUNT,
+ configProps.getProperty(ProducerConfigConstants.COLLECTION_MAX_COUNT));
+ configProps.remove(ProducerConfigConstants.COLLECTION_MAX_COUNT);
+ }
+ // Replace deprecated key
+ if (configProps.containsKey(ProducerConfigConstants.AGGREGATION_MAX_COUNT)) {
+ configProps.setProperty(AGGREGATION_MAX_COUNT,
+ configProps.getProperty(ProducerConfigConstants.AGGREGATION_MAX_COUNT));
+ configProps.remove(ProducerConfigConstants.AGGREGATION_MAX_COUNT);
+ }
+ return configProps;
+ }
+
/**
* Validate configuration properties for {@link FlinkKinesisProducer}.
*/
- public static void validateProducerConfiguration(Properties config) {
+ public static KinesisProducerConfiguration validateProducerConfiguration(Properties config) {
checkNotNull(config, "config can not be null");
validateAwsConfiguration(config);
- validateOptionalPositiveLongProperty(config, ProducerConfigConstants.COLLECTION_MAX_COUNT,
- "Invalid value given for maximum number of items to pack into a PutRecords request. Must be a valid non-negative long value.");
+ // Override KPL default value if it's not specified by user
+ if (!config.containsKey(RATE_LIMIT)) {
+ config.setProperty(RATE_LIMIT, DEFAULT_RATE_LIMIT);
+ }
- validateOptionalPositiveLongProperty(config, ProducerConfigConstants.AGGREGATION_MAX_COUNT,
- "Invalid value given for maximum number of items to pack into an aggregated record. Must be a valid non-negative long value.");
+ return KinesisProducerConfiguration.fromProperties(config);
}
/**
diff --git a/flink-connectors/flink-connector-kinesis/src/test/java/org/apache/flink/streaming/connectors/kinesis/FlinkKinesisConsumerMigrationTest.java b/flink-connectors/flink-connector-kinesis/src/test/java/org/apache/flink/streaming/connectors/kinesis/FlinkKinesisConsumerMigrationTest.java
index af84420e6e3eb..364560c40ddd3 100644
--- a/flink-connectors/flink-connector-kinesis/src/test/java/org/apache/flink/streaming/connectors/kinesis/FlinkKinesisConsumerMigrationTest.java
+++ b/flink-connectors/flink-connector-kinesis/src/test/java/org/apache/flink/streaming/connectors/kinesis/FlinkKinesisConsumerMigrationTest.java
@@ -18,100 +18,133 @@
package org.apache.flink.streaming.connectors.kinesis;
import org.apache.flink.api.common.functions.RuntimeContext;
+import org.apache.flink.core.testutils.OneShotLatch;
import org.apache.flink.streaming.api.TimeCharacteristic;
import org.apache.flink.streaming.api.functions.source.SourceFunction;
import org.apache.flink.streaming.api.operators.StreamSource;
-import org.apache.flink.streaming.connectors.kinesis.config.ConsumerConfigConstants;
+import org.apache.flink.streaming.connectors.kinesis.config.AWSConfigConstants;
import org.apache.flink.streaming.connectors.kinesis.internals.KinesisDataFetcher;
-import org.apache.flink.streaming.connectors.kinesis.model.KinesisStreamShard;
import org.apache.flink.streaming.connectors.kinesis.model.SequenceNumber;
import org.apache.flink.streaming.connectors.kinesis.model.StreamShardMetadata;
import org.apache.flink.streaming.connectors.kinesis.serialization.KinesisDeserializationSchema;
import org.apache.flink.streaming.connectors.kinesis.testutils.KinesisShardIdGenerator;
+import org.apache.flink.streaming.runtime.tasks.OperatorStateHandles;
import org.apache.flink.streaming.util.AbstractStreamOperatorTestHarness;
+import org.apache.flink.streaming.util.OperatorSnapshotUtil;
+import org.apache.flink.streaming.util.migration.MigrationTestUtil;
+import org.apache.flink.streaming.util.migration.MigrationVersion;
-import com.amazonaws.services.kinesis.model.Shard;
+import org.junit.Ignore;
import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+import org.mockito.invocation.InvocationOnMock;
+import org.mockito.stubbing.Answer;
-import java.net.URL;
+import java.util.Arrays;
+import java.util.Collection;
import java.util.HashMap;
import java.util.List;
import java.util.Properties;
+import java.util.concurrent.atomic.AtomicReference;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertNotEquals;
+import static org.junit.Assert.assertTrue;
+import static org.mockito.Mockito.doAnswer;
import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
/**
* Tests for checking whether {@link FlinkKinesisConsumer} can restore from snapshots that were
- * done using the Flink 1.1 {@code FlinkKinesisConsumer}.
+ * done using an older {@code FlinkKinesisConsumer}.
+ *
+ * For regenerating the binary snapshot files run {@link #writeSnapshot()} on the corresponding
+ * Flink release-* branch.
*/
+@RunWith(Parameterized.class)
public class FlinkKinesisConsumerMigrationTest {
+ /**
+ * TODO change this to the corresponding savepoint version to be written (e.g. {@link MigrationVersion#v1_3} for 1.3)
+ * TODO and remove all @Ignore annotations on the writeSnapshot() method to generate savepoints
+ */
+ private final MigrationVersion flinkGenerateSavepointVersion = null;
+
+ private static final HashMap TEST_STATE = new HashMap<>();
+ static {
+ StreamShardMetadata shardMetadata = new StreamShardMetadata();
+ shardMetadata.setStreamName("fakeStream1");
+ shardMetadata.setShardId(KinesisShardIdGenerator.generateFromShardOrder(0));
+
+ TEST_STATE.put(shardMetadata, new SequenceNumber("987654321"));
+ }
+
+ private final MigrationVersion testMigrateVersion;
+
+ @Parameterized.Parameters(name = "Migration Savepoint: {0}")
+ public static Collection parameters () {
+ return Arrays.asList(MigrationVersion.v1_3);
+ }
+
+ public FlinkKinesisConsumerMigrationTest(MigrationVersion testMigrateVersion) {
+ this.testMigrateVersion = testMigrateVersion;
+ }
+
+ /**
+ * Manually run this to write binary snapshot data.
+ */
+ @Ignore
@Test
- public void testRestoreFromFlink11WithEmptyState() throws Exception {
- Properties testConfig = new Properties();
- testConfig.setProperty(ConsumerConfigConstants.AWS_REGION, "us-east-1");
- testConfig.setProperty(ConsumerConfigConstants.AWS_CREDENTIALS_PROVIDER, "BASIC");
- testConfig.setProperty(ConsumerConfigConstants.AWS_ACCESS_KEY_ID, "accessKeyId");
- testConfig.setProperty(ConsumerConfigConstants.AWS_SECRET_ACCESS_KEY, "secretKey");
+ public void writeSnapshot() throws Exception {
+ writeSnapshot("src/test/resources/kinesis-consumer-migration-test-flink" + flinkGenerateSavepointVersion + "-snapshot", TEST_STATE);
- final DummyFlinkKafkaConsumer consumerFunction = new DummyFlinkKafkaConsumer<>(testConfig);
+ // write empty state snapshot
+ writeSnapshot("src/test/resources/kinesis-consumer-migration-test-flink" + flinkGenerateSavepointVersion + "-empty-snapshot", new HashMap<>());
+ }
+
+ @Test
+ public void testRestoreWithEmptyState() throws Exception {
+ final DummyFlinkKinesisConsumer consumerFunction = new DummyFlinkKinesisConsumer<>(mock(KinesisDataFetcher.class));
- StreamSource> consumerOperator = new StreamSource<>(consumerFunction);
+ StreamSource> consumerOperator = new StreamSource<>(consumerFunction);
final AbstractStreamOperatorTestHarness testHarness =
new AbstractStreamOperatorTestHarness<>(consumerOperator, 1, 1, 0);
- testHarness.setTimeCharacteristic(TimeCharacteristic.ProcessingTime);
-
testHarness.setup();
- // restore state from binary snapshot file using legacy method
- testHarness.initializeStateFromLegacyCheckpoint(
- getResourceFilename("kinesis-consumer-migration-test-flink1.1-empty-snapshot"));
+ MigrationTestUtil.restoreFromSnapshot(
+ testHarness,
+ "src/test/resources/kinesis-consumer-migration-test-flink" + testMigrateVersion + "-empty-snapshot", testMigrateVersion);
testHarness.open();
// assert that no state was restored
- assertEquals(null, consumerFunction.getRestoredState());
+ assertTrue(consumerFunction.getRestoredState().isEmpty());
consumerOperator.close();
consumerOperator.cancel();
}
@Test
- public void testRestoreFromFlink11() throws Exception {
- Properties testConfig = new Properties();
- testConfig.setProperty(ConsumerConfigConstants.AWS_REGION, "us-east-1");
- testConfig.setProperty(ConsumerConfigConstants.AWS_CREDENTIALS_PROVIDER, "BASIC");
- testConfig.setProperty(ConsumerConfigConstants.AWS_ACCESS_KEY_ID, "accessKeyId");
- testConfig.setProperty(ConsumerConfigConstants.AWS_SECRET_ACCESS_KEY, "secretKey");
+ public void testRestore() throws Exception {
+ final DummyFlinkKinesisConsumer consumerFunction = new DummyFlinkKinesisConsumer<>(mock(KinesisDataFetcher.class));
- final DummyFlinkKafkaConsumer consumerFunction = new DummyFlinkKafkaConsumer<>(testConfig);
-
- StreamSource> consumerOperator =
+ StreamSource> consumerOperator =
new StreamSource<>(consumerFunction);
final AbstractStreamOperatorTestHarness testHarness =
new AbstractStreamOperatorTestHarness<>(consumerOperator, 1, 1, 0);
- testHarness.setTimeCharacteristic(TimeCharacteristic.ProcessingTime);
-
testHarness.setup();
- // restore state from binary snapshot file using legacy method
- testHarness.initializeStateFromLegacyCheckpoint(
- getResourceFilename("kinesis-consumer-migration-test-flink1.1-snapshot"));
+ MigrationTestUtil.restoreFromSnapshot(
+ testHarness,
+ "src/test/resources/kinesis-consumer-migration-test-flink" + testMigrateVersion + "-snapshot", testMigrateVersion);
testHarness.open();
- // the expected state in "kafka-consumer-migration-test-flink1.1-snapshot"
- final HashMap expectedState = new HashMap<>();
- expectedState.put(KinesisStreamShard.convertToStreamShardMetadata(new KinesisStreamShard("fakeStream1",
- new Shard().withShardId(KinesisShardIdGenerator.generateFromShardOrder(0)))),
- new SequenceNumber("987654321"));
-
- // assert that state is correctly restored from legacy checkpoint
+ // assert that state is correctly restored
assertNotEquals(null, consumerFunction.getRestoredState());
assertEquals(1, consumerFunction.getRestoredState().size());
- assertEquals(expectedState, consumerFunction.getRestoredState());
+ assertEquals(TEST_STATE, consumerFunction.getRestoredState());
consumerOperator.close();
consumerOperator.cancel();
@@ -119,31 +152,87 @@ public void testRestoreFromFlink11() throws Exception {
// ------------------------------------------------------------------------
- private static String getResourceFilename(String filename) {
- ClassLoader cl = FlinkKinesisConsumerMigrationTest.class.getClassLoader();
- URL resource = cl.getResource(filename);
- if (resource == null) {
- throw new NullPointerException("Missing snapshot resource.");
+ @SuppressWarnings("unchecked")
+ private void writeSnapshot(String path, HashMap state) throws Exception {
+ final OneShotLatch latch = new OneShotLatch();
+
+ final KinesisDataFetcher fetcher = mock(KinesisDataFetcher.class);
+ doAnswer(new Answer() {
+ @Override
+ public Object answer(InvocationOnMock invocationOnMock) throws Throwable {
+ latch.trigger();
+ return null;
+ }
+ }).when(fetcher).runFetcher();
+ when(fetcher.snapshotState()).thenReturn(state);
+
+ final DummyFlinkKinesisConsumer consumer = new DummyFlinkKinesisConsumer<>(fetcher);
+
+ StreamSource> consumerOperator = new StreamSource<>(consumer);
+
+ final AbstractStreamOperatorTestHarness testHarness =
+ new AbstractStreamOperatorTestHarness<>(consumerOperator, 1, 1, 0);
+
+ testHarness.setTimeCharacteristic(TimeCharacteristic.ProcessingTime);
+
+ testHarness.setup();
+ testHarness.open();
+
+ final AtomicReference error = new AtomicReference<>();
+
+ // run the source asynchronously
+ Thread runner = new Thread() {
+ @Override
+ public void run() {
+ try {
+ consumer.run(mock(SourceFunction.SourceContext.class));
+ } catch (Throwable t) {
+ t.printStackTrace();
+ error.set(t);
+ }
+ }
+ };
+ runner.start();
+
+ if (!latch.isTriggered()) {
+ latch.await();
+ }
+
+ final OperatorStateHandles snapshot;
+ synchronized (testHarness.getCheckpointLock()) {
+ snapshot = testHarness.snapshot(0L, 0L);
}
- return resource.getFile();
+
+ OperatorSnapshotUtil.writeStateHandle(snapshot, path);
+
+ consumerOperator.close();
+ runner.join();
}
- private static class DummyFlinkKafkaConsumer extends FlinkKinesisConsumer {
- private static final long serialVersionUID = 1L;
+ private static class DummyFlinkKinesisConsumer extends FlinkKinesisConsumer {
+
+ private KinesisDataFetcher mockFetcher;
+
+ private static Properties dummyConfig = new Properties();
+ static {
+ dummyConfig.setProperty(AWSConfigConstants.AWS_REGION, "us-east-1");
+ dummyConfig.setProperty(AWSConfigConstants.AWS_ACCESS_KEY_ID, "accessKeyId");
+ dummyConfig.setProperty(AWSConfigConstants.AWS_SECRET_ACCESS_KEY, "secretKey");
+ }
- @SuppressWarnings("unchecked")
- DummyFlinkKafkaConsumer(Properties properties) {
- super("test", mock(KinesisDeserializationSchema.class), properties);
+ DummyFlinkKinesisConsumer(KinesisDataFetcher mockFetcher) {
+ super("dummy-topic", mock(KinesisDeserializationSchema.class), dummyConfig);
+ this.mockFetcher = mockFetcher;
}
@Override
protected KinesisDataFetcher createFetcher(
List streams,
- SourceFunction.SourceContext sourceContext,
+ SourceContext sourceContext,
RuntimeContext runtimeContext,
Properties configProps,
- KinesisDeserializationSchema deserializationSchema) {
- return mock(KinesisDataFetcher.class);
+ KinesisDeserializationSchema deserializer) {
+ return mockFetcher;
}
}
}
diff --git a/flink-connectors/flink-connector-kinesis/src/test/java/org/apache/flink/streaming/connectors/kinesis/FlinkKinesisConsumerTest.java b/flink-connectors/flink-connector-kinesis/src/test/java/org/apache/flink/streaming/connectors/kinesis/FlinkKinesisConsumerTest.java
index a26e758e72687..4a007d5b2c009 100644
--- a/flink-connectors/flink-connector-kinesis/src/test/java/org/apache/flink/streaming/connectors/kinesis/FlinkKinesisConsumerTest.java
+++ b/flink-connectors/flink-connector-kinesis/src/test/java/org/apache/flink/streaming/connectors/kinesis/FlinkKinesisConsumerTest.java
@@ -22,6 +22,7 @@
import org.apache.flink.api.common.state.ListState;
import org.apache.flink.api.common.state.ListStateDescriptor;
import org.apache.flink.api.common.state.OperatorStateStore;
+import org.apache.flink.api.common.typeinfo.BasicTypeInfo;
import org.apache.flink.api.common.typeinfo.TypeInformation;
import org.apache.flink.api.java.tuple.Tuple2;
import org.apache.flink.api.java.typeutils.runtime.PojoSerializer;
@@ -32,7 +33,6 @@
import org.apache.flink.streaming.api.functions.source.SourceFunction;
import org.apache.flink.streaming.connectors.kinesis.config.AWSConfigConstants;
import org.apache.flink.streaming.connectors.kinesis.config.ConsumerConfigConstants;
-import org.apache.flink.streaming.connectors.kinesis.config.ProducerConfigConstants;
import org.apache.flink.streaming.connectors.kinesis.internals.KinesisDataFetcher;
import org.apache.flink.streaming.connectors.kinesis.model.KinesisStreamShard;
import org.apache.flink.streaming.connectors.kinesis.model.KinesisStreamShardState;
@@ -40,6 +40,7 @@
import org.apache.flink.streaming.connectors.kinesis.model.SequenceNumber;
import org.apache.flink.streaming.connectors.kinesis.model.StreamShardHandle;
import org.apache.flink.streaming.connectors.kinesis.model.StreamShardMetadata;
+import org.apache.flink.streaming.connectors.kinesis.serialization.KinesisDeserializationSchema;
import org.apache.flink.streaming.connectors.kinesis.testutils.KinesisShardIdGenerator;
import org.apache.flink.streaming.connectors.kinesis.testutils.TestableFlinkKinesisConsumer;
import org.apache.flink.streaming.connectors.kinesis.util.KinesisConfigUtil;
@@ -48,6 +49,7 @@
import com.amazonaws.services.kinesis.model.HashKeyRange;
import com.amazonaws.services.kinesis.model.SequenceNumberRange;
import com.amazonaws.services.kinesis.model.Shard;
+
import org.junit.Assert;
import org.junit.Rule;
import org.junit.Test;
@@ -60,6 +62,7 @@
import org.powermock.core.classloader.annotations.PrepareForTest;
import org.powermock.modules.junit4.PowerMockRunner;
+import java.io.IOException;
import java.util.ArrayList;
import java.util.HashMap;
import java.util.List;
@@ -501,38 +504,6 @@ public void testUnparsableLongForShardDiscoveryIntervalMillisInConfig() {
KinesisConfigUtil.validateConsumerConfiguration(testConfig);
}
- // ----------------------------------------------------------------------
- // FlinkKinesisConsumer.validateProducerConfiguration() tests
- // ----------------------------------------------------------------------
-
- @Test
- public void testUnparsableLongForCollectionMaxCountInConfig() {
- exception.expect(IllegalArgumentException.class);
- exception.expectMessage("Invalid value given for maximum number of items to pack into a PutRecords request");
-
- Properties testConfig = new Properties();
- testConfig.setProperty(ProducerConfigConstants.AWS_REGION, "us-east-1");
- testConfig.setProperty(ProducerConfigConstants.AWS_ACCESS_KEY_ID, "accessKeyId");
- testConfig.setProperty(ProducerConfigConstants.AWS_SECRET_ACCESS_KEY, "secretKey");
- testConfig.setProperty(ProducerConfigConstants.COLLECTION_MAX_COUNT, "unparsableLong");
-
- KinesisConfigUtil.validateProducerConfiguration(testConfig);
- }
-
- @Test
- public void testUnparsableLongForAggregationMaxCountInConfig() {
- exception.expect(IllegalArgumentException.class);
- exception.expectMessage("Invalid value given for maximum number of items to pack into an aggregated record");
-
- Properties testConfig = new Properties();
- testConfig.setProperty(ProducerConfigConstants.AWS_REGION, "us-east-1");
- testConfig.setProperty(ProducerConfigConstants.AWS_ACCESS_KEY_ID, "accessKeyId");
- testConfig.setProperty(ProducerConfigConstants.AWS_SECRET_ACCESS_KEY, "secretKey");
- testConfig.setProperty(ProducerConfigConstants.AGGREGATION_MAX_COUNT, "unparsableLong");
-
- KinesisConfigUtil.validateProducerConfiguration(testConfig);
- }
-
// ----------------------------------------------------------------------
// Tests related to state initialization
// ----------------------------------------------------------------------
@@ -710,38 +681,6 @@ public void testFetcherShouldNotBeRestoringFromFailureIfNotRestoringFromCheckpoi
consumer.run(Mockito.mock(SourceFunction.SourceContext.class));
}
- @Test
- @SuppressWarnings("unchecked")
- public void testFetcherShouldBeCorrectlySeededIfRestoringFromLegacyCheckpoint() throws Exception {
- HashMap fakeRestoredState = getFakeRestoredStore("all");
- HashMap legacyFakeRestoredState = new HashMap<>();
- for (Map.Entry kv : fakeRestoredState.entrySet()) {
- legacyFakeRestoredState.put(new KinesisStreamShard(kv.getKey().getStreamName(), kv.getKey().getShard()), kv.getValue());
- }
-
- KinesisDataFetcher mockedFetcher = Mockito.mock(KinesisDataFetcher.class);
- List shards = new ArrayList<>();
- shards.addAll(fakeRestoredState.keySet());
- when(mockedFetcher.discoverNewShardsToSubscribe()).thenReturn(shards);
- PowerMockito.whenNew(KinesisDataFetcher.class).withAnyArguments().thenReturn(mockedFetcher);
-
- // assume the given config is correct
- PowerMockito.mockStatic(KinesisConfigUtil.class);
- PowerMockito.doNothing().when(KinesisConfigUtil.class);
-
- TestableFlinkKinesisConsumer consumer = new TestableFlinkKinesisConsumer(
- "fakeStream", new Properties(), 10, 2);
- consumer.restoreState(legacyFakeRestoredState);
- consumer.open(new Configuration());
- consumer.run(Mockito.mock(SourceFunction.SourceContext.class));
-
- for (Map.Entry restoredShard : fakeRestoredState.entrySet()) {
- Mockito.verify(mockedFetcher).registerNewSubscribedShardState(
- new KinesisStreamShardState(KinesisDataFetcher.convertToStreamShardMetadata(restoredShard.getKey()),
- restoredShard.getKey(), restoredShard.getValue()));
- }
- }
-
@Test
@SuppressWarnings("unchecked")
public void testFetcherShouldBeCorrectlySeededIfRestoringFromCheckpoint() throws Exception {
@@ -1062,4 +1001,35 @@ private HashMap getFakeRestoredStore(String s
return fakeRestoredState;
}
+
+ /**
+ * A non-serializable {@link KinesisDeserializationSchema} (because it is a nested class with reference
+ * to the enclosing class, which is not serializable) used for testing.
+ */
+ private final class NonSerializableDeserializationSchema implements KinesisDeserializationSchema {
+ @Override
+ public String deserialize(byte[] recordValue, String partitionKey, String seqNum, long approxArrivalTimestamp, String stream, String shardId) throws IOException {
+ return new String(recordValue);
+ }
+
+ @Override
+ public TypeInformation getProducedType() {
+ return BasicTypeInfo.STRING_TYPE_INFO;
+ }
+ }
+
+ /**
+ * A static, serializable {@link KinesisDeserializationSchema}.
+ */
+ private static final class SerializableDeserializationSchema implements KinesisDeserializationSchema {
+ @Override
+ public String deserialize(byte[] recordValue, String partitionKey, String seqNum, long approxArrivalTimestamp, String stream, String shardId) throws IOException {
+ return new String(recordValue);
+ }
+
+ @Override
+ public TypeInformation getProducedType() {
+ return BasicTypeInfo.STRING_TYPE_INFO;
+ }
+ }
}
diff --git a/flink-connectors/flink-connector-kinesis/src/test/java/org/apache/flink/streaming/connectors/kinesis/FlinkKinesisProducerTest.java b/flink-connectors/flink-connector-kinesis/src/test/java/org/apache/flink/streaming/connectors/kinesis/FlinkKinesisProducerTest.java
new file mode 100644
index 0000000000000..ac03cfed0c898
--- /dev/null
+++ b/flink-connectors/flink-connector-kinesis/src/test/java/org/apache/flink/streaming/connectors/kinesis/FlinkKinesisProducerTest.java
@@ -0,0 +1,161 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.streaming.connectors.kinesis;
+
+import org.apache.flink.streaming.connectors.kinesis.config.AWSConfigConstants;
+import org.apache.flink.streaming.connectors.kinesis.serialization.KinesisSerializationSchema;
+import org.apache.flink.streaming.util.serialization.SimpleStringSchema;
+import org.apache.flink.util.InstantiationUtil;
+
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.ExpectedException;
+
+import java.nio.ByteBuffer;
+import java.util.Properties;
+
+import static org.junit.Assert.assertTrue;
+
+/**
+ * Suite of {@link FlinkKinesisProducer} tests.
+ */
+public class FlinkKinesisProducerTest {
+
+ @Rule
+ public ExpectedException exception = ExpectedException.none();
+
+ // ----------------------------------------------------------------------
+ // Tests to verify serializability
+ // ----------------------------------------------------------------------
+
+ @Test
+ public void testCreateWithNonSerializableDeserializerFails() {
+ exception.expect(IllegalArgumentException.class);
+ exception.expectMessage("The provided serialization schema is not serializable");
+
+ Properties testConfig = new Properties();
+ testConfig.setProperty(AWSConfigConstants.AWS_REGION, "us-east-1");
+ testConfig.setProperty(AWSConfigConstants.AWS_ACCESS_KEY_ID, "accessKeyId");
+ testConfig.setProperty(AWSConfigConstants.AWS_SECRET_ACCESS_KEY, "secretKey");
+
+ new FlinkKinesisProducer<>(new NonSerializableSerializationSchema(), testConfig);
+ }
+
+ @Test
+ public void testCreateWithSerializableDeserializer() {
+ Properties testConfig = new Properties();
+ testConfig.setProperty(AWSConfigConstants.AWS_REGION, "us-east-1");
+ testConfig.setProperty(AWSConfigConstants.AWS_ACCESS_KEY_ID, "accessKeyId");
+ testConfig.setProperty(AWSConfigConstants.AWS_SECRET_ACCESS_KEY, "secretKey");
+
+ new FlinkKinesisProducer<>(new SerializableSerializationSchema(), testConfig);
+ }
+
+ @Test
+ public void testConfigureWithNonSerializableCustomPartitionerFails() {
+ exception.expect(IllegalArgumentException.class);
+ exception.expectMessage("The provided custom partitioner is not serializable");
+
+ Properties testConfig = new Properties();
+ testConfig.setProperty(AWSConfigConstants.AWS_REGION, "us-east-1");
+ testConfig.setProperty(AWSConfigConstants.AWS_ACCESS_KEY_ID, "accessKeyId");
+ testConfig.setProperty(AWSConfigConstants.AWS_SECRET_ACCESS_KEY, "secretKey");
+
+ new FlinkKinesisProducer<>(new SimpleStringSchema(), testConfig)
+ .setCustomPartitioner(new NonSerializableCustomPartitioner());
+ }
+
+ @Test
+ public void testConfigureWithSerializableCustomPartitioner() {
+ Properties testConfig = new Properties();
+ testConfig.setProperty(AWSConfigConstants.AWS_REGION, "us-east-1");
+ testConfig.setProperty(AWSConfigConstants.AWS_ACCESS_KEY_ID, "accessKeyId");
+ testConfig.setProperty(AWSConfigConstants.AWS_SECRET_ACCESS_KEY, "secretKey");
+
+ new FlinkKinesisProducer<>(new SimpleStringSchema(), testConfig)
+ .setCustomPartitioner(new SerializableCustomPartitioner());
+ }
+
+ @Test
+ public void testConsumerIsSerializable() {
+ Properties testConfig = new Properties();
+ testConfig.setProperty(AWSConfigConstants.AWS_REGION, "us-east-1");
+ testConfig.setProperty(AWSConfigConstants.AWS_ACCESS_KEY_ID, "accessKeyId");
+ testConfig.setProperty(AWSConfigConstants.AWS_SECRET_ACCESS_KEY, "secretKey");
+
+ FlinkKinesisProducer consumer = new FlinkKinesisProducer<>(new SimpleStringSchema(), testConfig);
+ assertTrue(InstantiationUtil.isSerializable(consumer));
+ }
+
+ // ----------------------------------------------------------------------
+ // Utility test classes
+ // ----------------------------------------------------------------------
+
+ /**
+ * A non-serializable {@link KinesisSerializationSchema} (because it is a nested class with reference
+ * to the enclosing class, which is not serializable) used for testing.
+ */
+ private final class NonSerializableSerializationSchema implements KinesisSerializationSchema {
+ @Override
+ public ByteBuffer serialize(String element) {
+ return ByteBuffer.wrap(element.getBytes());
+ }
+
+ @Override
+ public String getTargetStream(String element) {
+ return "test-stream";
+ }
+ }
+
+ /**
+ * A static, serializable {@link KinesisSerializationSchema}.
+ */
+ private static final class SerializableSerializationSchema implements KinesisSerializationSchema {
+ @Override
+ public ByteBuffer serialize(String element) {
+ return ByteBuffer.wrap(element.getBytes());
+ }
+
+ @Override
+ public String getTargetStream(String element) {
+ return "test-stream";
+ }
+ }
+
+ /**
+ * A non-serializable {@link KinesisPartitioner} (because it is a nested class with reference
+ * to the enclosing class, which is not serializable) used for testing.
+ */
+ private final class NonSerializableCustomPartitioner extends KinesisPartitioner {
+ @Override
+ public String getPartitionId(String element) {
+ return "test-partition";
+ }
+ }
+
+ /**
+ * A static, serializable {@link KinesisPartitioner}.
+ */
+ private static final class SerializableCustomPartitioner extends KinesisPartitioner {
+ @Override
+ public String getPartitionId(String element) {
+ return "test-partition";
+ }
+ }
+}
diff --git a/flink-connectors/flink-connector-kinesis/src/test/java/org/apache/flink/streaming/connectors/kinesis/manualtests/ManualConsumerProducerTest.java b/flink-connectors/flink-connector-kinesis/src/test/java/org/apache/flink/streaming/connectors/kinesis/manualtests/ManualConsumerProducerTest.java
index 2915e2f6da1b3..a7470dc166e2f 100644
--- a/flink-connectors/flink-connector-kinesis/src/test/java/org/apache/flink/streaming/connectors/kinesis/manualtests/ManualConsumerProducerTest.java
+++ b/flink-connectors/flink-connector-kinesis/src/test/java/org/apache/flink/streaming/connectors/kinesis/manualtests/ManualConsumerProducerTest.java
@@ -25,8 +25,8 @@
import org.apache.flink.streaming.connectors.kinesis.FlinkKinesisConsumer;
import org.apache.flink.streaming.connectors.kinesis.FlinkKinesisProducer;
import org.apache.flink.streaming.connectors.kinesis.KinesisPartitioner;
+import org.apache.flink.streaming.connectors.kinesis.config.AWSConfigConstants;
import org.apache.flink.streaming.connectors.kinesis.config.ConsumerConfigConstants;
-import org.apache.flink.streaming.connectors.kinesis.config.ProducerConfigConstants;
import org.apache.flink.streaming.connectors.kinesis.examples.ProduceIntoKinesis;
import org.apache.flink.streaming.connectors.kinesis.serialization.KinesisSerializationSchema;
import org.apache.flink.streaming.util.serialization.SimpleStringSchema;
@@ -56,9 +56,9 @@ public static void main(String[] args) throws Exception {
DataStream simpleStringStream = see.addSource(new ProduceIntoKinesis.EventsGenerator());
Properties kinesisProducerConfig = new Properties();
- kinesisProducerConfig.setProperty(ProducerConfigConstants.AWS_REGION, pt.getRequired("region"));
- kinesisProducerConfig.setProperty(ProducerConfigConstants.AWS_ACCESS_KEY_ID, pt.getRequired("accessKey"));
- kinesisProducerConfig.setProperty(ProducerConfigConstants.AWS_SECRET_ACCESS_KEY, pt.getRequired("secretKey"));
+ kinesisProducerConfig.setProperty(AWSConfigConstants.AWS_REGION, pt.getRequired("region"));
+ kinesisProducerConfig.setProperty(AWSConfigConstants.AWS_ACCESS_KEY_ID, pt.getRequired("accessKey"));
+ kinesisProducerConfig.setProperty(AWSConfigConstants.AWS_SECRET_ACCESS_KEY, pt.getRequired("secretKey"));
FlinkKinesisProducer kinesis = new FlinkKinesisProducer<>(
new KinesisSerializationSchema() {
diff --git a/flink-connectors/flink-connector-kinesis/src/test/java/org/apache/flink/streaming/connectors/kinesis/manualtests/ManualProducerTest.java b/flink-connectors/flink-connector-kinesis/src/test/java/org/apache/flink/streaming/connectors/kinesis/manualtests/ManualProducerTest.java
index 8abf4bb2ef655..fb49169bc8110 100644
--- a/flink-connectors/flink-connector-kinesis/src/test/java/org/apache/flink/streaming/connectors/kinesis/manualtests/ManualProducerTest.java
+++ b/flink-connectors/flink-connector-kinesis/src/test/java/org/apache/flink/streaming/connectors/kinesis/manualtests/ManualProducerTest.java
@@ -23,7 +23,7 @@
import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
import org.apache.flink.streaming.connectors.kinesis.FlinkKinesisProducer;
import org.apache.flink.streaming.connectors.kinesis.KinesisPartitioner;
-import org.apache.flink.streaming.connectors.kinesis.config.ProducerConfigConstants;
+import org.apache.flink.streaming.connectors.kinesis.config.AWSConfigConstants;
import org.apache.flink.streaming.connectors.kinesis.examples.ProduceIntoKinesis;
import org.apache.flink.streaming.connectors.kinesis.serialization.KinesisSerializationSchema;
@@ -53,9 +53,9 @@ public static void main(String[] args) throws Exception {
DataStream simpleStringStream = see.addSource(new ProduceIntoKinesis.EventsGenerator());
Properties kinesisProducerConfig = new Properties();
- kinesisProducerConfig.setProperty(ProducerConfigConstants.AWS_REGION, pt.getRequired("region"));
- kinesisProducerConfig.setProperty(ProducerConfigConstants.AWS_ACCESS_KEY_ID, pt.getRequired("accessKey"));
- kinesisProducerConfig.setProperty(ProducerConfigConstants.AWS_SECRET_ACCESS_KEY, pt.getRequired("secretKey"));
+ kinesisProducerConfig.setProperty(AWSConfigConstants.AWS_REGION, pt.getRequired("region"));
+ kinesisProducerConfig.setProperty(AWSConfigConstants.AWS_ACCESS_KEY_ID, pt.getRequired("accessKey"));
+ kinesisProducerConfig.setProperty(AWSConfigConstants.AWS_SECRET_ACCESS_KEY, pt.getRequired("secretKey"));
FlinkKinesisProducer kinesis = new FlinkKinesisProducer<>(
new KinesisSerializationSchema() {
diff --git a/flink-connectors/flink-connector-kinesis/src/test/java/org/apache/flink/streaming/connectors/kinesis/testutils/ExactlyOnceValidatingConsumerThread.java b/flink-connectors/flink-connector-kinesis/src/test/java/org/apache/flink/streaming/connectors/kinesis/testutils/ExactlyOnceValidatingConsumerThread.java
index 75356efca8428..1336652226f13 100644
--- a/flink-connectors/flink-connector-kinesis/src/test/java/org/apache/flink/streaming/connectors/kinesis/testutils/ExactlyOnceValidatingConsumerThread.java
+++ b/flink-connectors/flink-connector-kinesis/src/test/java/org/apache/flink/streaming/connectors/kinesis/testutils/ExactlyOnceValidatingConsumerThread.java
@@ -21,7 +21,7 @@
import org.apache.flink.api.common.functions.RichFlatMapFunction;
import org.apache.flink.api.common.restartstrategy.RestartStrategies;
import org.apache.flink.configuration.Configuration;
-import org.apache.flink.streaming.api.checkpoint.Checkpointed;
+import org.apache.flink.streaming.api.checkpoint.ListCheckpointed;
import org.apache.flink.streaming.api.datastream.DataStream;
import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
import org.apache.flink.streaming.connectors.kinesis.FlinkKinesisConsumer;
@@ -29,11 +29,14 @@
import org.apache.flink.streaming.util.serialization.SimpleStringSchema;
import org.apache.flink.test.util.SuccessException;
import org.apache.flink.util.Collector;
+import org.apache.flink.util.Preconditions;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import java.util.BitSet;
+import java.util.Collections;
+import java.util.List;
import java.util.Properties;
import java.util.concurrent.atomic.AtomicReference;
@@ -95,7 +98,7 @@ public void run() {
return new Thread(exactlyOnceValidationConsumer);
}
- private static class ExactlyOnceValidatingMapper implements FlatMapFunction, Checkpointed {
+ private static class ExactlyOnceValidatingMapper implements FlatMapFunction, ListCheckpointed {
private static final Logger LOG = LoggerFactory.getLogger(ExactlyOnceValidatingMapper.class);
@@ -126,13 +129,18 @@ public void flatMap(String value, Collector out) throws Exception {
}
@Override
- public BitSet snapshotState(long checkpointId, long checkpointTimestamp) throws Exception {
- return validator;
+ public List snapshotState(long checkpointId, long timestamp) throws Exception {
+ return Collections.singletonList(validator);
}
@Override
- public void restoreState(BitSet state) throws Exception {
- this.validator = state;
+ public void restoreState(List state) throws Exception {
+ // we expect either 1 or 0 elements
+ if (state.size() == 1) {
+ validator = state.get(0);
+ } else {
+ Preconditions.checkState(state.isEmpty());
+ }
}
}
diff --git a/flink-connectors/flink-connector-kinesis/src/test/java/org/apache/flink/streaming/connectors/kinesis/util/KinesisConfigUtilTest.java b/flink-connectors/flink-connector-kinesis/src/test/java/org/apache/flink/streaming/connectors/kinesis/util/KinesisConfigUtilTest.java
new file mode 100644
index 0000000000000..3b000588f4b91
--- /dev/null
+++ b/flink-connectors/flink-connector-kinesis/src/test/java/org/apache/flink/streaming/connectors/kinesis/util/KinesisConfigUtilTest.java
@@ -0,0 +1,68 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.streaming.connectors.kinesis.util;
+
+import org.apache.flink.streaming.connectors.kinesis.FlinkKinesisConsumer;
+import org.apache.flink.streaming.connectors.kinesis.config.AWSConfigConstants;
+import org.apache.flink.streaming.connectors.kinesis.config.ProducerConfigConstants;
+
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.ExpectedException;
+import org.junit.runner.RunWith;
+import org.powermock.core.classloader.annotations.PrepareForTest;
+import org.powermock.modules.junit4.PowerMockRunner;
+
+import java.util.Properties;
+
+import static org.junit.Assert.assertEquals;
+
+/**
+ * Tests for KinesisConfigUtil.
+ */
+@RunWith(PowerMockRunner.class)
+@PrepareForTest({FlinkKinesisConsumer.class, KinesisConfigUtil.class})
+public class KinesisConfigUtilTest {
+ @Rule
+ private ExpectedException exception = ExpectedException.none();
+
+ @Test
+ public void testUnparsableLongForProducerConfiguration() {
+ exception.expect(IllegalArgumentException.class);
+ exception.expectMessage("Error trying to set field RateLimit with the value 'unparsableLong'");
+
+ Properties testConfig = new Properties();
+ testConfig.setProperty(AWSConfigConstants.AWS_REGION, "us-east-1");
+ testConfig.setProperty("RateLimit", "unparsableLong");
+
+ KinesisConfigUtil.validateProducerConfiguration(testConfig);
+ }
+
+ @Test
+ public void testReplaceDeprecatedKeys() {
+ Properties testConfig = new Properties();
+ testConfig.setProperty(AWSConfigConstants.AWS_REGION, "us-east-1");
+ // these deprecated keys should be replaced
+ testConfig.setProperty(ProducerConfigConstants.AGGREGATION_MAX_COUNT, "1");
+ testConfig.setProperty(ProducerConfigConstants.COLLECTION_MAX_COUNT, "2");
+ Properties replacedConfig = KinesisConfigUtil.replaceDeprecatedProducerKeys(testConfig);
+
+ assertEquals("1", replacedConfig.getProperty(KinesisConfigUtil.AGGREGATION_MAX_COUNT));
+ assertEquals("2", replacedConfig.getProperty(KinesisConfigUtil.COLLECTION_MAX_COUNT));
+ }
+}
diff --git a/flink-connectors/flink-connector-kinesis/src/test/resources/kinesis-consumer-migration-test-flink1.1-empty-snapshot b/flink-connectors/flink-connector-kinesis/src/test/resources/kinesis-consumer-migration-test-flink1.1-empty-snapshot
deleted file mode 100644
index f4dd96d211342..0000000000000
Binary files a/flink-connectors/flink-connector-kinesis/src/test/resources/kinesis-consumer-migration-test-flink1.1-empty-snapshot and /dev/null differ
diff --git a/flink-connectors/flink-connector-kinesis/src/test/resources/kinesis-consumer-migration-test-flink1.1-snapshot b/flink-connectors/flink-connector-kinesis/src/test/resources/kinesis-consumer-migration-test-flink1.1-snapshot
deleted file mode 100644
index b60402e848327..0000000000000
Binary files a/flink-connectors/flink-connector-kinesis/src/test/resources/kinesis-consumer-migration-test-flink1.1-snapshot and /dev/null differ
diff --git a/flink-connectors/flink-connector-kinesis/src/test/resources/kinesis-consumer-migration-test-flink1.3-empty-snapshot b/flink-connectors/flink-connector-kinesis/src/test/resources/kinesis-consumer-migration-test-flink1.3-empty-snapshot
new file mode 100644
index 0000000000000..aa981c0a61b83
Binary files /dev/null and b/flink-connectors/flink-connector-kinesis/src/test/resources/kinesis-consumer-migration-test-flink1.3-empty-snapshot differ
diff --git a/flink-connectors/flink-connector-kinesis/src/test/resources/kinesis-consumer-migration-test-flink1.3-snapshot b/flink-connectors/flink-connector-kinesis/src/test/resources/kinesis-consumer-migration-test-flink1.3-snapshot
new file mode 100644
index 0000000000000..ddf8a4d8d0834
Binary files /dev/null and b/flink-connectors/flink-connector-kinesis/src/test/resources/kinesis-consumer-migration-test-flink1.3-snapshot differ
diff --git a/flink-connectors/flink-connector-rabbitmq/src/test/java/org/apache/flink/streaming/connectors/rabbitmq/RMQSourceTest.java b/flink-connectors/flink-connector-rabbitmq/src/test/java/org/apache/flink/streaming/connectors/rabbitmq/RMQSourceTest.java
index 05ae8108b5a68..f180e786c2f28 100644
--- a/flink-connectors/flink-connector-rabbitmq/src/test/java/org/apache/flink/streaming/connectors/rabbitmq/RMQSourceTest.java
+++ b/flink-connectors/flink-connector-rabbitmq/src/test/java/org/apache/flink/streaming/connectors/rabbitmq/RMQSourceTest.java
@@ -51,8 +51,8 @@
import java.io.IOException;
import java.util.ArrayDeque;
-import java.util.List;
import java.util.Random;
+import java.util.Set;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertNotNull;
@@ -180,12 +180,12 @@ public void testCheckpointing() throws Exception {
testHarnessCopy.initializeState(data);
testHarnessCopy.open();
- ArrayDeque>> deque = sourceCopy.getRestoredState();
- List messageIds = deque.getLast().f1;
+ ArrayDeque>> deque = sourceCopy.getRestoredState();
+ Set messageIds = deque.getLast().f1;
assertEquals(numIds, messageIds.size());
if (messageIds.size() > 0) {
- assertEquals(lastSnapshotId, (long) Long.valueOf(messageIds.get(messageIds.size() - 1)));
+ assertTrue(messageIds.contains(Long.toString(lastSnapshotId)));
}
// check if the messages are being acknowledged and the transaction committed
@@ -339,7 +339,7 @@ public TypeInformation getProducedType() {
private class RMQTestSource extends RMQSource {
- private ArrayDeque>> restoredState;
+ private ArrayDeque>> restoredState;
public RMQTestSource() {
super(new RMQConnectionConfig.Builder().setHost("hostTest")
@@ -353,7 +353,7 @@ public void initializeState(FunctionInitializationContext context) throws Except
this.restoredState = this.pendingCheckpoints;
}
- public ArrayDeque>> getRestoredState() {
+ public ArrayDeque>> getRestoredState() {
return this.restoredState;
}
diff --git a/flink-connectors/flink-connector-twitter/pom.xml b/flink-connectors/flink-connector-twitter/pom.xml
index 0f1e44a5d5dcc..2d8d62a39593c 100644
--- a/flink-connectors/flink-connector-twitter/pom.xml
+++ b/flink-connectors/flink-connector-twitter/pom.xml
@@ -77,12 +77,23 @@ under the License.
+ com.google.guava:guava
com.twitter:hbc-core
com.twitter:joauth
org.apache.httpcomponents:httpclient
org.apache.httpcomponents:httpcore
+
+
+ com.google
+ org.apache.flink.twitter.shaded.com.google
+
+ com.google.protobuf.**
+ com.google.inject.**
+
+
+
diff --git a/flink-connectors/flink-hbase/pom.xml b/flink-connectors/flink-hbase/pom.xml
index b900d23ded2b3..e18fe39280c8b 100644
--- a/flink-connectors/flink-hbase/pom.xml
+++ b/flink-connectors/flink-hbase/pom.xml
@@ -127,14 +127,6 @@ under the License.
flink-streaming-java_${scala.binary.version}
${project.version}
provided
-
-
-
-
- com.google.guava
- guava
-
-
diff --git a/flink-connectors/flink-hcatalog/pom.xml b/flink-connectors/flink-hcatalog/pom.xml
index 10ca36d8acaf2..1e77d7df443f4 100644
--- a/flink-connectors/flink-hcatalog/pom.xml
+++ b/flink-connectors/flink-hcatalog/pom.xml
@@ -70,6 +70,32 @@ under the License.
+
+ org.apache.maven.plugins
+ maven-shade-plugin
+
+
+ shade-flink
+
+
+
+ com.google.guava:guava
+
+
+
+
+ com.google
+ org.apache.flink.hcatalog.shaded.com.google
+
+ com.google.protobuf.**
+ com.google.inject.**
+
+
+
+
+
+
+
net.alchim31.maven
diff --git a/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/JDBCInputFormat.java b/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/JDBCInputFormat.java
index b7ac7446a410d..7d088147ae829 100644
--- a/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/JDBCInputFormat.java
+++ b/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/JDBCInputFormat.java
@@ -144,7 +144,7 @@ public void openInputFormat() {
dbConn = DriverManager.getConnection(dbURL, username, password);
}
statement = dbConn.prepareStatement(queryTemplate, resultSetType, resultSetConcurrency);
- if (fetchSize > 0) {
+ if (fetchSize == Integer.MIN_VALUE || fetchSize > 0) {
statement.setFetchSize(fetchSize);
}
} catch (SQLException se) {
@@ -390,7 +390,8 @@ public JDBCInputFormatBuilder setRowTypeInfo(RowTypeInfo rowTypeInfo) {
}
public JDBCInputFormatBuilder setFetchSize(int fetchSize) {
- Preconditions.checkArgument(fetchSize > 0, "Illegal value %s for fetchSize, has to be positive.", fetchSize);
+ Preconditions.checkArgument(fetchSize == Integer.MIN_VALUE || fetchSize > 0,
+ "Illegal value %s for fetchSize, has to be positive or Integer.MIN_VALUE.", fetchSize);
format.fetchSize = fetchSize;
return this;
}
diff --git a/flink-connectors/flink-jdbc/src/test/java/org/apache/flink/api/java/io/jdbc/JDBCInputFormatTest.java b/flink-connectors/flink-jdbc/src/test/java/org/apache/flink/api/java/io/jdbc/JDBCInputFormatTest.java
index f7a86e5afd222..10e8c66a7ddfa 100644
--- a/flink-connectors/flink-jdbc/src/test/java/org/apache/flink/api/java/io/jdbc/JDBCInputFormatTest.java
+++ b/flink-connectors/flink-jdbc/src/test/java/org/apache/flink/api/java/io/jdbc/JDBCInputFormatTest.java
@@ -113,6 +113,17 @@ public void testInvalidFetchSize() {
.finish();
}
+ @Test
+ public void testValidFetchSizeIntegerMin() {
+ jdbcInputFormat = JDBCInputFormat.buildJDBCInputFormat()
+ .setDrivername(DRIVER_CLASS)
+ .setDBUrl(DB_URL)
+ .setQuery(SELECT_ALL_BOOKS)
+ .setRowTypeInfo(ROW_TYPE_INFO)
+ .setFetchSize(Integer.MIN_VALUE)
+ .finish();
+ }
+
@Test
public void testDefaultFetchSizeIsUsedIfNotConfiguredOtherwise() throws SQLException, ClassNotFoundException {
jdbcInputFormat = JDBCInputFormat.buildJDBCInputFormat()
diff --git a/flink-connectors/pom.xml b/flink-connectors/pom.xml
index bc3f82f686c44..2ed3b7974098b 100644
--- a/flink-connectors/pom.xml
+++ b/flink-connectors/pom.xml
@@ -54,6 +54,7 @@ under the License.
flink-connector-nifi
flink-connector-cassandra
flink-connector-filesystem
+ flink-connector-eventhubs
+
+ org.apache.flink
+ flink-shaded-guava
+
+
org.apache.flink
flink-test-utils_${scala.binary.version}
diff --git a/flink-contrib/flink-storm-examples/src/test/java/org/apache/flink/storm/join/SingleJoinITCase.java b/flink-contrib/flink-storm-examples/src/test/java/org/apache/flink/storm/join/SingleJoinITCase.java
index 83531bab38a2e..5d406dba72a7f 100644
--- a/flink-contrib/flink-storm-examples/src/test/java/org/apache/flink/storm/join/SingleJoinITCase.java
+++ b/flink-contrib/flink-storm-examples/src/test/java/org/apache/flink/storm/join/SingleJoinITCase.java
@@ -20,7 +20,7 @@
import org.apache.flink.streaming.util.StreamingProgramTestBase;
-import com.google.common.base.Joiner;
+import org.apache.flink.shaded.guava18.com.google.common.base.Joiner;
/**
* Test for the SingleJoin example.
diff --git a/flink-core/src/main/java/org/apache/flink/configuration/BlobServerOptions.java b/flink-core/src/main/java/org/apache/flink/configuration/BlobServerOptions.java
index e27c29f2e0b46..019580a6705dd 100644
--- a/flink-core/src/main/java/org/apache/flink/configuration/BlobServerOptions.java
+++ b/flink-core/src/main/java/org/apache/flink/configuration/BlobServerOptions.java
@@ -22,7 +22,7 @@
import static org.apache.flink.configuration.ConfigOptions.key;
/**
- * Configuration options for the BlobServer.
+ * Configuration options for the BlobServer and BlobCache.
*/
@PublicEvolving
public class BlobServerOptions {
@@ -73,4 +73,18 @@ public class BlobServerOptions {
public static final ConfigOption SSL_ENABLED =
key("blob.service.ssl.enabled")
.defaultValue(true);
+
+ /**
+ * Cleanup interval of the blob caches at the task managers (in seconds).
+ *
+ * Whenever a job is not referenced at the cache anymore, we set a TTL and let the periodic
+ * cleanup task (executed every CLEANUP_INTERVAL seconds) remove its blob files after this TTL
+ * has passed. This means that a blob will be retained at most 2 * CLEANUP_INTERVAL
+ * seconds after not being referenced anymore. Therefore, a recovery still has the chance to use
+ * existing files rather than to download them again.
+ */
+ public static final ConfigOption CLEANUP_INTERVAL =
+ key("blob.service.cleanup.interval")
+ .defaultValue(3_600L) // once per hour
+ .withDeprecatedKeys("library-cache-manager.cleanup.interval");
}
diff --git a/flink-core/src/main/java/org/apache/flink/configuration/ConfigConstants.java b/flink-core/src/main/java/org/apache/flink/configuration/ConfigConstants.java
index 4c6c62a51c8ca..4153e456e3160 100644
--- a/flink-core/src/main/java/org/apache/flink/configuration/ConfigConstants.java
+++ b/flink-core/src/main/java/org/apache/flink/configuration/ConfigConstants.java
@@ -178,7 +178,10 @@ public final class ConfigConstants {
/**
* The config parameter defining the cleanup interval of the library cache manager.
+ *
+ * @deprecated use {@link BlobServerOptions#CLEANUP_INTERVAL} instead
*/
+ @Deprecated
public static final String LIBRARY_CACHE_MANAGER_CLEANUP_INTERVAL = "library-cache-manager.cleanup.interval";
/**
@@ -1253,8 +1256,12 @@ public final class ConfigConstants {
/**
* The default library cache manager cleanup interval in seconds
+ *
+ * @deprecated use {@link BlobServerOptions#CLEANUP_INTERVAL} instead
*/
- public static final long DEFAULT_LIBRARY_CACHE_MANAGER_CLEANUP_INTERVAL = 3600;
+ @Deprecated
+ public static final long DEFAULT_LIBRARY_CACHE_MANAGER_CLEANUP_INTERVAL =
+ BlobServerOptions.CLEANUP_INTERVAL.defaultValue();
/**
* The default network port to connect to for communication with the job manager.
diff --git a/flink-core/src/main/java/org/apache/flink/configuration/Configuration.java b/flink-core/src/main/java/org/apache/flink/configuration/Configuration.java
index d6f1decf3f69f..dfcd04fb97eae 100644
--- a/flink-core/src/main/java/org/apache/flink/configuration/Configuration.java
+++ b/flink-core/src/main/java/org/apache/flink/configuration/Configuration.java
@@ -79,7 +79,7 @@ public Configuration(Configuration other) {
}
// --------------------------------------------------------------------------------------------
-
+
/**
* Returns the class associated with the given key as a string.
*
diff --git a/flink-core/src/main/java/org/apache/flink/configuration/GlobalConfiguration.java b/flink-core/src/main/java/org/apache/flink/configuration/GlobalConfiguration.java
index ea9f8bfc97e0c..4569ebe0acf0d 100644
--- a/flink-core/src/main/java/org/apache/flink/configuration/GlobalConfiguration.java
+++ b/flink-core/src/main/java/org/apache/flink/configuration/GlobalConfiguration.java
@@ -28,6 +28,8 @@
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
+import javax.annotation.Nullable;
+
/**
* Global configuration object for Flink. Similar to Java properties configuration
* objects it includes key-value pairs which represent the framework's configuration.
@@ -46,24 +48,6 @@ private GlobalConfiguration() {}
// --------------------------------------------------------------------------------------------
- private static Configuration dynamicProperties = null;
-
- /**
- * Set the process-wide dynamic properties to be merged with the loaded configuration.
- */
- public static void setDynamicProperties(Configuration dynamicProperties) {
- GlobalConfiguration.dynamicProperties = new Configuration(dynamicProperties);
- }
-
- /**
- * Get the dynamic properties.
- */
- public static Configuration getDynamicProperties() {
- return GlobalConfiguration.dynamicProperties;
- }
-
- // --------------------------------------------------------------------------------------------
-
/**
* Loads the global configuration from the environment. Fails if an error occurs during loading. Returns an
* empty configuration object if the environment variable is not set. In production this variable is set but
@@ -76,18 +60,30 @@ public static Configuration loadConfiguration() {
if (configDir == null) {
return new Configuration();
}
- return loadConfiguration(configDir);
+ return loadConfiguration(configDir, null);
}
/**
* Loads the configuration files from the specified directory.
*
* YAML files are supported as configuration files.
- *
+ *
* @param configDir
* the directory which contains the configuration files
*/
public static Configuration loadConfiguration(final String configDir) {
+ return loadConfiguration(configDir, null);
+ }
+
+ /**
+ * Loads the configuration files from the specified directory. If the dynamic properties
+ * configuration is not null, then it is added to the loaded configuration.
+ *
+ * @param configDir directory to load the configuration from
+ * @param dynamicProperties configuration file containing the dynamic properties. Null if none.
+ * @return The configuration loaded from the given configuration directory
+ */
+ public static Configuration loadConfiguration(final String configDir, @Nullable final Configuration dynamicProperties) {
if (configDir == null) {
throw new IllegalArgumentException("Given configuration directory is null, cannot load configuration");
@@ -109,13 +105,29 @@ public static Configuration loadConfiguration(final String configDir) {
"' (" + confDirFile.getAbsolutePath() + ") does not exist.");
}
- Configuration conf = loadYAMLResource(yamlConfigFile);
+ Configuration configuration = loadYAMLResource(yamlConfigFile);
- if(dynamicProperties != null) {
- conf.addAll(dynamicProperties);
+ if (dynamicProperties != null) {
+ configuration.addAll(dynamicProperties);
+ }
+
+ return configuration;
+ }
+
+ /**
+ * Loads the global configuration and adds the given dynamic properties
+ * configuration.
+ *
+ * @param dynamicProperties The given dynamic properties
+ * @return Returns the loaded global configuration with dynamic properties
+ */
+ public static Configuration loadConfigurationWithDynamicProperties(Configuration dynamicProperties) {
+ final String configDir = System.getenv(ConfigConstants.ENV_FLINK_CONF_DIR);
+ if (configDir == null) {
+ return new Configuration(dynamicProperties);
}
- return conf;
+ return loadConfiguration(configDir, dynamicProperties);
}
/**
diff --git a/flink-core/src/main/java/org/apache/flink/configuration/RestOptions.java b/flink-core/src/main/java/org/apache/flink/configuration/RestOptions.java
new file mode 100644
index 0000000000000..a2a20136ea40e
--- /dev/null
+++ b/flink-core/src/main/java/org/apache/flink/configuration/RestOptions.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.configuration;
+
+import org.apache.flink.annotation.Internal;
+
+import static org.apache.flink.configuration.ConfigOptions.key;
+
+/**
+ * Configuration parameters for REST communication.
+ */
+@Internal
+public class RestOptions {
+ /**
+ * The address that the server binds itself to / the client connects to.
+ */
+ public static final ConfigOption REST_ADDRESS =
+ key("rest.address")
+ .defaultValue("localhost");
+
+ /**
+ * The port that the server listens on / the client connects to.
+ */
+ public static final ConfigOption REST_PORT =
+ key("rest.port")
+ .defaultValue(9067);
+}
diff --git a/flink-core/src/main/java/org/apache/flink/migration/util/MigrationInstantiationUtil.java b/flink-core/src/main/java/org/apache/flink/migration/util/MigrationInstantiationUtil.java
deleted file mode 100644
index 69e4e6daa668e..0000000000000
--- a/flink-core/src/main/java/org/apache/flink/migration/util/MigrationInstantiationUtil.java
+++ /dev/null
@@ -1,96 +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.migration.util;
-
-import org.apache.flink.annotation.PublicEvolving;
-import org.apache.flink.util.InstantiationUtil;
-
-import java.io.ByteArrayInputStream;
-import java.io.IOException;
-import java.io.InputStream;
-import java.io.ObjectInputStream;
-import java.io.ObjectStreamClass;
-
-/**
- * Utility class to deserialize legacy classes for migration.
- */
-@PublicEvolving
-public final class MigrationInstantiationUtil {
-
- public static class ClassLoaderObjectInputStream extends InstantiationUtil.ClassLoaderObjectInputStream {
-
- private static final String ARRAY_PREFIX = "[L";
- private static final String FLINK_BASE_PACKAGE = "org.apache.flink.";
- private static final String FLINK_MIGRATION_PACKAGE = "org.apache.flink.migration.";
-
- public ClassLoaderObjectInputStream(InputStream in, ClassLoader classLoader) throws IOException {
- super(in, classLoader);
- }
-
- @Override
- protected Class> resolveClass(ObjectStreamClass desc) throws IOException, ClassNotFoundException {
- final String className = desc.getName();
-
- // the flink package may be at position 0 (regular class) or position 2 (array)
- final int flinkPackagePos;
- if ((flinkPackagePos = className.indexOf(FLINK_BASE_PACKAGE)) == 0 ||
- (flinkPackagePos == 2 && className.startsWith(ARRAY_PREFIX)))
- {
- final String modClassName = flinkPackagePos == 0 ?
- FLINK_MIGRATION_PACKAGE + className.substring(FLINK_BASE_PACKAGE.length()) :
- ARRAY_PREFIX + FLINK_MIGRATION_PACKAGE + className.substring(2 + FLINK_BASE_PACKAGE.length());
-
- try {
- return classLoader != null ?
- Class.forName(modClassName, false, classLoader) :
- Class.forName(modClassName);
- }
- catch (ClassNotFoundException ignored) {}
- }
-
- // either a non-Flink class, or not located in the migration package
- return super.resolveClass(desc);
- }
- }
-
- public static T deserializeObject(byte[] bytes, ClassLoader cl) throws IOException, ClassNotFoundException {
- return deserializeObject(new ByteArrayInputStream(bytes), cl);
- }
-
- @SuppressWarnings("unchecked")
- public static T deserializeObject(InputStream in, ClassLoader cl) throws IOException, ClassNotFoundException {
- final ClassLoader old = Thread.currentThread().getContextClassLoader();
- try (ObjectInputStream oois = new ClassLoaderObjectInputStream(in, cl)) {
- Thread.currentThread().setContextClassLoader(cl);
- return (T) oois.readObject();
- } finally {
- Thread.currentThread().setContextClassLoader(old);
- }
- }
-
- // --------------------------------------------------------------------------------------------
-
- /**
- * Private constructor to prevent instantiation.
- */
- private MigrationInstantiationUtil() {
- throw new IllegalAccessError();
- }
-
-}
diff --git a/flink-core/src/main/java/org/apache/flink/migration/util/SerializedValue.java b/flink-core/src/main/java/org/apache/flink/migration/util/SerializedValue.java
deleted file mode 100644
index 6fa29d3c554c7..0000000000000
--- a/flink-core/src/main/java/org/apache/flink/migration/util/SerializedValue.java
+++ /dev/null
@@ -1,98 +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.migration.util;
-
-import org.apache.flink.annotation.PublicEvolving;
-import org.apache.flink.util.InstantiationUtil;
-
-import java.io.IOException;
-import java.util.Arrays;
-
-/**
- * This class is used to transfer (via serialization) objects whose classes are not available
- * in the system class loader. When those objects are deserialized without access to their
- * special class loader, the deserialization fails with a {@code ClassNotFoundException}.
- *
- * To work around that issue, the SerializedValue serialized data immediately into a byte array.
- * When send through RPC or another service that uses serialization, only the byte array is
- * transferred. The object is deserialized later (upon access) and requires the accessor to
- * provide the corresponding class loader.
- *
- * @param The type of the value held.
- * @deprecated Only used internally when migrating from previous savepoint versions.
- */
-@Deprecated
-@PublicEvolving
-public class SerializedValue implements java.io.Serializable {
-
- private static final long serialVersionUID = -3564011643393683761L;
-
- /** The serialized data */
- private final byte[] serializedData;
-
- private SerializedValue(byte[] serializedData) {
- this.serializedData = serializedData;
- }
-
- public SerializedValue(T value) throws IOException {
- this.serializedData = value == null ? null : InstantiationUtil.serializeObject(value);
- }
-
- @SuppressWarnings("unchecked")
- public T deserializeValue(ClassLoader loader) throws IOException, ClassNotFoundException {
- return serializedData == null ? null : (T) MigrationInstantiationUtil.deserializeObject(serializedData, loader);
- }
-
- /**
- * Returns the serialized value or