diff --git a/build.gradle b/build.gradle index 5d89998e7..ffa273d64 100644 --- a/build.gradle +++ b/build.gradle @@ -141,7 +141,10 @@ project(':datastream-common') { compile "com.linkedin.pegasus:restli-server:$pegasusVersion" compile "com.intellij:annotations:$intellijAnnotationsVersion" compile "com.google.guava:guava:$guavaVersion" - compile "com.linkedin.kafka.clients:li-apache-kafka-clients:$LIKafkaVersion" + compile "com.linkedin.kafka:kafka-clients:$kafkaVersion" + compile "com.fasterxml.jackson.core:jackson-annotations:$jacksonVersion" + compile "com.fasterxml.jackson.core:jackson-core:$jacksonVersion" + compile "com.fasterxml.jackson.core:jackson-databind:$jacksonVersion" testCompile "org.mockito:mockito-core:$mockitoVersion" } } @@ -150,17 +153,16 @@ project(':datastream-server-api') { dependencies { compile project(':datastream-common') compile project(':datastream-utils') - compile "com.linkedin.kafka.clients:li-apache-kafka-clients:$LIKafkaVersion" } } project(':datastream-utils') { dependencies { compile project(':datastream-common') - compile "com.101tec:zkclient:$zkclientVersion" + compile "org.apache.helix:zookeeper-api:$helixZkclientVersion" compile "com.google.guava:guava:$guavaVersion" - testCompile project(":datastream-kafka_$scalaSuffix") - testCompile project(":datastream-testcommon_$scalaSuffix") + testCompile project(":datastream-kafka") + testCompile project(":datastream-testcommon") } } @@ -170,7 +172,7 @@ project(':datastream-file-connector') { compile project(':datastream-common') compile project(':datastream-utils') testCompile project(':datastream-server') - testCompile project(":datastream-testcommon_$scalaSuffix") + testCompile project(":datastream-testcommon") testCompile "org.mockito:mockito-core:$mockitoVersion" } } @@ -186,17 +188,16 @@ project(':datastream-directory') { } } -project(":datastream-kafka_$scalaSuffix") { +project(":datastream-kafka") { dependencies { compile "com.linkedin.kafka:kafka_$scalaSuffix:$kafkaVersion" - compile "com.linkedin.kafka.clients:li-apache-kafka-clients:$LIKafkaVersion" compile "commons-httpclient:commons-httpclient:$commonsHttpClientVersion" compile project(':datastream-server') compile project(':datastream-server-api') compile project(':datastream-utils') - testCompile project(":datastream-testcommon_$scalaSuffix") + testCompile project(":datastream-testcommon") testCompile "org.mockito:mockito-core:$mockitoVersion" tasks.create(name: "copyDependentLibs", type: Copy) { @@ -214,9 +215,8 @@ project(":datastream-kafka_$scalaSuffix") { project(':datastream-kafka-factory-impl') { dependencies { compile project(':datastream-kafka-connector') - compile "com.linkedin.kafka.clients:li-apache-kafka-clients:$LIKafkaVersion" - testCompile project(":datastream-kafka_$scalaSuffix") + testCompile project(":datastream-kafka") } } @@ -224,24 +224,23 @@ project(':datastream-kafka-connector') { dependencies { compile project(':datastream-server-api') compile project(':datastream-common') - compile project(":datastream-kafka_$scalaSuffix") - compile "com.linkedin.kafka.clients:li-apache-kafka-clients:$LIKafkaVersion" + compile project(":datastream-kafka") compile "org.apache.httpcomponents:httpclient:$apacheHttpClientVersion" compile "commons-validator:commons-validator:$commonsValidatorVersion" compile "org.apache.commons:commons-lang3:$commonslang3Version" compile "javax.xml.bind:jaxb-api:2.3.1" - testCompile project(":datastream-kafka_$scalaSuffix") - testCompile project(":datastream-testcommon_$scalaSuffix") + testCompile project(":datastream-kafka") + testCompile project(":datastream-testcommon") testCompile project(':datastream-kafka-factory-impl') testCompile "org.mockito:mockito-core:$mockitoVersion" } } -project(":datastream-testcommon_$scalaSuffix") { +project(":datastream-testcommon") { dependencies { compile project(':datastream-server-api') - compile project(":datastream-kafka_$scalaSuffix") + compile project(":datastream-kafka") compile project(':datastream-common') compile project(':datastream-utils') compile project(':datastream-server') @@ -262,7 +261,7 @@ project(':datastream-tools') { compile project(':datastream-common') compile project(':datastream-client') compile project(':datastream-server') - compile project(":datastream-testcommon_$scalaSuffix") + compile project(":datastream-testcommon") compile "commons-cli:commons-cli:$commonsCliVersion" } @@ -285,12 +284,12 @@ project(':datastream-tools') { from(project(':datastream-file-connector').configurations.runtime) { into("libs/") } from(project(':datastream-directory').jar) { into("libs/") } from(project(':datastream-directory').configurations.runtime) { into("libs/") } - from(project(":datastream-testcommon_$scalaSuffix").jar) { into("libs/") } - from(project(":datastream-testcommon_$scalaSuffix").configurations.runtime) { into("libs/") } + from(project(":datastream-testcommon").jar) { into("libs/") } + from(project(":datastream-testcommon").configurations.runtime) { into("libs/") } from(project(':datastream-common').jar) { into("libs/") } from(project(':datastream-common').configurations.runtime) { into("libs/") } - from(project(":datastream-kafka_$scalaSuffix").jar) { into("libs/") } - from(project(":datastream-kafka_$scalaSuffix").configurations.runtime) { into("libs/") } + from(project(":datastream-kafka").jar) { into("libs/") } + from(project(":datastream-kafka").configurations.runtime) { into("libs/") } from(project(':datastream-kafka-connector').jar) { into("libs/") } from(project(':datastream-kafka-connector').configurations.runtime) { into("libs/") } duplicatesStrategy 'exclude' @@ -309,8 +308,8 @@ project(':datastream-client') { compile project(':datastream-common') compile project(':datastream-utils') - testCompile project(":datastream-testcommon_$scalaSuffix") - testCompile project(":datastream-kafka_$scalaSuffix") + testCompile project(":datastream-testcommon") + testCompile project(":datastream-kafka") testCompile project(':datastream-server') testCompile "org.mockito:mockito-core:$mockitoVersion" } @@ -319,19 +318,19 @@ project(':datastream-client') { project(':datastream-server') { dependencies { - compile "com.101tec:zkclient:$zkclientVersion" - compile "org.codehaus.jackson:jackson-core-asl:$jacksonVersion" - compile "org.codehaus.jackson:jackson-mapper-asl:$jacksonVersion" + compile "com.fasterxml.jackson.core:jackson-annotations:$jacksonVersion" + compile "com.fasterxml.jackson.core:jackson-core:$jacksonVersion" + compile "com.fasterxml.jackson.core:jackson-databind:$jacksonVersion" compile project(':datastream-server-api') compile project(':datastream-common') compile project(':datastream-utils') compile project(':datastream-client') - testCompile project(":datastream-kafka_$scalaSuffix") + testCompile project(":datastream-kafka") testCompile "com.linkedin.kafka:kafka_$scalaSuffix:$kafkaVersion" testCompile project(':datastream-client') - testCompile project(":datastream-testcommon_$scalaSuffix") + testCompile project(":datastream-testcommon") testCompile "org.mockito:mockito-core:$mockitoVersion" } @@ -355,19 +354,19 @@ project(':datastream-server-restli') { compile "com.linkedin.pegasus:restli-netty-standalone:$pegasusVersion" compile "com.linkedin.pegasus:r2-jetty:$pegasusVersion" compile "com.linkedin.parseq:parseq:$parseqVersion" - compile "com.101tec:zkclient:$zkclientVersion" - compile "org.codehaus.jackson:jackson-core-asl:$jacksonVersion" - compile "org.codehaus.jackson:jackson-mapper-asl:$jacksonVersion" + compile "com.fasterxml.jackson.core:jackson-annotations:$jacksonVersion" + compile "com.fasterxml.jackson.core:jackson-core:$jacksonVersion" + compile "com.fasterxml.jackson.core:jackson-databind:$jacksonVersion" compile "org.apache.commons:commons-lang3:$commonslang3Version" compile project(':datastream-server') compile project(':datastream-common') - testCompile project(":datastream-kafka_$scalaSuffix") + testCompile project(":datastream-kafka") testCompile "com.linkedin.kafka:kafka_$scalaSuffix:$kafkaVersion" testCompile project(':datastream-client') testCompile project(':datastream-file-connector') - testCompile project(":datastream-testcommon_$scalaSuffix") + testCompile project(":datastream-testcommon") testCompile "org.mockito:mockito-core:$mockitoVersion" } diff --git a/datastream-common/src/main/java/com/linkedin/datastream/avrogenerator/AvroJson.java b/datastream-common/src/main/java/com/linkedin/datastream/avrogenerator/AvroJson.java index a75942d6c..3419e3da8 100644 --- a/datastream-common/src/main/java/com/linkedin/datastream/avrogenerator/AvroJson.java +++ b/datastream-common/src/main/java/com/linkedin/datastream/avrogenerator/AvroJson.java @@ -13,9 +13,10 @@ import java.util.Map; import org.apache.avro.Schema; -import org.codehaus.jackson.JsonFactory; -import org.codehaus.jackson.JsonGenerator; -import org.codehaus.jackson.map.ObjectMapper; + +import com.fasterxml.jackson.core.JsonFactory; +import com.fasterxml.jackson.core.JsonGenerator; +import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.base.CaseFormat; diff --git a/datastream-common/src/main/java/com/linkedin/datastream/common/DatastreamEvent.java b/datastream-common/src/main/java/com/linkedin/datastream/common/DatastreamEvent.java index 546a9a448..42a13b28b 100644 --- a/datastream-common/src/main/java/com/linkedin/datastream/common/DatastreamEvent.java +++ b/datastream-common/src/main/java/com/linkedin/datastream/common/DatastreamEvent.java @@ -6,12 +6,12 @@ // CHECKSTYLE:OFF /** * Autogenerated by Avro - * + * * DO NOT EDIT DIRECTLY */ package com.linkedin.datastream.common; -@SuppressWarnings("all") +@SuppressWarnings("serial") public class DatastreamEvent extends org.apache.avro.specific.SpecificRecordBase implements org.apache.avro.specific.SpecificRecord { public static final org.apache.avro.Schema SCHEMA$ = org.apache.avro.Schema.parse("{\"type\":\"record\",\"name\":\"DatastreamEvent\",\"namespace\":\"com.linkedin.datastream.common\",\"fields\":[{\"name\":\"metadata\",\"type\":{\"type\":\"map\",\"values\":\"string\"},\"doc\":\"Event metadata.\"},{\"name\":\"key\",\"type\":\"bytes\",\"doc\":\"serialized key.\"},{\"name\":\"payload\",\"type\":\"bytes\",\"doc\":\"serialized payload data.\"},{\"name\":\"previous_payload\",\"type\":\"bytes\",\"doc\":\"serialized previous payload data.\"}]}"); /** Event metadata. */ @@ -23,7 +23,7 @@ public class DatastreamEvent extends org.apache.avro.specific.SpecificRecordBase /** serialized previous payload data. */ public java.nio.ByteBuffer previous_payload; public org.apache.avro.Schema getSchema() { return SCHEMA$; } - // Used by DatumWriter. Applications should not call. + // Used by DatumWriter. Applications should not call. public java.lang.Object get(int field$) { switch (field$) { case 0: return metadata; @@ -33,7 +33,7 @@ public java.lang.Object get(int field$) { default: throw new org.apache.avro.AvroRuntimeException("Bad index"); } } - // Used by DatumReader. Applications should not call. + // Used by DatumReader. Applications should not call. @SuppressWarnings(value="unchecked") public void put(int field$, java.lang.Object value$) { switch (field$) { diff --git a/datastream-common/src/main/java/com/linkedin/datastream/common/JsonUtils.java b/datastream-common/src/main/java/com/linkedin/datastream/common/JsonUtils.java index b7aaeed69..bc96fa7c7 100644 --- a/datastream-common/src/main/java/com/linkedin/datastream/common/JsonUtils.java +++ b/datastream-common/src/main/java/com/linkedin/datastream/common/JsonUtils.java @@ -5,24 +5,26 @@ */ package com.linkedin.datastream.common; + import java.io.IOException; import java.io.StringWriter; import java.time.Instant; import org.apache.commons.lang.Validate; -import org.codehaus.jackson.JsonGenerator; -import org.codehaus.jackson.JsonParser; -import org.codehaus.jackson.annotate.JsonIgnore; -import org.codehaus.jackson.map.DeserializationConfig; -import org.codehaus.jackson.map.DeserializationContext; -import org.codehaus.jackson.map.JsonDeserializer; -import org.codehaus.jackson.map.JsonSerializer; -import org.codehaus.jackson.map.ObjectMapper; -import org.codehaus.jackson.map.SerializerProvider; -import org.codehaus.jackson.type.TypeReference; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import com.fasterxml.jackson.annotation.JsonIgnore; +import com.fasterxml.jackson.core.JsonGenerator; +import com.fasterxml.jackson.core.JsonParser; +import com.fasterxml.jackson.core.type.TypeReference; +import com.fasterxml.jackson.databind.DeserializationContext; +import com.fasterxml.jackson.databind.DeserializationFeature; +import com.fasterxml.jackson.databind.JsonDeserializer; +import com.fasterxml.jackson.databind.JsonSerializer; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.fasterxml.jackson.databind.SerializerProvider; + /** * Utility class for converting objects and JSON strings. @@ -34,12 +36,11 @@ public final class JsonUtils { private static final ObjectMapper MAPPER = new ObjectMapper(); static { - MAPPER.configure(DeserializationConfig.Feature.FAIL_ON_UNKNOWN_PROPERTIES, false); + MAPPER.configure(DeserializationFeature.FAIL_ON_UNKNOWN_PROPERTIES, false); - final DeserializationConfig config = MAPPER.getDeserializationConfig(); - config.addMixInAnnotations(Datastream.class, IgnoreDatastreamSetPausedMixIn.class); - config.addMixInAnnotations(DatastreamSource.class, IgnoreDatastreamSourceSetPartitionsMixIn.class); - config.addMixInAnnotations(DatastreamDestination.class, IgnoreDatastreamDestinationSetPartitionsMixIn.class); + MAPPER.addMixIn(Datastream.class, IgnoreDatastreamSetPausedMixIn.class); + MAPPER.addMixIn(DatastreamSource.class, IgnoreDatastreamSourceSetPartitionsMixIn.class); + MAPPER.addMixIn(DatastreamDestination.class, IgnoreDatastreamDestinationSetPartitionsMixIn.class); } /** diff --git a/datastream-common/src/main/java/com/linkedin/datastream/metrics/DynamicMetricsManager.java b/datastream-common/src/main/java/com/linkedin/datastream/metrics/DynamicMetricsManager.java index c18d04071..39ef431e2 100644 --- a/datastream-common/src/main/java/com/linkedin/datastream/metrics/DynamicMetricsManager.java +++ b/datastream-common/src/main/java/com/linkedin/datastream/metrics/DynamicMetricsManager.java @@ -129,6 +129,7 @@ private ConcurrentHashMap> getClassMet */ @SuppressWarnings("unchecked") private T getMetric(String name, Class clazz) { + Validate.notNull(clazz, "metric class argument is null."); if (clazz.equals(Counter.class)) { return (T) _metricRegistry.counter(name); } else if (clazz.equals(Meter.class)) { @@ -136,53 +137,23 @@ private T getMetric(String name, Class clazz) { } else if (clazz.equals(Histogram.class)) { return (T) _metricRegistry.histogram(name); } else if (clazz.equals(Gauge.class)) { - return (T) new ResettableGauge<>(); + throw new IllegalArgumentException("This method doesn't work with Gauges."); } else if (clazz.equals(Timer.class)) { - return (T) new Timer(); + return (T) _metricRegistry.timer(name); } else { throw new IllegalArgumentException("Invalid metric type: " + clazz); } } - /** - * Internal method to create and register a metric with the registry. If the metric with the same - * name has already been registered before, it will be returned instead of creating a new one as - * metric registry forbids duplicate registrations. For an existing Gauge metric, we replace its - * value supplier with the new supplier passed in. - * @param simpleName namespace of the metric - * @param key optional key for the metric (eg. source name) - * @param metricName actual name of the metric - * @param metricClass class of the metric type - * @param supplier optional supplier for Gauge metric (not used for non-Gauge metrics) - * @param metric type - * @param value type for the supplier - */ - @SuppressWarnings("unchecked") - private T doRegisterMetric(String simpleName, String key, String metricName, - Class metricClass, Supplier supplier) { - validateArguments(simpleName, metricName); - Validate.notNull(metricClass, "metric class argument is null."); - - String fullMetricName = MetricRegistry.name(simpleName, key, metricName); - - Metric metric = getMetric(fullMetricName, metricClass); + private void countReference(String fullMetricName) { _registeredMetricRefCount.compute(fullMetricName, (localKey, val) -> (val == null) ? 1 : val + 1); - if (metric instanceof ResettableGauge) { - Validate.notNull(supplier, "null supplier to Gauge"); - ((ResettableGauge) metric).setSupplier(supplier); - - try { - // Gauge needs explicit registration - _metricRegistry.register(fullMetricName, metric); - } catch (IllegalArgumentException e) { - // This can happen with parallel unit tests - } - } - // _indexedMetrics update is left to the createOrUpdate APIs which is only needed // if the same metrics are accessed through both registerMetric and createOrUpdate. + } - return (T) metric; + private String formatName(String simpleName, String key, String metricName) { + validateArguments(simpleName, metricName); + return MetricRegistry.name(simpleName, key, metricName); } /** @@ -196,8 +167,12 @@ private T doRegisterMetric(String simpleName, String key, */ @SuppressWarnings("unchecked") public T registerMetric(String simpleName, String key, String metricName, Class metricClass) { - Validate.isTrue(!metricClass.equals(Gauge.class), "please call registerGauge() to register a Gauge metric."); - return doRegisterMetric(simpleName, key, metricName, metricClass, null); + if (Gauge.class.isAssignableFrom(metricClass)) { + throw new IllegalArgumentException("Cannot register Gauges; use registerGauge"); + } + String name = formatName(simpleName, key, metricName); + countReference(name); + return getMetric(name, metricClass); } /** @@ -221,7 +196,9 @@ public T registerMetric(String classSimpleName, String metric */ @SuppressWarnings("unchecked") public Gauge registerGauge(String simpleName, String key, String metricName, Supplier supplier) { - return doRegisterMetric(simpleName, key, metricName, Gauge.class, supplier); + String name = formatName(simpleName, key, metricName); + countReference(name); + return _metricRegistry.gauge(name, () -> new ResettableGauge(supplier)); } /** @@ -231,9 +208,16 @@ public Gauge registerGauge(String simpleName, String key, String metricNa * @param supplier value supplier for the Gauge * @return the metric just registered or previously registered one */ - @SuppressWarnings("unchecked") public Gauge registerGauge(String simpleName, String metricName, Supplier supplier) { - return doRegisterMetric(simpleName, null, metricName, Gauge.class, supplier); + return registerGauge(simpleName, null, metricName, supplier); + } + + /** + * Explicitly set the Supplier for a Gauge. + */ + public void setGauge(Gauge gauge, Supplier supplier) { + Validate.isTrue(gauge instanceof ResettableGauge, "Unsupported Gauge impl."); + ((ResettableGauge) gauge).setSupplier(supplier); } /** diff --git a/datastream-file-connector/src/main/java/com/linkedin/datastream/connectors/file/diag/FilePositionKey.java b/datastream-file-connector/src/main/java/com/linkedin/datastream/connectors/file/diag/FilePositionKey.java index 492c3a9f3..9d54aff6f 100644 --- a/datastream-file-connector/src/main/java/com/linkedin/datastream/connectors/file/diag/FilePositionKey.java +++ b/datastream-file-connector/src/main/java/com/linkedin/datastream/connectors/file/diag/FilePositionKey.java @@ -8,11 +8,12 @@ import java.time.Instant; import java.util.Objects; -import org.codehaus.jackson.annotate.JsonProperty; -import org.codehaus.jackson.map.annotate.JsonDeserialize; -import org.codehaus.jackson.map.annotate.JsonSerialize; import org.jetbrains.annotations.NotNull; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.databind.annotation.JsonDeserialize; +import com.fasterxml.jackson.databind.annotation.JsonSerialize; + import com.linkedin.datastream.common.JsonUtils.InstantDeserializer; import com.linkedin.datastream.common.JsonUtils.InstantSerializer; import com.linkedin.datastream.common.diag.PositionKey; diff --git a/datastream-kafka-connector/src/main/java/com/linkedin/datastream/connectors/kafka/AbstractKafkaBasedConnectorTask.java b/datastream-kafka-connector/src/main/java/com/linkedin/datastream/connectors/kafka/AbstractKafkaBasedConnectorTask.java index a9ecff0b8..98918c33e 100644 --- a/datastream-kafka-connector/src/main/java/com/linkedin/datastream/connectors/kafka/AbstractKafkaBasedConnectorTask.java +++ b/datastream-kafka-connector/src/main/java/com/linkedin/datastream/connectors/kafka/AbstractKafkaBasedConnectorTask.java @@ -37,9 +37,9 @@ import org.apache.kafka.common.KafkaException; import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.errors.WakeupException; -import org.codehaus.jackson.type.TypeReference; import org.slf4j.Logger; +import com.fasterxml.jackson.core.type.TypeReference; import com.google.common.annotations.VisibleForTesting; import com.google.common.collect.Sets; @@ -90,10 +90,11 @@ abstract public class AbstractKafkaBasedConnectorTask implements Runnable, Consu // lifecycle private volatile Thread _connectorTaskThread; protected volatile boolean _shutdown = false; + private volatile boolean _skipOnPartitionsRevoked = false; protected volatile long _lastPolledTimeMillis = System.currentTimeMillis(); protected volatile long _lastPollCompletedTimeMillis = 0; protected final CountDownLatch _startedLatch = new CountDownLatch(1); - protected final CountDownLatch _stoppedLatch = new CountDownLatch(1); + private final CountDownLatch _stoppedLatch = new CountDownLatch(1); private final AtomicBoolean _metricDeregistered = new AtomicBoolean(false); // config @@ -255,6 +256,13 @@ protected String getTaskName() { return _taskName; } + /** + * Exposing the flag for overridden classes + */ + protected boolean getSkipOnPartitionsRevoked() { + return _skipOnPartitionsRevoked; + } + /** * Translate the Kafka consumer records if necessary and send the batch of records to destination. * @param records the Kafka consumer records @@ -432,17 +440,22 @@ public void run() { } finally { if (null != _consumer) { try { + _skipOnPartitionsRevoked = true; _consumer.close(); } catch (Exception e) { _logger.warn(String.format("Got exception on consumer close for task %s.", _taskName), e); } } postShutdownHook(); - _stoppedLatch.countDown(); + countDownStoppedLatch(); _logger.info("{} stopped", _taskName); } } + protected void countDownStoppedLatch() { + _stoppedLatch.countDown(); + } + /** * Signal task to stop */ @@ -764,6 +777,10 @@ protected void updateConsumerAssignment(Collection partitions) { @Override public void onPartitionsRevoked(Collection topicPartitions) { + if (_skipOnPartitionsRevoked) { + _logger.info("Skipping commit in onPartitionsRevoked during consumer.close()"); + return; + } _logger.info("Partition ownership revoked for {}, checkpointing.", topicPartitions); _kafkaTopicPartitionTracker.onPartitionsRevoked(topicPartitions); if (!_shutdown && !topicPartitions.isEmpty()) { // there is a commit at the end of the run method, skip extra commit in shouldDie mode. @@ -1099,4 +1116,9 @@ public static String getKafkaGroupId(DatastreamTask task, GroupIdConstructor gro public String getConsumerAutoOffsetResetConfig() { return _consumerProps.getProperty(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, ""); } + + @VisibleForTesting + public long getStoppedLatchCount() { + return _stoppedLatch.getCount(); + } } \ No newline at end of file diff --git a/datastream-kafka-connector/src/main/java/com/linkedin/datastream/connectors/kafka/AbstractKafkaConnector.java b/datastream-kafka-connector/src/main/java/com/linkedin/datastream/connectors/kafka/AbstractKafkaConnector.java index 1fcc8689b..66fd80481 100644 --- a/datastream-kafka-connector/src/main/java/com/linkedin/datastream/connectors/kafka/AbstractKafkaConnector.java +++ b/datastream-kafka-connector/src/main/java/com/linkedin/datastream/connectors/kafka/AbstractKafkaConnector.java @@ -102,7 +102,10 @@ public abstract class AbstractKafkaConnector implements Connector, DiagnosticsAw // multiple concurrent threads. If access is required to both maps then the order of synchronization must be // _runningTasks followed by _tasksToStop to prevent deadlocks. private final Map _runningTasks = new HashMap<>(); - private final Map _tasksToStop = new HashMap<>(); + + // _tasksPendingStop contains the tasks that are pending stop across various assignment changes. The periodic health + // check call will attempt to stop these tasks until they are not stopped / are stuck somewhere in stop path. + private final Map _tasksPendingStop = new HashMap<>(); // A daemon executor to constantly check whether all tasks are running and restart them if not. private final ScheduledExecutorService _daemonThreadExecutorService = @@ -154,23 +157,25 @@ public synchronized void onAssignmentChange(List tasks) { _logger.info("onAssignmentChange called with tasks {}", tasks); synchronized (_runningTasks) { + Map runningTasksToStop = new HashMap<>(); Set toCancel = new HashSet<>(_runningTasks.keySet()); tasks.forEach(toCancel::remove); if (toCancel.size() > 0) { // Mark the connector task as stopped so that, in case stopping the task here fails for any reason in // restartDeadTasks the task is not restarted - synchronized (_tasksToStop) { + synchronized (_tasksPendingStop) { toCancel.forEach(task -> { - _tasksToStop.put(task, _runningTasks.get(task)); + runningTasksToStop.put(task, _runningTasks.get(task)); + _tasksPendingStop.put(task, _runningTasks.get(task)); _runningTasks.remove(task); }); } - stopUnassignedTasks(); + scheduleTasksToStop(runningTasksToStop); } boolean toCallRestartDeadTasks = false; - synchronized (_tasksToStop) { + synchronized (_tasksPendingStop) { for (DatastreamTask task : tasks) { ConnectorTaskEntry connectorTaskEntry = _runningTasks.get(task); if (connectorTaskEntry != null) { @@ -180,16 +185,17 @@ public synchronized void onAssignmentChange(List tasks) { // This is necessary because DatastreamTaskImpl.hashCode() does not take into account all the // fields/properties of the DatastreamTask (e.g. dependencies). _runningTasks.remove(task); - _runningTasks.put(task, connectorTaskEntry); } else { - if (_tasksToStop.containsKey(task)) { + // If a pending stop task is reassigned to this host, we'd have to ensure to restart the + // task or replace the connectorTaskEntry for that task in the restartDeadTasks function. + if (_tasksPendingStop.containsKey(task)) { toCallRestartDeadTasks = true; - connectorTaskEntry = _tasksToStop.remove(task); + connectorTaskEntry = _tasksPendingStop.remove(task); } else { connectorTaskEntry = createKafkaConnectorTask(task); } - _runningTasks.put(task, connectorTaskEntry); } + _runningTasks.put(task, connectorTaskEntry); } } // If any tasks pending stop were re-assigned to this host we explicitly call restartDeadTasks to ensure @@ -259,22 +265,17 @@ protected void restartDeadTasks() { if (isConnectorTaskDead(connectorTaskEntry)) { _logger.warn("Detected that the kafka connector task is not running for datastream task {}. Restarting it", datastreamTask.getDatastreamTaskName()); - if (isTaskThreadDead(connectorTaskEntry)) { - _logger.warn("Task thread for datastream task {} has died. No need to attempt to stop the task", - datastreamTask.getDatastreamTaskName()); + // If stoppedTask is null it means that attempting to stop the task failed and that it will be retired again + // (in the next restartDeadTasks iteration). + // If we dont successfully stop the task before creating another connector task we can potentially end up with + // two connector tasks instances running in parallel. This is possible because the acquire method acts as a + // re-entrant lock if the same host calls the acquire method for the same task multiple times. + DatastreamTask stoppedTask = stopTask(datastreamTask, connectorTaskEntry); + if (stoppedTask != null) { deadDatastreamTasks.add(datastreamTask); } else { - // If stoppedTask is null it means that attempting to stop the task failed and that it will be retired again - // (in the next restartDeadTasks iteration). - // If we dont successfully stop the task before creating another connector task we can potentially end up with - // two connector tasks instances running in parallel. This is possible because the acquire method acts as a - // re-entrant lock if the same host calls the acquire method for the same task multiple times. - DatastreamTask stoppedTask = stopTask(datastreamTask, connectorTaskEntry); - if (stoppedTask != null) { - deadDatastreamTasks.add(datastreamTask); - } else { - _logger.error("Connector task for datastream task {} could not be stopped.", datastreamTask.getDatastreamTaskName()); - } + _logger.error("Connector task for datastream task {} could not be stopped.", + datastreamTask.getDatastreamTaskName()); } } else { _logger.info("Connector task for datastream task {} is healthy", datastreamTask.getDatastreamTaskName()); @@ -293,8 +294,8 @@ protected void restartDeadTasks() { * Returns the number of tasks yet to be stopped. */ int getTasksToStopCount() { - synchronized (_tasksToStop) { - return _tasksToStop.size(); + synchronized (_tasksPendingStop) { + return _tasksPendingStop.size(); } } @@ -308,44 +309,50 @@ int getRunningTasksCount() { } /** - * Attempt to stop the unassigned tasks. + * Attempt to stop the unassigned tasks from the _tasksToStop map. */ private void stopUnassignedTasks() { - synchronized (_tasksToStop) { - if (_tasksToStop.size() == 0) { - _logger.info("No tasks to stop"); - return; - } + scheduleTasksToStop(_tasksPendingStop); + } - // Spawn a separate thread to attempt stopping the connectorTask. The connectorTask will be canceled if it - // does not stop within a certain amount of time. This will force cleanup of connectorTasks which take too long - // to stop, or are stuck indefinitely. A separate thread is spawned to handle this because the Coordinator - // requires that this step completely because we call this from onAssignmentChange() (assignment thread gets - // killed if it takes too long) and restartDeadTasks which must complete quickly. - List> stopTaskFutures = _tasksToStop.keySet().stream() - .map(task -> asyncStopTask(task, _tasksToStop.get(task))) - .collect(Collectors.toList()); + /** + * Attempt to stop the unassigned tasks from the argument map. + */ + private void scheduleTasksToStop(Map tasks) { + if (tasks.size() == 0) { + _logger.info("No tasks to stop"); + return; + } - _shutdownExecutorService.submit(() -> { - List toRemoveTasks = stopTaskFutures.stream().map(stopTaskFuture -> { - try { - return stopTaskFuture.get(CANCEL_TASK_TIMEOUT.plus(POST_CANCEL_TASK_TIMEOUT).toMillis(), TimeUnit.MILLISECONDS); - } catch (ExecutionException | InterruptedException | TimeoutException e) { - _logger.warn("Stop task future failed with exception", e); - } - return null; - }).filter(Objects::nonNull).collect(Collectors.toList()); - - if (toRemoveTasks.size() > 0) { - synchronized (_tasksToStop) { - // Its possible that while stopping the task was pending there was another onAssignmentChange event - // which reassigned the task back to this host and the task was moved back to _runningTasks. In this - // case the remove operation here will be a no-op. - toRemoveTasks.forEach(_tasksToStop::remove); - } + // Spawn a separate thread to attempt stopping the connectorTask. The connectorTask will be canceled if it + // does not stop within a certain amount of time. This will force cleanup of connectorTasks which take too long + // to stop, or are stuck indefinitely. A separate thread is spawned to handle this because the Coordinator + // requires that this step completely because we call this from onAssignmentChange() (assignment thread gets + // killed if it takes too long) and restartDeadTasks which must complete quickly. + List> stopTaskFutures = tasks.keySet().stream() + .map(task -> asyncStopTask(task, tasks.get(task))) + .collect(Collectors.toList()); + + _shutdownExecutorService.submit(() -> { + List toRemoveTasks = stopTaskFutures.stream().map(stopTaskFuture -> { + try { + return stopTaskFuture.get(CANCEL_TASK_TIMEOUT.plus(POST_CANCEL_TASK_TIMEOUT).toMillis(), + TimeUnit.MILLISECONDS); + } catch (ExecutionException | InterruptedException | TimeoutException e) { + _logger.warn("Stop task future failed with exception", e); } - }); - } + return null; + }).filter(Objects::nonNull).collect(Collectors.toList()); + + if (toRemoveTasks.size() > 0) { + synchronized (_tasksPendingStop) { + // Its possible that while stopping the task was pending there was another onAssignmentChange event + // which reassigned the task back to this host and the task was moved back to _runningTasks. In this + // case the remove operation here will be a no-op. + toRemoveTasks.forEach(_tasksPendingStop::remove); + } + } + }); } @NotNull @@ -362,6 +369,11 @@ private Future asyncStopTask(DatastreamTask task, ConnectorTaskE */ private DatastreamTask stopTask(DatastreamTask datastreamTask, ConnectorTaskEntry connectorTaskEntry) { try { + if (isTaskThreadDead(connectorTaskEntry)) { + _logger.warn("Task thread for datastream task {} has died. No need to attempt to stop the task", + datastreamTask.getDatastreamTaskName()); + return datastreamTask; + } connectorTaskEntry.setPendingStop(); AbstractKafkaBasedConnectorTask connectorTask = connectorTaskEntry.getConnectorTask(); connectorTask.stop(); @@ -420,10 +432,10 @@ public void stop() { _runningTasks.forEach(this::asyncStopTask); _runningTasks.clear(); } - synchronized (_tasksToStop) { + synchronized (_tasksPendingStop) { // Try to stop the tasks - _tasksToStop.forEach(this::asyncStopTask); - _tasksToStop.clear(); + _tasksPendingStop.forEach(this::asyncStopTask); + _tasksPendingStop.clear(); } _logger.info("Start to shut down the shutdown executor and wait up to {} ms.", SHUTDOWN_EXECUTOR_SHUTDOWN_TIMEOUT.toMillis()); diff --git a/datastream-kafka-connector/src/main/java/com/linkedin/datastream/connectors/kafka/KafkaBasedConnectorConfig.java b/datastream-kafka-connector/src/main/java/com/linkedin/datastream/connectors/kafka/KafkaBasedConnectorConfig.java index cbab995b4..2bbf8db1c 100644 --- a/datastream-kafka-connector/src/main/java/com/linkedin/datastream/connectors/kafka/KafkaBasedConnectorConfig.java +++ b/datastream-kafka-connector/src/main/java/com/linkedin/datastream/connectors/kafka/KafkaBasedConnectorConfig.java @@ -13,7 +13,8 @@ import com.linkedin.datastream.common.VerifiableProperties; import com.linkedin.datastream.kafka.factory.KafkaConsumerFactory; import com.linkedin.datastream.kafka.factory.KafkaConsumerFactoryImpl; - +import com.linkedin.datastream.server.callbackstatus.CallbackStatusFactory; +import com.linkedin.datastream.server.callbackstatus.CallbackStatusWithComparableOffsetsFactory; /** * Configs for Kafka-based connectors. @@ -36,6 +37,8 @@ public class KafkaBasedConnectorConfig { public static final String DAEMON_THREAD_INTERVAL_SECONDS = "daemonThreadIntervalInSeconds"; public static final String NON_GOOD_STATE_THRESHOLD_MILLIS = "nonGoodStateThresholdMs"; public static final String PROCESSING_DELAY_LOG_THRESHOLD_MILLIS = "processingDelayLogThreshold"; + private static final String CONFIG_CALLBACK_STATUS_STRATEGY_FACTORY_CLASS = "callbackStatusStrategyFactoryClass"; + // config value to enable Kafka partition management for KafkaMirrorConnector public static final String ENABLE_PARTITION_ASSIGNMENT = "enablePartitionAssignment"; public static final long DEFAULT_NON_GOOD_STATE_THRESHOLD_MILLIS = Duration.ofMinutes(10).toMillis(); @@ -72,6 +75,9 @@ public class KafkaBasedConnectorConfig { private final long _nonGoodStateThresholdMillis; private final boolean _enablePartitionAssignment; + // Kafka based pub sub framework uses Long as their offset type, hence instantiating a Long parameterized factory + private final CallbackStatusFactory _callbackStatusStrategyFactory; + /** * Constructor for KafkaBasedConnectorConfig. * @param properties Properties to use for creating config. @@ -110,6 +116,13 @@ public KafkaBasedConnectorConfig(Properties properties) { INCLUDE_DATASTREAM_NAME_IN_CONSUMER_CLIENT_ID, DEFAULT_INCLUDE_DATASTREAM_NAME_IN_CONSUMER_CLIENT_ID); _enablePartitionAssignment = verifiableProperties.getBoolean(ENABLE_PARTITION_ASSIGNMENT, Boolean.FALSE); + String callbackStatusStrategyFactoryClass = verifiableProperties.getString(CONFIG_CALLBACK_STATUS_STRATEGY_FACTORY_CLASS, + CallbackStatusWithComparableOffsetsFactory.class.getName()); + _callbackStatusStrategyFactory = ReflectionUtils.createInstance(callbackStatusStrategyFactoryClass); + if (_callbackStatusStrategyFactory == null) { + throw new DatastreamRuntimeException("Unable to instantiate factory class: " + callbackStatusStrategyFactoryClass); + } + String factory = verifiableProperties.getString(CONFIG_CONSUMER_FACTORY_CLASS, KafkaConsumerFactoryImpl.class.getName()); _consumerFactory = ReflectionUtils.createInstance(factory); @@ -197,4 +210,8 @@ public long getProcessingDelayLogThresholdMillis() { public boolean getEnablePartitionAssignment() { return _enablePartitionAssignment; } + + public CallbackStatusFactory getCallbackStatusStrategyFactory() { + return _callbackStatusStrategyFactory; + } } diff --git a/datastream-kafka-connector/src/main/java/com/linkedin/datastream/connectors/kafka/KafkaConnector.java b/datastream-kafka-connector/src/main/java/com/linkedin/datastream/connectors/kafka/KafkaConnector.java index f13d2eb31..5355a2f9b 100644 --- a/datastream-kafka-connector/src/main/java/com/linkedin/datastream/connectors/kafka/KafkaConnector.java +++ b/datastream-kafka-connector/src/main/java/com/linkedin/datastream/connectors/kafka/KafkaConnector.java @@ -18,10 +18,11 @@ import org.apache.commons.lang.StringUtils; import org.apache.kafka.clients.consumer.Consumer; import org.apache.kafka.common.PartitionInfo; -import org.codehaus.jackson.type.TypeReference; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import com.fasterxml.jackson.core.type.TypeReference; + import com.linkedin.datastream.common.Datastream; import com.linkedin.datastream.common.DatastreamMetadataConstants; import com.linkedin.datastream.common.DatastreamSource; diff --git a/datastream-kafka-connector/src/main/java/com/linkedin/datastream/connectors/kafka/KafkaConnectorDiagUtils.java b/datastream-kafka-connector/src/main/java/com/linkedin/datastream/connectors/kafka/KafkaConnectorDiagUtils.java index f1848c57c..0a60767ee 100644 --- a/datastream-kafka-connector/src/main/java/com/linkedin/datastream/connectors/kafka/KafkaConnectorDiagUtils.java +++ b/datastream-kafka-connector/src/main/java/com/linkedin/datastream/connectors/kafka/KafkaConnectorDiagUtils.java @@ -12,9 +12,10 @@ import java.util.Map; import org.apache.commons.lang3.StringUtils; -import org.codehaus.jackson.type.TypeReference; import org.slf4j.Logger; +import com.fasterxml.jackson.core.type.TypeReference; + import com.linkedin.datastream.common.JsonUtils; diff --git a/datastream-kafka-connector/src/main/java/com/linkedin/datastream/connectors/kafka/KafkaConsumerOffsetsResponse.java b/datastream-kafka-connector/src/main/java/com/linkedin/datastream/connectors/kafka/KafkaConsumerOffsetsResponse.java index 693b3a956..d228b47a3 100644 --- a/datastream-kafka-connector/src/main/java/com/linkedin/datastream/connectors/kafka/KafkaConsumerOffsetsResponse.java +++ b/datastream-kafka-connector/src/main/java/com/linkedin/datastream/connectors/kafka/KafkaConsumerOffsetsResponse.java @@ -8,7 +8,7 @@ import java.util.HashMap; import java.util.Map; -import org.codehaus.jackson.annotate.JsonProperty; +import com.fasterxml.jackson.annotation.JsonProperty; /** diff --git a/datastream-kafka-connector/src/main/java/com/linkedin/datastream/connectors/kafka/KafkaDatastreamStatesResponse.java b/datastream-kafka-connector/src/main/java/com/linkedin/datastream/connectors/kafka/KafkaDatastreamStatesResponse.java index d189d825e..2890d2294 100644 --- a/datastream-kafka-connector/src/main/java/com/linkedin/datastream/connectors/kafka/KafkaDatastreamStatesResponse.java +++ b/datastream-kafka-connector/src/main/java/com/linkedin/datastream/connectors/kafka/KafkaDatastreamStatesResponse.java @@ -11,17 +11,19 @@ import java.util.Set; import org.apache.kafka.common.TopicPartition; -import org.codehaus.jackson.JsonParser; -import org.codehaus.jackson.Version; -import org.codehaus.jackson.annotate.JsonPropertyOrder; -import org.codehaus.jackson.map.DeserializationConfig; -import org.codehaus.jackson.map.DeserializationContext; -import org.codehaus.jackson.map.JsonDeserializer; -import org.codehaus.jackson.map.KeyDeserializer; -import org.codehaus.jackson.map.ObjectMapper; -import org.codehaus.jackson.map.annotate.JsonSerialize; -import org.codehaus.jackson.map.module.SimpleModule; -import org.codehaus.jackson.map.ser.ToStringSerializer; + +import com.fasterxml.jackson.annotation.JsonPropertyOrder; +import com.fasterxml.jackson.core.JsonParser; +import com.fasterxml.jackson.core.Version; +import com.fasterxml.jackson.databind.DeserializationContext; +import com.fasterxml.jackson.databind.DeserializationFeature; +import com.fasterxml.jackson.databind.JsonDeserializer; +import com.fasterxml.jackson.databind.JsonNode; +import com.fasterxml.jackson.databind.KeyDeserializer; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.fasterxml.jackson.databind.annotation.JsonSerialize; +import com.fasterxml.jackson.databind.module.SimpleModule; +import com.fasterxml.jackson.databind.ser.std.ToStringSerializer; import com.linkedin.datastream.common.JsonUtils; import com.linkedin.datastream.server.FlushlessEventProducerHandler; @@ -106,7 +108,7 @@ public static KafkaDatastreamStatesResponse fromJson(String json) { simpleModule.addDeserializer(TopicPartition.class, TopicPartitionDeserializer.getInstance()); ObjectMapper mapper = new ObjectMapper(); mapper.registerModule(simpleModule); - mapper.configure(DeserializationConfig.Feature.FAIL_ON_UNKNOWN_PROPERTIES, false); + mapper.configure(DeserializationFeature.FAIL_ON_UNKNOWN_PROPERTIES, false); return JsonUtils.fromJson(json, KafkaDatastreamStatesResponse.class, mapper); } @@ -195,7 +197,7 @@ static TopicPartitionDeserializer getInstance() { @Override public TopicPartition deserialize(JsonParser jp, DeserializationContext ctxt) throws IOException { - String topicPartition = jp.getCodec().readTree(jp).getTextValue(); + String topicPartition = ((JsonNode) jp.getCodec().readTree(jp)).asText(); return topicPartitionFromString(topicPartition); } } diff --git a/datastream-kafka-connector/src/main/java/com/linkedin/datastream/connectors/kafka/KafkaTopicPartitionStatsResponse.java b/datastream-kafka-connector/src/main/java/com/linkedin/datastream/connectors/kafka/KafkaTopicPartitionStatsResponse.java index b1fed74ea..888764914 100644 --- a/datastream-kafka-connector/src/main/java/com/linkedin/datastream/connectors/kafka/KafkaTopicPartitionStatsResponse.java +++ b/datastream-kafka-connector/src/main/java/com/linkedin/datastream/connectors/kafka/KafkaTopicPartitionStatsResponse.java @@ -10,8 +10,8 @@ import java.util.Map; import java.util.Set; -import org.codehaus.jackson.annotate.JsonCreator; -import org.codehaus.jackson.annotate.JsonProperty; +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; /** diff --git a/datastream-kafka-connector/src/main/java/com/linkedin/datastream/connectors/kafka/PausedSourcePartitionMetadata.java b/datastream-kafka-connector/src/main/java/com/linkedin/datastream/connectors/kafka/PausedSourcePartitionMetadata.java index c054b8bc8..030763df5 100644 --- a/datastream-kafka-connector/src/main/java/com/linkedin/datastream/connectors/kafka/PausedSourcePartitionMetadata.java +++ b/datastream-kafka-connector/src/main/java/com/linkedin/datastream/connectors/kafka/PausedSourcePartitionMetadata.java @@ -11,7 +11,8 @@ import java.util.function.BooleanSupplier; import org.apache.commons.lang.exception.ExceptionUtils; -import org.codehaus.jackson.annotate.JsonPropertyOrder; + +import com.fasterxml.jackson.annotation.JsonPropertyOrder; /** diff --git a/datastream-kafka-connector/src/main/java/com/linkedin/datastream/connectors/kafka/mirrormaker/KafkaMirrorMakerConnectorTask.java b/datastream-kafka-connector/src/main/java/com/linkedin/datastream/connectors/kafka/mirrormaker/KafkaMirrorMakerConnectorTask.java index e14dee799..676a170c9 100644 --- a/datastream-kafka-connector/src/main/java/com/linkedin/datastream/connectors/kafka/mirrormaker/KafkaMirrorMakerConnectorTask.java +++ b/datastream-kafka-connector/src/main/java/com/linkedin/datastream/connectors/kafka/mirrormaker/KafkaMirrorMakerConnectorTask.java @@ -19,8 +19,8 @@ import java.util.regex.Pattern; import java.util.stream.Collectors; -import org.I0Itec.zkclient.exception.ZkInterruptedException; import org.apache.commons.lang3.StringUtils; +import org.apache.helix.zookeeper.zkclient.exception.ZkInterruptedException; import org.apache.kafka.clients.consumer.Consumer; import org.apache.kafka.clients.consumer.ConsumerConfig; import org.apache.kafka.clients.consumer.ConsumerRecord; @@ -162,7 +162,8 @@ public KafkaMirrorMakerConnectorTask(KafkaBasedConnectorConfig config, Datastrea LOG.info("Destination topic prefix has been set to {}", _destinationTopicPrefix); if (_isFlushlessModeEnabled) { - _flushlessProducer = new FlushlessEventProducerHandler<>(_producer); + _flushlessProducer = new FlushlessEventProducerHandler(_producer, + config.getCallbackStatusStrategyFactory()); _flowControlEnabled = config.getConnectorProps().getBoolean(CONFIG_FLOW_CONTROL_ENABLED, false); _maxInFlightMessagesThreshold = config.getConnectorProps().getLong(CONFIG_MAX_IN_FLIGHT_MSGS_THRESHOLD, DEFAULT_MAX_IN_FLIGHT_MSGS_THRESHOLD); @@ -352,9 +353,14 @@ public void run() { LOG.info("Trying to acquire the lock on datastreamTask: {}", _datastreamTask); _datastreamTask.acquire(LOCK_ACQUIRE_TIMEOUT); } catch (DatastreamRuntimeException ex) { + // setting _stoppedLatch count to 0 since the lock couldn't be acquired, + // as a non-zero stoppedLatch value won't let the task to be stopped. + countDownStoppedLatch(); LOG.error(String.format("Failed to acquire lock for datastreamTask %s", _datastreamTask), ex); _dynamicMetricsManager.createOrUpdateMeter(generateMetricsPrefix(_connectorName, CLASS_NAME), _datastreamName, TASK_LOCK_ACQUIRE_ERROR_RATE, 1); + // This exception should not be swallowed as it is fatal and can cause multiple instances + // to work on the same task concurrently throw ex; } } @@ -369,6 +375,10 @@ public void stop() { @Override public void onPartitionsRevoked(Collection partitions) { + if (getSkipOnPartitionsRevoked()) { + _logger.info("Skipping commit in onPartitionsRevoked during consumer.close()"); + return; + } super.onPartitionsRevoked(partitions); _topicManager.onPartitionsRevoked(partitions); } diff --git a/datastream-kafka-connector/src/test/java/com/linkedin/datastream/connectors/kafka/TestAbstractKafkaConnector.java b/datastream-kafka-connector/src/test/java/com/linkedin/datastream/connectors/kafka/TestAbstractKafkaConnector.java index 17e242b0b..39f78a1a3 100644 --- a/datastream-kafka-connector/src/test/java/com/linkedin/datastream/connectors/kafka/TestAbstractKafkaConnector.java +++ b/datastream-kafka-connector/src/test/java/com/linkedin/datastream/connectors/kafka/TestAbstractKafkaConnector.java @@ -14,6 +14,10 @@ import java.util.Collections; import java.util.List; import java.util.Properties; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.TimeUnit; +import java.util.stream.IntStream; import java.util.stream.Stream; import org.slf4j.Logger; @@ -84,7 +88,7 @@ public void testOnAssignmentChangeReassignment() { props.setProperty("daemonThreadIntervalInSeconds", "2"); // With failStopTaskOnce set to true the AbstractKafkaBasedConnectorTask.stop is configured // to fail the first time with InterruptedException and pass the second time. - TestKafkaConnector connector = new TestKafkaConnector(false, props, true); + TestKafkaConnector connector = new TestKafkaConnector(false, props, true, false); DatastreamTaskImpl datastreamTask1 = new DatastreamTaskImpl(); datastreamTask1.setTaskPrefix("testtask1"); connector.onAssignmentChange(Collections.singletonList(datastreamTask1)); @@ -114,7 +118,7 @@ public void testOnAssignmentChangeStopTaskFailure() { props.setProperty("daemonThreadIntervalInSeconds", "2"); // With failStopTaskOnce set to true the AbstractKafkaBasedConnectorTask.stop is configured // to fail the first time with InterruptedException and pass the second time. - TestKafkaConnector connector = new TestKafkaConnector(false, props, true); + TestKafkaConnector connector = new TestKafkaConnector(false, props, true, false); DatastreamTaskImpl datastreamTask = new DatastreamTaskImpl(); datastreamTask.setTaskPrefix("testtask1"); connector.onAssignmentChange(Collections.singletonList(datastreamTask)); @@ -138,6 +142,51 @@ public void testOnAssignmentChangeStopTaskFailure() { connector.stop(); } + @Test + public void testOnAssignmentChangeMultipleReassignments() throws InterruptedException { + Properties props = new Properties(); + // Reduce time interval between calls to restartDeadTasks to force invocation of stopTasks + props.setProperty("daemonThreadIntervalInSeconds", "2"); + // With failStopTaskOnce set to true the AbstractKafkaBasedConnectorTask.stop is configured + // to fail the first time with InterruptedException and pass the second time. + TestKafkaConnector connector = new TestKafkaConnector(false, props, true, false); + + // first task assignment assigns task 1 + List firstTaskAssignment = getTaskListInRange(1, 2); + connector.onAssignmentChange(firstTaskAssignment); + connector.start(null); + Assert.assertEquals(connector.getRunningTasksCount(), 1); + + // second task assignment assigns task 2,3,4,5 and takes out task 1 + List secondTaskAssignment = getTaskListInRange(2, 6); + + // during the assignment, the _taskToStop map count need to be less than 1, as only task 1 would be taken out. + ExecutorService executor = Executors.newFixedThreadPool(2); + executor.execute(() -> connector.onAssignmentChange(secondTaskAssignment)); + executor.execute(() -> Assert.assertTrue(connector.getTasksToStopCount() <= 1)); + + awaitForExecution(executor, 50L); + Assert.assertTrue(connector.getTasksToStopCount() >= 1); // the count of the _taskToStopTracker + Assert.assertEquals(connector.getRunningTasksCount(), 4); + + // second task assignment keeps task 5, assigns task 6,7,8 and takes out task 2,3,4 + List thirdTaskAssignment = getTaskListInRange(5, 9); + + // during the assignment, the _taskToStop map count need to be less than 4, as task 2,3,4 would be taken out and task 1 if not already stopped. + executor = Executors.newFixedThreadPool(2); + executor.execute(() -> connector.onAssignmentChange(thirdTaskAssignment)); + executor.execute(() -> Assert.assertTrue(connector.getTasksToStopCount() <= 4)); + + awaitForExecution(executor, 50L); + Assert.assertTrue(connector.getTasksToStopCount() >= 3); // the count of the _taskToStopTracker + + // Wait for restartDeadTasks to be called to attempt another stopTasks call + PollUtils.poll(() -> connector.getCreateTaskCalled() >= 3, Duration.ofSeconds(1).toMillis(), + Duration.ofSeconds(10).toMillis()); + Assert.assertEquals(connector.getRunningTasksCount(), 4); + connector.stop(); + } + @Test public void testCalculateThreadStartDelay() { Properties props = new Properties(); @@ -191,6 +240,26 @@ public void testRestartThrowsException() { connector.stop(); } + // helper method to generate the tasks in a range for assignment + private List getTaskListInRange(int start, int end) { + List taskAssignmentList = new ArrayList<>(); + IntStream.range(start, end).forEach(index -> { + DatastreamTaskImpl dt = new DatastreamTaskImpl(); + dt.setTaskPrefix("testtask" + index); + taskAssignmentList.add(dt); + }); + return taskAssignmentList; + } + + // helper method to await on the executor for the given timeout period + private void awaitForExecution(ExecutorService executor, Long timeUnitMs) throws InterruptedException { + try { + executor.awaitTermination(timeUnitMs, TimeUnit.MILLISECONDS); + } finally { + executor.shutdownNow(); + } + } + /** * Dummy implementation of {@link AbstractKafkaConnector} for testing purposes */ @@ -199,19 +268,34 @@ public class TestKafkaConnector extends AbstractKafkaConnector { private boolean _failStopTaskOnce; private int _createTaskCalled = 0; private int _stopTaskCalled = 0; + private boolean _taskThreadDead; /** * Constructor for TestKafkaConnector * @param restartThrows Indicates whether calling {@link #restartDeadTasks()} * for the first time should throw a {@link RuntimeException} * @param props Configuration properties to use + * @param failStopTaskOnce Fails Stopping task once + * @param taskThreadDead Mocks if the task thread is dead or alive */ - public TestKafkaConnector(boolean restartThrows, Properties props, boolean failStopTaskOnce) { + public TestKafkaConnector(boolean restartThrows, Properties props, boolean failStopTaskOnce, boolean taskThreadDead) { super("test", props, new KafkaGroupIdConstructor( Boolean.parseBoolean(props.getProperty(IS_GROUP_ID_HASHING_ENABLED, Boolean.FALSE.toString())), "TestkafkaConnectorCluster"), "TestkafkaConnectorCluster", LOG); _restartThrows = restartThrows; _failStopTaskOnce = failStopTaskOnce; + _taskThreadDead = taskThreadDead; + } + + /** + * Constructor for TestKafkaConnector + * @param restartThrows Indicates whether calling {@link #restartDeadTasks()} + * for the first time should throw a {@link RuntimeException} + * @param props Configuration properties to use + * @param failStopTaskOnce Fails Stopping task once + */ + public TestKafkaConnector(boolean restartThrows, Properties props, boolean failStopTaskOnce) { + this(restartThrows, props, failStopTaskOnce, true); } @Override @@ -241,7 +325,7 @@ protected boolean isConnectorTaskDead(ConnectorTaskEntry connectorTaskEntry) { @Override protected boolean isTaskThreadDead(ConnectorTaskEntry connectorTaskEntry) { - return true; + return _taskThreadDead; } @Override diff --git a/datastream-kafka-connector/src/test/java/com/linkedin/datastream/connectors/kafka/mirrormaker/KafkaMirrorMakerConnectorTestUtils.java b/datastream-kafka-connector/src/test/java/com/linkedin/datastream/connectors/kafka/mirrormaker/KafkaMirrorMakerConnectorTestUtils.java index 1110183a8..358f63f59 100644 --- a/datastream-kafka-connector/src/test/java/com/linkedin/datastream/connectors/kafka/mirrormaker/KafkaMirrorMakerConnectorTestUtils.java +++ b/datastream-kafka-connector/src/test/java/com/linkedin/datastream/connectors/kafka/mirrormaker/KafkaMirrorMakerConnectorTestUtils.java @@ -27,8 +27,6 @@ import com.linkedin.datastream.connectors.kafka.KafkaBasedConnectorConfig; import com.linkedin.datastream.connectors.kafka.KafkaBasedConnectorConfigBuilder; import com.linkedin.datastream.connectors.kafka.LiKafkaConsumerFactory; -import com.linkedin.datastream.connectors.kafka.NoOpAuditor; -import com.linkedin.datastream.connectors.kafka.NoOpSegmentDeserializer; import com.linkedin.datastream.server.DatastreamTaskImpl; import com.linkedin.datastream.testutil.DatastreamEmbeddedZookeeperKafkaCluster; @@ -163,24 +161,10 @@ static Thread runKafkaMirrorMakerConnectorTask(KafkaMirrorMakerConnectorTask con static KafkaBasedConnectorConfigBuilder getKafkaBasedConnectorConfigBuilder() { return new KafkaBasedConnectorConfigBuilder() - .setConsumerProps(getKafkaConsumerProperties()) .setPausePartitionOnError(true) .setPauseErrorPartitionDuration(Duration.ofSeconds(5)); } - /** - * Returns properties that will be used to configure Kafka consumer in BMM. - * Right now it returns No Op Segment Deserializer and No Op Auditor, as BMM doesn't need to assemble/disassemble - * any message, it just needs to do byte-byte copying. - * @return Properties to be used by Kafka consumer in BMM. - */ - static Properties getKafkaConsumerProperties() { - Properties props = new Properties(); - props.put("segment.deserializer.class", NoOpSegmentDeserializer.class.getCanonicalName()); - props.put("auditor.class", NoOpAuditor.class.getCanonicalName()); - return props; - } - /** * Get the default config properties of a Kafka-based connector * @param override Configuration properties to override default config properties diff --git a/datastream-kafka-connector/src/test/java/com/linkedin/datastream/connectors/kafka/mirrormaker/TestKafkaConsumerOffsets.java b/datastream-kafka-connector/src/test/java/com/linkedin/datastream/connectors/kafka/mirrormaker/TestKafkaConsumerOffsets.java index bdff8c108..824d9439b 100644 --- a/datastream-kafka-connector/src/test/java/com/linkedin/datastream/connectors/kafka/mirrormaker/TestKafkaConsumerOffsets.java +++ b/datastream-kafka-connector/src/test/java/com/linkedin/datastream/connectors/kafka/mirrormaker/TestKafkaConsumerOffsets.java @@ -13,12 +13,13 @@ import java.util.Optional; import java.util.stream.IntStream; -import org.codehaus.jackson.type.TypeReference; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.testng.Assert; import org.testng.annotations.Test; +import com.fasterxml.jackson.core.type.TypeReference; + import com.linkedin.datastream.common.Datastream; import com.linkedin.datastream.common.JsonUtils; import com.linkedin.datastream.common.PollUtils; diff --git a/datastream-kafka-connector/src/test/java/com/linkedin/datastream/connectors/kafka/mirrormaker/TestKafkaMirrorMakerConnector.java b/datastream-kafka-connector/src/test/java/com/linkedin/datastream/connectors/kafka/mirrormaker/TestKafkaMirrorMakerConnector.java index a3234e510..19529aa79 100644 --- a/datastream-kafka-connector/src/test/java/com/linkedin/datastream/connectors/kafka/mirrormaker/TestKafkaMirrorMakerConnector.java +++ b/datastream-kafka-connector/src/test/java/com/linkedin/datastream/connectors/kafka/mirrormaker/TestKafkaMirrorMakerConnector.java @@ -21,12 +21,12 @@ import org.apache.kafka.clients.consumer.ConsumerConfig; import org.apache.kafka.common.TopicPartition; -import org.codehaus.jackson.type.TypeReference; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.testng.Assert; import org.testng.annotations.Test; +import com.fasterxml.jackson.core.type.TypeReference; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableSet; import com.google.common.collect.Sets; diff --git a/datastream-kafka-connector/src/test/java/com/linkedin/datastream/connectors/kafka/mirrormaker/TestKafkaMirrorMakerConnectorTask.java b/datastream-kafka-connector/src/test/java/com/linkedin/datastream/connectors/kafka/mirrormaker/TestKafkaMirrorMakerConnectorTask.java index 5e851f052..a1ad5f114 100644 --- a/datastream-kafka-connector/src/test/java/com/linkedin/datastream/connectors/kafka/mirrormaker/TestKafkaMirrorMakerConnectorTask.java +++ b/datastream-kafka-connector/src/test/java/com/linkedin/datastream/connectors/kafka/mirrormaker/TestKafkaMirrorMakerConnectorTask.java @@ -462,6 +462,35 @@ public List getMetricInfos() { }, DynamicMetricsManager.getInstance()); } + @Test + public void testLatchCountVarOnLockAcquireFailure() throws InterruptedException { + KafkaBasedConnectorConfig connectorConfig = KafkaMirrorMakerConnectorTestUtils.getKafkaBasedConnectorConfigBuilder() + .setEnablePartitionManaged(true) + .build(); + + Datastream ds = KafkaMirrorMakerConnectorTestUtils.createDatastream("ds", "test:1111", "*"); + + DatastreamTaskImpl spyDatastreamTask = spy(new DatastreamTaskImpl(Collections.singletonList(ds))); + spyDatastreamTask.setTaskPrefix("test"); + + // explicitly throwing exception when lock is acquired + doThrow(DatastreamRuntimeException.class).when(spyDatastreamTask).acquire(any()); + + KafkaMirrorMakerConnectorTask connectorTask = + new KafkaMirrorMakerConnectorTask(connectorConfig, spyDatastreamTask, "mirrormaker", false, + new KafkaMirrorMakerGroupIdConstructor(false, "testCluster")); + + // Before acquiring the lock of any task, the latch counter should be 1 + Assert.assertEquals(connectorTask.getStoppedLatchCount(), 1); + + Thread t = new Thread(connectorTask); + t.start(); + t.join(); + + // Irrespective of success or failure to acquire, the latch counter should become 0 + Assert.assertEquals(connectorTask.getStoppedLatchCount(), 0); + } + @Test public void testPartitionManagedLockReleaseOnInterruptException() throws InterruptedException { Datastream datastream = KafkaMirrorMakerConnectorTestUtils.createDatastream("pizzaStream", _broker, "\\w+Pizza"); @@ -801,7 +830,7 @@ public void testAutoPauseOnSendFailure() throws Exception { new MockDatastreamEventProducer((r) -> new String((byte[]) r.getEvents().get(0).key().get()).equals("key-2")); task.setEventProducer(datastreamProducer); - Properties consumerProps = KafkaMirrorMakerConnectorTestUtils.getKafkaConsumerProperties(); + Properties consumerProps = new Properties(); consumerProps.put(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "earliest"); KafkaMirrorMakerConnectorTask connectorTask = KafkaMirrorMakerConnectorTestUtils.createKafkaMirrorMakerConnectorTask(task, @@ -884,7 +913,7 @@ public void testAutoPauseAndResumeOnSendFailure() throws Exception { new MockDatastreamEventProducer((r) -> new String((byte[]) r.getEvents().get(0).key().get()).equals("key-2")); task.setEventProducer(datastreamProducer); - Properties consumerProps = KafkaMirrorMakerConnectorTestUtils.getKafkaConsumerProperties(); + Properties consumerProps = new Properties(); consumerProps.put(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "earliest"); KafkaMirrorMakerConnectorTask connectorTask = KafkaMirrorMakerConnectorTestUtils.createKafkaMirrorMakerConnectorTask(task, @@ -960,7 +989,7 @@ private void testValidateTaskDiesOnRewindFailure(boolean failOnGetLastCheckpoint createAndConnectZkAdapter(task); - Properties consumerProps = KafkaMirrorMakerConnectorTestUtils.getKafkaConsumerProperties(); + Properties consumerProps = new Properties(); consumerProps.put(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "earliest"); KafkaBasedConnectorConfig connectorConfig = KafkaMirrorMakerConnectorTestUtils .getKafkaBasedConnectorConfigBuilder() diff --git a/datastream-kafka-connector/src/test/java/com/linkedin/datastream/connectors/kafka/mirrormaker/TestKafkaTopicPartitionStats.java b/datastream-kafka-connector/src/test/java/com/linkedin/datastream/connectors/kafka/mirrormaker/TestKafkaTopicPartitionStats.java index c2ce4b701..f3b829565 100644 --- a/datastream-kafka-connector/src/test/java/com/linkedin/datastream/connectors/kafka/mirrormaker/TestKafkaTopicPartitionStats.java +++ b/datastream-kafka-connector/src/test/java/com/linkedin/datastream/connectors/kafka/mirrormaker/TestKafkaTopicPartitionStats.java @@ -15,10 +15,10 @@ import java.util.Optional; import java.util.Set; -import org.codehaus.jackson.type.TypeReference; import org.testng.Assert; import org.testng.annotations.Test; +import com.fasterxml.jackson.core.type.TypeReference; import com.google.common.collect.ImmutableSet; import com.linkedin.datastream.common.Datastream; diff --git a/datastream-kafka-factory-impl/src/main/java/com/linkedin/datastream/connectors/kafka/LiKafkaConsumerFactory.java b/datastream-kafka-factory-impl/src/main/java/com/linkedin/datastream/connectors/kafka/LiKafkaConsumerFactory.java index 20b02316c..f6077d23c 100644 --- a/datastream-kafka-factory-impl/src/main/java/com/linkedin/datastream/connectors/kafka/LiKafkaConsumerFactory.java +++ b/datastream-kafka-factory-impl/src/main/java/com/linkedin/datastream/connectors/kafka/LiKafkaConsumerFactory.java @@ -8,10 +8,10 @@ import java.util.Properties; import org.apache.kafka.clients.consumer.Consumer; +import org.apache.kafka.clients.consumer.KafkaConsumer; import org.apache.kafka.common.serialization.ByteArrayDeserializer; import com.linkedin.datastream.kafka.factory.KafkaConsumerFactory; -import com.linkedin.kafka.clients.consumer.LiKafkaConsumerImpl; /** @@ -23,6 +23,6 @@ public class LiKafkaConsumerFactory implements KafkaConsumerFactory createConsumer(Properties properties) { properties.put("key.deserializer", ByteArrayDeserializer.class.getCanonicalName()); properties.put("value.deserializer", ByteArrayDeserializer.class.getCanonicalName()); - return new LiKafkaConsumerImpl<>(properties); + return new KafkaConsumer<>(properties); } } diff --git a/datastream-kafka-factory-impl/src/main/java/com/linkedin/datastream/connectors/kafka/NoOpAuditor.java b/datastream-kafka-factory-impl/src/main/java/com/linkedin/datastream/connectors/kafka/NoOpAuditor.java deleted file mode 100644 index d39e00011..000000000 --- a/datastream-kafka-factory-impl/src/main/java/com/linkedin/datastream/connectors/kafka/NoOpAuditor.java +++ /dev/null @@ -1,52 +0,0 @@ -/** - * Copyright 2019 LinkedIn Corporation. All rights reserved. - * Licensed under the BSD 2-Clause License. See the LICENSE file in the project root for license information. - * See the NOTICE file in the project root for additional information regarding copyright ownership. - */ -package com.linkedin.datastream.connectors.kafka; - -import java.util.Map; -import java.util.concurrent.TimeUnit; - -import com.linkedin.kafka.clients.auditing.AuditType; -import com.linkedin.kafka.clients.auditing.Auditor; - - -/** - * An {@link Auditor} implementation that does nothing - * @param Record key type - * @param Record value type - */ -public class NoOpAuditor implements Auditor { - - @Override - public void configure(Map configs) { - - } - - @Override - public void start() { - - } - - @Override - public Object auditToken(K key, V value) { - return null; - } - - @Override - public void record(Object auditToken, String topic, Long timestamp, Long messageCount, Long bytesCount, - AuditType auditType) { - - } - - @Override - public void close(long timeout, TimeUnit unit) { - - } - - @Override - public void close() { - - } -} \ No newline at end of file diff --git a/datastream-kafka-factory-impl/src/main/java/com/linkedin/datastream/connectors/kafka/NoOpSegmentDeserializer.java b/datastream-kafka-factory-impl/src/main/java/com/linkedin/datastream/connectors/kafka/NoOpSegmentDeserializer.java deleted file mode 100644 index 9808be0d0..000000000 --- a/datastream-kafka-factory-impl/src/main/java/com/linkedin/datastream/connectors/kafka/NoOpSegmentDeserializer.java +++ /dev/null @@ -1,32 +0,0 @@ -/** - * Copyright 2019 LinkedIn Corporation. All rights reserved. - * Licensed under the BSD 2-Clause License. See the LICENSE file in the project root for license information. - * See the NOTICE file in the project root for additional information regarding copyright ownership. - */ -package com.linkedin.datastream.connectors.kafka; - -import java.util.Map; - -import org.apache.kafka.common.serialization.Deserializer; - -import com.linkedin.kafka.clients.largemessage.LargeMessageSegment; - - -/** - * A {@link Deserializer} implementation that does nothing. - */ -public class NoOpSegmentDeserializer implements Deserializer { - @Override - public void configure(Map configs, boolean isKey) { - } - - @Override - public LargeMessageSegment deserialize(String s, byte[] bytes) { - return null; - } - - @Override - public void close() { - - } -} diff --git a/datastream-kafka/src/main/java/com/linkedin/datastream/kafka/EmbeddedKafkaCluster.java b/datastream-kafka/src/main/java/com/linkedin/datastream/kafka/EmbeddedKafkaCluster.java index 96241861d..832d449ca 100644 --- a/datastream-kafka/src/main/java/com/linkedin/datastream/kafka/EmbeddedKafkaCluster.java +++ b/datastream-kafka/src/main/java/com/linkedin/datastream/kafka/EmbeddedKafkaCluster.java @@ -13,10 +13,11 @@ import java.util.Properties; import java.util.concurrent.TimeUnit; +import org.apache.kafka.common.utils.SystemTime; import org.apache.kafka.common.utils.Time; import kafka.server.KafkaConfig; -import kafka.server.KafkaServer; +import kafka.server.KafkaServerStartable; import com.linkedin.datastream.common.FileUtils; import com.linkedin.datastream.common.NetworkUtils; @@ -32,7 +33,7 @@ public class EmbeddedKafkaCluster { private final String _brokers; - private final List _brokerList; + private final List _brokerList; private final List _logDirs; /** @@ -114,16 +115,15 @@ public void startup() { properties.setProperty("log.cleaner.enable", Boolean.FALSE.toString()); //to save memory properties.setProperty("offsets.topic.num.partitions", "1"); - KafkaServer broker = startBroker(properties); + KafkaServerStartable broker = startBroker(properties); _brokerList.add(broker); _logDirs.add(logDir); } } - private KafkaServer startBroker(Properties props) { - KafkaServer server = new KafkaServer(KafkaConfig.fromProps(props), new SystemTime(), - scala.Option.apply(""), scala.collection.JavaConversions.asScalaBuffer(Collections.emptyList())); + private KafkaServerStartable startBroker(Properties props) { + KafkaServerStartable server = new KafkaServerStartable(KafkaConfig.fromProps(props)); server.startup(); return server; } @@ -155,7 +155,7 @@ public String getZkConnection() { * Shut down the Kafka cluster */ public void shutdown() { - for (KafkaServer broker : _brokerList) { + for (KafkaServerStartable broker : _brokerList) { try { broker.shutdown(); } catch (Exception e) { @@ -179,7 +179,7 @@ public String toString() { return sb.toString(); } - static class SystemTime implements Time { + static class EmbeddedSystemTime extends SystemTime implements Time { public long milliseconds() { return System.currentTimeMillis(); } diff --git a/datastream-kafka/src/main/java/com/linkedin/datastream/kafka/KafkaTestUtils.java b/datastream-kafka/src/main/java/com/linkedin/datastream/kafka/KafkaTestUtils.java index 87aee5989..17ac1919f 100644 --- a/datastream-kafka/src/main/java/com/linkedin/datastream/kafka/KafkaTestUtils.java +++ b/datastream-kafka/src/main/java/com/linkedin/datastream/kafka/KafkaTestUtils.java @@ -45,6 +45,17 @@ public interface ReaderCallback { boolean onMessage(byte[] key, byte[] value) throws IOException; } + /** + * Interface for the callback invoked whenever broadcast messages are read + */ + public interface BroadcastReaderCallbackTest { + + /** + * Callback invoked whenever a broadcast message is read to be consumed + */ + boolean onMessage(byte[] key, byte[] value, int partition) throws IOException; + } + private KafkaTestUtils() { } @@ -122,6 +133,40 @@ public static boolean topicExists(AdminClient adminClient, String topic) { return false; } + /** + * Consume broadcast messages from a given partition of a Kafka topic, using given BroadcastReaderCallbackTest + * + * @param topic Topic to be consumed + * @param brokerList Kafka broker list for the topic + * @param callback Broadcast message consumer callback + * @throws Exception + */ + public static void readTopic(String topic, String brokerList, BroadcastReaderCallbackTest callback) throws Exception { + Validate.notNull(topic); + Validate.notNull(brokerList); + Validate.notNull(callback); + + KafkaConsumer consumer = createConsumer(brokerList); + consumer.subscribe(Collections.singletonList(topic)); + + boolean keepGoing = true; + long now = System.currentTimeMillis(); + do { + ConsumerRecords records = consumer.poll(1000); + for (ConsumerRecord record : records.records(topic)) { + if (!callback.onMessage(record.key(), record.value(), record.partition())) { + keepGoing = false; + break; + } + } + + // Guard against buggy test which can hang forever + if (System.currentTimeMillis() - now >= DEFAULT_TIMEOUT_MS) { + throw new TimeoutException("Timed out before reading all messages"); + } + } while (keepGoing); + } + /** * Consume messages from a given partition of a Kafka topic, using given ReaderCallback */ diff --git a/datastream-kafka/src/main/java/com/linkedin/datastream/kafka/KafkaTransportProvider.java b/datastream-kafka/src/main/java/com/linkedin/datastream/kafka/KafkaTransportProvider.java index 1d7910dc6..dffc6c17e 100644 --- a/datastream-kafka/src/main/java/com/linkedin/datastream/kafka/KafkaTransportProvider.java +++ b/datastream-kafka/src/main/java/com/linkedin/datastream/kafka/KafkaTransportProvider.java @@ -25,12 +25,14 @@ import com.linkedin.datastream.common.BrooklinEnvelope; import com.linkedin.datastream.common.BrooklinEnvelopeMetadataConstants; +import com.linkedin.datastream.common.DatastreamRuntimeException; import com.linkedin.datastream.common.ErrorLogger; import com.linkedin.datastream.metrics.BrooklinMeterInfo; import com.linkedin.datastream.metrics.BrooklinMetricInfo; import com.linkedin.datastream.metrics.DynamicMetricsManager; import com.linkedin.datastream.metrics.MetricsAware; import com.linkedin.datastream.server.DatastreamProducerRecord; +import com.linkedin.datastream.server.DatastreamProducerRecordBuilder; import com.linkedin.datastream.server.DatastreamTask; import com.linkedin.datastream.server.api.transport.DatastreamRecordMetadata; import com.linkedin.datastream.server.api.transport.SendCallback; @@ -59,6 +61,8 @@ public class KafkaTransportProvider implements TransportProvider { private final Meter _eventByteWriteRate; private final Meter _eventTransportErrorRate; + private final Properties _transportProviderProperties; + private boolean _isUnassigned; /** @@ -83,6 +87,7 @@ public KafkaTransportProvider(DatastreamTask datastreamTask, List sentToPartitions = new ArrayList<>(); + try { + for (; partition < partitionCount; partition++) { + DatastreamProducerRecord producerRecord = DatastreamProducerRecordBuilder.copyProducerRecord(record, partition); + send(destinationUri, producerRecord, ((metadata, exception) -> { + if (exception != null) { + LOG.error("Failed to broadcast record {} to partition {}", producerRecord, metadata.getPartition()); + } else { + LOG.debug("Sent broadcast record {} to partition {}", producerRecord, metadata.getPartition()); + } + // We simply invoke onEventComplete on each "send" completion. No additional book-keeping is done in broadcast + // regarding individual send succeeding/failing. Client will need to do that through onEventComplete. + // Eg, client will have to track if broadcast is complete on all partitions if they want a guaranteed broadcast. + onEventComplete.onCompletion(metadata, exception); + })); + sentToPartitions.add(partition); + } + return new DatastreamRecordMetadata(record.getCheckpoint(), topicName, sentToPartitions, true, partitionCount); + } catch (DatastreamRuntimeException ex) { + LOG.error("Broadcast send failed for record {} at partition {}/{} because of exception: {} ", + record, partition, partitionCount, ex); + throw ex; + } + } + @Override public void send(String destinationUri, DatastreamProducerRecord record, SendCallback onSendComplete) { String topicName = KafkaTransportProviderUtils.getTopicName(destinationUri); diff --git a/datastream-kafka/src/main/java/com/linkedin/datastream/kafka/factory/LiKafkaProducerFactory.java b/datastream-kafka/src/main/java/com/linkedin/datastream/kafka/factory/LiKafkaProducerFactory.java deleted file mode 100644 index 7a67f54ec..000000000 --- a/datastream-kafka/src/main/java/com/linkedin/datastream/kafka/factory/LiKafkaProducerFactory.java +++ /dev/null @@ -1,122 +0,0 @@ -/** - * Copyright 2019 LinkedIn Corporation. All rights reserved. - * Licensed under the BSD 2-Clause License. See the LICENSE file in the project root for license information. - * See the NOTICE file in the project root for additional information regarding copyright ownership. - */ -package com.linkedin.datastream.kafka.factory; - -import java.util.Properties; - -import org.apache.commons.lang.Validate; -import org.apache.kafka.clients.producer.Producer; -import org.apache.kafka.clients.producer.ProducerConfig; -import org.apache.kafka.common.serialization.ByteArraySerializer; - -import com.linkedin.datastream.common.VerifiableProperties; -import com.linkedin.kafka.clients.producer.LiKafkaProducerConfig; -import com.linkedin.kafka.clients.producer.LiKafkaProducerImpl; - - -/** - * A factory for creating Kafka {@link Producer} instances - */ -public class LiKafkaProducerFactory implements KafkaProducerFactory { - // New producer configurations. Please look at http://kafka.apache.org/documentation.html#producerconfigs for - // more details on what these configs mean. - // The configs below should ensure that there is no data loss in the Kafka pipeline - // http://www.slideshare.net/JiangjieQin/no-data-loss-pipeline-with-apache-kafka-49753844 - private static final String CFG_REQUEST_REQUIRED_ACKS = "acks"; - private static final String DEFAULT_REQUEST_REQUIRED_ACKS = "-1"; - - private static final String CFG_REQUEST_TIMEOUT_MS = "request.timeout.ms"; - private static final String DEFAULT_REQUEST_TIMEOUT_MS = "120000"; // 120 seconds - - private static final String CFG_RETRY_BACKOFF_MS = "retry.backoff.ms"; - private static final String DEFAULT_RETRY_BACKOFF_MS = "2000"; // 2 seconds - - private static final String CFG_METADATA_EXPIRY_MS = "metadata.max.age.ms"; - private static final String DEFAULT_METADATA_EXPIRY_MS = "300000"; // 300 seconds - - // This is per partition batch size - private static final String CFG_MAX_PARTITION_BYTES = "batch.size"; - private static final String DEFAULT_MAX_PARTITION_BYTES = "102400"; // 100 KB - - private static final String CFG_TOTAL_MEMORY_BYTES = "buffer.memory"; - private static final String DEFAULT_TOTAL_MEMORY_BYTES = "524288000"; // 512 MB - - // Time to wait for batching the sends. Since we have max.in.flight.requests.per.connection to 1. We get batching - // through that ala Nagle algorithm, Hence turning this off by setting it to 0. - private static final String CFG_LINGER_MS = "linger.ms"; - private static final String DEFAULT_LINGER_MS = "0"; - - // Size of the socket buffer used while sending data to the broker. - private static final String CFG_SEND_BUFFER_BYTES = "send.buffer.bytes"; - private static final String DEFAULT_SEND_BUFFER_BYTES = "131072"; // 128 KB - - // Size of the socket buffer used while receiving data from the broker. - private static final String CFG_RECEIVE_BUFFER_BYTES = "receive.buffer.bytes"; - private static final String DEFAULT_RECEIVE_BUFFER_BYTES = "131072"; // 128 KB - - private static final String CFG_MAX_REQUEST_SIZE = "max.request.size"; - private static final String DEFAULT_MAX_REQUEST_SIZE = "104857600"; // 100 MB - - private static final String CFG_RECONNECT_BACKOFF_MS = "reconnect.backoff.ms"; - private static final String DEFAULT_RECONNECT_BACKOFF_MS = "500"; - - private static final String CFG_MAX_BLOCK_MS = "max.block.ms"; - private static final String DEFAULT_MAX_BLOCK_MS = String.valueOf(Integer.MAX_VALUE); - - private static final String CFG_MAX_IN_FLIGHT_REQUESTS_PER_CONNECTION = "max.in.flight.requests.per.connection"; - private static final String DEFAULT_MAX_IN_FLIGHT_REQUESTS_PER_CONNECTION = "1"; - - private static final String CFG_REQUEST_RETRIES = "retries"; - private static final String DEFAULT_REQUEST_RETRIES = String.valueOf(Integer.MAX_VALUE); - - private static final String CFG_COMPRESSION_TYPE = "compression.type"; - private static final String DEFAULT_COMPRESSION_TYPE = "gzip"; - - private static final String DEFAULT_ENABLE_LARGE_MESSAGE = "false"; - /* Package Visible */ - static Properties buildProducerProperties(Properties prop, String clientId, String brokers, String enableLargeMessage) { - prop.put(ProducerConfig.CLIENT_ID_CONFIG, clientId); - prop.put(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, brokers); - prop.put(LiKafkaProducerConfig.LARGE_MESSAGE_ENABLED_CONFIG, enableLargeMessage); - prop.putIfAbsent(CFG_RETRY_BACKOFF_MS, DEFAULT_RETRY_BACKOFF_MS); - prop.putIfAbsent(CFG_REQUEST_TIMEOUT_MS, DEFAULT_REQUEST_TIMEOUT_MS); - prop.putIfAbsent(CFG_METADATA_EXPIRY_MS, DEFAULT_METADATA_EXPIRY_MS); - prop.putIfAbsent(CFG_MAX_PARTITION_BYTES, DEFAULT_MAX_PARTITION_BYTES); - prop.putIfAbsent(CFG_TOTAL_MEMORY_BYTES, DEFAULT_TOTAL_MEMORY_BYTES); - prop.putIfAbsent(CFG_REQUEST_REQUIRED_ACKS, DEFAULT_REQUEST_REQUIRED_ACKS); - prop.putIfAbsent(CFG_LINGER_MS, DEFAULT_LINGER_MS); - prop.putIfAbsent(CFG_SEND_BUFFER_BYTES, DEFAULT_SEND_BUFFER_BYTES); - prop.putIfAbsent(CFG_RECEIVE_BUFFER_BYTES, DEFAULT_RECEIVE_BUFFER_BYTES); - prop.putIfAbsent(CFG_MAX_REQUEST_SIZE, DEFAULT_MAX_REQUEST_SIZE); - prop.putIfAbsent(CFG_RECONNECT_BACKOFF_MS, DEFAULT_RECONNECT_BACKOFF_MS); - prop.putIfAbsent(CFG_MAX_BLOCK_MS, DEFAULT_MAX_BLOCK_MS); - prop.putIfAbsent(CFG_MAX_IN_FLIGHT_REQUESTS_PER_CONNECTION, DEFAULT_MAX_IN_FLIGHT_REQUESTS_PER_CONNECTION); - prop.putIfAbsent(CFG_REQUEST_RETRIES, DEFAULT_REQUEST_RETRIES); - prop.putIfAbsent(CFG_COMPRESSION_TYPE, DEFAULT_COMPRESSION_TYPE); - return prop; - } - - @Override - public Producer createProducer(Properties transportProps) { - VerifiableProperties transportProviderProperties = new VerifiableProperties(transportProps); - String clientId = transportProviderProperties.getString(ProducerConfig.CLIENT_ID_CONFIG); - String bootstrapServers = transportProviderProperties.getString(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG); - Properties producerConfig = transportProviderProperties.getDomainProperties(DOMAIN_PRODUCER); - - Validate.notEmpty(clientId, "clientId cannot be empty."); - Validate.notEmpty(bootstrapServers, "bootstrapServers cannot be empty."); - - producerConfig = buildProducerProperties(producerConfig, clientId, bootstrapServers, DEFAULT_ENABLE_LARGE_MESSAGE); - - // Default DeSerializer for Key and Payload - producerConfig.putIfAbsent(LiKafkaProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, - ByteArraySerializer.class.getCanonicalName()); - producerConfig.putIfAbsent(LiKafkaProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, - ByteArraySerializer.class.getCanonicalName()); - - return new LiKafkaProducerImpl<>(producerConfig); - } -} diff --git a/datastream-kafka/src/test/java/com/linkedin/datastream/kafka/TestKafkaTransportProvider.java b/datastream-kafka/src/test/java/com/linkedin/datastream/kafka/TestKafkaTransportProvider.java index 827e083f2..60ba26114 100644 --- a/datastream-kafka/src/test/java/com/linkedin/datastream/kafka/TestKafkaTransportProvider.java +++ b/datastream-kafka/src/test/java/com/linkedin/datastream/kafka/TestKafkaTransportProvider.java @@ -9,6 +9,7 @@ import java.util.ArrayList; import java.util.Collections; import java.util.HashMap; +import java.util.HashSet; import java.util.List; import java.util.Map; import java.util.Properties; @@ -182,27 +183,27 @@ public void testReAssignBuggyProducer() throws Exception { @Test public void testSendHappyPath() throws Exception { - testEventSend(1, 1, 0, true, true, "test"); + testEventSendOrBroadcast(1, 1, 0, true, true, "test", false); } @Test public void testSendWithoutPartitionNumber() throws Exception { - testEventSend(1, 2, -1, true, true, "test"); + testEventSendOrBroadcast(1, 2, -1, true, true, "test", false); } @Test public void testEventWithoutKeyAndPartition() throws Exception { - testEventSend(1, 2, -1, false, true, "test"); + testEventSendOrBroadcast(1, 2, -1, false, true, "test", false); } @Test public void testEventWithoutKeyNOrValue() throws Exception { - testEventSend(1, 2, 0, false, false, "test"); + testEventSendOrBroadcast(1, 2, 0, false, false, "test", false); } @Test public void testEventWithoutKeyValueAndPartition() throws Exception { - testEventSend(1, 2, -1, false, false, "test"); + testEventSendOrBroadcast(1, 2, -1, false, false, "test", false); } @Test @@ -279,8 +280,14 @@ public void testSendMultipleEventsInSingleDatastreamProducerRecord() throws Exce Assert.assertNotNull(DynamicMetricsManager.getInstance().getMetric(producerCountMetricName)); } - private void testEventSend(int numberOfEvents, int numberOfPartitions, int partition, boolean includeKey, - boolean includeValue, String metricsPrefix) throws Exception { + @Test + public void testBroadcastHappyPath() throws Exception { + testEventSendOrBroadcast(1, 3, -1, true, true, "broadcast", true); + } + + // Helper method. + private void testEventSendOrBroadcast(int numberOfEvents, int numberOfPartitions, int partition, boolean includeKey, + boolean includeValue, String metricsPrefix, boolean isBroadcast) throws Exception { String topicName = getUniqueTopicName(); if (metricsPrefix != null) { @@ -296,31 +303,52 @@ private void testEventSend(int numberOfEvents, int numberOfPartitions, int parti TransportProvider transportProvider = provider.assignTransportProvider(task); provider.createTopic(destinationUri, numberOfPartitions, new Properties(), ds); - //KafkaTestUtils.waitForTopicCreation(_adminClient, topicName, _kafkaCluster.getBrokers()); - LOG.info(String.format("Topic %s created with %d partitions and topic properties %s", topicName, numberOfPartitions, new Properties())); List datastreamEvents = - createEvents(topicName, partition, numberOfEvents, includeKey, includeValue); + createEvents(topicName, partition, numberOfEvents, includeKey, includeValue, isBroadcast); LOG.info(String.format("Trying to send %d events to topic %s", datastreamEvents.size(), topicName)); final Integer[] callbackCalled = {0}; for (DatastreamProducerRecord event : datastreamEvents) { - transportProvider.send(destinationUri, event, ((metadata, exception) -> callbackCalled[0]++)); + if (isBroadcast) { + transportProvider.broadcast(destinationUri, event, ((metadata, exception) -> callbackCalled[0]++)); + } else { + transportProvider.send(destinationUri, event, ((metadata, exception) -> callbackCalled[0]++)); + } } - // wait until all messages were acked, to ensure all events were successfully sent to the topic - Assert.assertTrue(PollUtils.poll(() -> callbackCalled[0] == datastreamEvents.size(), 1000, 10000), - "Send callback was not called; likely topic was not created in time"); + if (isBroadcast) { + // wait until all messages were acked, to ensure all events were successfully sent to the topic + Assert.assertTrue(PollUtils.poll(() -> callbackCalled[0] == (datastreamEvents.size() * numberOfPartitions), 1000, 10000), + "Send callback was not called; likely topic was not created in time"); + } else { + // wait until all messages were acked, to ensure all events were successfully sent to the topic + Assert.assertTrue(PollUtils.poll(() -> callbackCalled[0] == datastreamEvents.size(), 1000, 10000), + "Send callback was not called; likely topic was not created in time"); + } LOG.info(String.format("Trying to read events from the topicName %s partition %d", topicName, partition)); Map events = new HashMap<>(); - KafkaTestUtils.readTopic(topicName, partition, _kafkaCluster.getBrokers(), (key, value) -> { - events.put(new String(key), new String(value)); - return events.size() < numberOfEvents; - }); + Set partitionsRead = new HashSet<>(); + if (isBroadcast) { + KafkaTestUtils.readTopic(topicName, _kafkaCluster.getBrokers(), (key, value, recordPartition) -> { + events.put(new String(key), new String(value)); + partitionsRead.add(recordPartition); + return (partitionsRead.size() < numberOfPartitions) || (events.size() < numberOfEvents); + }); + } else { + KafkaTestUtils.readTopic(topicName, partition, _kafkaCluster.getBrokers(), (key, value) -> { + events.put(new String(key), new String(value)); + return events.size() < numberOfEvents; + }); + } + + if (isBroadcast) { + Assert.assertEquals(partitionsRead.size(), numberOfPartitions); + } if (metricsPrefix != null) { // verify that configured metrics prefix was used @@ -351,13 +379,17 @@ private void testEventSend(int numberOfEvents, int numberOfPartitions, int parti } } - private byte[] createMessage(String text) { return text.getBytes(); } private List createEvents(String topicName, int partition, int numberOfEvents, boolean includeKey, boolean includeValue) { + return createEvents(topicName, partition, numberOfEvents, includeKey, includeValue, false); + } + + private List createEvents(String topicName, int partition, int numberOfEvents, + boolean includeKey, boolean includeValue, boolean isBroadcastEvent) { Datastream stream = new Datastream(); stream.setName("datastream_" + topicName); stream.setConnectorName("dummyConnector"); @@ -390,10 +422,14 @@ private List createEvents(String topicName, int partit DatastreamProducerRecordBuilder builder = new DatastreamProducerRecordBuilder(); builder.setEventsSourceTimestamp(System.currentTimeMillis()); builder.addEvent(new BrooklinEnvelope(keyValue, payloadValue, previousPayloadValue, new HashMap<>())); - if (partition >= 0) { - builder.setPartition(partition); + if (isBroadcastEvent) { + builder.setIsBroadcastRecord(true); } else { - builder.setPartitionKey(key); + if (partition >= 0) { + builder.setPartition(partition); + } else { + builder.setPartitionKey(key); + } } builder.setSourceCheckpoint("test"); diff --git a/datastream-server-api/src/main/java/com/linkedin/datastream/server/DatastreamEventProducer.java b/datastream-server-api/src/main/java/com/linkedin/datastream/server/DatastreamEventProducer.java index 8d54ba6e6..f982ed620 100644 --- a/datastream-server-api/src/main/java/com/linkedin/datastream/server/DatastreamEventProducer.java +++ b/datastream-server-api/src/main/java/com/linkedin/datastream/server/DatastreamEventProducer.java @@ -5,6 +5,7 @@ */ package com.linkedin.datastream.server; +import com.linkedin.datastream.server.api.transport.DatastreamRecordMetadata; import com.linkedin.datastream.server.api.transport.SendCallback; @@ -47,4 +48,15 @@ public interface DatastreamEventProducer { */ default void enablePeriodicFlushOnSend(boolean enableFlushOnSend) { } + + /** + * Broadcast event onto the transport. Broadcast callback.onComplete should be reasonably fast + * for the same reason as in send. + * + * @param event event to broadcast + * @param callback callback to be called on completion of each send + */ + default DatastreamRecordMetadata broadcast(DatastreamProducerRecord event, SendCallback callback) { + throw new UnsupportedOperationException("Broadcast not supported by event producer"); + } } diff --git a/datastream-server-api/src/main/java/com/linkedin/datastream/server/DatastreamProducerRecord.java b/datastream-server-api/src/main/java/com/linkedin/datastream/server/DatastreamProducerRecord.java index ba520454e..4263e193e 100644 --- a/datastream-server-api/src/main/java/com/linkedin/datastream/server/DatastreamProducerRecord.java +++ b/datastream-server-api/src/main/java/com/linkedin/datastream/server/DatastreamProducerRecord.java @@ -25,6 +25,7 @@ public class DatastreamProducerRecord { private final Optional _destination; private final String _checkpoint; private final long _eventsSourceTimestamp; + private final boolean _isBroadcastRecord; private final List _events; @@ -36,11 +37,16 @@ public class DatastreamProducerRecord { DatastreamProducerRecord(List events, Optional partition, Optional partitionKey, String checkpoint, long eventsSourceTimestamp) { - this(events, partition, partitionKey, Optional.empty(), checkpoint, eventsSourceTimestamp); + this(events, partition, partitionKey, Optional.empty(), checkpoint, eventsSourceTimestamp, false); } DatastreamProducerRecord(List events, Optional partition, Optional partitionKey, Optional destination, String checkpoint, long eventsSourceTimestamp) { + this(events, partition, partitionKey, destination, checkpoint, eventsSourceTimestamp, false); + } + + DatastreamProducerRecord(List events, Optional partition, Optional partitionKey, + Optional destination, String checkpoint, long eventsSourceTimestamp, boolean isBroadcastRecord) { Validate.notNull(events, "null event"); events.forEach((e) -> Validate.notNull(e, "null event")); Validate.isTrue(eventsSourceTimestamp > 0, "events source timestamp is invalid"); @@ -51,6 +57,7 @@ public class DatastreamProducerRecord { _checkpoint = checkpoint; _eventsSourceTimestamp = eventsSourceTimestamp; _destination = destination; + _isBroadcastRecord = isBroadcastRecord; } /** @@ -117,9 +124,14 @@ public Optional getPartition() { return _partition; } + public boolean isBroadcastRecord() { + return _isBroadcastRecord; + } + @Override public String toString() { - return String.format("%s @ partitionKey=%s partition=%d", _events, _partitionKey.orElse(null), _partition.orElse(-1)); + return String.format("%s @ partitionKey=%s partition=%d isBroadcastRecord=%s", + _events, _partitionKey.orElse(null), _partition.orElse(-1), _isBroadcastRecord); } @Override diff --git a/datastream-server-api/src/main/java/com/linkedin/datastream/server/DatastreamProducerRecordBuilder.java b/datastream-server-api/src/main/java/com/linkedin/datastream/server/DatastreamProducerRecordBuilder.java index f53087f79..7bb971e1f 100644 --- a/datastream-server-api/src/main/java/com/linkedin/datastream/server/DatastreamProducerRecordBuilder.java +++ b/datastream-server-api/src/main/java/com/linkedin/datastream/server/DatastreamProducerRecordBuilder.java @@ -30,6 +30,7 @@ public class DatastreamProducerRecordBuilder { private long _eventsSourceTimestamp; private Optional _partitionKey = Optional.empty(); private Optional _destination = Optional.empty(); + private boolean _isBroadcastRecord = false; /** * Partition to which this DatastreamProducerRecord should be produced. If the partition is not set, TransportProvider @@ -84,6 +85,10 @@ public void setEventsSourceTimestamp(long eventsSourceTimestamp) { _eventsSourceTimestamp = eventsSourceTimestamp; } + public void setIsBroadcastRecord(boolean isBroadcastRecord) { + _isBroadcastRecord = isBroadcastRecord; + } + /** * Build the DatastreamProducerRecord. * @return @@ -91,6 +96,29 @@ public void setEventsSourceTimestamp(long eventsSourceTimestamp) { */ public DatastreamProducerRecord build() { return new DatastreamProducerRecord(_events, _partition, _partitionKey, _destination, _sourceCheckpoint, - _eventsSourceTimestamp); + _eventsSourceTimestamp, _isBroadcastRecord); + } + + /** + * Create DatastreamProducerRecord copied from another record and overriding the partition number + * + * @param record datastream record to be copied + * @param partition partition to override + * @return copiedDatastreamProducerRecord + */ + public static DatastreamProducerRecord copyProducerRecord(DatastreamProducerRecord record, int partition) { + DatastreamProducerRecordBuilder builder = new DatastreamProducerRecordBuilder(); + record.getEvents().forEach(builder::addEvent); + builder.setPartition(partition); + if (record.getPartitionKey().isPresent()) { + builder.setPartitionKey(record.getPartitionKey().get()); + } + if (record.getDestination().isPresent()) { + builder.setDestination(record.getDestination().get()); + } + builder.setSourceCheckpoint(record.getCheckpoint()); + builder.setEventsSourceTimestamp(record.getEventsSourceTimestamp()); + builder.setIsBroadcastRecord(record.isBroadcastRecord()); + return builder.build(); } } diff --git a/datastream-server-api/src/main/java/com/linkedin/datastream/server/FlushlessEventProducerHandler.java b/datastream-server-api/src/main/java/com/linkedin/datastream/server/FlushlessEventProducerHandler.java index 28ba57f4b..64db16792 100644 --- a/datastream-server-api/src/main/java/com/linkedin/datastream/server/FlushlessEventProducerHandler.java +++ b/datastream-server-api/src/main/java/com/linkedin/datastream/server/FlushlessEventProducerHandler.java @@ -5,14 +5,9 @@ */ package com.linkedin.datastream.server; -import java.util.Collections; import java.util.Comparator; -import java.util.LinkedHashSet; import java.util.Map; import java.util.Optional; -import java.util.PriorityQueue; -import java.util.Queue; -import java.util.Set; import java.util.concurrent.ConcurrentHashMap; import java.util.stream.Collectors; @@ -20,7 +15,8 @@ import org.slf4j.LoggerFactory; import com.linkedin.datastream.server.api.transport.SendCallback; - +import com.linkedin.datastream.server.callbackstatus.CallbackStatus; +import com.linkedin.datastream.server.callbackstatus.CallbackStatusFactory; /** @@ -36,15 +32,24 @@ public class FlushlessEventProducerHandler> { private static final Logger LOG = LoggerFactory.getLogger(FlushlessEventProducerHandler.class); + private final CallbackStatusFactory _callbackStatusFactory; private final DatastreamEventProducer _eventProducer; - private final ConcurrentHashMap _callbackStatusMap = new ConcurrentHashMap<>(); + private final ConcurrentHashMap> _callbackStatusMap = new ConcurrentHashMap<>(); /** * Constructor for FlushlessEventProducerHandler */ - public FlushlessEventProducerHandler(DatastreamEventProducer eventProducer) { + public FlushlessEventProducerHandler(DatastreamEventProducer eventProducer, CallbackStatusFactory callbackStatusFactory) { _eventProducer = eventProducer; _eventProducer.enablePeriodicFlushOnSend(false); + _callbackStatusFactory = callbackStatusFactory; + } + + /** + * Creating a new instance of the OffsetCheckpointTrackingStrategy to be used for checkpointing + */ + private CallbackStatus createCallbackStatusInstance() { + return _callbackStatusFactory.createCallbackStatusStrategy(); } /** @@ -73,7 +78,7 @@ public void clear(String source, int partition) { */ public void send(DatastreamProducerRecord record, String source, int sourcePartition, T sourceCheckpoint, SendCallback callback) { SourcePartition sp = new SourcePartition(source, sourcePartition); - CallbackStatus status = _callbackStatusMap.computeIfAbsent(sp, d -> new CallbackStatus()); + CallbackStatus status = _callbackStatusMap.computeIfAbsent(sp, d -> createCallbackStatusInstance()); status.register(sourceCheckpoint); _eventProducer.send(record, ((metadata, exception) -> { if (exception != null) { @@ -92,7 +97,7 @@ public void send(DatastreamProducerRecord record, String source, int sourceParti * acknowledged. */ public Optional getAckCheckpoint(String source, int sourcePartition) { - CallbackStatus status = _callbackStatusMap.get(new SourcePartition(source, sourcePartition)); + CallbackStatus status = _callbackStatusMap.get(new SourcePartition(source, sourcePartition)); return Optional.ofNullable(status).map(CallbackStatus::getAckCheckpoint); } @@ -100,7 +105,7 @@ public Optional getAckCheckpoint(String source, int sourcePartition) { * Get the in-flight count of messages yet to be acknowledged for a given source and sourcePartition */ public long getInFlightCount(String source, int sourcePartition) { - CallbackStatus status = _callbackStatusMap.get(new SourcePartition(source, sourcePartition)); + CallbackStatus status = _callbackStatusMap.get(new SourcePartition(source, sourcePartition)); return status != null ? status.getInFlightCount() : 0; } @@ -117,7 +122,7 @@ public Map getInFlightMessagesCounts() { * Get the in-flight count of messages yet to be acknowledged for a given source and sourcePartition */ public long getAckMessagesPastCheckpointCount(String source, int sourcePartition) { - CallbackStatus status = _callbackStatusMap.get(new SourcePartition(source, sourcePartition)); + CallbackStatus status = _callbackStatusMap.get(new SourcePartition(source, sourcePartition)); return status != null ? status.getAckMessagesPastCheckpointCount() : 0; } @@ -141,7 +146,7 @@ public Map getAckMessagesPastCheckpointCounts() { public Optional getAckCheckpoint(T currentCheckpoint, Comparator checkpointComparator) { T lowWaterMark = null; - for (CallbackStatus status : _callbackStatusMap.values()) { + for (CallbackStatus status : _callbackStatusMap.values()) { if (status.getInFlightCount() > 0) { T checkpoint = status.getAckCheckpoint(); if (checkpoint == null) { @@ -181,75 +186,4 @@ public String toString() { return getSource() + "-" + getPartition(); } } - - /** - * Helper class to store the callback status of the inflight events. - */ - private class CallbackStatus { - - private T _currentCheckpoint = null; - private T _highWaterMark = null; - - private final Queue _acked = new PriorityQueue<>(); - private final Set _inFlight = Collections.synchronizedSet(new LinkedHashSet<>()); - - public T getAckCheckpoint() { - return _currentCheckpoint; - } - - public long getInFlightCount() { - return _inFlight.size(); - } - - public long getAckMessagesPastCheckpointCount() { - return _acked.size(); - } - - /** - * Registers the given checkpoint by adding it to the set of in-flight checkpoints. - * @param checkpoint the checkpoint to register - */ - public synchronized void register(T checkpoint) { - _inFlight.add(checkpoint); - } - - /** - * The checkpoint acknowledgement can be received out of order. In that case we need to keep track - * of the high watermark, and only update the ackCheckpoint when we are sure all events before it has - * been received. - */ - public synchronized void ack(T checkpoint) { - if (!_inFlight.remove(checkpoint)) { - LOG.error("Internal state error; could not remove checkpoint {}", checkpoint); - } - _acked.add(checkpoint); - - if (_highWaterMark == null || _highWaterMark.compareTo(checkpoint) < 0) { - _highWaterMark = checkpoint; - } - - if (_inFlight.isEmpty()) { - // Queue is empty, update to high water mark. - _currentCheckpoint = _highWaterMark; - _acked.clear(); - } else { - // Update the checkpoint to the largest acked message that is still smaller than the first in-flight message - T max = null; - T first = _inFlight.iterator().next(); - while (!_acked.isEmpty() && _acked.peek().compareTo(first) < 0) { - max = _acked.poll(); - } - if (max != null) { - if (_currentCheckpoint != null && max.compareTo(_currentCheckpoint) < 0) { - // max is less than current checkpoint, should not happen - LOG.error( - "Internal error: checkpoints should progress in increasing order. Resolved checkpoint as {} which is " - + "less than current checkpoint of {}", - max, _currentCheckpoint); - } - _currentCheckpoint = max; - } - } - } - } } diff --git a/datastream-server-api/src/main/java/com/linkedin/datastream/server/api/transport/DatastreamRecordMetadata.java b/datastream-server-api/src/main/java/com/linkedin/datastream/server/api/transport/DatastreamRecordMetadata.java index 4d6c9fe52..479ab09a0 100644 --- a/datastream-server-api/src/main/java/com/linkedin/datastream/server/api/transport/DatastreamRecordMetadata.java +++ b/datastream-server-api/src/main/java/com/linkedin/datastream/server/api/transport/DatastreamRecordMetadata.java @@ -5,6 +5,9 @@ */ package com.linkedin.datastream.server.api.transport; +import java.util.List; + + /** * Metadata of the successfully produced datastream record */ @@ -16,6 +19,12 @@ public class DatastreamRecordMetadata { private final int _eventIndex; private final int _sourcePartition; + // Broadcast record metadata. + private final boolean _isBroadcastRecord; + private final List _sentToPartitions; + private final int _partitionCount; + private final boolean _isMessageSerializationError; + /** * Construct an instance of DatastreamRecordMetadata. Defaults the event index to 0 and source partition to -1. * @param checkpoint checkpoint string @@ -28,6 +37,10 @@ public DatastreamRecordMetadata(String checkpoint, String topic, int partition) _partition = partition; _eventIndex = 0; _sourcePartition = -1; + _isBroadcastRecord = false; + _sentToPartitions = null; + _partitionCount = -1; + _isMessageSerializationError = false; } /** @@ -44,6 +57,49 @@ public DatastreamRecordMetadata(String checkpoint, String topic, int partition, _partition = partition; _eventIndex = eventIndex; _sourcePartition = sourcePartition; + _isBroadcastRecord = false; + _sentToPartitions = null; + _partitionCount = -1; + _isMessageSerializationError = false; + } + + /** + * Construct an instance of DatastreamRecordMetadata. + * + * @param checkpoint checkpoint string + * @param topic Kafka topic name + * @param sentToPartitions List of partitions numbers where send was attempted + * @param isBroadcastRecord Boolean to indicate if metadata record indicates broadcast + * @param partitionCount Total number of partitions of the topic + */ + public DatastreamRecordMetadata(String checkpoint, String topic, List sentToPartitions, + boolean isBroadcastRecord, int partitionCount) { + _checkpoint = checkpoint; + _topic = topic; + _sourcePartition = -1; + _sentToPartitions = sentToPartitions; + _eventIndex = 0; + _isBroadcastRecord = isBroadcastRecord; + _partition = -1; + _partitionCount = partitionCount; + _isMessageSerializationError = false; + } + + /** + * Construct an instance of DatastreamRecordMetadata. + * + * @param isMessageSerializationError Indicates is serialization error was encountered in EventProducer + */ + public DatastreamRecordMetadata(boolean isMessageSerializationError) { + _checkpoint = null; + _topic = null; + _sourcePartition = -1; + _sentToPartitions = null; + _eventIndex = 0; + _isBroadcastRecord = true; + _partition = -1; + _partitionCount = -1; + _isMessageSerializationError = isMessageSerializationError; } /** @@ -87,4 +143,20 @@ public String toString() { return String.format("Checkpoint: %s, Topic: %s, Destination Partition: %d, Event Index: %d, Source Partition: %d", _checkpoint, _topic, _partition, _eventIndex, _sourcePartition); } + + public boolean isBroadcastRecord() { + return _isBroadcastRecord; + } + + public List getSentToPartitions() { + return _sentToPartitions; + } + + public int getPartitionCount() { + return _partitionCount; + } + + public boolean isMessageSerializationError() { + return _isMessageSerializationError; + } } diff --git a/datastream-server-api/src/main/java/com/linkedin/datastream/server/api/transport/TransportProvider.java b/datastream-server-api/src/main/java/com/linkedin/datastream/server/api/transport/TransportProvider.java index fa7531a25..db7865282 100644 --- a/datastream-server-api/src/main/java/com/linkedin/datastream/server/api/transport/TransportProvider.java +++ b/datastream-server-api/src/main/java/com/linkedin/datastream/server/api/transport/TransportProvider.java @@ -38,4 +38,31 @@ public interface TransportProvider { * Flush to make sure that the current set of events that are in the buffer gets flushed to the server. */ void flush(); + + /** + * Broadcast for sending the record to all consumers/endpoints. Broadcast could involve invoking "send" to multiple + * endpoints. Broadcast is a best-effort strategy, there is no guarantee that the record send to each endpoint will succeed. + * onEventComplete will be called on completion of record send to each endpoint and each onEventComplete callback will + * contain result of send completion to that endpoint. + * + * If a client wants to build guaranteed broadcast semantics or needs to do additional book keeping (like which endpoints + * broadcast was successful, etc), it will be client's responsibility to do book-keeping on successful/failed "send" + * on each endpoint through onEventComplete callback and implement its own strategy (like retries, etc). + * + * DatastreamRecordMetadata will be returned after "send" method is called on each endpoint. + * + * For e.g., for Kafka this means sending the record to all topic partitions (i.e. partition each partition + * is the broadcast endpoint for Kafka). When record send to each partition completes onEventComplete will be + * invoked if provided. The returned DatastreamRecordMetadata will contain total partition count and which partitions + * "send" was invoked on. + * + * @param destination the destination topic to which the record should be broadcasted. + * @param record DatastreamEvent that needs to be broadcasted to the stream. + * @param onEventComplete Callback that will be called at send completion to each endpoint. This is an optional + * callback. For e.g., for Kafka this callback would be invoked when send to each partition + * completes. + */ + default DatastreamRecordMetadata broadcast(String destination, DatastreamProducerRecord record, SendCallback onEventComplete) { + throw new UnsupportedOperationException("Transport Provider does not support broadcast"); + } } diff --git a/datastream-server-api/src/main/java/com/linkedin/datastream/server/callbackstatus/CallbackStatus.java b/datastream-server-api/src/main/java/com/linkedin/datastream/server/callbackstatus/CallbackStatus.java new file mode 100644 index 000000000..c2c0720bb --- /dev/null +++ b/datastream-server-api/src/main/java/com/linkedin/datastream/server/callbackstatus/CallbackStatus.java @@ -0,0 +1,41 @@ +/** + * Copyright 2021 LinkedIn Corporation. All rights reserved. + * Licensed under the BSD 2-Clause License. See the LICENSE file in the project root for license information. + * See the NOTICE file in the project root for additional information regarding copyright ownership. + */ +package com.linkedin.datastream.server.callbackstatus; + +/** + * Abstract class to track the callback status of the inflight events. + * + * @param Type of the comparable or non comparable checkpoint object internally used by the connector. + */ +abstract public class CallbackStatus { + + /** + * Get the latest checkpoint to be acked + */ + abstract public T getAckCheckpoint(); + + /** + * Get the count of the records which are in flight + */ + abstract public long getInFlightCount(); + + /** + * Get the count of the records which are all acked from the producer + */ + abstract public long getAckMessagesPastCheckpointCount(); + + /** + * Registers the given checkpoint. + * @param checkpoint is the latest record acked by the producer of the underlying pub sub framework + */ + abstract public void register(T checkpoint); + + /** + * The checkpoint acknowledgement maintains the last successfully checkpoint-ed entry with + * either comparing or without comparing the offsets. + */ + abstract public void ack(T checkpoint); +} \ No newline at end of file diff --git a/datastream-server-api/src/main/java/com/linkedin/datastream/server/callbackstatus/CallbackStatusFactory.java b/datastream-server-api/src/main/java/com/linkedin/datastream/server/callbackstatus/CallbackStatusFactory.java new file mode 100644 index 000000000..508cd7d17 --- /dev/null +++ b/datastream-server-api/src/main/java/com/linkedin/datastream/server/callbackstatus/CallbackStatusFactory.java @@ -0,0 +1,18 @@ +/** + * Copyright 2021 LinkedIn Corporation. All rights reserved. + * Licensed under the BSD 2-Clause License. See the LICENSE file in the project root for license information. + * See the NOTICE file in the project root for additional information regarding copyright ownership. + */ +package com.linkedin.datastream.server.callbackstatus; + +/** + * Interface for CallbackStatus Factories + * @param The type of the offset position that the underlying pub-sub system uses. + */ +public interface CallbackStatusFactory { + /** + * Creates a callback status strategy that checkpoints the consumer offset on successful produce of that record + * @return CallbackStatus strategy construct + */ + CallbackStatus createCallbackStatusStrategy(); +} \ No newline at end of file diff --git a/datastream-server-api/src/main/java/com/linkedin/datastream/server/callbackstatus/CallbackStatusWithComparableOffsets.java b/datastream-server-api/src/main/java/com/linkedin/datastream/server/callbackstatus/CallbackStatusWithComparableOffsets.java new file mode 100644 index 000000000..24b9e2b6e --- /dev/null +++ b/datastream-server-api/src/main/java/com/linkedin/datastream/server/callbackstatus/CallbackStatusWithComparableOffsets.java @@ -0,0 +1,108 @@ +/** + * Copyright 2021 LinkedIn Corporation. All rights reserved. + * Licensed under the BSD 2-Clause License. See the LICENSE file in the project root for license information. + * See the NOTICE file in the project root for additional information regarding copyright ownership. + */ +package com.linkedin.datastream.server.callbackstatus; + +import java.util.Collections; +import java.util.LinkedHashSet; +import java.util.PriorityQueue; +import java.util.Queue; +import java.util.Set; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + + +/** + * Helper class to store the callback status of the inflight events with comparable offsets. + * + * @param Type of the comparable checkpoint object internally used by the connector. + */ +public class CallbackStatusWithComparableOffsets> extends CallbackStatus { + + private static final Logger LOG = LoggerFactory.getLogger(CallbackStatusWithComparableOffsets.class); + + private final Queue _acked = new PriorityQueue<>(); + private final Set _inFlight = Collections.synchronizedSet(new LinkedHashSet<>()); + + private T _highWaterMark = null; + + // the last checkpoint-ed record's offset + protected T _currentCheckpoint = null; + + /** + * Get the latest checkpoint to be acked + * @return Type of the comparable checkpoint object internally used by the connector. + */ + @Override + public T getAckCheckpoint() { + return _currentCheckpoint; + } + + /** + * Get the count of the records which are in flight + */ + @Override + public long getInFlightCount() { + return _inFlight.size(); + } + + /** + * Get the count of the records which are all acked from the producer + */ + @Override + public long getAckMessagesPastCheckpointCount() { + return _acked.size(); + } + + /** + * Registers the given checkpoint by adding it to the set of in-flight checkpoints. + * @param checkpoint is the latest record acked by the producer of the underlying pub sub framework + */ + @Override + public synchronized void register(T checkpoint) { + _inFlight.add(checkpoint); + } + + /** + * The checkpoint acknowledgement can be received out of order. In that case we need to keep track + * of the high watermark, and only update the ackCheckpoint when we are sure all events before it has + * been received. + */ + @Override + public synchronized void ack(T checkpoint) { + if (!_inFlight.remove(checkpoint)) { + LOG.error("Internal state error; could not remove checkpoint {}", checkpoint); + } + _acked.add(checkpoint); + + if (_highWaterMark == null || _highWaterMark.compareTo(checkpoint) < 0) { + _highWaterMark = checkpoint; + } + + if (_inFlight.isEmpty()) { + // Queue is empty, update to high water mark. + _currentCheckpoint = _highWaterMark; + _acked.clear(); + } else { + // Update the checkpoint to the largest acked message that is still smaller than the first in-flight message + T max = null; + T first = _inFlight.iterator().next(); + while (!_acked.isEmpty() && _acked.peek().compareTo(first) < 0) { + max = _acked.poll(); + } + if (max != null) { + if (_currentCheckpoint != null && max.compareTo(_currentCheckpoint) < 0) { + // max is less than current checkpoint, should not happen + LOG.error( + "Internal error: checkpoints should progress in increasing order. Resolved checkpoint as {} which is " + + "less than current checkpoint of {}", + max, _currentCheckpoint); + } + _currentCheckpoint = max; + } + } + } +} \ No newline at end of file diff --git a/datastream-server-api/src/main/java/com/linkedin/datastream/server/callbackstatus/CallbackStatusWithComparableOffsetsFactory.java b/datastream-server-api/src/main/java/com/linkedin/datastream/server/callbackstatus/CallbackStatusWithComparableOffsetsFactory.java new file mode 100644 index 000000000..15d09657c --- /dev/null +++ b/datastream-server-api/src/main/java/com/linkedin/datastream/server/callbackstatus/CallbackStatusWithComparableOffsetsFactory.java @@ -0,0 +1,23 @@ +/** + * Copyright 2021 LinkedIn Corporation. All rights reserved. + * Licensed under the BSD 2-Clause License. See the LICENSE file in the project root for license information. + * See the NOTICE file in the project root for additional information regarding copyright ownership. + */ +package com.linkedin.datastream.server.callbackstatus; + +/** + * Factory implementation for Callback Status With Comparable Offsets + * @param The type of the offset position that the underlying pub-sub system uses. + */ +public class CallbackStatusWithComparableOffsetsFactory> implements CallbackStatusFactory { + + /** + * Creates a callback status strategy that checkpoints the consumer offset on successful produce of that record + * with comparable offsets + * @return CallbackStatus strategy construct + */ + @Override + public CallbackStatus createCallbackStatusStrategy() { + return new CallbackStatusWithComparableOffsets(); + } +} diff --git a/datastream-server-api/src/main/java/com/linkedin/datastream/server/callbackstatus/CallbackStatusWithNonComparableOffsets.java b/datastream-server-api/src/main/java/com/linkedin/datastream/server/callbackstatus/CallbackStatusWithNonComparableOffsets.java new file mode 100644 index 000000000..90d16f2c7 --- /dev/null +++ b/datastream-server-api/src/main/java/com/linkedin/datastream/server/callbackstatus/CallbackStatusWithNonComparableOffsets.java @@ -0,0 +1,99 @@ +/** + * Copyright 2021 LinkedIn Corporation. All rights reserved. + * Licensed under the BSD 2-Clause License. See the LICENSE file in the project root for license information. + * See the NOTICE file in the project root for additional information regarding copyright ownership. + */ +package com.linkedin.datastream.server.callbackstatus; + +import java.util.Collections; +import java.util.Deque; +import java.util.HashSet; +import java.util.LinkedHashSet; +import java.util.LinkedList; +import java.util.Set; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + + +/** + * Helper class to store the callback status of the inflight events with non comparable offsets. + * + * @param Type of the non comparable checkpoint object internally used by the connector. + */ +public class CallbackStatusWithNonComparableOffsets extends CallbackStatus { + + private static final Logger LOG = LoggerFactory.getLogger(CallbackStatusWithNonComparableOffsets.class); + + // Hashset storing all the records which are yet to be acked + private final Set _inFlight = Collections.synchronizedSet(new LinkedHashSet<>()); + + // Deque to store all the messages which are inflight after the last consumer checkpoint is made + private final Deque _inFlightAfterLastConsumerCheckpoint = new LinkedList<>(); + + // Hashset storing all the records for which the ack is received + private final Set _acked = Collections.synchronizedSet(new HashSet<>()); + + // the last checkpoint-ed record's offset + protected T _currentCheckpoint = null; + + /** + * Get the latest checkpoint to be acked + * @return Type of the comparable checkpoint object internally used by the connector. + */ + @Override + public T getAckCheckpoint() { + return _currentCheckpoint; + } + + /** + * Get the count of the records which are in flight + */ + @Override + public long getInFlightCount() { + return _inFlight.size(); + } + + /** + * Get the count of the records which are all acked from the producer + */ + @Override + public long getAckMessagesPastCheckpointCount() { + return _acked.size(); + } + + /** + * Registers the given checkpoint by adding it to the deque of in-flight checkpoints. + * @param checkpoint is the latest record acked by the producer of the underlying pub sub framework + */ + @Override + public synchronized void register(T checkpoint) { + _inFlight.add(checkpoint); + _inFlightAfterLastConsumerCheckpoint.offerLast(checkpoint); + } + + /** + * The checkpoint acknowledgement can be received out of order. So here, we track the checkpoints by adding + * them in the _acked set and only update the _currentCheckpoint if a contiguous sequence of offsets are ack-ed + * from the front of the queue. + */ + @Override + public synchronized void ack(T checkpoint) { + // adding the checkpoint in the _acked set + _acked.add(checkpoint); + + // removing the checkpoint from the _inFlight set as we got acknowledgement for this checkpoint from producer + _inFlight.remove(checkpoint); + + // Until a contiguous sequence of offsets are not ack-ed from the producer for all the consumed records, we can't + // commit new checkpoint to consumer. This loops checks for that contiguous acked offsets. + while (!_inFlightAfterLastConsumerCheckpoint.isEmpty() && !_acked.isEmpty() && _acked.contains( + _inFlightAfterLastConsumerCheckpoint.peekFirst())) { + _currentCheckpoint = _inFlightAfterLastConsumerCheckpoint.pollFirst(); + + if (!_acked.remove(_currentCheckpoint)) { + LOG.error("Internal state error; could not remove checkpoint {}", _currentCheckpoint); + } + } + } +} \ No newline at end of file diff --git a/datastream-server-api/src/main/java/com/linkedin/datastream/server/callbackstatus/CallbackStatusWithNonComparableOffsetsFactory.java b/datastream-server-api/src/main/java/com/linkedin/datastream/server/callbackstatus/CallbackStatusWithNonComparableOffsetsFactory.java new file mode 100644 index 000000000..36bab86ab --- /dev/null +++ b/datastream-server-api/src/main/java/com/linkedin/datastream/server/callbackstatus/CallbackStatusWithNonComparableOffsetsFactory.java @@ -0,0 +1,23 @@ +/** + * Copyright 2021 LinkedIn Corporation. All rights reserved. + * Licensed under the BSD 2-Clause License. See the LICENSE file in the project root for license information. + * See the NOTICE file in the project root for additional information regarding copyright ownership. + */ +package com.linkedin.datastream.server.callbackstatus; + +/** + * Factory implementation for Callback Status With Non Comparable Offsets + * @param The type of the offset position that the underlying pub-sub system uses. + */ +public class CallbackStatusWithNonComparableOffsetsFactory> implements CallbackStatusFactory { + + /** + * Creates a callback status strategy that checkpoints the consumer offset on successful produce of that record + * with non comparable offsets + * @return CallbackStatus strategy construct + */ + @Override + public CallbackStatus createCallbackStatusStrategy() { + return new CallbackStatusWithNonComparableOffsets(); + } +} diff --git a/datastream-server-api/src/test/java/com/linkedin/datastream/server/TestFlushlessEventProducerHandler.java b/datastream-server-api/src/test/java/com/linkedin/datastream/server/TestFlushlessEventProducerHandler.java index 289c1e26a..c7c2e1955 100644 --- a/datastream-server-api/src/test/java/com/linkedin/datastream/server/TestFlushlessEventProducerHandler.java +++ b/datastream-server-api/src/test/java/com/linkedin/datastream/server/TestFlushlessEventProducerHandler.java @@ -18,8 +18,12 @@ import org.testng.annotations.Test; import com.linkedin.datastream.common.BrooklinEnvelope; +import com.linkedin.datastream.common.ReflectionUtils; import com.linkedin.datastream.server.api.transport.DatastreamRecordMetadata; import com.linkedin.datastream.server.api.transport.SendCallback; +import com.linkedin.datastream.server.callbackstatus.CallbackStatusFactory; +import com.linkedin.datastream.server.callbackstatus.CallbackStatusWithComparableOffsetsFactory; +import com.linkedin.datastream.server.callbackstatus.CallbackStatusWithNonComparableOffsetsFactory; import static com.linkedin.datastream.server.FlushlessEventProducerHandler.SourcePartition; @@ -32,11 +36,18 @@ public class TestFlushlessEventProducerHandler { private static final String TOPIC = "MyTopic"; private static final Random RANDOM = new Random(); - @Test - public void testSingleRecord() { - RandomEventProducer eventProducer = new RandomEventProducer(); - FlushlessEventProducerHandler handler = new FlushlessEventProducerHandler<>(eventProducer); + private static final CallbackStatusFactory OFFSET_CHECKPOINT_TRACKING_STRATEGY_WITH_NON_COMPARABLE_OFFSETS = + ReflectionUtils.createInstance(CallbackStatusWithNonComparableOffsetsFactory.class.getName()); + + private static final CallbackStatusFactory OFFSET_CHECKPOINT_TRACKING_STRATEGY_WITH_COMPARABLE_OFFSETS = + ReflectionUtils.createInstance(CallbackStatusWithComparableOffsetsFactory.class.getName()); + /** + * Helper function to test the scenario of sending single record for both comparable and non comparable offsets + * @param eventProducer is the interface for Connectors to send events to the designated destination + * @param handler is the flushless producer handler that does offset checkpoint management + */ + private void testSingleRecordWithGivenHandler(DatastreamEventProducer eventProducer, FlushlessEventProducerHandler handler) { long checkpoint = 1; DatastreamProducerRecord record = getDatastreamProducerRecord(checkpoint, TOPIC, 1); @@ -54,11 +65,28 @@ public void testSingleRecord() { } @Test - public void testMultipleSends() { + public void testSingleRecordWithComparableOffsets() { + RandomEventProducer eventProducer = new RandomEventProducer(); + FlushlessEventProducerHandler handler = + new FlushlessEventProducerHandler<>(eventProducer, OFFSET_CHECKPOINT_TRACKING_STRATEGY_WITH_COMPARABLE_OFFSETS); + testSingleRecordWithGivenHandler(eventProducer, handler); + } + + @Test + public void testSingleRecordWithNonComparableOffsets() { + RandomEventProducer eventProducer = new RandomEventProducer(); + FlushlessEventProducerHandler handler = new FlushlessEventProducerHandler<>(eventProducer, + OFFSET_CHECKPOINT_TRACKING_STRATEGY_WITH_NON_COMPARABLE_OFFSETS); + testSingleRecordWithGivenHandler(eventProducer, handler); + } + + @Test + public void testMultipleSendsWithComparableOffsets() { RandomEventProducer eventProducer = new RandomEventProducer(); - FlushlessEventProducerHandler handler = new FlushlessEventProducerHandler<>(eventProducer); + FlushlessEventProducerHandler handler = + new FlushlessEventProducerHandler<>(eventProducer, OFFSET_CHECKPOINT_TRACKING_STRATEGY_WITH_COMPARABLE_OFFSETS); - // Send 1000 messages to 100 partitions + // Send 1000 messages to 10 partitions for (int i = 0; i < 10; i++) { SourcePartition tp = new SourcePartition(TOPIC, i); for (int j = 0; j < 100; j++) { @@ -74,6 +102,7 @@ public void testMultipleSends() { Assert.assertTrue(ackOffset < minOffsetPending, "Not true that " + ackOffset + " is less than " + minOffsetPending); } + // event producer calls ack on the last element eventProducer.processOne(); for (int par = 0; par < 10; par++) { @@ -86,10 +115,35 @@ public void testMultipleSends() { } @Test - public void testOutOfOrderAck() { + public void testMultipleSendsWithNonComparableOffsets() { RandomEventProducer eventProducer = new RandomEventProducer(); - FlushlessEventProducerHandler handler = new FlushlessEventProducerHandler<>(eventProducer); + FlushlessEventProducerHandler handler = + new FlushlessEventProducerHandler<>(eventProducer, OFFSET_CHECKPOINT_TRACKING_STRATEGY_WITH_NON_COMPARABLE_OFFSETS); + // Send 1000 messages to 10 partitions + for (int i = 0; i < 10; i++) { + SourcePartition tp = new SourcePartition(TOPIC, i); + for (int j = 0; j < 100; j++) { + sendEvent(tp, handler, j); + } + } + + for (int i = 0; i < 1000; i++) { + eventProducer.processOne(); + } + + for (int par = 0; par < 10; par++) { + Assert.assertEquals(handler.getInFlightCount(TOPIC, par), 0); + } + } + + /** + * Helper function to test the scenario of sending multiple records for both comparable and non comparable offsets + * and validating acking behaviors + * @param eventProducer is the interface for Connectors to send events to the designated destination + * @param handler is the flushless producer handler that does offset checkpoint management + */ + private void testOutOfOrderAckForGivenHandler(RandomEventProducer eventProducer, FlushlessEventProducerHandler handler) { int partition = 0; SourcePartition tp = new SourcePartition(TOPIC, partition); @@ -102,12 +156,14 @@ public void testOutOfOrderAck() { eventProducer.process(tp, 4); // inflight result: 0, 1, 2, 3 Assert.assertEquals(handler.getAckCheckpoint(TOPIC, partition), Optional.empty(), "Safe checkpoint should be empty"); + Assert.assertEquals(handler.getInFlightCount(TOPIC, partition), 4, "Number of inflight messages should be 4"); Assert.assertEquals(handler.getAckMessagesPastCheckpointCount(TOPIC, partition), 1); // simulate callback for checkpoint 2 eventProducer.process(tp, 2); // inflight result: 0, 1, 3 Assert.assertEquals(handler.getAckCheckpoint(TOPIC, partition), Optional.empty(), "Safe checkpoint should be empty"); + Assert.assertEquals(handler.getInFlightCount(TOPIC, partition), 3, "Number of inflight messages should be 3"); Assert.assertEquals(handler.getAckMessagesPastCheckpointCount(TOPIC, partition), 2); @@ -115,12 +171,14 @@ public void testOutOfOrderAck() { eventProducer.process(tp, 0); // inflight result: 1, 3 Assert.assertEquals(handler.getAckCheckpoint(TOPIC, partition).get(), Long.valueOf(0), "Safe checkpoint should be 0"); + Assert.assertEquals(handler.getInFlightCount(TOPIC, partition), 2, "Number of inflight messages should be 2"); Assert.assertEquals(handler.getAckMessagesPastCheckpointCount(TOPIC, partition), 2); // simulate callback for checkpoint 1 eventProducer.process(tp, 0); // inflight result: 3 Assert.assertEquals(handler.getAckCheckpoint(TOPIC, partition).get(), Long.valueOf(2), "Safe checkpoint should be 1"); + Assert.assertEquals(handler.getInFlightCount(TOPIC, partition), 1, "Number of inflight messages should be 1"); Assert.assertEquals(handler.getAckMessagesPastCheckpointCount(TOPIC, partition), 1); @@ -134,15 +192,15 @@ public void testOutOfOrderAck() { eventProducer.process(tp, 0); // inflight result: 5 Assert.assertEquals(handler.getAckCheckpoint(TOPIC, partition).get(), Long.valueOf(4), "Safe checkpoint should be 4"); + Assert.assertEquals(handler.getInFlightCount(TOPIC, partition), 1, "Number of inflight messages should be 1"); Assert.assertEquals(handler.getAckMessagesPastCheckpointCount(TOPIC, partition), 0); // simulate callback for checkpoint 5 eventProducer.process(tp, 0); // inflight result: empty Assert.assertEquals(handler.getAckCheckpoint(TOPIC, partition).get(), Long.valueOf(5), "Safe checkpoint should be 5"); - Assert.assertEquals(handler.getAckMessagesPastCheckpointCount(TOPIC, partition), 0); - Assert.assertEquals(handler.getInFlightCount(TOPIC, partition), 0, "Number of inflight messages should be 0"); + Assert.assertEquals(handler.getAckMessagesPastCheckpointCount(TOPIC, partition), 0); // send another event with checkpoint 6 sendEvent(tp, handler, 6); @@ -153,14 +211,32 @@ public void testOutOfOrderAck() { eventProducer.process(tp, 0); // inflight result: empty Assert.assertEquals(handler.getAckCheckpoint(TOPIC, partition).get(), Long.valueOf(6), "Safe checkpoint should be 6"); + Assert.assertEquals(handler.getInFlightCount(TOPIC, partition), 0, "Number of inflight messages should be 0"); Assert.assertEquals(handler.getAckMessagesPastCheckpointCount(TOPIC, partition), 0); } @Test - public void testBackwardsOrderAck() { + public void testOutOfOrderAckForComparableOffsets() { + RandomEventProducer eventProducer = new RandomEventProducer(); + FlushlessEventProducerHandler handler = + new FlushlessEventProducerHandler<>(eventProducer, OFFSET_CHECKPOINT_TRACKING_STRATEGY_WITH_COMPARABLE_OFFSETS); + testOutOfOrderAckForGivenHandler(eventProducer, handler); + } + + @Test + public void testOutOfOrderAckForNonComparableOffsets() { RandomEventProducer eventProducer = new RandomEventProducer(); - FlushlessEventProducerHandler handler = new FlushlessEventProducerHandler<>(eventProducer); + FlushlessEventProducerHandler handler = + new FlushlessEventProducerHandler<>(eventProducer, OFFSET_CHECKPOINT_TRACKING_STRATEGY_WITH_NON_COMPARABLE_OFFSETS); + testOutOfOrderAckForGivenHandler(eventProducer, handler); + } + /** + * Helper function to test the behavior of backwards order record acking for both comparable and non comparable offsets + * @param randomEventProducer is the event producer to send records to randomized topic partition + * @param handler is the flushless producer handler that does offset checkpoint management + */ + private void testBackwardsOrderAckWithGivenHandler(RandomEventProducer randomEventProducer, FlushlessEventProducerHandler handler) { int partition = 0; SourcePartition tp = new SourcePartition(TOPIC, partition); @@ -171,19 +247,35 @@ public void testBackwardsOrderAck() { // acknowledge the checkpoints in backward (descending order) to simulate worst case scenario for (int i = 999; i > 0; i--) { - eventProducer.process(tp, i); + randomEventProducer.process(tp, i); // validate that checkpoint has to be empty because oldest message was not yet acknowledged Assert.assertEquals(handler.getAckCheckpoint(TOPIC, partition), Optional.empty(), "Safe checkpoint should be empty"); } // finally process the oldest message - eventProducer.process(tp, 0); + randomEventProducer.process(tp, 0); // validate that the checkpoint was finally updated to 999 Assert.assertEquals(handler.getAckCheckpoint(TOPIC, partition).get(), Long.valueOf(999), "Safe checkpoint should be 999"); } + @Test + public void testBackwardsOrderAckWithComparableOffsets() { + RandomEventProducer eventProducer = new RandomEventProducer(); + FlushlessEventProducerHandler handler = + new FlushlessEventProducerHandler<>(eventProducer, OFFSET_CHECKPOINT_TRACKING_STRATEGY_WITH_COMPARABLE_OFFSETS); + testBackwardsOrderAckWithGivenHandler(eventProducer, handler); + } + + @Test + public void testBackwardsOrderAckWithNonComparableOffsets() { + RandomEventProducer eventProducer = new RandomEventProducer(); + FlushlessEventProducerHandler handler = + new FlushlessEventProducerHandler<>(eventProducer, OFFSET_CHECKPOINT_TRACKING_STRATEGY_WITH_NON_COMPARABLE_OFFSETS); + testBackwardsOrderAckWithGivenHandler(eventProducer, handler); + } + private void sendEvent(SourcePartition tp, FlushlessEventProducerHandler handler, long checkpoint) { DatastreamProducerRecord record = getDatastreamProducerRecord(checkpoint, tp.getKey(), tp.getValue()); handler.send(record, tp.getSource(), tp.getPartition(), checkpoint, null); diff --git a/datastream-server-restli/src/test/java/com/linkedin/datastream/server/TestCoordinator.java b/datastream-server-restli/src/test/java/com/linkedin/datastream/server/TestCoordinator.java index 30fd9c459..fd2e73607 100644 --- a/datastream-server-restli/src/test/java/com/linkedin/datastream/server/TestCoordinator.java +++ b/datastream-server-restli/src/test/java/com/linkedin/datastream/server/TestCoordinator.java @@ -2532,6 +2532,68 @@ public void testDatastreamDeleteUponTTLExpire() throws Exception { }, 200, Duration.ofSeconds(30).toMillis()); } + @Test + public void testMultipleDatastreamDeleteUponTTLExpire() throws Exception { + TestSetup setup = createTestCoordinator(); + + String[] streamNames = {"TestDatastreamTTLExpire1", "TestDatastreamTTLExpire2", "TestDatastreamTTLExpire3"}; + Datastream[] streams = DatastreamTestUtils.createDatastreams(DummyConnector.CONNECTOR_TYPE, streamNames); + streams[0].getSource().setConnectionString(DummyConnector.VALID_DUMMY_SOURCE); + streams[1].getSource().setConnectionString(DummyConnector.VALID_DUMMY_SOURCE); + streams[2].getSource().setConnectionString(DummyConnector.VALID_DUMMY_SOURCE); + + streams[0].getMetadata().put(DatastreamMetadataConstants.TASK_PREFIX, "MyPrefix"); + streams[1].getMetadata().put(DatastreamMetadataConstants.TASK_PREFIX, "MyPrefix"); + streams[2].getMetadata().put(DatastreamMetadataConstants.TASK_PREFIX, "MyPrefix3"); + + // stream1 and stream2 expire in 1 minute from now and should get deleted when stream3 is created + long createTime = Instant.now().toEpochMilli(); + long expireTTL = Duration.ofMinutes(1).toMillis(); + + streams[0].getMetadata().put(CREATION_MS, String.valueOf(createTime)); + streams[0].getMetadata().put(TTL_MS, String.valueOf(expireTTL)); + streams[1].getMetadata().put(CREATION_MS, String.valueOf(createTime)); + streams[1].getMetadata().put(TTL_MS, String.valueOf(expireTTL)); + + // Creation should go through as TTL is not considered for freshly created streams (INITIALIZING) + CreateResponse createResponse = setup._resource.create(streams[0]); + Assert.assertNull(createResponse.getError()); + Assert.assertEquals(createResponse.getStatus(), HttpStatus.S_201_CREATED); + createResponse = setup._resource.create(streams[1]); + Assert.assertNull(createResponse.getError()); + Assert.assertEquals(createResponse.getStatus(), HttpStatus.S_201_CREATED); + + // Sleep for 1 minute to wait for stream1 and stream2 to expire. + Thread.sleep(Duration.ofMinutes(1).toMillis()); + + // Creating a stream3 which should trigger stream1 to be deleted + createResponse = setup._resource.create(streams[2]); + Assert.assertNull(createResponse.getError()); + Assert.assertEquals(createResponse.getStatus(), HttpStatus.S_201_CREATED); + + // Poll up to 30s for stream1 to get deleted + PollUtils.poll(() -> { + try { + setup._resource.get(streams[0].getName()); + return false; + } catch (RestLiServiceException e) { + Assert.assertEquals(e.getStatus(), HttpStatus.S_404_NOT_FOUND); + return true; + } + }, 200, Duration.ofSeconds(30).toMillis()); + + // Poll up to 30s for stream2 to get deleted + PollUtils.poll(() -> { + try { + setup._resource.get(streams[1].getName()); + return false; + } catch (RestLiServiceException e) { + Assert.assertEquals(e.getStatus(), HttpStatus.S_404_NOT_FOUND); + return true; + } + }, 200, Duration.ofSeconds(30).toMillis()); + } + @Test public void testDoNotAssignExpiredStreams() throws Exception { TestSetup setup = createTestCoordinator(); diff --git a/datastream-server/src/main/java/com/linkedin/datastream/server/CachedDatastreamReader.java b/datastream-server/src/main/java/com/linkedin/datastream/server/CachedDatastreamReader.java index 7d6578fe6..e4d324ad9 100644 --- a/datastream-server/src/main/java/com/linkedin/datastream/server/CachedDatastreamReader.java +++ b/datastream-server/src/main/java/com/linkedin/datastream/server/CachedDatastreamReader.java @@ -15,7 +15,7 @@ import java.util.concurrent.ConcurrentHashMap; import java.util.stream.Collectors; -import org.I0Itec.zkclient.exception.ZkNoNodeException; +import org.apache.helix.zookeeper.zkclient.exception.ZkNoNodeException; import org.slf4j.Logger; import org.slf4j.LoggerFactory; diff --git a/datastream-server/src/main/java/com/linkedin/datastream/server/Coordinator.java b/datastream-server/src/main/java/com/linkedin/datastream/server/Coordinator.java index 75a38c4e9..833b33d47 100644 --- a/datastream-server/src/main/java/com/linkedin/datastream/server/Coordinator.java +++ b/datastream-server/src/main/java/com/linkedin/datastream/server/Coordinator.java @@ -819,7 +819,7 @@ private void initializeTask(DatastreamTask task, Set failedDatas Map checkpoints = producer.loadCheckpoints(task); taskImpl.setCheckpoints(checkpoints); } catch (Exception e) { - _log.warn("Failed to initialize {} task", task.getDatastreamTaskName()); + _log.error("Failed to initialize task: " + task.getDatastreamTaskName(), e); if (retryAndSaveError) { _eventQueue.put(CoordinatorEvent.createHandleInstanceErrorEvent(ExceptionUtils.getRootCauseMessage(e))); failedDatastreamTasks.add(task); @@ -1005,6 +1005,9 @@ private void handleDatastreamAddOrDelete() { // Get the list of all datastreams List allStreams = _datastreamCache.getAllDatastreams(true); + // List of active streams that are not expired or deleted. Used for checking for a duplicate stream when deciding + // whether to delete a datastream tasks and topic or not. + List activeStreams = allStreams.stream().filter(ds -> !isDeletingOrExpired(ds)).collect(Collectors.toList()); // do nothing if there are zero datastreams if (allStreams.isEmpty()) { @@ -1036,7 +1039,7 @@ private void handleDatastreamAddOrDelete() { _log.info("Trying to hard delete datastream {} (reason={})", ds, ds.getStatus() == DatastreamStatus.DELETING ? "deleting" : "expired"); - hardDeleteDatastream(ds, allStreams); + hardDeleteDatastream(ds, activeStreams); } } @@ -1059,7 +1062,7 @@ private void handleDatastreamAddOrDelete() { _eventQueue.put(CoordinatorEvent.createLeaderDoAssignmentEvent(false)); } - private void hardDeleteDatastream(Datastream ds, List allStreams) { + private void hardDeleteDatastream(Datastream ds, List activeStreams) { String taskPrefix; if (DatastreamUtils.containsTaskPrefix(ds)) { taskPrefix = DatastreamUtils.getTaskPrefix(ds); @@ -1067,7 +1070,7 @@ private void hardDeleteDatastream(Datastream ds, List allStreams) { taskPrefix = DatastreamTaskImpl.getTaskPrefix(ds); } - Optional duplicateStream = allStreams.stream() + Optional duplicateStream = activeStreams.stream() .filter(DatastreamUtils::containsTaskPrefix) .filter(x -> !x.getName().equals(ds.getName()) && DatastreamUtils.getTaskPrefix(x).equals(taskPrefix)) .findFirst(); diff --git a/datastream-server/src/main/java/com/linkedin/datastream/server/DatastreamTaskImpl.java b/datastream-server/src/main/java/com/linkedin/datastream/server/DatastreamTaskImpl.java index 9ff6a9135..43a4929b6 100644 --- a/datastream-server/src/main/java/com/linkedin/datastream/server/DatastreamTaskImpl.java +++ b/datastream-server/src/main/java/com/linkedin/datastream/server/DatastreamTaskImpl.java @@ -19,11 +19,12 @@ import java.util.stream.Collectors; import org.apache.commons.lang.Validate; -import org.codehaus.jackson.annotate.JsonIgnore; import org.jetbrains.annotations.TestOnly; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import com.fasterxml.jackson.annotation.JsonIgnore; + import com.linkedin.datastream.common.Datastream; import com.linkedin.datastream.common.DatastreamDestination; import com.linkedin.datastream.common.DatastreamMetadataConstants; diff --git a/datastream-server/src/main/java/com/linkedin/datastream/server/EventProducer.java b/datastream-server/src/main/java/com/linkedin/datastream/server/EventProducer.java index 30f6a7b33..8344cde18 100644 --- a/datastream-server/src/main/java/com/linkedin/datastream/server/EventProducer.java +++ b/datastream-server/src/main/java/com/linkedin/datastream/server/EventProducer.java @@ -200,40 +200,75 @@ private void validateEventRecord(DatastreamProducerRecord record) { } } + @Override + public DatastreamRecordMetadata broadcast(DatastreamProducerRecord record, SendCallback sendEventCallback) { + return helperSendOrBroadcast(record, sendEventCallback, true); + } + + @Override + public void send(DatastreamProducerRecord record, SendCallback sendCallback) { + helperSendOrBroadcast(record, sendCallback, false); + } + /** * Send the event onto the underlying transport. * @param record the datastream event - * @param sendCallback the callback to be invoked after the event is sent to the destination + * @param sendEventCallback the callback to be invoked after the event is sent to the destination + * + * @return For broadcast return DatastreamRecordMetadata got from transport provider broadcast, null for send */ - @Override - public void send(DatastreamProducerRecord record, SendCallback sendCallback) { + private DatastreamRecordMetadata helperSendOrBroadcast(DatastreamProducerRecord record, + SendCallback sendEventCallback, boolean isBroadcast) { + DatastreamRecordMetadata broadcastMetadata = null; + try { validateEventRecord(record); - try { - record.serializeEvents(_datastreamTask.getDestinationSerDes()); - } catch (Exception e) { - if (_skipMessageOnSerializationErrors) { - _logger.info(String.format("Skipping the message on serialization error as configured. " - + "Datastream name: %s, Datastream task name: %s", - getDatastreamName(), _datastreamTask.getDatastreamTaskName()), e); - _dynamicMetricsManager.createOrUpdateCounter(MODULE, getDatastreamName(), - DROPPED_SENT_FROM_SERIALIZATION_ERROR, 1); - _dynamicMetricsManager.createOrUpdateCounter(MODULE, AGGREGATE, DROPPED_SENT_FROM_SERIALIZATION_ERROR, 1); - return; + record.serializeEvents(_datastreamTask.getDestinationSerDes()); + } catch (NullPointerException e) { + String errorMessage = String.format("Validation failed for record %s exception %s", record, e); + _logger.warn(errorMessage, e); + throw new DatastreamRuntimeException(errorMessage, e); + } catch (Exception e) { + if (_skipMessageOnSerializationErrors) { + _logger.info(String.format("Skipping the message on serialization error as configured. " + + "Datastream name: %s, Datastream task name: %s", + getDatastreamName(), _datastreamTask.getDatastreamTaskName()), e); + _dynamicMetricsManager.createOrUpdateCounter(MODULE, getDatastreamName(), + DROPPED_SENT_FROM_SERIALIZATION_ERROR, 1); + _dynamicMetricsManager.createOrUpdateCounter(MODULE, AGGREGATE, DROPPED_SENT_FROM_SERIALIZATION_ERROR, 1); + if (isBroadcast) { + return new DatastreamRecordMetadata(true); } - throw e; + return null; + } else { + String errorMessage = String.format("Failed to send event %s because of serialization exception %s", record, e); + _logger.warn(errorMessage, e); + throw new DatastreamRuntimeException(errorMessage, e); } + } + try { // Send the event to the transport String destination = record.getDestination().orElse(_datastreamTask.getDatastreamDestination().getConnectionString()); record.setEventsSendTimestamp(System.currentTimeMillis()); long recordEventsSourceTimestamp = record.getEventsSourceTimestamp(); long recordEventsSendTimestamp = record.getEventsSendTimestamp().orElse(0L); - _transportProvider.send(destination, record, - (metadata, exception) -> onSendCallback(metadata, exception, sendCallback, recordEventsSourceTimestamp, - recordEventsSendTimestamp)); + if (isBroadcast) { + broadcastMetadata = _transportProvider.broadcast(destination, record, + (metadata, exception) -> onSendCallback(metadata, exception, sendEventCallback, recordEventsSourceTimestamp, + recordEventsSendTimestamp)); + _logger.debug("Broadcast completed with {}", broadcastMetadata); + if (broadcastMetadata.isMessageSerializationError()) { + _logger.warn("Broadcast of record {} to destination {} failed because of serialization error.", + record, destination); + } + } else { + _transportProvider.send(destination, record, + (metadata, exception) -> onSendCallback(metadata, exception, sendEventCallback, recordEventsSourceTimestamp, + recordEventsSendTimestamp)); + } } catch (Exception e) { String errorMessage = String.format("Failed to send the event %s exception %s", record, e); _logger.warn(errorMessage, e); @@ -245,6 +280,8 @@ public void send(DatastreamProducerRecord record, SendCallback sendCallback) { if (_enableFlushOnSend && Instant.now().isAfter(_lastFlushTime.plus(_flushInterval))) { flush(); } + + return broadcastMetadata; } // Report SLA metrics for aggregate, connector and task diff --git a/datastream-server/src/main/java/com/linkedin/datastream/server/assignment/BroadcastStrategy.java b/datastream-server/src/main/java/com/linkedin/datastream/server/assignment/BroadcastStrategy.java index 238936a74..799b47bf8 100644 --- a/datastream-server/src/main/java/com/linkedin/datastream/server/assignment/BroadcastStrategy.java +++ b/datastream-server/src/main/java/com/linkedin/datastream/server/assignment/BroadcastStrategy.java @@ -89,10 +89,12 @@ public Map> assign(List datastreams tasksAvailableToReuse.stream().collect(Collectors.groupingBy(DatastreamTask::getTaskPrefix)); int instancePos = 0; for (DatastreamGroup dg : datastreams) { + Set uniqueDatastreamTaskNamesSet = new HashSet<>(); List reuseTasksPerDg = reuseTaskMap.getOrDefault(dg.getTaskPrefix(), Collections.emptyList()); int numTasks = getNumTasks(dg, instances.size()); - for (int taskPos = 0; taskPos < numTasks; taskPos++) { + int taskPos = 0; + while (taskPos < numTasks) { String instance = instances.get(instancePos); DatastreamTask foundDatastreamTask = currentAssignmentCopy.get(instance) @@ -102,10 +104,16 @@ public Map> assign(List datastreams .orElseGet(() -> getOrCreateDatastreamTask(reuseTasksPerDg, dg)); currentAssignmentCopy.get(instance).remove(foundDatastreamTask); + + // Prevent duplicate datastream task names + if (!uniqueDatastreamTaskNamesSet.add(foundDatastreamTask.getDatastreamTaskName())) { + continue; + } newAssignment.get(instance).add(foundDatastreamTask); - // Move to the next instance + // Move to the next task and instance instancePos = (instancePos + 1) % instances.size(); + taskPos++; } } diff --git a/datastream-server/src/main/java/com/linkedin/datastream/server/assignment/LoadBasedPartitionAssigner.java b/datastream-server/src/main/java/com/linkedin/datastream/server/assignment/LoadBasedPartitionAssigner.java index 6d67f4bb6..3f27a5e9e 100644 --- a/datastream-server/src/main/java/com/linkedin/datastream/server/assignment/LoadBasedPartitionAssigner.java +++ b/datastream-server/src/main/java/com/linkedin/datastream/server/assignment/LoadBasedPartitionAssigner.java @@ -11,19 +11,20 @@ import java.util.Comparator; import java.util.HashMap; import java.util.HashSet; +import java.util.IntSummaryStatistics; import java.util.List; import java.util.Map; import java.util.PriorityQueue; import java.util.Set; import java.util.concurrent.ConcurrentHashMap; -import java.util.concurrent.atomic.AtomicInteger; -import java.util.function.Supplier; import java.util.stream.Collectors; import org.apache.commons.lang.StringUtils; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import com.codahale.metrics.Gauge; + import com.google.common.annotations.VisibleForTesting; import com.linkedin.datastream.common.DatastreamRuntimeException; @@ -49,11 +50,11 @@ public class LoadBasedPartitionAssigner implements MetricsAware { private static final String MIN_PARTITIONS_ACROSS_TASKS = "minPartitionsAcrossTasks"; private static final String MAX_PARTITIONS_ACROSS_TASKS = "maxPartitionsAcrossTasks"; - private final Map _partitionAssignmentStatsMap = - new ConcurrentHashMap<>(); private final int _defaultPartitionBytesInKBRate; private final int _defaultPartitionMsgsInRate; + private final Map _metricsForDatastream = new ConcurrentHashMap<>(); + /** * Constructor of LoadBasedPartitionAssigner * @param defaultPartitionBytesInKBRate default bytesIn rate in KB for partition @@ -80,17 +81,17 @@ public Map> assignPartitions( ClusterThroughputInfo throughputInfo, Map> currentAssignment, List unassignedPartitions, DatastreamGroupPartitionsMetadata partitionMetadata, int maxPartitionsPerTask) { String datastreamGroupName = partitionMetadata.getDatastreamGroup().getName(); - Map partitionInfoMap = throughputInfo.getPartitionInfoMap(); + Map partitionInfoMap = new HashMap<>(throughputInfo.getPartitionInfoMap()); Set tasksWithChangedPartition = new HashSet<>(); // filter out all the tasks for the current datastream group, and retain assignments in a map - Map> newPartitions = new HashMap<>(); + Map> newPartitionAssignmentMap = new HashMap<>(); currentAssignment.values().forEach(tasks -> tasks.forEach(task -> { if (task.getTaskPrefix().equals(datastreamGroupName)) { Set retainedPartitions = new HashSet<>(task.getPartitionsV2()); retainedPartitions.retainAll(partitionMetadata.getPartitions()); - newPartitions.put(task.getId(), retainedPartitions); + newPartitionAssignmentMap.put(task.getId(), retainedPartitions); if (retainedPartitions.size() != task.getPartitionsV2().size()) { tasksWithChangedPartition.add(task.getId()); } @@ -98,7 +99,7 @@ public Map> assignPartitions( })); int numPartitions = partitionMetadata.getPartitions().size(); - int numTasks = newPartitions.size(); + int numTasks = newPartitionAssignmentMap.size(); validatePartitionCountAndThrow(datastreamGroupName, numTasks, numPartitions, maxPartitionsPerTask); // sort the current assignment's tasks on total throughput @@ -106,9 +107,13 @@ public Map> assignPartitions( PartitionThroughputInfo defaultPartitionInfo = new PartitionThroughputInfo(_defaultPartitionBytesInKBRate, _defaultPartitionMsgsInRate, ""); - newPartitions.forEach((task, partitions) -> { + newPartitionAssignmentMap.forEach((task, partitions) -> { int totalThroughput = partitions.stream() - .mapToInt(p -> partitionInfoMap.getOrDefault(p, defaultPartitionInfo).getBytesInKBRate()) + .mapToInt(p -> { + String topic = extractTopicFromPartition(p); + PartitionThroughputInfo defaultValue = partitionInfoMap.getOrDefault(topic, defaultPartitionInfo); + return partitionInfoMap.getOrDefault(p, defaultValue).getBytesInKBRate(); + }) .sum(); taskThroughputMap.put(task, totalThroughput); }); @@ -119,7 +124,16 @@ public Map> assignPartitions( if (partitionInfoMap.containsKey(partition)) { recognizedPartitions.add(partition); } else { - unrecognizedPartitions.add(partition); + // If the partition level information is not found, try finding topic level information. It is always better + // than no information about the partition. Update the map with that information so that it can be used in later + // part of the code. + String topic = extractTopicFromPartition(partition); + if (partitionInfoMap.containsKey(topic)) { + partitionInfoMap.put(partition, partitionInfoMap.get(topic)); + recognizedPartitions.add(partition); + } else { + unrecognizedPartitions.add(partition); + } } } @@ -132,8 +146,8 @@ public Map> assignPartitions( // build a priority queue of tasks based on throughput // only add tasks that can accommodate more partitions in the queue - List tasks = newPartitions.keySet().stream() - .filter(t -> newPartitions.get(t).size() < maxPartitionsPerTask) + List tasks = newPartitionAssignmentMap.keySet().stream() + .filter(t -> newPartitionAssignmentMap.get(t).size() < maxPartitionsPerTask) .collect(Collectors.toList()); PriorityQueue taskQueue = new PriorityQueue<>(Comparator.comparing(taskThroughputMap::get)); taskQueue.addAll(tasks); @@ -143,10 +157,10 @@ public Map> assignPartitions( String heaviestPartition = recognizedPartitions.remove(recognizedPartitions.size() - 1); int heaviestPartitionThroughput = partitionInfoMap.get(heaviestPartition).getBytesInKBRate(); String lightestTask = taskQueue.poll(); - newPartitions.get(lightestTask).add(heaviestPartition); + newPartitionAssignmentMap.get(lightestTask).add(heaviestPartition); taskThroughputMap.put(lightestTask, taskThroughputMap.get(lightestTask) + heaviestPartitionThroughput); tasksWithChangedPartition.add(lightestTask); - int currentNumPartitions = newPartitions.get(lightestTask).size(); + int currentNumPartitions = newPartitionAssignmentMap.get(lightestTask).size(); // don't put the task back in the queue if the number of its partitions is maxed out if (currentNumPartitions < maxPartitionsPerTask) { taskQueue.add(lightestTask); @@ -158,48 +172,49 @@ public Map> assignPartitions( Collections.shuffle(unrecognizedPartitions); int index = 0; for (String partition : unrecognizedPartitions) { - index = findTaskWithRoomForAPartition(tasks, newPartitions, index, maxPartitionsPerTask); + index = findTaskWithRoomForAPartition(tasks, newPartitionAssignmentMap, index, maxPartitionsPerTask); String currentTask = tasks.get(index); - newPartitions.get(currentTask).add(partition); + newPartitionAssignmentMap.get(currentTask).add(partition); tasksWithChangedPartition.add(currentTask); index = (index + 1) % tasks.size(); unrecognizedPartitionCountPerTask.put(currentTask, unrecognizedPartitionCountPerTask.getOrDefault(currentTask, 0) + 1); } - AtomicInteger minPartitionsAcrossTasks = new AtomicInteger(Integer.MAX_VALUE); - AtomicInteger maxPartitionsAcrossTasks = new AtomicInteger(0); // build the new assignment using the new partitions for the affected datastream's tasks - Map> newAssignments = new HashMap<>(); - currentAssignment.keySet().forEach(instance -> { - Set oldTasks = currentAssignment.get(instance); - Set newTasks = oldTasks.stream() - .map(task -> { - int partitionCount = newPartitions.containsKey(task.getId()) ? newPartitions.get(task.getId()).size() : - task.getPartitionsV2().size(); - - minPartitionsAcrossTasks.set(Math.min(minPartitionsAcrossTasks.get(), partitionCount)); - maxPartitionsAcrossTasks.set(Math.max(maxPartitionsAcrossTasks.get(), partitionCount)); - if (tasksWithChangedPartition.contains(task.getId())) { - DatastreamTaskImpl newTask = new DatastreamTaskImpl((DatastreamTaskImpl) task, newPartitions.get(task.getId())); - saveStats(partitionInfoMap, taskThroughputMap, unrecognizedPartitionCountPerTask, task, partitionCount, newTask); - return newTask; - } + Map> newAssignments = currentAssignment.entrySet().stream() + .collect(Collectors.toMap(x -> x.getKey(), x -> x.getValue().stream() + .map(task -> { + if (tasksWithChangedPartition.contains(task.getId())) { + Set newAssignment = newPartitionAssignmentMap.get(task.getId()); + DatastreamTaskImpl newTask = new DatastreamTaskImpl((DatastreamTaskImpl) task, newAssignment); + saveStats(partitionInfoMap, taskThroughputMap, unrecognizedPartitionCountPerTask, task, newAssignment.size(), newTask); + return newTask; + } else { return task; - }).collect(Collectors.toSet()); - newAssignments.put(instance, newTasks); - }); + } + }) + .collect(Collectors.toSet()))); + + IntSummaryStatistics stats = newAssignments.values().stream() + .flatMap(x -> x.stream()) // flatten + .filter(x -> x.getTaskPrefix().equals(datastreamGroupName)) + .collect(Collectors.summarizingInt(x -> x.getPartitionsV2().size())); // update metrics - PartitionAssignmentStats stats = new PartitionAssignmentStats(minPartitionsAcrossTasks.get(), - maxPartitionsAcrossTasks.get()); String taskPrefix = partitionMetadata.getDatastreamGroup().getTaskPrefix(); - updateMetricsForDatastream(taskPrefix, stats); + DatastreamMetrics metrics = metricsForDatastream(taskPrefix); + metrics.minPartitionsAcrossTasks(stats.getMin()); + metrics.maxPartitionsAcrossTasks(stats.getMax()); LOG.info("Assignment stats for {}. Min partitions across tasks: {}, max partitions across tasks: {}", taskPrefix, - stats.getMinPartitionsAcrossTasks(), stats.getMaxPartitionsAcrossTasks()); + stats.getMin(), stats.getMax()); return newAssignments; } + private DatastreamMetrics metricsForDatastream(String taskPrefix) { + return _metricsForDatastream.computeIfAbsent(taskPrefix, (x) -> new DatastreamMetrics(x)); + } + private void saveStats(Map partitionInfoMap, Map taskThroughputMap, Map unrecognizedPartitionCountPerTask, DatastreamTask task, int partitionCount, DatastreamTaskImpl newTask) { @@ -244,25 +259,6 @@ int findTaskWithRoomForAPartition(List tasks, Map> p throw new DatastreamRuntimeException("No tasks found that can host an additional partition"); } - void updateMetricsForDatastream(String datastream, PartitionAssignmentStats stats) { - if (!_partitionAssignmentStatsMap.containsKey(datastream)) { - registerLoadBasedPartitionAssignmentMetrics(datastream); - } - _partitionAssignmentStatsMap.put(datastream, stats); - } - - private void registerLoadBasedPartitionAssignmentMetrics(String datastream) { - Supplier minPartitionsAcrossTasksSupplier = () -> _partitionAssignmentStatsMap - .getOrDefault(datastream, PartitionAssignmentStats.DEFAULT).getMinPartitionsAcrossTasks(); - DYNAMIC_METRICS_MANAGER.registerGauge(CLASS_NAME, datastream, MIN_PARTITIONS_ACROSS_TASKS, - minPartitionsAcrossTasksSupplier); - - Supplier maxPartitionsAcrossTasksSupplier = () -> _partitionAssignmentStatsMap - .getOrDefault(datastream, PartitionAssignmentStats.DEFAULT).getMaxPartitionsAcrossTasks(); - DYNAMIC_METRICS_MANAGER.registerGauge(CLASS_NAME, datastream, MAX_PARTITIONS_ACROSS_TASKS, - maxPartitionsAcrossTasksSupplier); - } - /** * {@inheritDoc} */ @@ -278,13 +274,31 @@ public List getMetricInfos() { } void cleanupMetrics() { - _partitionAssignmentStatsMap.keySet().forEach(this::unregisterMetricsForDatastream); - _partitionAssignmentStatsMap.clear(); + _metricsForDatastream.keySet().forEach(this::unregisterMetricsForDatastream); } void unregisterMetricsForDatastream(String datastream) { - DYNAMIC_METRICS_MANAGER.unregisterMetric(CLASS_NAME, datastream, MIN_PARTITIONS_ACROSS_TASKS); - DYNAMIC_METRICS_MANAGER.unregisterMetric(CLASS_NAME, datastream, MAX_PARTITIONS_ACROSS_TASKS); + // cleanup existing DatastreamMetrics object, then remove it + _metricsForDatastream.compute(datastream, (k, v) -> { + if (v != null) { + v.cleanup(); + } + return null; + }); + } + + /** + * + * @param partition partition name + * @return topic name + */ + static String extractTopicFromPartition(String partition) { + String topic = partition; + int index = partition.lastIndexOf('-'); + if (index > -1) { + topic = partition.substring(0, index); + } + return topic; } static class PartitionAssignmentStatPerTask { @@ -346,39 +360,31 @@ public String toJson() throws IOException { return JsonUtils.toJson(this); } } - /** - * Encapsulates assignment metrics for a single datastream group - */ - private static class PartitionAssignmentStats { - private final int _minPartitionsAcrossTasks; - private final int _maxPartitionsAcrossTasks; - public static final PartitionAssignmentStats DEFAULT = new PartitionAssignmentStats(0, 0); + private static class DatastreamMetrics { + private final String taskPrefix; + private final Gauge minPartitionsAcrossTasks; + private final Gauge maxPartitionsAcrossTasks; + + DatastreamMetrics(String taskPrefix) { + this.taskPrefix = taskPrefix; + minPartitionsAcrossTasks = DYNAMIC_METRICS_MANAGER.registerGauge(CLASS_NAME, taskPrefix, + MIN_PARTITIONS_ACROSS_TASKS, () -> 0); + maxPartitionsAcrossTasks = DYNAMIC_METRICS_MANAGER.registerGauge(CLASS_NAME, taskPrefix, + MAX_PARTITIONS_ACROSS_TASKS, () -> 0); + } - /** - * Creates an instance of {@link PartitionAssignmentStats} - * @param minPartitionsAcrossTasks Minimum number of partitions across tasks - * @param maxPartitionsAcrossTasks Maximum number of partitions across tasks - */ - public PartitionAssignmentStats(int minPartitionsAcrossTasks, int maxPartitionsAcrossTasks) { - _minPartitionsAcrossTasks = minPartitionsAcrossTasks; - _maxPartitionsAcrossTasks = maxPartitionsAcrossTasks; + void cleanup() { + DYNAMIC_METRICS_MANAGER.unregisterMetric(CLASS_NAME, taskPrefix, MIN_PARTITIONS_ACROSS_TASKS); + DYNAMIC_METRICS_MANAGER.unregisterMetric(CLASS_NAME, taskPrefix, MAX_PARTITIONS_ACROSS_TASKS); } - /** - * Gets the minimum number of partitions across tasks - * @return Minimum number of partitions across tasks - */ - public int getMinPartitionsAcrossTasks() { - return _minPartitionsAcrossTasks; + void minPartitionsAcrossTasks(int min) { + DYNAMIC_METRICS_MANAGER.setGauge(minPartitionsAcrossTasks, () -> min); } - /** - * Gets the maximum number of partitions across tasks - * @return Maximum number of partitions across tasks - */ - public int getMaxPartitionsAcrossTasks() { - return _maxPartitionsAcrossTasks; + void maxPartitionsAcrossTasks(int max) { + DYNAMIC_METRICS_MANAGER.setGauge(maxPartitionsAcrossTasks, () -> max); } } } diff --git a/datastream-server/src/main/java/com/linkedin/datastream/server/assignment/LoadBasedTaskCountEstimator.java b/datastream-server/src/main/java/com/linkedin/datastream/server/assignment/LoadBasedTaskCountEstimator.java index b562a6b5d..87c0970aa 100644 --- a/datastream-server/src/main/java/com/linkedin/datastream/server/assignment/LoadBasedTaskCountEstimator.java +++ b/datastream-server/src/main/java/com/linkedin/datastream/server/assignment/LoadBasedTaskCountEstimator.java @@ -73,8 +73,11 @@ public int getTaskCount(ClusterThroughputInfo throughputInfo, List assig // total throughput in KB/sec int totalThroughput = allPartitions.stream() - .map(p -> throughputMap.getOrDefault(p, defaultThroughputInfo)) - .mapToInt(PartitionThroughputInfo::getBytesInKBRate) + .mapToInt(p -> { + String topic = LoadBasedPartitionAssigner.extractTopicFromPartition(p); + PartitionThroughputInfo defaultValue = throughputMap.getOrDefault(topic, defaultThroughputInfo); + return throughputMap.getOrDefault(p, defaultValue).getBytesInKBRate(); + }) .sum(); LOG.info("Total throughput in all {} partitions for datastream {}: {}KB/sec, assigned partitions: {} " + "unassigned partitions: {}", allPartitions.size(), datastreamName, totalThroughput, diff --git a/datastream-server/src/main/java/com/linkedin/datastream/server/assignment/LoadbalancingStrategy.java b/datastream-server/src/main/java/com/linkedin/datastream/server/assignment/LoadbalancingStrategy.java index ad416663d..12ffeb42e 100644 --- a/datastream-server/src/main/java/com/linkedin/datastream/server/assignment/LoadbalancingStrategy.java +++ b/datastream-server/src/main/java/com/linkedin/datastream/server/assignment/LoadbalancingStrategy.java @@ -102,8 +102,9 @@ private List getDatastreamTasks(List datastream currentAssignment.values().forEach(currentlyAssignedDatastreamTasks::addAll); for (DatastreamGroup dg : datastreams) { + Set uniqueDatastreamTaskNamesSet = new HashSet<>(); Set tasksForDatastreamGroup = currentlyAssignedDatastreamTasks.stream() - .filter(x -> x.getTaskPrefix().equals(dg.getTaskPrefix())) + .filter(x -> x.getTaskPrefix().equals(dg.getTaskPrefix()) && uniqueDatastreamTaskNamesSet.add(x.getDatastreamTaskName())) .collect(Collectors.toSet()); // If there are no datastream tasks that are currently assigned for this datastream. diff --git a/datastream-server/src/main/java/com/linkedin/datastream/server/assignment/StickyMulticastStrategy.java b/datastream-server/src/main/java/com/linkedin/datastream/server/assignment/StickyMulticastStrategy.java index e4bd6f6a1..cef388021 100644 --- a/datastream-server/src/main/java/com/linkedin/datastream/server/assignment/StickyMulticastStrategy.java +++ b/datastream-server/src/main/java/com/linkedin/datastream/server/assignment/StickyMulticastStrategy.java @@ -135,6 +135,7 @@ public Map> assign(List datastreams // STEP 1: keep assignments from previous instances, if possible. for (DatastreamGroup dg : datastreams) { int numTasks = constructExpectedNumberOfTasks(dg, instances.size()); + Set uniqueDatastreamTaskNamesSet = new HashSet<>(); Set allAliveTasks = new HashSet<>(); for (String instance : instances) { if (numTasks <= 0) { @@ -142,7 +143,7 @@ public Map> assign(List datastreams } List foundDatastreamTasks = Optional.ofNullable(currentAssignmentCopy.get(instance)).map(c -> - c.stream().filter(x -> x.getTaskPrefix().equals(dg.getTaskPrefix()) && !allAliveTasks.contains(x)) + c.stream().filter(x -> x.getTaskPrefix().equals(dg.getTaskPrefix()) && uniqueDatastreamTaskNamesSet.add(x.getDatastreamTaskName())) .collect(Collectors.toList())).orElse(Collections.emptyList()); allAliveTasks.addAll(foundDatastreamTasks); @@ -205,7 +206,8 @@ public Map> assign(List datastreams int minTasksPerInstance = tasksTotal / instances.size(); // some rebalance to increase the task count in instances below the minTasksPerInstance - while (newAssignment.get(instancesBySize.get(0)).size() < minTasksPerInstance) { + while (newAssignment.get(instancesBySize.get(0)).size() + _imbalanceThreshold < newAssignment.get( + instancesBySize.get(instancesBySize.size() - 1)).size()) { String smallInstance = instancesBySize.get(0); String largeInstance = instancesBySize.get(instancesBySize.size() - 1); diff --git a/datastream-server/src/main/java/com/linkedin/datastream/server/providers/FileBasedPartitionThroughputProvider.java b/datastream-server/src/main/java/com/linkedin/datastream/server/providers/FileBasedPartitionThroughputProvider.java index f4580070f..540abcbc4 100644 --- a/datastream-server/src/main/java/com/linkedin/datastream/server/providers/FileBasedPartitionThroughputProvider.java +++ b/datastream-server/src/main/java/com/linkedin/datastream/server/providers/FileBasedPartitionThroughputProvider.java @@ -14,9 +14,10 @@ import org.apache.commons.lang.NotImplementedException; import org.apache.commons.lang3.StringUtils; -import org.codehaus.jackson.JsonNode; -import org.codehaus.jackson.map.ObjectMapper; -import org.codehaus.jackson.type.TypeReference; + +import com.fasterxml.jackson.core.type.TypeReference; +import com.fasterxml.jackson.databind.JsonNode; +import com.fasterxml.jackson.databind.ObjectMapper; import com.linkedin.datastream.server.ClusterThroughputInfo; import com.linkedin.datastream.server.DatastreamGroup; @@ -87,7 +88,7 @@ private HashMap readThroughputInfoFromFile(File f try { JsonNode root = mapper.readTree(file); JsonNode allStats = root.get(ROOT_NODE_NAME); - Iterator clusterNames = allStats.getFieldNames(); + Iterator clusterNames = allStats.fieldNames(); while (clusterNames.hasNext()) { String key = clusterNames.next(); @@ -127,7 +128,7 @@ private ClusterThroughputInfo getClusterThroughputInfoFromNode(ObjectMapper mapp HashMap partitionInfoMap = new HashMap<>(); try { - HashMap partitionStats = mapper.readValue(clusterStats, mapTypeRef); + HashMap partitionStats = mapper.readValue(clusterStats.toString(), mapTypeRef); for (String partition : partitionStats.keySet()) { String value = partitionStats.get(partition); String[] tokens = StringUtils.split(value, ","); diff --git a/datastream-server/src/main/java/com/linkedin/datastream/server/providers/ZookeeperCheckpointProvider.java b/datastream-server/src/main/java/com/linkedin/datastream/server/providers/ZookeeperCheckpointProvider.java index fa92e8489..f72a1bf60 100644 --- a/datastream-server/src/main/java/com/linkedin/datastream/server/providers/ZookeeperCheckpointProvider.java +++ b/datastream-server/src/main/java/com/linkedin/datastream/server/providers/ZookeeperCheckpointProvider.java @@ -16,10 +16,11 @@ import org.apache.commons.lang.StringUtils; import org.apache.commons.lang.Validate; -import org.codehaus.jackson.type.TypeReference; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import com.fasterxml.jackson.core.type.TypeReference; + import com.linkedin.datastream.common.JsonUtils; import com.linkedin.datastream.metrics.BrooklinHistogramInfo; import com.linkedin.datastream.metrics.BrooklinMeterInfo; diff --git a/datastream-server/src/main/java/com/linkedin/datastream/server/zk/ZkAdapter.java b/datastream-server/src/main/java/com/linkedin/datastream/server/zk/ZkAdapter.java index cd3916ea2..13d7a2f4b 100644 --- a/datastream-server/src/main/java/com/linkedin/datastream/server/zk/ZkAdapter.java +++ b/datastream-server/src/main/java/com/linkedin/datastream/server/zk/ZkAdapter.java @@ -28,12 +28,12 @@ import java.util.function.Function; import java.util.stream.Collectors; -import org.I0Itec.zkclient.IZkChildListener; -import org.I0Itec.zkclient.IZkDataListener; -import org.I0Itec.zkclient.IZkStateListener; -import org.I0Itec.zkclient.exception.ZkException; -import org.I0Itec.zkclient.exception.ZkNoNodeException; import org.apache.commons.lang3.StringUtils; +import org.apache.helix.zookeeper.zkclient.IZkChildListener; +import org.apache.helix.zookeeper.zkclient.IZkDataListener; +import org.apache.helix.zookeeper.zkclient.IZkStateListener; +import org.apache.helix.zookeeper.zkclient.exception.ZkException; +import org.apache.helix.zookeeper.zkclient.exception.ZkNoNodeException; import org.apache.zookeeper.CreateMode; import org.apache.zookeeper.Watcher; import org.slf4j.Logger; @@ -274,10 +274,6 @@ public void connect(boolean reinitOnNewSession) { _leaderElectionListener = new ZkLeaderElectionListener(); } - if (_liveInstancesProvider == null) { - _liveInstancesProvider = new ZkBackedLiveInstanceListProvider(); - } - // create a globally unique instance name and create a live instance node in ZooKeeper _instanceName = createLiveInstanceNode(); @@ -353,11 +349,6 @@ private void closeZkListeners(boolean isDisconnect, boolean isSessionExpired) { _zkclient.unsubscribeDataChanges(KeyBuilder.liveInstance(_cluster, _currentSubscription), _leaderElectionListener); _currentSubscription = null; } - - if (_liveInstancesProvider != null) { - _liveInstancesProvider.close(); - _liveInstancesProvider = null; - } } if (_datastreamList != null) { @@ -365,6 +356,11 @@ private void closeZkListeners(boolean isDisconnect, boolean isSessionExpired) { _datastreamList = null; } + if (_liveInstancesProvider != null) { + _liveInstancesProvider.close(); + _liveInstancesProvider = null; + } + if (_targetAssignmentProvider != null) { _targetAssignmentProvider.close(); _targetAssignmentProvider = null; @@ -1835,9 +1831,9 @@ public void handleStateChanged(Watcher.Event.KeeperState state) { } @Override - public void handleNewSession() { + public void handleNewSession(final String sessionId) { synchronized (_zkSessionLock) { - LOG.info("ZkStateChangeListener::A new session has been established."); + LOG.info("ZkStateChangeListener::A new session with ID {} has been established.", sessionId); if (_reinitOnNewSession) { onNewSession(); } diff --git a/datastream-server/src/test/java/com/linkedin/datastream/server/TestLoadBasedTaskCountEstimator.java b/datastream-server/src/test/java/com/linkedin/datastream/server/TestLoadBasedTaskCountEstimator.java index f439e66f9..d68fa835a 100644 --- a/datastream-server/src/test/java/com/linkedin/datastream/server/TestLoadBasedTaskCountEstimator.java +++ b/datastream-server/src/test/java/com/linkedin/datastream/server/TestLoadBasedTaskCountEstimator.java @@ -99,4 +99,15 @@ public void partitionsHaveDefaultWeightTest() { int taskCount = estimator.getTaskCount(throughputInfo, assignedPartitions, unassignedPartitions, "test"); Assert.assertTrue(taskCount > 0); } + + @Test + public void throughputTaskEstimatorWithTopicLevelInformation() { + ClusterThroughputInfo throughputInfo = _provider.getThroughputInfo("fruit"); + List assignedPartitions = Collections.emptyList(); + List unassignedPartitions = Arrays.asList("apple-0", "apple-1", "apple-2", "banana-0"); + LoadBasedTaskCountEstimator estimator = new LoadBasedTaskCountEstimator(TASK_CAPACITY_MBPS, + TASK_CAPACITY_UTILIZATION_PCT, DEFAULT_BYTES_IN_KB_RATE, DEFAULT_MSGS_IN_RATE); + int taskCount = estimator.getTaskCount(throughputInfo, assignedPartitions, unassignedPartitions, "test"); + Assert.assertEquals(taskCount, 4); + } } diff --git a/datastream-server/src/test/java/com/linkedin/datastream/server/assignment/TestBroadcastStrategy.java b/datastream-server/src/test/java/com/linkedin/datastream/server/assignment/TestBroadcastStrategy.java index 0c69bbb03..9bc7c27b1 100644 --- a/datastream-server/src/test/java/com/linkedin/datastream/server/assignment/TestBroadcastStrategy.java +++ b/datastream-server/src/test/java/com/linkedin/datastream/server/assignment/TestBroadcastStrategy.java @@ -151,6 +151,32 @@ public void testDontCreateNewTasksWhenCalledSecondTime() { } } + @Test + public void testSameTaskIsNotAssignedToMoreThanOneInstance() { + List instances = Arrays.asList("instance1", "instance2", "instance3"); + int numDatastreams = 5; + List datastreams = generateDatastreams("ds", 5); + BroadcastStrategy strategy = new BroadcastStrategy(Optional.empty()); + Map> assignment = strategy.assign(datastreams, instances, new HashMap<>()); + // Copying the assignment to simulate the scenario where two instances have the same task, + // which is possible when the previous leader gets interrupted while updating the assignment. + assignment.get("instance1").addAll(assignment.get("instance2")); + + // Create a new task with the same name but different partitions, should dedupe out + Optional oldTask = assignment.get("instance1") + .stream() + .findFirst(); + Assert.assertTrue(oldTask.isPresent()); + DatastreamTaskImpl newTask = new DatastreamTaskImpl(oldTask.get().getDatastreams(), oldTask.get().getId(), Arrays.asList(0, 1, 2)); + assignment.get("instance1").add(newTask); + + Map> newAssignment = strategy.assign(datastreams, instances, assignment); + Set newAssignmentTasks = newAssignment.values().stream().flatMap(Set::stream).collect(Collectors.toSet()); + List newAssignmentTasksList = newAssignment.values().stream().flatMap(Set::stream).collect(Collectors.toList()); + Assert.assertEquals(newAssignmentTasks.size(), newAssignmentTasksList.size()); + Assert.assertEquals(newAssignmentTasks.size(), instances.size() * numDatastreams); + } + @Test public void testRemoveDatastreamTasksWhenDatastreamIsDeleted() { List instances = Arrays.asList("instance1", "instance2", "instance3"); diff --git a/datastream-server/src/test/java/com/linkedin/datastream/server/assignment/TestLoadBasedPartitionAssigner.java b/datastream-server/src/test/java/com/linkedin/datastream/server/assignment/TestLoadBasedPartitionAssigner.java index f0abe90e7..935c01297 100644 --- a/datastream-server/src/test/java/com/linkedin/datastream/server/assignment/TestLoadBasedPartitionAssigner.java +++ b/datastream-server/src/test/java/com/linkedin/datastream/server/assignment/TestLoadBasedPartitionAssigner.java @@ -13,12 +13,15 @@ import java.util.List; import java.util.Map; import java.util.Set; +import java.util.function.Predicate; import org.mockito.Mockito; import org.testng.Assert; import org.testng.annotations.BeforeClass; import org.testng.annotations.Test; +import com.codahale.metrics.Gauge; +import com.codahale.metrics.Metric; import com.codahale.metrics.MetricRegistry; import com.linkedin.datastream.common.Datastream; @@ -34,6 +37,7 @@ import com.linkedin.datastream.server.PartitionThroughputInfo; import com.linkedin.datastream.server.zk.ZkAdapter; import com.linkedin.datastream.testutil.DatastreamTestUtils; +import com.linkedin.datastream.testutil.MetricsTestUtils; import static org.mockito.Matchers.anyString; @@ -94,6 +98,11 @@ public void assignFromScratchTest() { Assert.assertEquals(statObj.getTotalPartitions(), 1); Assert.assertEquals(statObj.getPartitionsWithUnknownThroughput(), 0); Assert.assertEquals(statObj.getThroughputRateInKBps(), 5); + + assertMetricEquals("LoadBasedPartitionAssigner.ds1.minPartitionsAcrossTasks", 1); + assertMetricEquals("LoadBasedPartitionAssigner.ds1.maxPartitionsAcrossTasks", 1); + + MetricsTestUtils.verifyMetrics(assigner, DynamicMetricsManager.getInstance()); } @Test @@ -148,6 +157,13 @@ public void newAssignmentRetainsTasksFromOtherDatastreamsTest() { Assert.assertEquals(statObj.getTotalPartitions(), 1); Assert.assertEquals(statObj.getPartitionsWithUnknownThroughput(), 0); Assert.assertEquals(statObj.getThroughputRateInKBps(), 5); + + assertMetric("LoadBasedPartitionAssigner.ds1.minPartitionsAcrossTasks", (Integer x) -> x > 0); + assertMetric("LoadBasedPartitionAssigner.ds1.maxPartitionsAcrossTasks", (Integer x) -> x <= 3); + assertMetric("LoadBasedPartitionAssigner.ds2.minPartitionsAcrossTasks", (Integer x) -> x > 0); + assertMetric("LoadBasedPartitionAssigner.ds2.maxPartitionsAcrossTasks", (Integer x) -> x <= 3); + + MetricsTestUtils.verifyMetrics(assigner, DynamicMetricsManager.getInstance()); } @Test @@ -183,6 +199,11 @@ public void assignmentDistributesPartitionsWhenThroughputInfoIsMissingTest() { Assert.assertEquals(statObj.getTotalPartitions(), 2); Assert.assertEquals(statObj.getPartitionsWithUnknownThroughput(), 2); Assert.assertEquals(statObj.getThroughputRateInKBps(), 0); + + assertMetricEquals("LoadBasedPartitionAssigner.ds1.minPartitionsAcrossTasks", 2); + assertMetricEquals("LoadBasedPartitionAssigner.ds1.maxPartitionsAcrossTasks", 2); + + MetricsTestUtils.verifyMetrics(assigner, DynamicMetricsManager.getInstance()); } @Test @@ -216,8 +237,55 @@ public void lightestTaskGetsNewPartitionTest() { // verify that task in instance1 got the new partition Assert.assertEquals(task3.getPartitionsV2().size(), 3); Assert.assertTrue(task3.getPartitionsV2().contains("P4")); + + assertMetricEquals("LoadBasedPartitionAssigner.ds1.minPartitionsAcrossTasks", 1); + assertMetricEquals("LoadBasedPartitionAssigner.ds1.maxPartitionsAcrossTasks", 3); + + MetricsTestUtils.verifyMetrics(assigner, DynamicMetricsManager.getInstance()); + } + + @Test + public void lightestTaskGetsNewPartitionWithTopicMetricsTest() { + List unassignedPartitions = Arrays.asList("P-2", "P-3"); + Map throughputInfoMap = new HashMap<>(); + throughputInfoMap.put("P-1", new PartitionThroughputInfo(5, 5, "P-1")); + throughputInfoMap.put("R", new PartitionThroughputInfo(5, 5, "R")); + throughputInfoMap.put("T", new PartitionThroughputInfo(50, 5, "T")); + throughputInfoMap.put("P", new PartitionThroughputInfo(40, 5, "P")); + ClusterThroughputInfo throughputInfo = new ClusterThroughputInfo("dummy", throughputInfoMap); + + Datastream ds1 = DatastreamTestUtils.createDatastreams(DummyConnector.CONNECTOR_TYPE, "ds1")[0]; + ds1.getSource().setPartitions(0); + ds1.getMetadata().put(DatastreamMetadataConstants.TASK_PREFIX, DatastreamTaskImpl.getTaskPrefix(ds1)); + Map> currentAssignment = new HashMap<>(); + DatastreamTask task1 = createTaskForDatastream(ds1, Arrays.asList("P-1", "R-1")); + DatastreamTask task2 = createTaskForDatastream(ds1, Collections.singletonList("T-1")); + currentAssignment.put("instance1", new HashSet<>(Collections.singletonList(task1))); + currentAssignment.put("instance2", new HashSet<>(Collections.singletonList(task2))); + + DatastreamGroupPartitionsMetadata metadata = new DatastreamGroupPartitionsMetadata(new DatastreamGroup( + Collections.singletonList(ds1)), Arrays.asList("P-1", "P-2", "P-3", "R-1", "T-1")); + + LoadBasedPartitionAssigner assigner = new LoadBasedPartitionAssigner(5, 10); + Map> newAssignment = assigner.assignPartitions(throughputInfo, currentAssignment, + unassignedPartitions, metadata, Integer.MAX_VALUE); + + DatastreamTask task3 = (DatastreamTask) newAssignment.get("instance1").toArray()[0]; + + // verify that task in instance1 got the new partition + Assert.assertEquals(task3.getPartitionsV2().size(), 3); + Assert.assertTrue(task3.getPartitionsV2().contains("P-3")); + + DatastreamTask task4 = (DatastreamTask) newAssignment.get("instance2").toArray()[0]; + + // verify that task in instance1 got the new partition + Assert.assertEquals(task4.getPartitionsV2().size(), 2); + Assert.assertTrue(task4.getPartitionsV2().contains("P-2")); + + MetricsTestUtils.verifyMetrics(assigner, DynamicMetricsManager.getInstance()); } + @Test public void throwsExceptionWhenNotEnoughRoomForAllPartitionsTest() { List unassignedPartitions = Arrays.asList("P4", "P5"); @@ -274,6 +342,11 @@ public void taskWithRoomGetsNewPartitionTest() { // verify that task in instance2 got the new partition Assert.assertEquals(task3.getPartitionsV2().size(), 2); Assert.assertTrue(task3.getPartitionsV2().contains("P4")); + + assertMetricEquals("LoadBasedPartitionAssigner.ds1.minPartitionsAcrossTasks", 2); + assertMetricEquals("LoadBasedPartitionAssigner.ds1.maxPartitionsAcrossTasks", 2); + + MetricsTestUtils.verifyMetrics(assigner, DynamicMetricsManager.getInstance()); } @Test @@ -303,6 +376,8 @@ public void findTaskWithRoomForAPartitionTests() { partitionsMap2.get("T3").add("P2"); index2 = assigner.findTaskWithRoomForAPartition(tasks2, partitionsMap2, 1, 1); Assert.assertEquals(index2, 0); + + MetricsTestUtils.verifyMetrics(assigner, DynamicMetricsManager.getInstance()); } private DatastreamTask createTaskForDatastream(Datastream datastream) { @@ -328,4 +403,20 @@ private ClusterThroughputInfo getDummyClusterThroughputInfo(List partiti } return new ClusterThroughputInfo("dummy", partitionThroughputMap); } + + @SuppressWarnings("unchecked") + private void assertMetric(String name, Predicate predicate) { + Metric metric = DynamicMetricsManager.getInstance().getMetric(name); + Assert.assertNotNull(metric); + if (metric instanceof Gauge) { + T value = ((Gauge) metric).getValue(); + Assert.assertTrue(predicate.test(value), "(value " + value.toString() + ")"); + } else { + Assert.fail("unexpected metric type " + metric.getClass().getSimpleName()); + } + } + + private void assertMetricEquals(String name, T value) { + assertMetric(name, Predicate.isEqual(value)); + } } diff --git a/datastream-server/src/test/java/com/linkedin/datastream/server/assignment/TestLoadbalancingStrategy.java b/datastream-server/src/test/java/com/linkedin/datastream/server/assignment/TestLoadbalancingStrategy.java index 743c94345..c1d471076 100644 --- a/datastream-server/src/test/java/com/linkedin/datastream/server/assignment/TestLoadbalancingStrategy.java +++ b/datastream-server/src/test/java/com/linkedin/datastream/server/assignment/TestLoadbalancingStrategy.java @@ -11,6 +11,7 @@ import java.util.HashMap; import java.util.List; import java.util.Map; +import java.util.Optional; import java.util.Properties; import java.util.Set; import java.util.stream.Collectors; @@ -222,6 +223,37 @@ public void testLoadbalancingStrategyCreatesNewDatastreamTasksWhenNewDatastreamI } } + @Test + public void testLoadbalancingStrategySameTaskIsNotAssignedToMoreThanOneInstance() { + List instances = Arrays.asList("instance1", "instance2", "instance3"); + List ds = + Arrays.asList(DatastreamTestUtils.createDatastreams(DummyConnector.CONNECTOR_TYPE, "ds1", "ds2", "ds3", "ds4", "ds5")); + ds.forEach(x -> x.getSource().setPartitions(1)); + ds.forEach( + x -> x.getMetadata().put(DatastreamMetadataConstants.TASK_PREFIX, DatastreamTaskImpl.getTaskPrefix(x))); + List datastreams = + ds.stream().map(x -> new DatastreamGroup(Collections.singletonList(x))).collect(Collectors.toList()); + LoadbalancingStrategy strategy = new LoadbalancingStrategy(); + Map> assignment = strategy.assign(datastreams, instances, new HashMap<>()); + // Copying the assignment to simulate the scenario where two instances have the same task, + // which is possible when the previous leader gets interrupted while updating the assignment. + assignment.get("instance1").addAll(assignment.get("instance2")); + + // Create a new task with the same name but different partitions, should dedupe out + Optional oldTask = assignment.get("instance1") + .stream() + .findFirst(); + Assert.assertTrue(oldTask.isPresent()); + DatastreamTaskImpl newTask = new DatastreamTaskImpl(oldTask.get().getDatastreams(), oldTask.get().getId(), Arrays.asList(0, 1, 2)); + assignment.get("instance1").add(newTask); + + Map> newAssignment = strategy.assign(datastreams, instances, assignment); + Set newAssignmentTasks = newAssignment.values().stream().flatMap(Set::stream).collect(Collectors.toSet()); + List newAssignmentTasksList = newAssignment.values().stream().flatMap(Set::stream).collect(Collectors.toList()); + Assert.assertEquals(newAssignmentTasks.size(), newAssignmentTasksList.size()); + Assert.assertEquals(newAssignmentTasks.size(), 5); + } + @Test public void testLoadbalancingStrategyRemovesTasksWhenDatastreamIsDeleted() { String[] instances = new String[]{"instance1", "instance2", "instance3"}; diff --git a/datastream-server/src/test/java/com/linkedin/datastream/server/assignment/TestStickyMulticastStrategy.java b/datastream-server/src/test/java/com/linkedin/datastream/server/assignment/TestStickyMulticastStrategy.java index 9aa8f8bfc..396d61c25 100644 --- a/datastream-server/src/test/java/com/linkedin/datastream/server/assignment/TestStickyMulticastStrategy.java +++ b/datastream-server/src/test/java/com/linkedin/datastream/server/assignment/TestStickyMulticastStrategy.java @@ -7,9 +7,11 @@ import java.util.ArrayList; import java.util.Arrays; +import java.util.Collection; import java.util.Collections; import java.util.Comparator; import java.util.HashMap; +import java.util.HashSet; import java.util.List; import java.util.Map; import java.util.Objects; @@ -294,6 +296,14 @@ public void testSameTaskIsNotAssignedToMoreThanOneInstance() { // which is possible when the previous leader gets interrupted while updating the assignment. assignment.get("instance1").addAll(assignment.get("instance2")); + // Create a new task with the same name but different partitions, should dedupe out + Optional oldTask = assignment.get("instance1") + .stream() + .findFirst(); + Assert.assertTrue(oldTask.isPresent()); + DatastreamTaskImpl newTask = new DatastreamTaskImpl(oldTask.get().getDatastreams(), oldTask.get().getId(), Arrays.asList(0, 1, 2)); + assignment.get("instance1").add(newTask); + Map> newAssignment = strategy.assign(datastreams, Arrays.asList(instances), assignment); Set newAssignmentTasks = newAssignment.values().stream().flatMap(Set::stream).collect(Collectors.toSet()); List newAssignmentTasksList = newAssignment.values().stream().flatMap(Set::stream).collect(Collectors.toList()); @@ -499,6 +509,80 @@ public void testExtraTasksAreNotAssignedDuringReassignment() { } } + @Test + public void testReBalancingTasksWithThreshold() { + String[] instances = new String[]{"instance1", "instance2", "instance3"}; + reBalancingTasksWithThresholdHelper(instances, new int[]{9, 9, 13}, 1); + reBalancingTasksWithThresholdHelper(instances, new int[]{9, 9, 13}, 2); + reBalancingTasksWithThresholdHelper(instances, new int[]{9, 9, 13}, 3); + + reBalancingTasksWithThresholdHelper(instances, new int[]{1, 10, 20}, 1); + reBalancingTasksWithThresholdHelper(instances, new int[]{1, 10, 20}, 2); + reBalancingTasksWithThresholdHelper(instances, new int[]{1, 10, 20}, 3); + + reBalancingTasksWithThresholdHelper(instances, new int[]{5, 5, 5}, 1); + reBalancingTasksWithThresholdHelper(instances, new int[]{5, 5, 5}, 2); + reBalancingTasksWithThresholdHelper(instances, new int[]{5, 5, 5}, 3); + + instances = new String[]{"instance1", "instance2"}; + reBalancingTasksWithThresholdHelper(instances, new int[]{9, 13}, 1); + reBalancingTasksWithThresholdHelper(instances, new int[]{9, 13}, 2); + reBalancingTasksWithThresholdHelper(instances, new int[]{9, 13}, 3); + + reBalancingTasksWithThresholdHelper(instances, new int[]{1, 10}, 1); + reBalancingTasksWithThresholdHelper(instances, new int[]{1, 10}, 2); + reBalancingTasksWithThresholdHelper(instances, new int[]{1, 10}, 3); + + reBalancingTasksWithThresholdHelper(instances, new int[]{5, 5}, 1); + reBalancingTasksWithThresholdHelper(instances, new int[]{5, 5}, 2); + reBalancingTasksWithThresholdHelper(instances, new int[]{5, 5}, 3); + + instances = new String[]{"instance1"}; + reBalancingTasksWithThresholdHelper(instances, new int[]{9}, 1); + reBalancingTasksWithThresholdHelper(instances, new int[]{9}, 2); + reBalancingTasksWithThresholdHelper(instances, new int[]{9}, 3); + } + + // this helper function tests the rebalancing of tasks with an imbalance threshold across the instances + // with StickyMulticastStrategy + private void reBalancingTasksWithThresholdHelper(String[] instances, int[] taskDistribution, int imbalanceThreshold) { + List datastreams = generateDatastreams("ds", 1); + StickyMulticastStrategy strategy = new StickyMulticastStrategy(Optional.empty(), imbalanceThreshold); + + // create a dummy current assignment following the parameterized task distribution; based on which the + // new assignment will be generated. + HashMap> currentDummyAssignment = new HashMap<>(); + for (int index = 0; index < instances.length; index += 1) { + currentDummyAssignment.put(instances[index], + getDummyTasksSet(taskDistribution[index], datastreams.get(0).getDatastreams())); + } + + int totalNumberTasks = currentDummyAssignment.values().stream().mapToInt(Collection::size).sum(); + // setting the total count of tasks as the max tasks for our single datastream, so that the new assignment + // distribution looks similar to the current assignment. + datastreams.get(0).getDatastreams().get(0).getMetadata().put(CFG_MAX_TASKS, Integer.toString(totalNumberTasks)); + + Map> assignment = + strategy.assign(datastreams, Arrays.asList(instances), currentDummyAssignment); + + Arrays.sort(instances, Comparator.comparing(x -> assignment.get(x).size())); + + int minTasksAssignedToInstance = assignment.get(instances[0]).size(); + int maxTasksAssignedToInstance = assignment.get(instances[instances.length - 1]).size(); + + Assert.assertTrue(maxTasksAssignedToInstance - minTasksAssignedToInstance <= imbalanceThreshold); + } + + // returns a dummy set with #numTasks tasks + private HashSet getDummyTasksSet(int numTasks, List datastreams) { + HashSet assignedTasks = new HashSet<>(); + while (numTasks > 0) { + assignedTasks.add(new DatastreamTaskImpl(datastreams)); + numTasks -= 1; + } + return assignedTasks; + } + private static String assignmentToString(Map> assignment) { StringBuilder builder = new StringBuilder(); assignment.keySet().stream().sorted().forEach(instance -> { diff --git a/datastream-server/src/test/java/com/linkedin/datastream/server/zk/TestZkAdapter.java b/datastream-server/src/test/java/com/linkedin/datastream/server/zk/TestZkAdapter.java index 69365b438..165d14c97 100644 --- a/datastream-server/src/test/java/com/linkedin/datastream/server/zk/TestZkAdapter.java +++ b/datastream-server/src/test/java/com/linkedin/datastream/server/zk/TestZkAdapter.java @@ -830,7 +830,7 @@ private void verifyZkListenersOfFollower(ZkClientInterceptingAdapter adapter2) { Assert.assertNotNull(adapter2.getLeaderElectionListener()); Assert.assertNotNull(adapter2.getAssignmentListProvider()); Assert.assertNotNull(adapter2.getStateChangeListener()); - Assert.assertNotNull(adapter2.getLiveInstancesProvider()); + Assert.assertNull(adapter2.getLiveInstancesProvider()); Assert.assertNull(adapter2.getDatastreamList()); Assert.assertNull(adapter2.getTargetAssignmentProvider()); } diff --git a/datastream-server/src/test/resources/partitionThroughput.json b/datastream-server/src/test/resources/partitionThroughput.json index 52bb1813f..a90a4c7e9 100644 --- a/datastream-server/src/test/resources/partitionThroughput.json +++ b/datastream-server/src/test/resources/partitionThroughput.json @@ -227,6 +227,10 @@ "donut" : { "BostonCreme-1" : "bytesInKB: 5000, msgIn:200", "BostonCreme-2" : "bytesInKB: 5000, msgIn:200" + }, + "fruit" : { + "apple" : "bytesInKB: 10000, msgIn:300", + "apple-2" : "bytesInKb: 8000, msgIn:200" } } } diff --git a/datastream-tools/src/main/java/com/linkedin/datastream/tools/DatastreamRestClientCli.java b/datastream-tools/src/main/java/com/linkedin/datastream/tools/DatastreamRestClientCli.java index c99c42634..c150af818 100644 --- a/datastream-tools/src/main/java/com/linkedin/datastream/tools/DatastreamRestClientCli.java +++ b/datastream-tools/src/main/java/com/linkedin/datastream/tools/DatastreamRestClientCli.java @@ -18,8 +18,9 @@ import org.apache.commons.cli.HelpFormatter; import org.apache.commons.cli.Options; import org.apache.commons.lang.StringUtils; -import org.codehaus.jackson.map.ObjectMapper; -import org.codehaus.jackson.type.TypeReference; + +import com.fasterxml.jackson.core.type.TypeReference; +import com.fasterxml.jackson.databind.ObjectMapper; import com.linkedin.data.template.StringMap; import com.linkedin.datastream.DatastreamRestClient; @@ -63,7 +64,7 @@ private static void printDatastreams(boolean noformat, List streams) String jsonValue = DatastreamUtils.toJSON(s); if (!noformat) { Object json = mapper.readValue(jsonValue, Object.class); - jsonValue = mapper.defaultPrettyPrintingWriter().writeValueAsString(json); + jsonValue = mapper.writerWithDefaultPrettyPrinter().writeValueAsString(json); } System.out.println(jsonValue); diff --git a/datastream-utils/src/main/java/com/linkedin/datastream/common/DatastreamUtils.java b/datastream-utils/src/main/java/com/linkedin/datastream/common/DatastreamUtils.java index d9be1c7fc..4dda17f13 100644 --- a/datastream-utils/src/main/java/com/linkedin/datastream/common/DatastreamUtils.java +++ b/datastream-utils/src/main/java/com/linkedin/datastream/common/DatastreamUtils.java @@ -21,7 +21,8 @@ import org.apache.commons.io.IOUtils; import org.apache.commons.lang.StringUtils; import org.apache.commons.lang.Validate; -import org.codehaus.jackson.type.TypeReference; + +import com.fasterxml.jackson.core.type.TypeReference; import com.linkedin.data.template.GetMode; import com.linkedin.data.template.StringMap; diff --git a/datastream-utils/src/main/java/com/linkedin/datastream/common/zk/ZkClient.java b/datastream-utils/src/main/java/com/linkedin/datastream/common/zk/ZkClient.java index 9be39dd27..b582dc3f8 100644 --- a/datastream-utils/src/main/java/com/linkedin/datastream/common/zk/ZkClient.java +++ b/datastream-utils/src/main/java/com/linkedin/datastream/common/zk/ZkClient.java @@ -6,27 +6,19 @@ package com.linkedin.datastream.common.zk; import java.nio.charset.StandardCharsets; -import java.util.Arrays; import java.util.List; import java.util.Random; import java.util.Stack; -import org.I0Itec.zkclient.ZkConnection; -import org.I0Itec.zkclient.exception.ZkInterruptedException; -import org.I0Itec.zkclient.exception.ZkMarshallingError; -import org.I0Itec.zkclient.exception.ZkNoNodeException; -import org.I0Itec.zkclient.exception.ZkNodeExistsException; -import org.I0Itec.zkclient.serialize.ZkSerializer; -import org.apache.zookeeper.CreateMode; -import org.apache.zookeeper.data.Stat; +import org.apache.helix.zookeeper.zkclient.exception.ZkMarshallingError; +import org.apache.helix.zookeeper.zkclient.exception.ZkNodeExistsException; +import org.apache.helix.zookeeper.zkclient.serialize.ZkSerializer; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import com.google.common.annotations.VisibleForTesting; - /** - * ZKClient is a wrapper of {@link org.I0Itec.zkclient.ZkClient}. It provides the following + * ZKClient is a wrapper of {@link org.apache.helix.zookeeper.impl.client.ZkClient}. It provides the following * basic features: *
    *
  1. tolerate network reconnects so the caller doesn't have to handle the retries
  2. @@ -34,7 +26,7 @@ *
  3. additional features like ensurePath to recursively create paths
  4. *
*/ -public class ZkClient extends org.I0Itec.zkclient.ZkClient { +public class ZkClient extends org.apache.helix.zookeeper.impl.client.ZkClient { public static final String ZK_PATH_SEPARATOR = "/"; public static final int DEFAULT_CONNECTION_TIMEOUT = 60 * 1000; public static final int DEFAULT_SESSION_TIMEOUT = 30 * 1000; @@ -86,71 +78,20 @@ public ZkClient(String zkServers, int sessionTimeoutMs, int connectionTimeoutMs, _zkSessionTimeoutMs = sessionTimeoutMs; } - @Override - public void close() throws ZkInterruptedException { - if (LOG.isTraceEnabled()) { - StackTraceElement[] calls = Thread.currentThread().getStackTrace(); - LOG.trace("closing zkclient. callStack: {}", Arrays.asList(calls)); - } - getEventLock().lock(); - try { - if (_connection == null) { - return; - } - LOG.info("closing zkclient: {}", ((ZkConnection) _connection).getZookeeper()); - super.close(); - } catch (ZkInterruptedException e) { - /* - * Workaround for HELIX-264: calling ZkClient#disconnect() in its own eventThread context will - * throw ZkInterruptedException and skip ZkConnection#disconnect() - */ - try { - /* - * ZkInterruptedException#construct() honors InterruptedException by calling - * Thread.currentThread().interrupt(); clear it first, so we can safely disconnect the - * zk-connection - */ - Thread.interrupted(); - _connection.close(); - /* - * restore interrupted status of current thread - */ - Thread.currentThread().interrupt(); - } catch (InterruptedException e1) { - throw new ZkInterruptedException(e1); - } - } finally { - getEventLock().unlock(); - LOG.info("closed zkclient"); - } - } - + /** + * Check if a zk path exists. Changes the access modified to public, its defined as protected in parent class. + */ @Override public boolean exists(final String path, final boolean watch) { - long startT = System.nanoTime(); - - try { - return retryUntilConnected(() -> _connection.exists(path, watch)); - } finally { - long endT = System.nanoTime(); - if (LOG.isTraceEnabled()) { - LOG.trace("exists, path: {}, time: {} ns", path, (endT - startT)); - } - } + return super.exists(path, watch); } + /** + * Get all children of zk path. Changes the access modified to public, its defined as protected in parent class. + */ @Override public List getChildren(final String path, final boolean watch) { - long startT = System.nanoTime(); - - try { - return retryUntilConnected(() -> _connection.getChildren(path, watch)); - } finally { - long endT = System.nanoTime(); - if (LOG.isTraceEnabled()) { - LOG.trace("getChildren, path: {}, time: {} ns", path, (endT - startT)); - } - } + return super.getChildren(path, watch); } /** @@ -212,80 +153,6 @@ public String ensureReadData(final String path) { return ensureReadData(path, _zkSessionTimeoutMs); } - @Override - @SuppressWarnings("unchecked") - protected T readData(final String path, final Stat stat, final boolean watch) { - long startT = System.nanoTime(); - try { - byte[] data = retryUntilConnected(() -> _connection.readData(path, stat, watch)); - return (T) deserialize(data); - } finally { - long endT = System.nanoTime(); - if (LOG.isTraceEnabled()) { - LOG.trace("readData, path: {}, time: {} ns", path, (endT - startT)); - } - } - } - - @Override - public void writeData(final String path, Object data, final int expectedVersion) { - long startT = System.nanoTime(); - try { - final byte[] bytes = serialize(data); - - retryUntilConnected(() -> { - _connection.writeData(path, bytes, expectedVersion); - return null; - }); - } finally { - long endT = System.nanoTime(); - if (LOG.isTraceEnabled()) { - LOG.trace("writeData, path: {}, time: {} ns", path, (endT - startT)); - } - } - } - - @Override - public String create(final String path, Object data, final CreateMode mode) throws RuntimeException { - if (path == null) { - throw new IllegalArgumentException("path must not be null."); - } - - long startT = System.nanoTime(); - try { - final byte[] bytes = data == null ? null : serialize(data); - - return retryUntilConnected(() -> _connection.create(path, bytes, mode)); - } finally { - long endT = System.nanoTime(); - if (LOG.isTraceEnabled()) { - LOG.trace("create, path: {}, time: {} ns", path, (endT - startT)); - } - } - } - - @Override - public boolean delete(final String path) { - long startT = System.nanoTime(); - try { - try { - retryUntilConnected(() -> { - _connection.delete(path); - return null; - }); - - return true; - } catch (ZkNoNodeException e) { - return false; - } - } finally { - long endT = System.nanoTime(); - if (LOG.isTraceEnabled()) { - LOG.trace("delete, path: {}, time: {} ns", path, (endT - startT)); - } - } - } - /** * Ensure that all the paths in the given full path String are created * @param path the zk path @@ -347,11 +214,6 @@ public T deserialize(byte[] data) { return (T) _zkSerializer.deserialize(data); } - @VisibleForTesting - public long getSessionId() { - return ((ZkConnection) _connection).getZookeeper().getSessionId(); - } - private static class ZKStringSerializer implements ZkSerializer { @Override public byte[] serialize(Object data) throws ZkMarshallingError { diff --git a/datastream-utils/src/test/java/com/linkedin/datastream/common/zk/TestZkClient.java b/datastream-utils/src/test/java/com/linkedin/datastream/common/zk/TestZkClient.java index ed0a6d3a0..9ee1ec447 100644 --- a/datastream-utils/src/test/java/com/linkedin/datastream/common/zk/TestZkClient.java +++ b/datastream-utils/src/test/java/com/linkedin/datastream/common/zk/TestZkClient.java @@ -8,8 +8,9 @@ import java.io.IOException; import java.util.List; -import org.I0Itec.zkclient.IZkChildListener; -import org.I0Itec.zkclient.IZkDataListener; +import org.apache.helix.zookeeper.zkclient.IZkChildListener; +import org.apache.helix.zookeeper.zkclient.IZkDataListener; +import org.apache.helix.zookeeper.zkclient.exception.ZkNoNodeException; import org.apache.zookeeper.CreateMode; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -63,6 +64,30 @@ public void testReadAndWriteRoundTrip() throws Exception { zkClient.close(); } + @Test + public void testCreateNoNodeException() throws Exception { + ZkClient zkClient = new ZkClient(_zkConnectionString); + + String electionPath = "/leaderelection"; + String electionNodeName = electionPath + "/coordinator-"; + + // now create this node with persistent mode + Assert.assertThrows(ZkNoNodeException.class, + () -> zkClient.create(electionNodeName, "test", CreateMode.PERSISTENT_SEQUENTIAL)); + } + + @Test + public void testCreateIllegalArgumentException() throws Exception { + ZkClient zkClient = new ZkClient(_zkConnectionString); + + String electionPath = "/leaderelection"; + String electionNodeName = electionPath + "/coordinator-"; + + // now create this node with persistent mode + Assert.assertThrows(NullPointerException.class, + () -> zkClient.create(null, "test", CreateMode.PERSISTENT_SEQUENTIAL)); + } + @Test public void testCreateEphemeralSequentialNode() throws Exception { ZkClient zkClient = new ZkClient(_zkConnectionString); diff --git a/gradle.properties b/gradle.properties index cee3d9542..c20535735 100644 --- a/gradle.properties +++ b/gradle.properties @@ -1,4 +1,4 @@ rest.model.compatibility=ignore org.gradle.daemon=false org.gradle.parallel=false -scalaSuffix=2.11 +scalaSuffix=2.12 diff --git a/gradle/dependency-versions.gradle b/gradle/dependency-versions.gradle index c6042e0df..fffe8b59b 100644 --- a/gradle/dependency-versions.gradle +++ b/gradle/dependency-versions.gradle @@ -1,7 +1,6 @@ ext { - LIKafkaVersion = "1.0.65" apacheHttpClientVersion = "4.5.3" - avroVersion = "1.7.7" + avroVersion = "1.9.2" commonsCliVersion = "1.2" commonsHttpClientVersion = "3.1" commonsIOVersion = "2.4" @@ -9,17 +8,16 @@ ext { commonsValidatorVersion = "1.5.1" guavaVersion = "25.0-jre" intellijAnnotationsVersion = "12.0" - jacksonVersion = "1.8.5" - kafkaVersion = "2.0.0.25" + jacksonVersion = "2.10.0" + kafkaVersion = "2.4.1.57" log4jVersion = "1.2.17" metricsCoreVersion = "4.1.0" mockitoVersion = "1.10.19" parseqVersion = "2.6.31" pegasusVersion = "29.14.0" - scalaVersion = "2.11" + scalaVersion = "2.12" slf4jVersion = "1.7.5" testngVersion = "7.1.0" - zkclientVersion = "0.11" zookeeperVersion = "3.4.13" - helixZkclientVersion = "1.0.1" + helixZkclientVersion = "1.0.2" } diff --git a/gradle/maven.gradle b/gradle/maven.gradle index fc6aca855..8febd361d 100644 --- a/gradle/maven.gradle +++ b/gradle/maven.gradle @@ -1,5 +1,5 @@ allprojects { - version = "2.0.0" + version = "4.0.0" } subprojects { diff --git a/scripts/git/commit-msg b/scripts/git/commit-msg index 75342ab67..749a5f4d8 100755 --- a/scripts/git/commit-msg +++ b/scripts/git/commit-msg @@ -1,4 +1,4 @@ -#!/usr/bin/env python +#!/usr/bin/python3 import sys import re @@ -102,7 +102,7 @@ def print_error(msg): def print_warning(msg): - print TerminalColors.warn(msg) + print(TerminalColors.warn(msg)) def process_commit_message(msg): diff --git a/settings.gradle b/settings.gradle index 67174fa1f..cfb5cbf29 100644 --- a/settings.gradle +++ b/settings.gradle @@ -11,18 +11,5 @@ include 'datastream-server-api' include 'datastream-server-restli' include 'datastream-tools' include 'datastream-utils' - -def scalaModules = [ - 'datastream-kafka', - 'datastream-testcommon' -] as HashSet - -scalaModules.each { - include it -} - -rootProject.children.each { - if (scalaModules.contains(it.name)) { - it.name = it.name + "_" + scalaSuffix - } -} +include 'datastream-testcommon' +include 'datastream-kafka' \ No newline at end of file