From 601bb27ea308e45acb1d91703f1a373c5cfd970a Mon Sep 17 00:00:00 2001 From: Suraj Kasi Date: Tue, 9 Nov 2021 09:47:09 -0800 Subject: [PATCH 01/32] Replace 101tec ZkClient with Helix ZkClient Use helix version 1.0.2. Extend common/zk/ZkClient.java from helix zookeeper client and refactoring some code. --- build.gradle | 4 +- .../KafkaMirrorMakerConnectorTask.java | 2 +- .../server/CachedDatastreamReader.java | 2 +- .../datastream/server/zk/ZkAdapter.java | 12 +- .../datastream/common/zk/ZkClient.java | 162 ++---------------- .../datastream/common/zk/TestZkClient.java | 4 +- gradle/dependency-versions.gradle | 2 +- 7 files changed, 29 insertions(+), 159 deletions(-) diff --git a/build.gradle b/build.gradle index 5d89998e7..a6ce388da 100644 --- a/build.gradle +++ b/build.gradle @@ -157,7 +157,7 @@ project(':datastream-server-api') { 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") @@ -319,7 +319,6 @@ 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" @@ -355,7 +354,6 @@ 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 "org.apache.commons:commons-lang3:$commonslang3Version" 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..973fb1fe9 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; 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/zk/ZkAdapter.java b/datastream-server/src/main/java/com/linkedin/datastream/server/zk/ZkAdapter.java index cd3916ea2..862bfbeab 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; @@ -1835,7 +1835,7 @@ 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."); if (_reinitOnNewSession) { 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..cb93fa1c9 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,19 +6,13 @@ 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; @@ -26,7 +20,7 @@ /** - * 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 +28,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 +80,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 +155,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,9 +216,12 @@ public T deserialize(byte[] data) { return (T) _zkSerializer.deserialize(data); } + /** + * Get Zk sessions session ID + */ @VisibleForTesting public long getSessionId() { - return ((ZkConnection) _connection).getZookeeper().getSessionId(); + return super.getSessionId(); } private static class ZKStringSerializer implements ZkSerializer { 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..de65aeed6 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,8 @@ 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.zookeeper.CreateMode; import org.slf4j.Logger; import org.slf4j.LoggerFactory; diff --git a/gradle/dependency-versions.gradle b/gradle/dependency-versions.gradle index c6042e0df..ce54aa96c 100644 --- a/gradle/dependency-versions.gradle +++ b/gradle/dependency-versions.gradle @@ -21,5 +21,5 @@ ext { testngVersion = "7.1.0" zkclientVersion = "0.11" zookeeperVersion = "3.4.13" - helixZkclientVersion = "1.0.1" + helixZkclientVersion = "1.0.2" } From 065450e6620e1ce5634a247d7830ef41f174ab56 Mon Sep 17 00:00:00 2001 From: Vaibhav Maheshwari Date: Mon, 29 Nov 2021 13:09:47 -0800 Subject: [PATCH 02/32] Use topic level throughput information when partition level information is unavailable. (#871) * Use topic level throughput information when partition level information is unavailable * Refactor code Co-authored-by: Vaibhav Maheshwari --- .../server/assignment/LoadBasedPartitionAssigner.java | 10 +++++++++- .../assignment/LoadBasedTaskCountEstimator.java | 11 +++++++++-- .../server/TestLoadBasedTaskCountEstimator.java | 11 +++++++++++ .../src/test/resources/partitionThroughput.json | 4 ++++ 4 files changed, 33 insertions(+), 3 deletions(-) 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..2195b7609 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 @@ -108,7 +108,15 @@ public Map> assignPartitions( newPartitions.forEach((task, partitions) -> { int totalThroughput = partitions.stream() - .mapToInt(p -> partitionInfoMap.getOrDefault(p, defaultPartitionInfo).getBytesInKBRate()) + .mapToInt(p -> { + int index = p.lastIndexOf('-'); + String topic = p; + if (index > -1) { + topic = p.substring(0, index); + } + PartitionThroughputInfo defaultValue = partitionInfoMap.getOrDefault(topic, defaultPartitionInfo); + return partitionInfoMap.getOrDefault(p, defaultValue).getBytesInKBRate(); + }) .sum(); taskThroughputMap.put(task, totalThroughput); }); 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..7e39dc3a5 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,15 @@ 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 -> { + int index = p.lastIndexOf('-'); + String topic = p; + if (index > -1) { + topic = p.substring(0, index); + } + 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/test/java/com/linkedin/datastream/server/TestLoadBasedTaskCountEstimator.java b/datastream-server/src/test/java/com/linkedin/datastream/server/TestLoadBasedTaskCountEstimator.java index f439e66f9..dbe55da9f 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); + int taskCount = estimator.getTaskCount(throughputInfo, assignedPartitions, unassignedPartitions); + Assert.assertEquals(taskCount, 4); + } } 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" } } } From b838a4b4d0555190669cc4a210cf1f24aeaaec4e Mon Sep 17 00:00:00 2001 From: Vaibhav Maheshwari Date: Wed, 1 Dec 2021 10:02:57 -0800 Subject: [PATCH 03/32] Fix compilation errors (#874) Co-authored-by: Vaibhav Maheshwari --- .../datastream/server/TestLoadBasedTaskCountEstimator.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) 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 dbe55da9f..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 @@ -106,8 +106,8 @@ public void throughputTaskEstimatorWithTopicLevelInformation() { 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); - int taskCount = estimator.getTaskCount(throughputInfo, assignedPartitions, unassignedPartitions); + 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); } } From 9906ecaa754e5b80c9f963043b70023c24eb925c Mon Sep 17 00:00:00 2001 From: Suraj Kasi Date: Thu, 2 Dec 2021 14:22:08 -0800 Subject: [PATCH 04/32] Removing unused older zk client version number variable and redundant methods --- .../com/linkedin/datastream/common/zk/ZkClient.java | 10 ---------- gradle/dependency-versions.gradle | 1 - 2 files changed, 11 deletions(-) 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 cb93fa1c9..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 @@ -16,8 +16,6 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import com.google.common.annotations.VisibleForTesting; - /** * ZKClient is a wrapper of {@link org.apache.helix.zookeeper.impl.client.ZkClient}. It provides the following @@ -216,14 +214,6 @@ public T deserialize(byte[] data) { return (T) _zkSerializer.deserialize(data); } - /** - * Get Zk sessions session ID - */ - @VisibleForTesting - public long getSessionId() { - return super.getSessionId(); - } - private static class ZKStringSerializer implements ZkSerializer { @Override public byte[] serialize(Object data) throws ZkMarshallingError { diff --git a/gradle/dependency-versions.gradle b/gradle/dependency-versions.gradle index ce54aa96c..7d4a034b9 100644 --- a/gradle/dependency-versions.gradle +++ b/gradle/dependency-versions.gradle @@ -19,7 +19,6 @@ ext { scalaVersion = "2.11" slf4jVersion = "1.7.5" testngVersion = "7.1.0" - zkclientVersion = "0.11" zookeeperVersion = "3.4.13" helixZkclientVersion = "1.0.2" } From 5a09717515dc5829385f8c01012ee2dfcf3b547e Mon Sep 17 00:00:00 2001 From: Suraj Kasi Date: Fri, 3 Dec 2021 18:39:17 -0800 Subject: [PATCH 05/32] Add exception unit tests for ZkClient create method --- .../datastream/common/zk/TestZkClient.java | 35 +++++++++++++++++++ 1 file changed, 35 insertions(+) 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 de65aeed6..5537f0e48 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 @@ -10,6 +10,7 @@ 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,40 @@ 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 + try { + zkClient.create(electionNodeName, "test", CreateMode.PERSISTENT_SEQUENTIAL); + } catch (ZkNoNodeException e) { + zkClient.close(); + return; + } + throw new Exception("Test failed, expected ZkNoNodeException"); + } + + @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 + try { + zkClient.create(null, "test", CreateMode.PERSISTENT_SEQUENTIAL); + } catch (NullPointerException e) { + zkClient.close(); + return; + } + throw new Exception("Test failed, expected NullPointerException"); + } + @Test public void testCreateEphemeralSequentialNode() throws Exception { ZkClient zkClient = new ZkClient(_zkConnectionString); From 49554b4540c761b72a9b2090abb462570ae9d122 Mon Sep 17 00:00:00 2001 From: Suraj Kasi Date: Mon, 6 Dec 2021 14:31:15 -0800 Subject: [PATCH 06/32] Update log message and refactor unit tests --- .../datastream/server/zk/ZkAdapter.java | 2 +- .../datastream/common/zk/TestZkClient.java | 18 ++++-------------- 2 files changed, 5 insertions(+), 15 deletions(-) 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 862bfbeab..4e78a27b1 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 @@ -1837,7 +1837,7 @@ public void handleStateChanged(Watcher.Event.KeeperState state) { @Override 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-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 5537f0e48..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 @@ -72,13 +72,8 @@ public void testCreateNoNodeException() throws Exception { String electionNodeName = electionPath + "/coordinator-"; // now create this node with persistent mode - try { - zkClient.create(electionNodeName, "test", CreateMode.PERSISTENT_SEQUENTIAL); - } catch (ZkNoNodeException e) { - zkClient.close(); - return; - } - throw new Exception("Test failed, expected ZkNoNodeException"); + Assert.assertThrows(ZkNoNodeException.class, + () -> zkClient.create(electionNodeName, "test", CreateMode.PERSISTENT_SEQUENTIAL)); } @Test @@ -89,13 +84,8 @@ public void testCreateIllegalArgumentException() throws Exception { String electionNodeName = electionPath + "/coordinator-"; // now create this node with persistent mode - try { - zkClient.create(null, "test", CreateMode.PERSISTENT_SEQUENTIAL); - } catch (NullPointerException e) { - zkClient.close(); - return; - } - throw new Exception("Test failed, expected NullPointerException"); + Assert.assertThrows(NullPointerException.class, + () -> zkClient.create(null, "test", CreateMode.PERSISTENT_SEQUENTIAL)); } @Test From 7380202b1522ce07423176667ef4d3ccf0c20590 Mon Sep 17 00:00:00 2001 From: Vaibhav Maheshwari Date: Mon, 6 Dec 2021 14:56:19 -0800 Subject: [PATCH 07/32] Loadbased Partition assigner not using topic level metrics to recognize partitions (#876) The load based estimator is considering the topic level metrics, but the assigner code is not using the information to distribute the partitions. Fixing the code to derive the partition level metrics. --- .../LoadBasedPartitionAssigner.java | 57 ++++++++++++------- .../LoadBasedTaskCountEstimator.java | 6 +- .../TestLoadBasedPartitionAssigner.java | 39 +++++++++++++ 3 files changed, 78 insertions(+), 24 deletions(-) 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 2195b7609..793b9e7b2 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 @@ -80,17 +80,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 +98,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,14 +106,10 @@ public Map> assignPartitions( PartitionThroughputInfo defaultPartitionInfo = new PartitionThroughputInfo(_defaultPartitionBytesInKBRate, _defaultPartitionMsgsInRate, ""); - newPartitions.forEach((task, partitions) -> { + newPartitionAssignmentMap.forEach((task, partitions) -> { int totalThroughput = partitions.stream() .mapToInt(p -> { - int index = p.lastIndexOf('-'); - String topic = p; - if (index > -1) { - topic = p.substring(0, index); - } + String topic = extractTopicFromPartition(p); PartitionThroughputInfo defaultValue = partitionInfoMap.getOrDefault(topic, defaultPartitionInfo); return partitionInfoMap.getOrDefault(p, defaultValue).getBytesInKBRate(); }) @@ -127,7 +123,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); + } } } @@ -140,8 +145,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); @@ -151,10 +156,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); @@ -166,9 +171,9 @@ 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); @@ -182,13 +187,13 @@ public Map> assignPartitions( Set oldTasks = currentAssignment.get(instance); Set newTasks = oldTasks.stream() .map(task -> { - int partitionCount = newPartitions.containsKey(task.getId()) ? newPartitions.get(task.getId()).size() : + int partitionCount = newPartitionAssignmentMap.containsKey(task.getId()) ? newPartitionAssignmentMap.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())); + DatastreamTaskImpl newTask = new DatastreamTaskImpl((DatastreamTaskImpl) task, newPartitionAssignmentMap.get(task.getId())); saveStats(partitionInfoMap, taskThroughputMap, unrecognizedPartitionCountPerTask, task, partitionCount, newTask); return newTask; } @@ -295,6 +300,20 @@ void unregisterMetricsForDatastream(String datastream) { DYNAMIC_METRICS_MANAGER.unregisterMetric(CLASS_NAME, datastream, MAX_PARTITIONS_ACROSS_TASKS); } + /** + * + * @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 { private int throughputRateInKBps; private int totalPartitions; 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 7e39dc3a5..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 @@ -74,11 +74,7 @@ public int getTaskCount(ClusterThroughputInfo throughputInfo, List assig // total throughput in KB/sec int totalThroughput = allPartitions.stream() .mapToInt(p -> { - int index = p.lastIndexOf('-'); - String topic = p; - if (index > -1) { - topic = p.substring(0, index); - } + String topic = LoadBasedPartitionAssigner.extractTopicFromPartition(p); PartitionThroughputInfo defaultValue = throughputMap.getOrDefault(topic, defaultThroughputInfo); return throughputMap.getOrDefault(p, defaultValue).getBytesInKBRate(); }) 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..8005e9b74 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 @@ -218,6 +218,45 @@ public void lightestTaskGetsNewPartitionTest() { Assert.assertTrue(task3.getPartitionsV2().contains("P4")); } + @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")); + } + @Test public void throwsExceptionWhenNotEnoughRoomForAllPartitionsTest() { List unassignedPartitions = Arrays.asList("P4", "P5"); From 7f9c97dbc9fe78059f8f4a44746c4309f1da7e7d Mon Sep 17 00:00:00 2001 From: Suraj Kasi Date: Mon, 13 Dec 2021 16:42:09 -0800 Subject: [PATCH 08/32] Updating Zookeeper version to 3.6.2.3 which has pagination support (3.6.2.1+) --- gradle/dependency-versions.gradle | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/gradle/dependency-versions.gradle b/gradle/dependency-versions.gradle index 7d4a034b9..1bdb0e424 100644 --- a/gradle/dependency-versions.gradle +++ b/gradle/dependency-versions.gradle @@ -19,6 +19,6 @@ ext { scalaVersion = "2.11" slf4jVersion = "1.7.5" testngVersion = "7.1.0" - zookeeperVersion = "3.4.13" + zookeeperVersion = "3.6.2.3" helixZkclientVersion = "1.0.2" } From 039c28ca65dbe9157c965d4b1db69c5c41f22796 Mon Sep 17 00:00:00 2001 From: Shrinand Thakkar Date: Wed, 5 Jan 2022 12:31:21 -0800 Subject: [PATCH 09/32] Flushless producer supporting both comparable and non comparable offsets (#873) * Flushless producer without comparing offsets * Flushless Producer Supporting Both Comparable and Non Comparable Offsets for Ack-ing * Fix comments * Making methods abstract * Created config support to use both strategies and added tests * Address comments * Address comments Co-authored-by: Shrinand Thakkar --- .../kafka/KafkaBasedConnectorConfig.java | 19 ++- .../KafkaMirrorMakerConnectorTask.java | 3 +- .../server/FlushlessEventProducerHandler.java | 102 +++------------ .../server/callbackstatus/CallbackStatus.java | 41 ++++++ .../callbackstatus/CallbackStatusFactory.java | 18 +++ .../CallbackStatusWithComparableOffsets.java | 108 ++++++++++++++++ ...ackStatusWithComparableOffsetsFactory.java | 23 ++++ ...allbackStatusWithNonComparableOffsets.java | 99 ++++++++++++++ ...StatusWithNonComparableOffsetsFactory.java | 23 ++++ .../TestFlushlessEventProducerHandler.java | 122 +++++++++++++++--- 10 files changed, 457 insertions(+), 101 deletions(-) create mode 100644 datastream-server-api/src/main/java/com/linkedin/datastream/server/callbackstatus/CallbackStatus.java create mode 100644 datastream-server-api/src/main/java/com/linkedin/datastream/server/callbackstatus/CallbackStatusFactory.java create mode 100644 datastream-server-api/src/main/java/com/linkedin/datastream/server/callbackstatus/CallbackStatusWithComparableOffsets.java create mode 100644 datastream-server-api/src/main/java/com/linkedin/datastream/server/callbackstatus/CallbackStatusWithComparableOffsetsFactory.java create mode 100644 datastream-server-api/src/main/java/com/linkedin/datastream/server/callbackstatus/CallbackStatusWithNonComparableOffsets.java create mode 100644 datastream-server-api/src/main/java/com/linkedin/datastream/server/callbackstatus/CallbackStatusWithNonComparableOffsetsFactory.java 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/mirrormaker/KafkaMirrorMakerConnectorTask.java b/datastream-kafka-connector/src/main/java/com/linkedin/datastream/connectors/kafka/mirrormaker/KafkaMirrorMakerConnectorTask.java index e14dee799..4d7c92e04 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 @@ -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); 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/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); From 87c7bec012a776cbb4a5be7d40903682be2d763e Mon Sep 17 00:00:00 2001 From: Vaibhav Maheshwari Date: Mon, 24 Jan 2022 16:23:38 -0800 Subject: [PATCH 10/32] Fix liveInstance provider call only in leader node (#879) Co-authored-by: Vaibhav Maheshwari --- .../linkedin/datastream/server/zk/ZkAdapter.java | 14 +++++--------- .../datastream/server/zk/TestZkAdapter.java | 2 +- 2 files changed, 6 insertions(+), 10 deletions(-) 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..3a28dc4a3 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 @@ -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; 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()); } From 2cd2b8bca59be9292fd7241758f130ee94bf8a06 Mon Sep 17 00:00:00 2001 From: Jhora Zakaryan Date: Tue, 1 Feb 2022 13:53:56 -0800 Subject: [PATCH 11/32] Fixed issue with missing exception message during task initialization (#882) * Fixed issue with missing exception message during task initialization * Changed log level to error --- .../main/java/com/linkedin/datastream/server/Coordinator.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) 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..3d98b2367 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); From 4d867ef8d61376fff6e3899cf6b4c9391f96c3ed Mon Sep 17 00:00:00 2001 From: Sridevi Nagaraj Date: Wed, 2 Feb 2022 12:17:12 -0800 Subject: [PATCH 12/32] Kafka upgrade (#881) * Kafka upgrade * Kafka upgrade * Removing temp version * Addressing comments: removing flag update after close to prevent threads misbehavior * Removing dependency on li-apache-kafka-clients --- build.gradle | 6 +- .../AbstractKafkaBasedConnectorTask.java | 5 +- .../KafkaMirrorMakerConnectorTestUtils.java | 16 --- .../TestKafkaMirrorMakerConnectorTask.java | 6 +- .../kafka/LiKafkaConsumerFactory.java | 4 +- .../connectors/kafka/NoOpAuditor.java | 52 -------- .../kafka/NoOpSegmentDeserializer.java | 32 ----- .../kafka/EmbeddedKafkaCluster.java | 5 +- .../kafka/factory/LiKafkaProducerFactory.java | 122 ------------------ gradle/dependency-versions.gradle | 3 +- 10 files changed, 14 insertions(+), 237 deletions(-) delete mode 100644 datastream-kafka-factory-impl/src/main/java/com/linkedin/datastream/connectors/kafka/NoOpAuditor.java delete mode 100644 datastream-kafka-factory-impl/src/main/java/com/linkedin/datastream/connectors/kafka/NoOpSegmentDeserializer.java delete mode 100644 datastream-kafka/src/main/java/com/linkedin/datastream/kafka/factory/LiKafkaProducerFactory.java diff --git a/build.gradle b/build.gradle index 5d89998e7..50b4e51d2 100644 --- a/build.gradle +++ b/build.gradle @@ -141,7 +141,7 @@ 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" testCompile "org.mockito:mockito-core:$mockitoVersion" } } @@ -150,7 +150,6 @@ project(':datastream-server-api') { dependencies { compile project(':datastream-common') compile project(':datastream-utils') - compile "com.linkedin.kafka.clients:li-apache-kafka-clients:$LIKafkaVersion" } } @@ -189,7 +188,6 @@ project(':datastream-directory') { project(":datastream-kafka_$scalaSuffix") { 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') @@ -214,7 +212,6 @@ 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") } @@ -225,7 +222,6 @@ project(':datastream-kafka-connector') { 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 "org.apache.httpcomponents:httpclient:$apacheHttpClientVersion" compile "commons-validator:commons-validator:$commonsValidatorVersion" compile "org.apache.commons:commons-lang3:$commonslang3Version" 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..f1d5ab4aa 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 @@ -90,6 +90,7 @@ abstract public class AbstractKafkaBasedConnectorTask implements Runnable, Consu // lifecycle private volatile Thread _connectorTaskThread; protected volatile boolean _shutdown = false; + private volatile boolean _failure = false; protected volatile long _lastPolledTimeMillis = System.currentTimeMillis(); protected volatile long _lastPollCompletedTimeMillis = 0; protected final CountDownLatch _startedLatch = new CountDownLatch(1); @@ -303,6 +304,8 @@ protected void rewindAndPausePartitionOnException(TopicPartition srcTopicPartiti } catch (Exception e) { // Seek to last checkpoint failed. Throw an exception to avoid any data loss scenarios where the consumed // offset can be committed even though the send for that offset has failed. + // This flag is used to address 2.4 kafka version behavior changes for onPartitionRevoked calls + _failure = true; String errorMessage = String.format("Partition rewind for %s failed due to ", srcTopicPartition); throw new DatastreamRuntimeException(errorMessage, e); } @@ -766,7 +769,7 @@ protected void updateConsumerAssignment(Collection partitions) { public void onPartitionsRevoked(Collection topicPartitions) { _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. + if (!_shutdown && !topicPartitions.isEmpty() && !_failure) { // there is a commit at the end of the run method, skip extra commit in shouldDie mode. try { maybeCommitOffsets(_consumer, true); // happens inline as part of poll } catch (Exception e) { 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/TestKafkaMirrorMakerConnectorTask.java b/datastream-kafka-connector/src/test/java/com/linkedin/datastream/connectors/kafka/mirrormaker/TestKafkaMirrorMakerConnectorTask.java index 5e851f052..29011da9d 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 @@ -801,7 +801,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 +884,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 +960,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-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..14672d47f 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,6 +13,7 @@ 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; @@ -122,7 +123,7 @@ public void startup() { } private KafkaServer startBroker(Properties props) { - KafkaServer server = new KafkaServer(KafkaConfig.fromProps(props), new SystemTime(), + KafkaServer server = new KafkaServer(KafkaConfig.fromProps(props), new EmbeddedSystemTime(), scala.Option.apply(""), scala.collection.JavaConversions.asScalaBuffer(Collections.emptyList())); server.startup(); return server; @@ -179,7 +180,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/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/gradle/dependency-versions.gradle b/gradle/dependency-versions.gradle index c6042e0df..1726366ae 100644 --- a/gradle/dependency-versions.gradle +++ b/gradle/dependency-versions.gradle @@ -1,5 +1,4 @@ ext { - LIKafkaVersion = "1.0.65" apacheHttpClientVersion = "4.5.3" avroVersion = "1.7.7" commonsCliVersion = "1.2" @@ -10,7 +9,7 @@ ext { guavaVersion = "25.0-jre" intellijAnnotationsVersion = "12.0" jacksonVersion = "1.8.5" - kafkaVersion = "2.0.0.25" + kafkaVersion = "2.4.1.42" log4jVersion = "1.2.17" metricsCoreVersion = "4.1.0" mockitoVersion = "1.10.19" From 37544cf8f386a57d3b2a5a264bf907aefe1719b1 Mon Sep 17 00:00:00 2001 From: Sridevi Nagaraj Date: Wed, 9 Mar 2022 09:38:05 -0800 Subject: [PATCH 13/32] skipping onPartitionsRevoked during consumer.close() call (#886) * Skip onpartitionsrevoked during consumer.close() * Bumping up version * Fixing warning messages * Fixing warning messages --- .../kafka/AbstractKafkaBasedConnectorTask.java | 18 ++++++++++++++---- .../KafkaMirrorMakerConnectorTask.java | 4 ++++ gradle/maven.gradle | 2 +- 3 files changed, 19 insertions(+), 5 deletions(-) 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 f1d5ab4aa..937090be9 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 @@ -90,7 +90,7 @@ abstract public class AbstractKafkaBasedConnectorTask implements Runnable, Consu // lifecycle private volatile Thread _connectorTaskThread; protected volatile boolean _shutdown = false; - private volatile boolean _failure = false; + private volatile boolean _skipOnPartitionsRevoked = false; protected volatile long _lastPolledTimeMillis = System.currentTimeMillis(); protected volatile long _lastPollCompletedTimeMillis = 0; protected final CountDownLatch _startedLatch = new CountDownLatch(1); @@ -256,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 @@ -304,8 +311,6 @@ protected void rewindAndPausePartitionOnException(TopicPartition srcTopicPartiti } catch (Exception e) { // Seek to last checkpoint failed. Throw an exception to avoid any data loss scenarios where the consumed // offset can be committed even though the send for that offset has failed. - // This flag is used to address 2.4 kafka version behavior changes for onPartitionRevoked calls - _failure = true; String errorMessage = String.format("Partition rewind for %s failed due to ", srcTopicPartition); throw new DatastreamRuntimeException(errorMessage, e); } @@ -435,6 +440,7 @@ 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); @@ -767,9 +773,13 @@ 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() && !_failure) { // there is a commit at the end of the run method, skip extra commit in shouldDie mode. + if (!_shutdown && !topicPartitions.isEmpty()) { // there is a commit at the end of the run method, skip extra commit in shouldDie mode. try { maybeCommitOffsets(_consumer, true); // happens inline as part of poll } catch (Exception e) { 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 4d7c92e04..c26003a29 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 @@ -370,6 +370,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/gradle/maven.gradle b/gradle/maven.gradle index fc6aca855..5ffd6c85b 100644 --- a/gradle/maven.gradle +++ b/gradle/maven.gradle @@ -1,5 +1,5 @@ allprojects { - version = "2.0.0" + version = "3.0.0" } subprojects { From d64e248b5389a4c5744a2473f32a70dde09608c2 Mon Sep 17 00:00:00 2001 From: Sridevi Nagaraj Date: Wed, 9 Mar 2022 09:42:17 -0800 Subject: [PATCH 14/32] Scale 2.12 upgrade (#895) --- gradle.properties | 2 +- gradle/dependency-versions.gradle | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) 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 1726366ae..8f77bcc72 100644 --- a/gradle/dependency-versions.gradle +++ b/gradle/dependency-versions.gradle @@ -15,7 +15,7 @@ ext { 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" From dbaffa6b384247cab713af64f3e09947c4e67269 Mon Sep 17 00:00:00 2001 From: Vaibhav Maheshwari Date: Wed, 9 Mar 2022 10:42:23 -0800 Subject: [PATCH 15/32] Upgrade avro and move jackson from codehaus to fasterxml (#894) Co-authored-by: Vaibhav Maheshwari --- build.gradle | 13 +++++--- .../datastream/avrogenerator/AvroJson.java | 7 +++-- .../datastream/common/DatastreamEvent.java | 8 ++--- .../linkedin/datastream/common/JsonUtils.java | 31 ++++++++++--------- .../connectors/file/diag/FilePositionKey.java | 7 +++-- .../AbstractKafkaBasedConnectorTask.java | 2 +- .../connectors/kafka/KafkaConnector.java | 3 +- .../kafka/KafkaConnectorDiagUtils.java | 3 +- .../kafka/KafkaConsumerOffsetsResponse.java | 2 +- .../kafka/KafkaDatastreamStatesResponse.java | 28 +++++++++-------- .../KafkaTopicPartitionStatsResponse.java | 4 +-- .../kafka/PausedSourcePartitionMetadata.java | 3 +- .../mirrormaker/TestKafkaConsumerOffsets.java | 3 +- .../TestKafkaMirrorMakerConnector.java | 2 +- .../TestKafkaTopicPartitionStats.java | 2 +- .../datastream/server/DatastreamTaskImpl.java | 3 +- .../FileBasedPartitionThroughputProvider.java | 11 ++++--- .../ZookeeperCheckpointProvider.java | 3 +- .../tools/DatastreamRestClientCli.java | 7 +++-- .../datastream/common/DatastreamUtils.java | 3 +- gradle/dependency-versions.gradle | 4 +-- 21 files changed, 84 insertions(+), 65 deletions(-) diff --git a/build.gradle b/build.gradle index 50b4e51d2..a7a161825 100644 --- a/build.gradle +++ b/build.gradle @@ -142,6 +142,9 @@ project(':datastream-common') { compile "com.intellij:annotations:$intellijAnnotationsVersion" compile "com.google.guava:guava:$guavaVersion" 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" } } @@ -316,8 +319,9 @@ 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') @@ -352,8 +356,9 @@ project(':datastream-server-restli') { 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') 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-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 937090be9..51fbf75f6 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; 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/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/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-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/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-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/gradle/dependency-versions.gradle b/gradle/dependency-versions.gradle index 8f77bcc72..ac4589e01 100644 --- a/gradle/dependency-versions.gradle +++ b/gradle/dependency-versions.gradle @@ -1,6 +1,6 @@ ext { apacheHttpClientVersion = "4.5.3" - avroVersion = "1.7.7" + avroVersion = "1.9.2" commonsCliVersion = "1.2" commonsHttpClientVersion = "3.1" commonsIOVersion = "2.4" @@ -8,7 +8,7 @@ ext { commonsValidatorVersion = "1.5.1" guavaVersion = "25.0-jre" intellijAnnotationsVersion = "12.0" - jacksonVersion = "1.8.5" + jacksonVersion = "2.10.0" kafkaVersion = "2.4.1.42" log4jVersion = "1.2.17" metricsCoreVersion = "4.1.0" From 26f32cad378b8d14dffb99ba7cdbd3c6766005f8 Mon Sep 17 00:00:00 2001 From: Suraj Nagaraja Kasi Date: Mon, 14 Mar 2022 09:50:23 -0700 Subject: [PATCH 16/32] Fix topic deletion when multiple duplicate streams expire at the same time (#897) Previously when all duplicate streams expire at the same time (i.e. all duplicate streams might have expired but were waiting for a DS add/delete event) we incorrectly identified that a duplicate stream is still active since its still present in zk. --- .../datastream/server/TestCoordinator.java | 62 +++++++++++++++++++ .../datastream/server/Coordinator.java | 9 ++- 2 files changed, 68 insertions(+), 3 deletions(-) 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/Coordinator.java b/datastream-server/src/main/java/com/linkedin/datastream/server/Coordinator.java index 3d98b2367..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 @@ -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(); From 4310926847a74a96d14d998f43a1bd0ff9271797 Mon Sep 17 00:00:00 2001 From: Shrinand Thakkar Date: Thu, 31 Mar 2022 14:56:43 -0700 Subject: [PATCH 17/32] Use 2.4.1.57 version open source kafka (#901) Co-authored-by: Shrinand Thakkar --- gradle/dependency-versions.gradle | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/gradle/dependency-versions.gradle b/gradle/dependency-versions.gradle index ac4589e01..9d36f5cd4 100644 --- a/gradle/dependency-versions.gradle +++ b/gradle/dependency-versions.gradle @@ -9,7 +9,7 @@ ext { guavaVersion = "25.0-jre" intellijAnnotationsVersion = "12.0" jacksonVersion = "2.10.0" - kafkaVersion = "2.4.1.42" + kafkaVersion = "2.4.1.57" log4jVersion = "1.2.17" metricsCoreVersion = "4.1.0" mockitoVersion = "1.10.19" From a67108bcff9e3c8f5c22dbc44e04f91f200cc1cc Mon Sep 17 00:00:00 2001 From: Ryanne Dolan Date: Tue, 12 Apr 2022 12:49:01 -0500 Subject: [PATCH 18/32] tests for min/max partitions per task metrics and minor code quality improvements (#887) Fixed bug where min/max partitions per task was being reported incorrectly. Previously, the min/max were the same across all datastreams. New setGauge method in DynamicMetricsManager enables explicitly setting a gauge value. This reduces some lifecycle management complexity in LoadBalancedPartitionAssigner. Incidental code quality improvements. Tests added to verify affected metrics (min/maxPartitionsPerTask). Fixed bug related to registration of Gauges and Timers (they weren't actually registered before). --- .../metrics/DynamicMetricsManager.java | 68 ++++----- .../LoadBasedPartitionAssigner.java | 135 ++++++++---------- .../TestLoadBasedPartitionAssigner.java | 52 +++++++ 3 files changed, 135 insertions(+), 120 deletions(-) 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-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 793b9e7b2..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 @@ -179,40 +180,41 @@ public Map> assignPartitions( 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 = newPartitionAssignmentMap.containsKey(task.getId()) ? newPartitionAssignmentMap.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, newPartitionAssignmentMap.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) { @@ -257,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} */ @@ -291,13 +274,17 @@ 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; + }); } /** @@ -373,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/test/java/com/linkedin/datastream/server/assignment/TestLoadBasedPartitionAssigner.java b/datastream-server/src/test/java/com/linkedin/datastream/server/assignment/TestLoadBasedPartitionAssigner.java index 8005e9b74..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,6 +237,11 @@ 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 @@ -255,8 +281,11 @@ public void lightestTaskGetsNewPartitionWithTopicMetricsTest() { // 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"); @@ -313,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 @@ -342,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) { @@ -367,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)); + } } From b821911e485799be83c0c4a39e5c8d2909307136 Mon Sep 17 00:00:00 2001 From: Shrinand Thakkar Date: Tue, 12 Apr 2022 14:02:21 -0700 Subject: [PATCH 19/32] fix rebalancing-tasks bug and added tests (#900) * fix rebalancing-tasks bug and added tests * add some more test cases with even number of instances and with single instance Co-authored-by: Shrinand Thakkar --- .../assignment/StickyMulticastStrategy.java | 3 +- .../TestStickyMulticastStrategy.java | 76 +++++++++++++++++++ 2 files changed, 78 insertions(+), 1 deletion(-) 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..eda56aa83 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 @@ -205,7 +205,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/test/java/com/linkedin/datastream/server/assignment/TestStickyMulticastStrategy.java b/datastream-server/src/test/java/com/linkedin/datastream/server/assignment/TestStickyMulticastStrategy.java index 9aa8f8bfc..4a5e9fea4 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; @@ -499,6 +501,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 -> { From 4f806accc87d7feb378592a9d6219f5289eabc98 Mon Sep 17 00:00:00 2001 From: Shrinand Thakkar Date: Tue, 12 Apr 2022 14:07:40 -0700 Subject: [PATCH 20/32] Refactor Stopping Tasks On Assignment Change of Tasks (#868) Co-authored-by: Shrinand Thakkar --- .../kafka/AbstractKafkaConnector.java | 106 ++++++++++-------- .../kafka/TestAbstractKafkaConnector.java | 69 ++++++++++++ 2 files changed, 128 insertions(+), 47 deletions(-) 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..4ffcb611d 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 @@ -293,8 +299,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 +314,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 @@ -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/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..6a08ccde1 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; @@ -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); + + // 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 */ From f6c7e7673ee92f26444f74837936d36a80b6318a Mon Sep 17 00:00:00 2001 From: Suraj Nagaraja Kasi Date: Mon, 18 Apr 2022 13:07:41 -0700 Subject: [PATCH 21/32] Change python dependency in commit-msg git hook (#904) --- scripts/git/commit-msg | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) 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): From a34bbdd08eb96bdd480bda39537c96d16e4af7b1 Mon Sep 17 00:00:00 2001 From: Shrinand Thakkar Date: Thu, 21 Apr 2022 12:15:18 -0700 Subject: [PATCH 22/32] Remove Scala Dependencies (#905) Remove minimally used Scala Dependencies from Brooklin code to avoid cross-scala versions builds. --- build.gradle | 44 +++++++++---------- .../kafka/EmbeddedKafkaCluster.java | 13 +++--- settings.gradle | 17 +------ 3 files changed, 30 insertions(+), 44 deletions(-) diff --git a/build.gradle b/build.gradle index a7a161825..efaf61f3d 100644 --- a/build.gradle +++ b/build.gradle @@ -161,8 +161,8 @@ project(':datastream-utils') { compile project(':datastream-common') compile "com.101tec:zkclient:$zkclientVersion" compile "com.google.guava:guava:$guavaVersion" - testCompile project(":datastream-kafka_$scalaSuffix") - testCompile project(":datastream-testcommon_$scalaSuffix") + testCompile project(":datastream-kafka") + testCompile project(":datastream-testcommon") } } @@ -172,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" } } @@ -188,7 +188,7 @@ project(':datastream-directory') { } } -project(":datastream-kafka_$scalaSuffix") { +project(":datastream-kafka") { dependencies { compile "com.linkedin.kafka:kafka_$scalaSuffix:$kafkaVersion" compile "commons-httpclient:commons-httpclient:$commonsHttpClientVersion" @@ -197,7 +197,7 @@ project(":datastream-kafka_$scalaSuffix") { 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) { @@ -216,7 +216,7 @@ project(':datastream-kafka-factory-impl') { dependencies { compile project(':datastream-kafka-connector') - testCompile project(":datastream-kafka_$scalaSuffix") + testCompile project(":datastream-kafka") } } @@ -224,23 +224,23 @@ project(':datastream-kafka-connector') { dependencies { compile project(':datastream-server-api') compile project(':datastream-common') - compile project(":datastream-kafka_$scalaSuffix") + 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') @@ -261,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" } @@ -284,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' @@ -308,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" } @@ -328,10 +328,10 @@ project(':datastream-server') { 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" } @@ -364,11 +364,11 @@ project(':datastream-server-restli') { 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-kafka/src/main/java/com/linkedin/datastream/kafka/EmbeddedKafkaCluster.java b/datastream-kafka/src/main/java/com/linkedin/datastream/kafka/EmbeddedKafkaCluster.java index 14672d47f..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 @@ -17,7 +17,7 @@ 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; @@ -33,7 +33,7 @@ public class EmbeddedKafkaCluster { private final String _brokers; - private final List _brokerList; + private final List _brokerList; private final List _logDirs; /** @@ -115,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 EmbeddedSystemTime(), - 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; } @@ -156,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) { 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 From 55027bc81cdbc95d711c4b9f24e04f218feb9f67 Mon Sep 17 00:00:00 2001 From: Suraj Nagaraja Kasi Date: Mon, 23 May 2022 13:21:42 -0700 Subject: [PATCH 23/32] Introduce broadcast API to TransportProvider (#903) Introduce braodcast API to TrasnportProvider This API will allow to sending events to all consumers/clients of TrasnsportProvider. TransportProvider will be able to optionally implement broadcast API. Example, for Kafka TrasnportProvider broadcast will send the event to all topic partitions. --- .../datastream/kafka/KafkaTestUtils.java | 45 +++++++++++ .../kafka/KafkaTransportProvider.java | 41 ++++++++++ .../kafka/TestKafkaTransportProvider.java | 80 ++++++++++++++----- .../server/DatastreamEventProducer.java | 12 +++ .../server/DatastreamProducerRecord.java | 16 +++- .../DatastreamProducerRecordBuilder.java | 30 ++++++- .../transport/DatastreamRecordMetadata.java | 72 +++++++++++++++++ .../api/transport/TransportProvider.java | 27 +++++++ .../datastream/server/EventProducer.java | 73 ++++++++++++----- 9 files changed, 353 insertions(+), 43 deletions(-) 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/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/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/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 From 981a32fb86585973cc95a740790324bee2ad8c12 Mon Sep 17 00:00:00 2001 From: Joseph Grogan Date: Mon, 6 Jun 2022 18:54:21 -0400 Subject: [PATCH 24/32] Dedupe tasks on LeaderDoAssignment (#906) * Update assignment to prevent duplicate task names * Address feedback and add test for same name but different fields * Simplify code since there cannot be duplicate datastream groups --- .../server/assignment/BroadcastStrategy.java | 12 +++++-- .../assignment/LoadbalancingStrategy.java | 3 +- .../assignment/StickyMulticastStrategy.java | 3 +- .../assignment/TestBroadcastStrategy.java | 26 +++++++++++++++ .../assignment/TestLoadbalancingStrategy.java | 32 +++++++++++++++++++ .../TestStickyMulticastStrategy.java | 8 +++++ 6 files changed, 80 insertions(+), 4 deletions(-) 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/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 eda56aa83..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); 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/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 4a5e9fea4..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 @@ -296,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()); From 5b30f5412d1a9aac2454b8b6e0b3f8ff50738326 Mon Sep 17 00:00:00 2001 From: Shrinand Thakkar Date: Wed, 8 Jun 2022 15:34:36 -0700 Subject: [PATCH 25/32] Fix Stopping Logic and Maintain Stopping Latch Counter (#877) * Fix Stopping Logic and Maintain Stopping Latch Counter * --amend Co-authored-by: Shrinand Thakkar --- .../AbstractKafkaBasedConnectorTask.java | 13 +++++++-- .../kafka/AbstractKafkaConnector.java | 28 +++++++++--------- .../KafkaMirrorMakerConnectorTask.java | 5 ++++ .../kafka/TestAbstractKafkaConnector.java | 23 ++++++++++++--- .../TestKafkaMirrorMakerConnectorTask.java | 29 +++++++++++++++++++ 5 files changed, 78 insertions(+), 20 deletions(-) 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 51fbf75f6..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 @@ -94,7 +94,7 @@ abstract public class AbstractKafkaBasedConnectorTask implements Runnable, Consu 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 @@ -447,11 +447,15 @@ public void run() { } } postShutdownHook(); - _stoppedLatch.countDown(); + countDownStoppedLatch(); _logger.info("{} stopped", _taskName); } } + protected void countDownStoppedLatch() { + _stoppedLatch.countDown(); + } + /** * Signal task to stop */ @@ -1112,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 4ffcb611d..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 @@ -265,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()); @@ -374,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(); 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 c26003a29..83069e667 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 @@ -353,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; } } 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 6a08ccde1..6bce68808 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 @@ -88,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)); @@ -118,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)); @@ -268,19 +268,34 @@ public class TestKafkaConnector extends AbstractKafkaConnector { private boolean _failStopTaskOnce; private int _createTaskCalled = 0; private int _stopTaskCalled = 0; + private boolean _taskThreadDead = true; /** * 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 @@ -310,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/TestKafkaMirrorMakerConnectorTask.java b/datastream-kafka-connector/src/test/java/com/linkedin/datastream/connectors/kafka/mirrormaker/TestKafkaMirrorMakerConnectorTask.java index 29011da9d..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"); From 51533bb8b3fe4caf608b618f5b240129c693ddcc Mon Sep 17 00:00:00 2001 From: Shrinand Thakkar Date: Thu, 9 Jun 2022 09:34:14 -0700 Subject: [PATCH 26/32] Fixing assignment change test to include task thread dead parameter after merging #877 (#908) Co-authored-by: Shrinand Thakkar --- .../connectors/kafka/TestAbstractKafkaConnector.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) 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 6bce68808..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 @@ -149,7 +149,7 @@ public void testOnAssignmentChangeMultipleReassignments() throws InterruptedExce 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); // first task assignment assigns task 1 List firstTaskAssignment = getTaskListInRange(1, 2); @@ -268,7 +268,7 @@ public class TestKafkaConnector extends AbstractKafkaConnector { private boolean _failStopTaskOnce; private int _createTaskCalled = 0; private int _stopTaskCalled = 0; - private boolean _taskThreadDead = true; + private boolean _taskThreadDead; /** * Constructor for TestKafkaConnector From 56572259e393fdae6fabe396aa90a4f92530dc3c Mon Sep 17 00:00:00 2001 From: Suraj Kasi Date: Tue, 9 Nov 2021 09:47:09 -0800 Subject: [PATCH 27/32] Replace 101tec ZkClient with Helix ZkClient Use helix version 1.0.2. Extend common/zk/ZkClient.java from helix zookeeper client and refactoring some code. --- build.gradle | 4 +- .../KafkaMirrorMakerConnectorTask.java | 2 +- .../server/CachedDatastreamReader.java | 2 +- .../datastream/server/zk/ZkAdapter.java | 12 +- .../datastream/common/zk/ZkClient.java | 162 ++---------------- .../datastream/common/zk/TestZkClient.java | 4 +- gradle/dependency-versions.gradle | 2 +- 7 files changed, 29 insertions(+), 159 deletions(-) diff --git a/build.gradle b/build.gradle index efaf61f3d..ffa273d64 100644 --- a/build.gradle +++ b/build.gradle @@ -159,7 +159,7 @@ project(':datastream-server-api') { 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") testCompile project(":datastream-testcommon") @@ -318,7 +318,6 @@ project(':datastream-client') { project(':datastream-server') { dependencies { - compile "com.101tec:zkclient:$zkclientVersion" compile "com.fasterxml.jackson.core:jackson-annotations:$jacksonVersion" compile "com.fasterxml.jackson.core:jackson-core:$jacksonVersion" compile "com.fasterxml.jackson.core:jackson-databind:$jacksonVersion" @@ -355,7 +354,6 @@ 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 "com.fasterxml.jackson.core:jackson-annotations:$jacksonVersion" compile "com.fasterxml.jackson.core:jackson-core:$jacksonVersion" compile "com.fasterxml.jackson.core:jackson-databind:$jacksonVersion" 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 83069e667..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; 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/zk/ZkAdapter.java b/datastream-server/src/main/java/com/linkedin/datastream/server/zk/ZkAdapter.java index 3a28dc4a3..9d2831c00 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; @@ -1831,7 +1831,7 @@ 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."); if (_reinitOnNewSession) { 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..cb93fa1c9 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,19 +6,13 @@ 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; @@ -26,7 +20,7 @@ /** - * 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 +28,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 +80,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 +155,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,9 +216,12 @@ public T deserialize(byte[] data) { return (T) _zkSerializer.deserialize(data); } + /** + * Get Zk sessions session ID + */ @VisibleForTesting public long getSessionId() { - return ((ZkConnection) _connection).getZookeeper().getSessionId(); + return super.getSessionId(); } private static class ZKStringSerializer implements ZkSerializer { 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..de65aeed6 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,8 @@ 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.zookeeper.CreateMode; import org.slf4j.Logger; import org.slf4j.LoggerFactory; diff --git a/gradle/dependency-versions.gradle b/gradle/dependency-versions.gradle index 9d36f5cd4..bc86e8992 100644 --- a/gradle/dependency-versions.gradle +++ b/gradle/dependency-versions.gradle @@ -20,5 +20,5 @@ ext { testngVersion = "7.1.0" zkclientVersion = "0.11" zookeeperVersion = "3.4.13" - helixZkclientVersion = "1.0.1" + helixZkclientVersion = "1.0.2" } From cdc0c6c0f2fc14aa43a970e00728d41d07f10423 Mon Sep 17 00:00:00 2001 From: Suraj Kasi Date: Thu, 2 Dec 2021 14:22:08 -0800 Subject: [PATCH 28/32] Removing unused older zk client version number variable and redundant methods --- .../com/linkedin/datastream/common/zk/ZkClient.java | 10 ---------- gradle/dependency-versions.gradle | 1 - 2 files changed, 11 deletions(-) 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 cb93fa1c9..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 @@ -16,8 +16,6 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import com.google.common.annotations.VisibleForTesting; - /** * ZKClient is a wrapper of {@link org.apache.helix.zookeeper.impl.client.ZkClient}. It provides the following @@ -216,14 +214,6 @@ public T deserialize(byte[] data) { return (T) _zkSerializer.deserialize(data); } - /** - * Get Zk sessions session ID - */ - @VisibleForTesting - public long getSessionId() { - return super.getSessionId(); - } - private static class ZKStringSerializer implements ZkSerializer { @Override public byte[] serialize(Object data) throws ZkMarshallingError { diff --git a/gradle/dependency-versions.gradle b/gradle/dependency-versions.gradle index bc86e8992..fffe8b59b 100644 --- a/gradle/dependency-versions.gradle +++ b/gradle/dependency-versions.gradle @@ -18,7 +18,6 @@ ext { scalaVersion = "2.12" slf4jVersion = "1.7.5" testngVersion = "7.1.0" - zkclientVersion = "0.11" zookeeperVersion = "3.4.13" helixZkclientVersion = "1.0.2" } From f4d6dcf7481c6e40905b8746fe4edc5dcdc74491 Mon Sep 17 00:00:00 2001 From: Suraj Kasi Date: Fri, 3 Dec 2021 18:39:17 -0800 Subject: [PATCH 29/32] Add exception unit tests for ZkClient create method --- .../datastream/common/zk/TestZkClient.java | 35 +++++++++++++++++++ 1 file changed, 35 insertions(+) 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 de65aeed6..5537f0e48 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 @@ -10,6 +10,7 @@ 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,40 @@ 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 + try { + zkClient.create(electionNodeName, "test", CreateMode.PERSISTENT_SEQUENTIAL); + } catch (ZkNoNodeException e) { + zkClient.close(); + return; + } + throw new Exception("Test failed, expected ZkNoNodeException"); + } + + @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 + try { + zkClient.create(null, "test", CreateMode.PERSISTENT_SEQUENTIAL); + } catch (NullPointerException e) { + zkClient.close(); + return; + } + throw new Exception("Test failed, expected NullPointerException"); + } + @Test public void testCreateEphemeralSequentialNode() throws Exception { ZkClient zkClient = new ZkClient(_zkConnectionString); From 608ec07d60065da3860c89eb0dd55dca1906c5be Mon Sep 17 00:00:00 2001 From: Suraj Kasi Date: Mon, 6 Dec 2021 14:31:15 -0800 Subject: [PATCH 30/32] Update log message and refactor unit tests --- .../datastream/server/zk/ZkAdapter.java | 2 +- .../datastream/common/zk/TestZkClient.java | 18 ++++-------------- 2 files changed, 5 insertions(+), 15 deletions(-) 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 9d2831c00..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 @@ -1833,7 +1833,7 @@ public void handleStateChanged(Watcher.Event.KeeperState state) { @Override 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-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 5537f0e48..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 @@ -72,13 +72,8 @@ public void testCreateNoNodeException() throws Exception { String electionNodeName = electionPath + "/coordinator-"; // now create this node with persistent mode - try { - zkClient.create(electionNodeName, "test", CreateMode.PERSISTENT_SEQUENTIAL); - } catch (ZkNoNodeException e) { - zkClient.close(); - return; - } - throw new Exception("Test failed, expected ZkNoNodeException"); + Assert.assertThrows(ZkNoNodeException.class, + () -> zkClient.create(electionNodeName, "test", CreateMode.PERSISTENT_SEQUENTIAL)); } @Test @@ -89,13 +84,8 @@ public void testCreateIllegalArgumentException() throws Exception { String electionNodeName = electionPath + "/coordinator-"; // now create this node with persistent mode - try { - zkClient.create(null, "test", CreateMode.PERSISTENT_SEQUENTIAL); - } catch (NullPointerException e) { - zkClient.close(); - return; - } - throw new Exception("Test failed, expected NullPointerException"); + Assert.assertThrows(NullPointerException.class, + () -> zkClient.create(null, "test", CreateMode.PERSISTENT_SEQUENTIAL)); } @Test From 50cadd4014055ef90c7b338c3e039c01e56d2347 Mon Sep 17 00:00:00 2001 From: Suraj Kasi Date: Mon, 13 Dec 2021 16:42:09 -0800 Subject: [PATCH 31/32] Updating Zookeeper version to 3.6.2.3 which has pagination support (3.6.2.1+) --- gradle/dependency-versions.gradle | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/gradle/dependency-versions.gradle b/gradle/dependency-versions.gradle index fffe8b59b..877bfc633 100644 --- a/gradle/dependency-versions.gradle +++ b/gradle/dependency-versions.gradle @@ -18,6 +18,6 @@ ext { scalaVersion = "2.12" slf4jVersion = "1.7.5" testngVersion = "7.1.0" - zookeeperVersion = "3.4.13" + zookeeperVersion = "3.6.2.3" helixZkclientVersion = "1.0.2" } From f06e55b017681ab3f625319dcee2b4243dd249e2 Mon Sep 17 00:00:00 2001 From: Suraj Kasi Date: Fri, 24 Jun 2022 13:53:54 -0700 Subject: [PATCH 32/32] Revert zookeeper version and update major version of brooklin Zookeeper version upgrade is not required. --- gradle/dependency-versions.gradle | 2 +- gradle/maven.gradle | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/gradle/dependency-versions.gradle b/gradle/dependency-versions.gradle index 877bfc633..fffe8b59b 100644 --- a/gradle/dependency-versions.gradle +++ b/gradle/dependency-versions.gradle @@ -18,6 +18,6 @@ ext { scalaVersion = "2.12" slf4jVersion = "1.7.5" testngVersion = "7.1.0" - zookeeperVersion = "3.6.2.3" + zookeeperVersion = "3.4.13" helixZkclientVersion = "1.0.2" } diff --git a/gradle/maven.gradle b/gradle/maven.gradle index 5ffd6c85b..8febd361d 100644 --- a/gradle/maven.gradle +++ b/gradle/maven.gradle @@ -1,5 +1,5 @@ allprojects { - version = "3.0.0" + version = "4.0.0" } subprojects {