From 39ef2af1fb8f5d5766c596dc6425d644f59bb20b Mon Sep 17 00:00:00 2001 From: Koki Kato Date: Tue, 7 Jun 2022 16:17:56 +0900 Subject: [PATCH 01/27] Add ArbitraryTopicTypeTest --- .../processor/ArbitraryTopicTypeTest.java | 166 ++++++++++++++++++ .../linecorp/decaton/testing/TestUtils.java | 2 +- 2 files changed, 167 insertions(+), 1 deletion(-) create mode 100644 processor/src/it/java/com/linecorp/decaton/processor/ArbitraryTopicTypeTest.java diff --git a/processor/src/it/java/com/linecorp/decaton/processor/ArbitraryTopicTypeTest.java b/processor/src/it/java/com/linecorp/decaton/processor/ArbitraryTopicTypeTest.java new file mode 100644 index 00000000..acf4dac4 --- /dev/null +++ b/processor/src/it/java/com/linecorp/decaton/processor/ArbitraryTopicTypeTest.java @@ -0,0 +1,166 @@ +/* + * Copyright 2022 LINE Corporation + * + * LINE Corporation licenses this file to you under the Apache License, + * version 2.0 (the "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at: + * + * https://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the + * License for the specific language governing permissions and limitations + * under the License. + */ + +package com.linecorp.decaton.processor; + +import java.time.Duration; +import java.util.Properties; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.CountDownLatch; +import java.util.function.Consumer; + +import org.apache.kafka.clients.producer.KafkaProducer; +import org.apache.kafka.clients.producer.ProducerRecord; +import org.apache.kafka.clients.producer.RecordMetadata; +import org.apache.kafka.common.serialization.Deserializer; +import org.apache.kafka.common.serialization.LongDeserializer; +import org.apache.kafka.common.serialization.LongSerializer; +import org.apache.kafka.common.serialization.Serializer; +import org.apache.kafka.common.serialization.StringDeserializer; +import org.junit.After; +import org.junit.Before; +import org.junit.ClassRule; +import org.junit.Test; + +import com.linecorp.decaton.client.kafka.PrintableAsciiStringSerializer; +import com.linecorp.decaton.processor.runtime.DecatonTask; +import com.linecorp.decaton.processor.runtime.ProcessorSubscription; +import com.linecorp.decaton.processor.runtime.ProcessorsBuilder; +import com.linecorp.decaton.processor.runtime.RetryConfig; +import com.linecorp.decaton.processor.runtime.SubscriptionBuilder; +import com.linecorp.decaton.processor.runtime.TaskExtractor; +import com.linecorp.decaton.testing.KafkaClusterRule; +import com.linecorp.decaton.testing.TestUtils; + +/** + * This test class verifies that {@link ProcessorSubscription} is capable of subscribing topics with + * - any key types other than String (our default) + * - any record types other than {@link DecatonTask} + */ +public class ArbitraryTopicTypeTest { + @ClassRule + public static KafkaClusterRule rule = new KafkaClusterRule(); + + private String topic; + private String retryTopic; + + @Before + public void setUp() { + topic = rule.admin().createRandomTopic(3, 3); + retryTopic = rule.admin().createRandomTopic(3, 3); + } + + @After + public void tearDown() { + rule.admin().deleteTopics(true, topic, retryTopic); + } + + private static final class TestProducer implements AutoCloseable { + private final KafkaProducer producer; + private final String topic; + + private TestProducer(String bootstrapServers, String topic, Serializer keySerializer, Serializer valueSerializer) { + final Properties properties = TestUtils.defaultProducerProps(bootstrapServers); + + this.topic = topic; + producer = new KafkaProducer<>(properties, keySerializer, valueSerializer); + } + + public CompletableFuture put(K key, V value) { + final ProducerRecord record = new ProducerRecord<>(topic, key, value); + + final CompletableFuture result = new CompletableFuture<>(); + producer.send(record, (metadata, exception) -> { + if (exception == null) { + result.complete(metadata); + } else { + result.completeExceptionally(exception); + } + }); + + return result; + } + + @Override + public void close() throws Exception { + producer.close(); + } + } + + private static final class TestTaskExtractor implements TaskExtractor { + private final String topic; + private final Deserializer deserializer; + + private TestTaskExtractor(String topic, Deserializer deserializer) { + this.topic = topic; + this.deserializer = deserializer; + } + + @Override + public DecatonTask extract(byte[] bytes) { + final T value = deserializer.deserialize(topic, bytes); + final TaskMetadata metadata = TaskMetadata.builder().build(); + return new DecatonTask<>(metadata, value, bytes); + } + } + + private void testRetryWithKeyValue( + Serializer keySerializer, + K key, + Serializer valueSerializer, + Deserializer valueDeserializer, + V value + ) throws Exception { + final CountDownLatch processLatch = new CountDownLatch(1); + final RetryConfig retryConfig = RetryConfig.builder().retryTopic(retryTopic).backoff(Duration.ofMillis(10)).build(); + final Consumer builderConfigurer = builder -> builder.processorsBuilder( + ProcessorsBuilder.consuming(topic, new TestTaskExtractor<>(topic, valueDeserializer)).thenProcess((context, task) -> { + if (context.metadata().retryCount() == 0) { + context.retry(); + } else { + processLatch.countDown(); + } + })).enableRetry(retryConfig); + + try (ProcessorSubscription subscription = TestUtils.subscription(rule.bootstrapServers(), builderConfigurer); + TestProducer producer = new TestProducer<>(rule.bootstrapServers(), topic, keySerializer, valueSerializer)) { + producer.put(key, value); + processLatch.await(); + } + } + + @Test(timeout = 30000) + public void testPrintableAsciiStringKeyValue() throws Exception { + testRetryWithKeyValue( + new PrintableAsciiStringSerializer(), + "abc", + new PrintableAsciiStringSerializer(), + new StringDeserializer(), + "value" + ); + } + + @Test(timeout = 30000) + public void testLongKeyValue() throws Exception { + testRetryWithKeyValue( + new LongSerializer(), + 123L, + new LongSerializer(), + new LongDeserializer(), + 100L + ); + } +} diff --git a/testing/src/main/java/com/linecorp/decaton/testing/TestUtils.java b/testing/src/main/java/com/linecorp/decaton/testing/TestUtils.java index e838ff60..16c6b659 100644 --- a/testing/src/main/java/com/linecorp/decaton/testing/TestUtils.java +++ b/testing/src/main/java/com/linecorp/decaton/testing/TestUtils.java @@ -61,7 +61,7 @@ private static int sequence() { return sequence.getAndIncrement(); } - private static Properties defaultProducerProps(String bootstrapServers) { + public static Properties defaultProducerProps(String bootstrapServers) { Properties props = new Properties(); props.setProperty(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, bootstrapServers); props.setProperty(ProducerConfig.CLIENT_ID_CONFIG, "test-client-" + sequence()); From b3048aea9309cbc99bddad96f681944928f32183 Mon Sep 17 00:00:00 2001 From: Koki Kato Date: Tue, 7 Jun 2022 12:18:21 +0900 Subject: [PATCH 02/27] Use bytes for record keys instead of String --- .../decaton/client/DecatonClient.java | 10 ++--- .../decaton/client/DecatonClientBuilder.java | 6 +-- .../decaton/client/KafkaProducerSupplier.java | 2 +- .../client/internal/DecatonClientImpl.java | 10 ++--- .../client/internal/DecatonTaskProducer.java | 6 +-- .../client/DecatonClientBuilderTest.java | 8 ++-- .../decaton/client/DecatonClientTest.java | 8 ++-- .../internal/DecatonClientImplTest.java | 41 ++++++++++--------- .../processor/CoreFunctionalityTest.java | 9 ++-- .../decaton/processor/RateLimiterTest.java | 12 +++--- .../decaton/processor/ProcessingContext.java | 2 +- .../decaton/processor/metrics/Metrics.java | 2 +- .../processors/CompactionProcessor.java | 17 ++++---- .../runtime/ProcessorSubscription.java | 8 ++-- .../internal/BlacklistedKeysFilter.java | 14 +++++-- .../runtime/internal/ConsumeManager.java | 8 ++-- .../runtime/internal/ConsumerSupplier.java | 7 ++-- .../internal/ProcessingContextImpl.java | 2 +- .../runtime/internal/SubPartitioner.java | 12 ++++-- .../runtime/internal/TaskRequest.java | 4 +- .../processors/CompactionProcessorTest.java | 5 ++- .../runtime/ProcessorSubscriptionTest.java | 28 +++++++------ .../runtime/internal/ConsumeManagerTest.java | 4 +- ...DecatonTaskRetryQueueingProcessorTest.java | 5 ++- .../runtime/internal/ProcessPipelineTest.java | 4 +- .../internal/ProcessingContextImplTest.java | 7 ++-- .../runtime/internal/SubPartitionerTest.java | 17 ++++---- .../linecorp/decaton/testing/TestUtils.java | 17 ++++---- .../testing/processor/ProcessOrdering.java | 13 +++--- .../testing/processor/ProcessedRecord.java | 2 +- .../testing/processor/ProcessorTestSuite.java | 19 +++++---- .../testing/processor/ProducedRecord.java | 2 +- .../testing/processor/SerialProcessing.java | 7 ++-- .../processor/TestTracingProducer.java | 10 ++--- 34 files changed, 176 insertions(+), 152 deletions(-) diff --git a/client/src/main/java/com/linecorp/decaton/client/DecatonClient.java b/client/src/main/java/com/linecorp/decaton/client/DecatonClient.java index 9af75e69..b7a70999 100644 --- a/client/src/main/java/com/linecorp/decaton/client/DecatonClient.java +++ b/client/src/main/java/com/linecorp/decaton/client/DecatonClient.java @@ -38,7 +38,7 @@ public interface DecatonClient extends AutoCloseable { * * @return a {@link CompletableFuture} which represents the result of task put. */ - CompletableFuture put(String key, T task); + CompletableFuture put(byte[] key, T task); /** * Put a task onto associated decaton queue with specifying arbitrary timestamp. @@ -47,7 +47,7 @@ public interface DecatonClient extends AutoCloseable { * @param timestamp milliseconds precision timestamp which is to be used to set timestamp of task metadata. * @return a {@link CompletableFuture} which represents the result of task put. */ - CompletableFuture put(String key, T task, long timestamp); + CompletableFuture put(byte[] key, T task, long timestamp); /** * Put a task onto associated decaton queue with specifying some fields of task metadata. @@ -56,7 +56,7 @@ public interface DecatonClient extends AutoCloseable { * @param overrideTaskMetadata taskMetaData which can be set by users and used for event publish. * @return a {@link CompletableFuture} which represents the result of task put. */ - CompletableFuture put(String key, T task, TaskMetadata overrideTaskMetadata); + CompletableFuture put(byte[] key, T task, TaskMetadata overrideTaskMetadata); /** * Put a task onto associated decaton queue. @@ -77,7 +77,7 @@ public interface DecatonClient extends AutoCloseable { * * @return a {@link CompletableFuture} which represents the result of task put. */ - CompletableFuture put(String key, T task, Consumer errorCallback); + CompletableFuture put(byte[] key, T task, Consumer errorCallback); /** * Put a task onto associated decaton queue with specifying arbitrary timestamp. @@ -100,7 +100,7 @@ public interface DecatonClient extends AutoCloseable { * * @return a {@link CompletableFuture} which represents the result of task put. */ - default CompletableFuture put(String key, T task, long timestamp, + default CompletableFuture put(byte[] key, T task, long timestamp, Consumer errorCallback) { CompletableFuture result = put(key, task, timestamp); result.exceptionally(e -> { diff --git a/client/src/main/java/com/linecorp/decaton/client/DecatonClientBuilder.java b/client/src/main/java/com/linecorp/decaton/client/DecatonClientBuilder.java index ef512e12..699d58d4 100644 --- a/client/src/main/java/com/linecorp/decaton/client/DecatonClientBuilder.java +++ b/client/src/main/java/com/linecorp/decaton/client/DecatonClientBuilder.java @@ -24,9 +24,9 @@ import org.apache.kafka.clients.producer.KafkaProducer; import org.apache.kafka.clients.producer.Producer; +import org.apache.kafka.common.serialization.ByteArraySerializer; import com.linecorp.decaton.client.internal.DecatonClientImpl; -import com.linecorp.decaton.client.kafka.PrintableAsciiStringSerializer; import com.linecorp.decaton.client.kafka.ProtocolBuffersKafkaSerializer; import com.linecorp.decaton.common.Serializer; import com.linecorp.decaton.protocol.Decaton.DecatonTaskRequest; @@ -54,9 +54,9 @@ public class DecatonClientBuilder { public static class DefaultKafkaProducerSupplier implements KafkaProducerSupplier { @Override - public Producer getProducer(Properties config) { + public Producer getProducer(Properties config) { return new KafkaProducer<>(config, - new PrintableAsciiStringSerializer(), + new ByteArraySerializer(), new ProtocolBuffersKafkaSerializer<>()); } } diff --git a/client/src/main/java/com/linecorp/decaton/client/KafkaProducerSupplier.java b/client/src/main/java/com/linecorp/decaton/client/KafkaProducerSupplier.java index bd9cb5b3..8d48efd2 100644 --- a/client/src/main/java/com/linecorp/decaton/client/KafkaProducerSupplier.java +++ b/client/src/main/java/com/linecorp/decaton/client/KafkaProducerSupplier.java @@ -39,5 +39,5 @@ public interface KafkaProducerSupplier { * @return an Kafka producer instance which implements {@link Producer}. The returned instance will be * closed along with {@link DecatonClient#close} being called. */ - Producer getProducer(Properties config); + Producer getProducer(Properties config); } diff --git a/client/src/main/java/com/linecorp/decaton/client/internal/DecatonClientImpl.java b/client/src/main/java/com/linecorp/decaton/client/internal/DecatonClientImpl.java index 27736a2b..f19b4205 100644 --- a/client/src/main/java/com/linecorp/decaton/client/internal/DecatonClientImpl.java +++ b/client/src/main/java/com/linecorp/decaton/client/internal/DecatonClientImpl.java @@ -62,7 +62,7 @@ public DecatonClientImpl(String topic, } @Override - public CompletableFuture put(String key, T task, long timestamp) { + public CompletableFuture put(byte[] key, T task, long timestamp) { TaskMetadataProto taskMetadata = TaskMetadataProto.newBuilder() .setTimestampMillis(timestamp) .setSourceApplicationId(applicationId) @@ -73,17 +73,17 @@ public CompletableFuture put(String key, T task, long timestamp) } @Override - public CompletableFuture put(String key, T task, TaskMetadata overrideTaskMetadata) { + public CompletableFuture put(byte[] key, T task, TaskMetadata overrideTaskMetadata) { return put(key, task, convertToTaskMetadataProto(overrideTaskMetadata)); } @Override - public CompletableFuture put(String key, T task) { + public CompletableFuture put(byte[] key, T task) { return put(key, task, timestampSupplier.get()); } @Override - public CompletableFuture put(String key, T task, Consumer errorCallback) { + public CompletableFuture put(byte[] key, T task, Consumer errorCallback) { return put(key, task, timestampSupplier.get(), errorCallback); } @@ -92,7 +92,7 @@ public void close() throws Exception { producer.close(); } - private CompletableFuture put(String key, T task, TaskMetadataProto taskMetadataProto) { + private CompletableFuture put(byte[] key, T task, TaskMetadataProto taskMetadataProto) { byte[] serializedTask = serializer.serialize(task); DecatonTaskRequest request = diff --git a/client/src/main/java/com/linecorp/decaton/client/internal/DecatonTaskProducer.java b/client/src/main/java/com/linecorp/decaton/client/internal/DecatonTaskProducer.java index 1f95d31b..809b9916 100644 --- a/client/src/main/java/com/linecorp/decaton/client/internal/DecatonTaskProducer.java +++ b/client/src/main/java/com/linecorp/decaton/client/internal/DecatonTaskProducer.java @@ -44,7 +44,7 @@ public class DecatonTaskProducer implements AutoCloseable { presetProducerConfig.put(ProducerConfig.MAX_IN_FLIGHT_REQUESTS_PER_CONNECTION, "1"); } - private final Producer producer; + private final Producer producer; private final String topic; private static Properties completeProducerConfig(Properties producerConfig) { @@ -61,8 +61,8 @@ public DecatonTaskProducer(String topic, Properties producerConfig, this.topic = topic; } - public CompletableFuture sendRequest(String key, DecatonTaskRequest request) { - ProducerRecord record = new ProducerRecord<>(topic, key, request); + public CompletableFuture sendRequest(byte[] key, DecatonTaskRequest request) { + ProducerRecord record = new ProducerRecord<>(topic, key, request); CompletableFuture result = new CompletableFuture<>(); producer.send(record, (metadata, exception) -> { diff --git a/client/src/test/java/com/linecorp/decaton/client/DecatonClientBuilderTest.java b/client/src/test/java/com/linecorp/decaton/client/DecatonClientBuilderTest.java index 08488b8c..dea6490f 100644 --- a/client/src/test/java/com/linecorp/decaton/client/DecatonClientBuilderTest.java +++ b/client/src/test/java/com/linecorp/decaton/client/DecatonClientBuilderTest.java @@ -44,12 +44,12 @@ public class DecatonClientBuilderTest { public MockitoRule rule = MockitoJUnit.rule(); @Mock - private Producer producer; + private Producer producer; @Captor - private ArgumentCaptor> recordCaptor; + private ArgumentCaptor> recordCaptor; - private ProducerRecord doProduce(DecatonClient dclient) { + private ProducerRecord doProduce(DecatonClient dclient) { dclient.put(null, HelloTask.getDefaultInstance()); verify(producer, times(1)).send(recordCaptor.capture(), any(Callback.class)); return recordCaptor.getValue(); @@ -69,7 +69,7 @@ public void testBuild() { .producerSupplier(config -> producer) .build(); - ProducerRecord record = doProduce(dclient); + ProducerRecord record = doProduce(dclient); assertEquals(topic, record.topic()); TaskMetadataProto metadata = record.value().getMetadata(); diff --git a/client/src/test/java/com/linecorp/decaton/client/DecatonClientTest.java b/client/src/test/java/com/linecorp/decaton/client/DecatonClientTest.java index b0c1fe43..f036d745 100644 --- a/client/src/test/java/com/linecorp/decaton/client/DecatonClientTest.java +++ b/client/src/test/java/com/linecorp/decaton/client/DecatonClientTest.java @@ -45,22 +45,22 @@ public class DecatonClientTest { @Spy private final DecatonClient decaton = new DecatonClient() { @Override - public CompletableFuture put(String key, HelloTask task) { + public CompletableFuture put(byte[] key, HelloTask task) { return null; } @Override - public CompletableFuture put(String key, HelloTask task, long timestamp) { + public CompletableFuture put(byte[] key, HelloTask task, long timestamp) { return null; } @Override - public CompletableFuture put(String key, HelloTask task, TaskMetadata overrideTaskMetadata) { + public CompletableFuture put(byte[] key, HelloTask task, TaskMetadata overrideTaskMetadata) { return null; } @Override - public CompletableFuture put(String key, HelloTask task, + public CompletableFuture put(byte[] key, HelloTask task, Consumer errorCallback) { return null; } diff --git a/client/src/test/java/com/linecorp/decaton/client/internal/DecatonClientImplTest.java b/client/src/test/java/com/linecorp/decaton/client/internal/DecatonClientImplTest.java index f55d003e..81891f7f 100644 --- a/client/src/test/java/com/linecorp/decaton/client/internal/DecatonClientImplTest.java +++ b/client/src/test/java/com/linecorp/decaton/client/internal/DecatonClientImplTest.java @@ -25,6 +25,7 @@ import static org.mockito.Mockito.times; import static org.mockito.Mockito.verify; +import java.nio.charset.StandardCharsets; import java.util.Properties; import java.util.function.Supplier; @@ -54,7 +55,7 @@ public class DecatonClientImplTest { public MockitoRule rule = MockitoJUnit.rule(); @Mock - private Producer producer; + private Producer producer; @Mock private Supplier timestampSupplier; @@ -62,7 +63,7 @@ public class DecatonClientImplTest { private DecatonClientImpl client; @Captor - private ArgumentCaptor> captor; + private ArgumentCaptor> captor; @Before public void setUp() { @@ -75,10 +76,10 @@ APPLICATION_ID, INSTANCE_ID, new Properties(), public void testTimestampFieldSetInternally() { doReturn(1234L).when(timestampSupplier).get(); - client.put("key", HelloTask.getDefaultInstance()); + client.put("key".getBytes(StandardCharsets.UTF_8), HelloTask.getDefaultInstance()); verify(producer, times(1)).send(captor.capture(), any(Callback.class)); - ProducerRecord record = captor.getValue(); + ProducerRecord record = captor.getValue(); assertNull(record.timestamp()); assertEquals(1234, record.value().getMetadata().getTimestampMillis()); } @@ -87,10 +88,10 @@ public void testTimestampFieldSetInternally() { public void testTimestampFieldSetInternallyWithCallback() { doReturn(1234L).when(timestampSupplier).get(); - client.put("key", HelloTask.getDefaultInstance(), ignored -> {}); + client.put("key".getBytes(StandardCharsets.UTF_8), HelloTask.getDefaultInstance(), ignored -> {}); verify(producer, times(1)).send(captor.capture(), any(Callback.class)); - ProducerRecord record = captor.getValue(); + ProducerRecord record = captor.getValue(); assertNull(record.timestamp()); assertEquals(1234, record.value().getMetadata().getTimestampMillis()); } @@ -99,10 +100,10 @@ public void testTimestampFieldSetInternallyWithCallback() { public void testTimestampFieldSetExternally() { doReturn(1234L).when(timestampSupplier).get(); - client.put("key", HelloTask.getDefaultInstance(), 5678); + client.put("key".getBytes(StandardCharsets.UTF_8), HelloTask.getDefaultInstance(), 5678); verify(producer, times(1)).send(captor.capture(), any(Callback.class)); - ProducerRecord record = captor.getValue(); + ProducerRecord record = captor.getValue(); assertNull(record.timestamp()); assertEquals(5678, record.value().getMetadata().getTimestampMillis()); } @@ -111,10 +112,10 @@ public void testTimestampFieldSetExternally() { public void testTimestampFieldSetExternallyWithCallback() { doReturn(1234L).when(timestampSupplier).get(); - client.put("key", HelloTask.getDefaultInstance(), 5678, ignored -> {}); + client.put("key".getBytes(StandardCharsets.UTF_8), HelloTask.getDefaultInstance(), 5678, ignored -> {}); verify(producer, times(1)).send(captor.capture(), any(Callback.class)); - ProducerRecord record = captor.getValue(); + ProducerRecord record = captor.getValue(); assertNull(record.timestamp()); assertEquals(5678, record.value().getMetadata().getTimestampMillis()); } @@ -123,10 +124,10 @@ public void testTimestampFieldSetExternallyWithCallback() { public void testTaskMetaDataSetExternally() { doReturn(1234L).when(timestampSupplier).get(); - client.put("key", HelloTask.getDefaultInstance(), TaskMetadata.builder() - .timestamp(5678L) - .scheduledTime(6912L) - .build()); + client.put("key".getBytes(StandardCharsets.UTF_8), HelloTask.getDefaultInstance(), TaskMetadata.builder() + .timestamp(5678L) + .scheduledTime(6912L) + .build()); verifyAndAssertTaskMetadata(5678L, 6912L); } @@ -135,9 +136,9 @@ public void testTaskMetaDataSetExternally() { public void testWithScheduledTimeSetExternally() { doReturn(1234L).when(timestampSupplier).get(); - client.put("key", HelloTask.getDefaultInstance(), TaskMetadata.builder() - .scheduledTime(181234L) - .build()); + client.put("key".getBytes(StandardCharsets.UTF_8), HelloTask.getDefaultInstance(), TaskMetadata.builder() + .scheduledTime(181234L) + .build()); verifyAndAssertTaskMetadata(1234L, 181234L); } @@ -146,10 +147,10 @@ public void testWithScheduledTimeSetExternally() { public void testWithEmptyTaskMetaDataSetExternally() { doReturn(1234L).when(timestampSupplier).get(); - client.put("key", HelloTask.getDefaultInstance(), TaskMetadata.builder().build()); + client.put("key".getBytes(StandardCharsets.UTF_8), HelloTask.getDefaultInstance(), TaskMetadata.builder().build()); verify(producer, times(1)).send(captor.capture(), any(Callback.class)); - ProducerRecord record = captor.getValue(); + ProducerRecord record = captor.getValue(); assertTrue(record.value().getMetadata().getTimestampMillis() > 0); assertNotNull(record.value().getMetadata().getSourceApplicationId()); assertNotNull(record.value().getMetadata().getSourceInstanceId()); @@ -157,7 +158,7 @@ public void testWithEmptyTaskMetaDataSetExternally() { private void verifyAndAssertTaskMetadata(long timestamp, long scheduledTime) { verify(producer, times(1)).send(captor.capture(), any(Callback.class)); - ProducerRecord record = captor.getValue(); + ProducerRecord record = captor.getValue(); assertNull(record.timestamp()); assertEquals(timestamp, record.value().getMetadata().getTimestampMillis()); assertEquals(scheduledTime, record.value().getMetadata().getScheduledTimeMillis()); diff --git a/processor/src/it/java/com/linecorp/decaton/processor/CoreFunctionalityTest.java b/processor/src/it/java/com/linecorp/decaton/processor/CoreFunctionalityTest.java index 23bab456..416bec65 100644 --- a/processor/src/it/java/com/linecorp/decaton/processor/CoreFunctionalityTest.java +++ b/processor/src/it/java/com/linecorp/decaton/processor/CoreFunctionalityTest.java @@ -18,6 +18,7 @@ import static org.junit.Assert.assertTrue; +import java.nio.ByteBuffer; import java.util.ArrayList; import java.util.HashMap; import java.util.List; @@ -153,17 +154,17 @@ public void testSingleThreadProcessing() throws Exception { // Note that this processing semantics is not be considered as Decaton specification which users can rely on. // Rather, this is just a expected behavior based on current implementation when we set concurrency to 1. ProcessingGuarantee noDuplicates = new ProcessingGuarantee() { - private final Map> produced = new HashMap<>(); - private final Map> processed = new HashMap<>(); + private final Map> produced = new HashMap<>(); + private final Map> processed = new HashMap<>(); @Override public synchronized void onProduce(ProducedRecord record) { - produced.computeIfAbsent(record.key(), key -> new ArrayList<>()).add(record.task()); + produced.computeIfAbsent(ByteBuffer.wrap(record.key()), key -> new ArrayList<>()).add(record.task()); } @Override public synchronized void onProcess(TaskMetadata metadata, ProcessedRecord record) { - processed.computeIfAbsent(record.key(), key -> new ArrayList<>()).add(record.task()); + processed.computeIfAbsent(ByteBuffer.wrap(record.key()), key -> new ArrayList<>()).add(record.task()); } @Override diff --git a/processor/src/it/java/com/linecorp/decaton/processor/RateLimiterTest.java b/processor/src/it/java/com/linecorp/decaton/processor/RateLimiterTest.java index 2ae3b496..7b5bde6f 100644 --- a/processor/src/it/java/com/linecorp/decaton/processor/RateLimiterTest.java +++ b/processor/src/it/java/com/linecorp/decaton/processor/RateLimiterTest.java @@ -18,6 +18,8 @@ import static org.junit.Assert.assertEquals; +import java.nio.ByteBuffer; +import java.nio.charset.StandardCharsets; import java.util.Collections; import java.util.HashSet; import java.util.Set; @@ -57,12 +59,12 @@ public void tearDown() { @Test(timeout = 30000) public void testPropertyDynamicSwitch() throws Exception { - Set keys = new HashSet<>(); + Set keys = new HashSet<>(); for (int i = 0; i < 10000; i++) { - keys.add("key" + i); + keys.add(ByteBuffer.wrap(("key" + i).getBytes(StandardCharsets.UTF_8))); } - Set processedKeys = Collections.synchronizedSet(new HashSet<>()); + Set processedKeys = Collections.synchronizedSet(new HashSet<>()); CountDownLatch processLatch = new CountDownLatch(keys.size()); DecatonProcessor processor = (context, task) -> { @@ -81,11 +83,11 @@ public void testPropertyDynamicSwitch() throws Exception { DecatonClient client = TestUtils.client(topicName, rule.bootstrapServers())) { int count = 0; - for (String key : keys) { + for (ByteBuffer key : keys) { if (++count % 1000 == 0) { rateProp.set((long) count / 10); } - client.put(key, HelloTask.getDefaultInstance()); + client.put(key.array(), HelloTask.getDefaultInstance()); } processLatch.await(); } diff --git a/processor/src/main/java/com/linecorp/decaton/processor/ProcessingContext.java b/processor/src/main/java/com/linecorp/decaton/processor/ProcessingContext.java index 7434c5de..8eeeb4ce 100644 --- a/processor/src/main/java/com/linecorp/decaton/processor/ProcessingContext.java +++ b/processor/src/main/java/com/linecorp/decaton/processor/ProcessingContext.java @@ -40,7 +40,7 @@ public interface ProcessingContext { * @return the key associated to the task now being processed. can be null if key isn't supplied for the * task. */ - String key(); + byte[] key(); /** * Returns the {@link Headers} which is associated to the source {@link ConsumerRecord} of the task. diff --git a/processor/src/main/java/com/linecorp/decaton/processor/metrics/Metrics.java b/processor/src/main/java/com/linecorp/decaton/processor/metrics/Metrics.java index 51ec0e60..e8e09a8f 100644 --- a/processor/src/main/java/com/linecorp/decaton/processor/metrics/Metrics.java +++ b/processor/src/main/java/com/linecorp/decaton/processor/metrics/Metrics.java @@ -104,7 +104,7 @@ public void close() { public class SubscriptionMetrics extends AbstractMetrics { volatile KafkaClientMetrics kafkaClientMetrics; - public void bindClientMetrics(Consumer consumer) { + public void bindClientMetrics(Consumer consumer) { kafkaClientMetrics = new KafkaClientMetrics(consumer, availableTags.subscriptionScope()); kafkaClientMetrics.bindTo(registry); } diff --git a/processor/src/main/java/com/linecorp/decaton/processor/processors/CompactionProcessor.java b/processor/src/main/java/com/linecorp/decaton/processor/processors/CompactionProcessor.java index 18699ae3..eaa872b8 100644 --- a/processor/src/main/java/com/linecorp/decaton/processor/processors/CompactionProcessor.java +++ b/processor/src/main/java/com/linecorp/decaton/processor/processors/CompactionProcessor.java @@ -16,6 +16,7 @@ package com.linecorp.decaton.processor.processors; +import java.nio.ByteBuffer; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentMap; import java.util.concurrent.ScheduledExecutorService; @@ -84,7 +85,7 @@ public enum CompactChoice { } private final ScheduledExecutorService executor; - private final ConcurrentMap> windowedTasks = new ConcurrentHashMap<>(); + private final ConcurrentMap> windowedTasks = new ConcurrentHashMap<>(); private final BiFunction, CompactingTask, CompactChoice> compactor; private final long lingerMillis; @@ -145,8 +146,8 @@ public CompactionProcessor( this(lingerMillis, compactor, null); } - private void flush(String key) throws InterruptedException { - CompactingTask task = windowedTasks.remove(key); + private void flush(byte[] key) throws InterruptedException { + CompactingTask task = windowedTasks.remove(ByteBuffer.wrap(key)); if (task == null) { return; } @@ -155,7 +156,7 @@ private void flush(String key) throws InterruptedException { // visible for testing Runnable flushTask(ProcessingContext context) { - String key = context.key(); + byte[] key = context.key(); return () -> { try { flush(key); @@ -182,15 +183,15 @@ private void scheduleFlush(ProcessingContext context) { @Override public void process(ProcessingContext context, T task) throws InterruptedException { - String key = context.key(); + byte[] key = context.key(); CompactingTask newTask = new CompactingTask<>(context.deferCompletion(), context, task); // Even though we do this read and following updates in separate operation, race condition can't be // happened because tasks are guaranteed to be serialized by it's key, so simultaneous processing // of tasks sharing the same key won't be happen. - CompactingTask prevTask = windowedTasks.get(key); + CompactingTask prevTask = windowedTasks.get(ByteBuffer.wrap(key)); if (prevTask == null) { - windowedTasks.put(key, newTask); + windowedTasks.put(ByteBuffer.wrap(key), newTask); scheduleFlush(context); return; } @@ -205,7 +206,7 @@ public void process(ProcessingContext context, T task) throws InterruptedExce case PICK_RIGHT: case PICK_EITHER: // Newer task has larger offset. We want to forward consumed offset. // Update the last task with new one. - Object oldEntry = windowedTasks.put(key, newTask); + Object oldEntry = windowedTasks.put(ByteBuffer.wrap(key), newTask); if (oldEntry == null) { // By race condition, there is a chance that the scheduled flush for preceding task just // got fired right after this method checked the key's existence at the beginning of this diff --git a/processor/src/main/java/com/linecorp/decaton/processor/runtime/ProcessorSubscription.java b/processor/src/main/java/com/linecorp/decaton/processor/runtime/ProcessorSubscription.java index 274a580b..ad368666 100644 --- a/processor/src/main/java/com/linecorp/decaton/processor/runtime/ProcessorSubscription.java +++ b/processor/src/main/java/com/linecorp/decaton/processor/runtime/ProcessorSubscription.java @@ -93,7 +93,7 @@ public void updateAssignment(Collection newAssignment) { } @Override - public void receive(ConsumerRecord record) { + public void receive(ConsumerRecord record) { TopicPartition tp = new TopicPartition(record.topic(), record.partition()); PartitionContext context = contexts.get(tp); @@ -130,7 +130,7 @@ public void receive(ConsumerRecord record) { } ProcessorSubscription(SubscriptionScope scope, - Supplier> consumerSupplier, + Supplier> consumerSupplier, Processors processors, ProcessorProperties props, SubscriptionStateListener stateListener, @@ -141,7 +141,7 @@ public void receive(ConsumerRecord record) { this.contexts = contexts; metrics = Metrics.withTags("subscription", scope.subscriptionId()).new SubscriptionMetrics(); - Consumer consumer = consumerSupplier.get(); + Consumer consumer = consumerSupplier.get(); if (props.get(CONFIG_BIND_CLIENT_METRICS).value()) { metrics.bindClientMetrics(consumer); } @@ -159,7 +159,7 @@ public void receive(ConsumerRecord record) { } public ProcessorSubscription(SubscriptionScope scope, - Supplier> consumerSupplier, + Supplier> consumerSupplier, Processors processors, ProcessorProperties props, SubscriptionStateListener stateListener) { diff --git a/processor/src/main/java/com/linecorp/decaton/processor/runtime/internal/BlacklistedKeysFilter.java b/processor/src/main/java/com/linecorp/decaton/processor/runtime/internal/BlacklistedKeysFilter.java index 9e27e1d1..c2ac193c 100644 --- a/processor/src/main/java/com/linecorp/decaton/processor/runtime/internal/BlacklistedKeysFilter.java +++ b/processor/src/main/java/com/linecorp/decaton/processor/runtime/internal/BlacklistedKeysFilter.java @@ -18,6 +18,7 @@ import static com.linecorp.decaton.processor.runtime.ProcessorProperties.CONFIG_IGNORE_KEYS; +import java.nio.charset.StandardCharsets; import java.util.HashSet; import java.util.Set; @@ -37,11 +38,18 @@ public BlacklistedKeysFilter(ProcessorProperties props) { .listen((oldValue, newValue) -> ignoreKeys = new HashSet<>(newValue)); } - public boolean shouldTake(ConsumerRecord record) { + public boolean shouldTake(ConsumerRecord record) { + final byte[] key = record.key(); + if (key == null) { + return true; + } + + final String stringKey = new String(key, StandardCharsets.UTF_8); + // Preceding isEmpty() check is for reducing tiny overhead applied for each contains() by calling // Object#hashCode. Since ignoreKeys should be empty for most cases.. - if (!ignoreKeys.isEmpty() && ignoreKeys.contains(record.key())) { - logger.debug("Ignore task which has key configured to ignore: {}", record.key()); + if (!ignoreKeys.isEmpty() && ignoreKeys.contains(stringKey)) { + logger.debug("Ignore task which has key configured to ignore: {}", stringKey); return false; } diff --git a/processor/src/main/java/com/linecorp/decaton/processor/runtime/internal/ConsumeManager.java b/processor/src/main/java/com/linecorp/decaton/processor/runtime/internal/ConsumeManager.java index 6775f8c4..a2b9c707 100644 --- a/processor/src/main/java/com/linecorp/decaton/processor/runtime/internal/ConsumeManager.java +++ b/processor/src/main/java/com/linecorp/decaton/processor/runtime/internal/ConsumeManager.java @@ -93,16 +93,16 @@ public interface ConsumerHandler { * Process a {@link ConsumerRecord} that has been consumed from the topic. * @param record a record that has been fetched from the target topic. */ - void receive(ConsumerRecord record); + void receive(ConsumerRecord record); } - private final Consumer consumer; + private final Consumer consumer; private final PartitionStates states; private final ConsumerHandler handler; private final SubscriptionMetrics metrics; private final AtomicBoolean consumerClosing; - public ConsumeManager(Consumer consumer, + public ConsumeManager(Consumer consumer, PartitionStates states, ConsumerHandler handler, SubscriptionMetrics metrics) { @@ -162,7 +162,7 @@ public void onPartitionsAssigned(Collection partitions) { */ public void poll() { Timer timer = Utils.timer(); - ConsumerRecords records = consumer.poll(POLL_TIMEOUT_MILLIS); + ConsumerRecords records = consumer.poll(POLL_TIMEOUT_MILLIS); metrics.consumerPollTime.record(timer.duration()); timer = Utils.timer(); diff --git a/processor/src/main/java/com/linecorp/decaton/processor/runtime/internal/ConsumerSupplier.java b/processor/src/main/java/com/linecorp/decaton/processor/runtime/internal/ConsumerSupplier.java index 39be5e60..65b171c2 100644 --- a/processor/src/main/java/com/linecorp/decaton/processor/runtime/internal/ConsumerSupplier.java +++ b/processor/src/main/java/com/linecorp/decaton/processor/runtime/internal/ConsumerSupplier.java @@ -26,9 +26,8 @@ import org.apache.kafka.clients.consumer.ConsumerConfig; import org.apache.kafka.clients.consumer.KafkaConsumer; import org.apache.kafka.common.serialization.ByteArrayDeserializer; -import org.apache.kafka.common.serialization.StringDeserializer; -public class ConsumerSupplier implements Supplier> { +public class ConsumerSupplier implements Supplier> { public static final int DEFAULT_MAX_POLL_RECORDS = 100; private static final Map configOverwrites = new HashMap() {{ @@ -42,8 +41,8 @@ public ConsumerSupplier(Properties config) { } @Override - public Consumer get() { - return new KafkaConsumer<>(mergedProps(), new StringDeserializer(), new ByteArrayDeserializer()); + public Consumer get() { + return new KafkaConsumer<>(mergedProps(), new ByteArrayDeserializer(), new ByteArrayDeserializer()); } private Properties mergedProps() { diff --git a/processor/src/main/java/com/linecorp/decaton/processor/runtime/internal/ProcessingContextImpl.java b/processor/src/main/java/com/linecorp/decaton/processor/runtime/internal/ProcessingContextImpl.java index b950ee2a..89cc8402 100644 --- a/processor/src/main/java/com/linecorp/decaton/processor/runtime/internal/ProcessingContextImpl.java +++ b/processor/src/main/java/com/linecorp/decaton/processor/runtime/internal/ProcessingContextImpl.java @@ -68,7 +68,7 @@ public TaskMetadata metadata() { } @Override - public String key() { + public byte[] key() { return request.key(); } diff --git a/processor/src/main/java/com/linecorp/decaton/processor/runtime/internal/SubPartitioner.java b/processor/src/main/java/com/linecorp/decaton/processor/runtime/internal/SubPartitioner.java index 6298d273..2128a404 100644 --- a/processor/src/main/java/com/linecorp/decaton/processor/runtime/internal/SubPartitioner.java +++ b/processor/src/main/java/com/linecorp/decaton/processor/runtime/internal/SubPartitioner.java @@ -16,6 +16,7 @@ package com.linecorp.decaton.processor.runtime.internal; +import java.nio.ByteBuffer; import java.util.concurrent.atomic.AtomicLong; class SubPartitioner { @@ -31,7 +32,7 @@ private static int toPositive(int number) { return number & 2147483647; } - public int partitionFor(String key) { + public int partitionFor(byte[] key) { if (key == null) { return toPositive((int) monotonicValueSupplier.getAndIncrement()) % bound; } else { @@ -40,8 +41,13 @@ public int partitionFor(String key) { // Here just by adding few bytes to the key we can "shift" hashing of the key and // can get back better distribution again in murmur2 result to evenly distribute keys // for subpartitions. - String shiftedKey = "s:" + key; - int hash = org.apache.kafka.common.utils.Utils.murmur2(shiftedKey.getBytes()); + // TODO: Eliminate array copy here + final ByteBuffer bb = ByteBuffer.allocate(key.length + 2); + bb.put((byte) 's'); + bb.put((byte) ':'); + bb.put(key); + + int hash = org.apache.kafka.common.utils.Utils.murmur2(bb.array()); return toPositive(hash) % bound; } } diff --git a/processor/src/main/java/com/linecorp/decaton/processor/runtime/internal/TaskRequest.java b/processor/src/main/java/com/linecorp/decaton/processor/runtime/internal/TaskRequest.java index 7d88eba0..64a013e7 100644 --- a/processor/src/main/java/com/linecorp/decaton/processor/runtime/internal/TaskRequest.java +++ b/processor/src/main/java/com/linecorp/decaton/processor/runtime/internal/TaskRequest.java @@ -32,7 +32,7 @@ public class TaskRequest { private final TopicPartition topicPartition; private final long recordOffset; private final OffsetState offsetState; - private final String key; + private final byte[] key; @ToString.Exclude private final Headers headers; @ToString.Exclude @@ -43,7 +43,7 @@ public class TaskRequest { public TaskRequest(TopicPartition topicPartition, long recordOffset, OffsetState offsetState, - String key, + byte[] key, Headers headers, RecordTraceHandle trace, byte[] rawRequestBytes) { diff --git a/processor/src/test/java/com/linecorp/decaton/processor/processors/CompactionProcessorTest.java b/processor/src/test/java/com/linecorp/decaton/processor/processors/CompactionProcessorTest.java index 254c42c5..44ee2134 100644 --- a/processor/src/test/java/com/linecorp/decaton/processor/processors/CompactionProcessorTest.java +++ b/processor/src/test/java/com/linecorp/decaton/processor/processors/CompactionProcessorTest.java @@ -27,6 +27,7 @@ import static org.mockito.Mockito.times; import static org.mockito.Mockito.verify; +import java.nio.charset.StandardCharsets; import java.util.Arrays; import java.util.concurrent.CountDownLatch; import java.util.concurrent.ScheduledThreadPoolExecutor; @@ -44,12 +45,12 @@ import org.mockito.junit.MockitoJUnit; import org.mockito.junit.MockitoRule; +import com.linecorp.decaton.processor.Completion; import com.linecorp.decaton.processor.DecatonProcessor; import com.linecorp.decaton.processor.ProcessingContext; import com.linecorp.decaton.processor.TaskMetadata; import com.linecorp.decaton.processor.processors.CompactionProcessor.CompactChoice; import com.linecorp.decaton.processor.processors.CompactionProcessor.CompactingTask; -import com.linecorp.decaton.processor.Completion; import com.linecorp.decaton.processor.runtime.DecatonTask; import com.linecorp.decaton.processor.runtime.ProcessorProperties; import com.linecorp.decaton.processor.runtime.internal.ProcessingContextImpl; @@ -101,7 +102,7 @@ private TaskInput put(DecatonProcessor processor, taskData, taskData.toByteArray()); TaskRequest request = new TaskRequest( - new TopicPartition("topic", 1), 1, null, name, null, NoopTrace.INSTANCE, null); + new TopicPartition("topic", 1), 1, null, name.getBytes(StandardCharsets.UTF_8), null, NoopTrace.INSTANCE, null); ProcessingContext context = spy(new ProcessingContextImpl<>("subscription", request, task, Arrays.asList(processor, downstream), null, diff --git a/processor/src/test/java/com/linecorp/decaton/processor/runtime/ProcessorSubscriptionTest.java b/processor/src/test/java/com/linecorp/decaton/processor/runtime/ProcessorSubscriptionTest.java index e9036643..98b0b3fe 100644 --- a/processor/src/test/java/com/linecorp/decaton/processor/runtime/ProcessorSubscriptionTest.java +++ b/processor/src/test/java/com/linecorp/decaton/processor/runtime/ProcessorSubscriptionTest.java @@ -25,6 +25,7 @@ import static org.mockito.Mockito.doAnswer; import static org.mockito.Mockito.verify; +import java.nio.charset.StandardCharsets; import java.time.Duration; import java.util.ArrayList; import java.util.Arrays; @@ -38,11 +39,11 @@ import java.util.concurrent.BlockingQueue; import java.util.concurrent.CountDownLatch; import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; import java.util.concurrent.Semaphore; +import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicReference; -import java.util.concurrent.Executors; -import java.util.concurrent.TimeUnit; import org.apache.kafka.clients.consumer.Consumer; import org.apache.kafka.clients.consumer.ConsumerRebalanceListener; @@ -61,7 +62,6 @@ import com.linecorp.decaton.processor.DecatonProcessor; import com.linecorp.decaton.processor.DeferredCompletion; -import com.linecorp.decaton.processor.ProcessingContext; import com.linecorp.decaton.processor.TaskMetadata; import com.linecorp.decaton.processor.runtime.SubscriptionStateListener.State; import com.linecorp.decaton.processor.runtime.internal.ConsumerSupplier; @@ -75,13 +75,13 @@ public class ProcessorSubscriptionTest { public MockitoRule rule = MockitoJUnit.rule(); @Mock - Consumer consumer; + Consumer consumer; /** * A mock consumer which exposes rebalance listener so that can be triggered manually * ({@link MockConsumer} doesn't simulate rebalance listener invocation. refs: KAFKA-6968). */ - private static class DecatonMockConsumer extends MockConsumer { + private static class DecatonMockConsumer extends MockConsumer { volatile ConsumerRebalanceListener rebalanceListener; private DecatonMockConsumer() { @@ -109,7 +109,7 @@ private static SubscriptionScope scope(String topic, long waitForProcessingOnClo ConsumerSupplier.DEFAULT_MAX_POLL_RECORDS); } - private static ProcessorSubscription subscription(Consumer consumer, + private static ProcessorSubscription subscription(Consumer consumer, SubscriptionStateListener listener, TopicPartition tp, DecatonProcessor processor) { @@ -172,7 +172,8 @@ public void testOffsetRegression() throws Exception { feedOffsets.add(100L); feedOffsets.add(101L); CountDownLatch processLatch = new CountDownLatch(1); - ProcessorSubscription subscription = subscription(consumer, ignored -> {}, tp, (context, task) -> { + ProcessorSubscription subscription = subscription(consumer, ignored -> { + }, tp, (context, task) -> { if ("101".equals(task)) { processLatch.countDown(); } @@ -195,7 +196,7 @@ public void testOffsetRegression() throws Exception { if (offset != null) { return new ConsumerRecords<>(singletonMap(tp, Collections.singletonList( // Feed one record, then a subsequent record of the regressing offset. - new ConsumerRecord<>(tp.topic(), tp.partition(), offset, "abc", + new ConsumerRecord<>(tp.topic(), tp.partition(), offset, "abc".getBytes(StandardCharsets.UTF_8), String.valueOf(offset).getBytes())))); } else { Thread.sleep(invocation.getArgument(0)); @@ -217,7 +218,7 @@ public void testTerminateAsync() throws Exception { TopicPartition tp = new TopicPartition("topic", 0); DecatonMockConsumer consumer = new DecatonMockConsumer() { @Override - public synchronized ConsumerRecords poll(Duration timeout) { + public synchronized ConsumerRecords poll(Duration timeout) { rebalanceListener.onPartitionsAssigned(assignment()); return super.poll(timeout); } @@ -262,9 +263,9 @@ public synchronized ConsumerRecords poll(Duration timeout) { // First task finishes synchronous part of processing, starts async processing // Second task blocks during synchronous part of processing // Third task will be queued behind it - consumer.addRecord(new ConsumerRecord<>(tp.topic(), tp.partition(), 10, "", NO_DATA)); - consumer.addRecord(new ConsumerRecord<>(tp.topic(), tp.partition(), 11, "", NO_DATA)); - consumer.addRecord(new ConsumerRecord<>(tp.topic(), tp.partition(), 12, "", NO_DATA)); + consumer.addRecord(new ConsumerRecord<>(tp.topic(), tp.partition(), 10, new byte[0], NO_DATA)); + consumer.addRecord(new ConsumerRecord<>(tp.topic(), tp.partition(), 11, new byte[0], NO_DATA)); + consumer.addRecord(new ConsumerRecord<>(tp.topic(), tp.partition(), 12, new byte[0], NO_DATA)); asyncProcessingStarted.await(); subscription.initiateShutdown(); assertTrue(consumer.committed(singleton(tp)).isEmpty()); @@ -280,7 +281,8 @@ public synchronized ConsumerRecords poll(Duration timeout) { @Test(timeout = 5000) public void closeWithoutStart() throws Exception { TopicPartition tp = new TopicPartition("topic", 0); - ProcessorSubscription subscription = subscription(consumer, null, tp, (context, task) -> {}); + ProcessorSubscription subscription = subscription(consumer, null, tp, (context, task) -> { + }); // The main point is that the below close returns within timeout. subscription.close(); verify(consumer).close(); diff --git a/processor/src/test/java/com/linecorp/decaton/processor/runtime/internal/ConsumeManagerTest.java b/processor/src/test/java/com/linecorp/decaton/processor/runtime/internal/ConsumeManagerTest.java index e6425a7c..61e6fe4d 100644 --- a/processor/src/test/java/com/linecorp/decaton/processor/runtime/internal/ConsumeManagerTest.java +++ b/processor/src/test/java/com/linecorp/decaton/processor/runtime/internal/ConsumeManagerTest.java @@ -62,10 +62,10 @@ public class ConsumeManagerTest { static final String TOPIC = "topic"; @Mock - Consumer consumer; + Consumer consumer; @Captor - ArgumentCaptor> recordsCaptor; + ArgumentCaptor> recordsCaptor; @Mock PartitionStates states; diff --git a/processor/src/test/java/com/linecorp/decaton/processor/runtime/internal/DecatonTaskRetryQueueingProcessorTest.java b/processor/src/test/java/com/linecorp/decaton/processor/runtime/internal/DecatonTaskRetryQueueingProcessorTest.java index 4cb459c9..0feb46f6 100644 --- a/processor/src/test/java/com/linecorp/decaton/processor/runtime/internal/DecatonTaskRetryQueueingProcessorTest.java +++ b/processor/src/test/java/com/linecorp/decaton/processor/runtime/internal/DecatonTaskRetryQueueingProcessorTest.java @@ -29,6 +29,7 @@ import static org.mockito.Mockito.times; import static org.mockito.Mockito.verify; +import java.nio.charset.StandardCharsets; import java.time.Duration; import java.util.Optional; import java.util.concurrent.CompletableFuture; @@ -76,13 +77,13 @@ public void setUp() { processor = new DecatonTaskRetryQueueingProcessor(scope, producer); doReturn(CompletableFuture.completedFuture(null)).when(producer).sendRequest(any(), any()); doReturn(new CompletionImpl()).when(context).deferCompletion(); - doReturn("key").when(context).key(); + doReturn("key".getBytes(StandardCharsets.UTF_8)).when(context).key(); doReturn(TaskMetadata.builder().build()).when(context).metadata(); } @Test public void testRetryRequest() throws InterruptedException { - String key = "key"; + byte[] key = "key".getBytes(StandardCharsets.UTF_8); TaskMetadata meta = TaskMetadata.builder() .sourceApplicationId("unit-test") diff --git a/processor/src/test/java/com/linecorp/decaton/processor/runtime/internal/ProcessPipelineTest.java b/processor/src/test/java/com/linecorp/decaton/processor/runtime/internal/ProcessPipelineTest.java index c36a696d..07f67473 100644 --- a/processor/src/test/java/com/linecorp/decaton/processor/runtime/internal/ProcessPipelineTest.java +++ b/processor/src/test/java/com/linecorp/decaton/processor/runtime/internal/ProcessPipelineTest.java @@ -32,6 +32,7 @@ import static org.mockito.Mockito.verify; import static org.mockito.Mockito.when; +import java.nio.charset.StandardCharsets; import java.time.Clock; import java.util.Collections; import java.util.Optional; @@ -92,7 +93,7 @@ public class ProcessPipelineTest { private static TaskRequest taskRequest() { return new TaskRequest( - new TopicPartition("topic", 1), 1, new OffsetState(1234), "TEST", null, NoopTrace.INSTANCE, REQUEST.toByteArray()); + new TopicPartition("topic", 1), 1, new OffsetState(1234), "TEST".getBytes(StandardCharsets.UTF_8), null, NoopTrace.INSTANCE, REQUEST.toByteArray()); } @Rule @@ -252,4 +253,3 @@ public void testScheduleThenProcess_Terminate() throws InterruptedException { assertFalse(request.offsetState().completion().isComplete()); } } - diff --git a/processor/src/test/java/com/linecorp/decaton/processor/runtime/internal/ProcessingContextImplTest.java b/processor/src/test/java/com/linecorp/decaton/processor/runtime/internal/ProcessingContextImplTest.java index 7bb03707..d40f616a 100644 --- a/processor/src/test/java/com/linecorp/decaton/processor/runtime/internal/ProcessingContextImplTest.java +++ b/processor/src/test/java/com/linecorp/decaton/processor/runtime/internal/ProcessingContextImplTest.java @@ -30,6 +30,7 @@ import static org.mockito.Mockito.times; import static org.mockito.Mockito.verify; +import java.nio.charset.StandardCharsets; import java.util.Arrays; import java.util.Collections; import java.util.concurrent.CompletableFuture; @@ -110,7 +111,7 @@ private static void terminateExecutor(ExecutorService executor) throws Interrupt private static ProcessingContextImpl context(RecordTraceHandle traceHandle, DecatonProcessor... processors) { TaskRequest request = new TaskRequest( - new TopicPartition("topic", 1), 1, null, "TEST", + new TopicPartition("topic", 1), 1, null, "TEST".getBytes(StandardCharsets.UTF_8), null, traceHandle, REQUEST.toByteArray()); DecatonTask task = new DecatonTask<>( TaskMetadata.fromProto(REQUEST.getMetadata()), TASK, TASK.toByteArray()); @@ -361,7 +362,7 @@ public void process(ProcessingContext context, byte[] task) } }); TaskRequest request = new TaskRequest( - new TopicPartition("topic", 1), 1, null, "TEST", null, null, REQUEST.toByteArray()); + new TopicPartition("topic", 1), 1, null, "TEST".getBytes(StandardCharsets.UTF_8), null, null, REQUEST.toByteArray()); DecatonTask task = new DecatonTask<>( TaskMetadata.fromProto(REQUEST.getMetadata()), TASK.toByteArray(), TASK.toByteArray()); @@ -405,7 +406,7 @@ public void process(ProcessingContext context, byte[] task) } }); TaskRequest request = new TaskRequest( - new TopicPartition("topic", 1), 1, null, "TEST", null, null, REQUEST.toByteArray()); + new TopicPartition("topic", 1), 1, null, "TEST".getBytes(StandardCharsets.UTF_8), null, null, REQUEST.toByteArray()); DecatonTask task = new DecatonTask<>( TaskMetadata.fromProto(REQUEST.getMetadata()), TASK.toByteArray(), TASK.toByteArray()); diff --git a/processor/src/test/java/com/linecorp/decaton/processor/runtime/internal/SubPartitionerTest.java b/processor/src/test/java/com/linecorp/decaton/processor/runtime/internal/SubPartitionerTest.java index 222904c8..25fc9a2e 100644 --- a/processor/src/test/java/com/linecorp/decaton/processor/runtime/internal/SubPartitionerTest.java +++ b/processor/src/test/java/com/linecorp/decaton/processor/runtime/internal/SubPartitionerTest.java @@ -19,6 +19,7 @@ import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertTrue; +import java.nio.charset.StandardCharsets; import java.util.ArrayList; import java.util.Arrays; import java.util.List; @@ -35,12 +36,12 @@ public class SubPartitionerTest { static final int[] SUBPARTITION_COUNTS = { 10, 17, 32, 64 }; static final double TARGET_STDDEV_RATIO = 0.05; - static final String[] keys = new String[DIST_KEYS_COUNT]; + static final byte[][] keys = new byte[DIST_KEYS_COUNT][]; @Before public void setUp() { for (int i = 0; i < keys.length; i++) { - keys[i] = String.valueOf(i); + keys[i] = String.valueOf(i).getBytes(StandardCharsets.UTF_8); } } @@ -54,14 +55,14 @@ private static double stddev(int[] counts) { @Test public void testEvenlyDistributedSelection() { for (int partitionCount : PARTITION_COUNTS) { - List> partitions = new ArrayList<>(partitionCount); + List> partitions = new ArrayList<>(partitionCount); for (int i = 0; i < partitionCount; i++) { partitions.add(new ArrayList<>()); } - for (String key : keys) { + for (byte[] key : keys) { // This is the way used to determine partition in Kafka's DefaultPartitioner - int partition = (Utils.murmur2(key.getBytes()) & 2147483647) % partitionCount; + int partition = (Utils.murmur2(key) & 2147483647) % partitionCount; partitions.get(partition).add(key); } @@ -71,10 +72,10 @@ public void testEvenlyDistributedSelection() { partitionCount, partStddev, Arrays.toString(partCounts)); for (int subpartitionCount : SUBPARTITION_COUNTS) { - for (List partition : partitions) { + for (List partition : partitions) { int[] counts = new int[subpartitionCount]; SubPartitioner subPartitioner = new SubPartitioner(counts.length); - for (String key : partition) { + for (byte[] key : partition) { int subPartition = subPartitioner.partitionFor(key); counts[subPartition]++; } @@ -96,7 +97,7 @@ public void testEvenlyDistributedSelection() { public void testConsistentSelectionForSameKeys() { for (int subpartitionCount : SUBPARTITION_COUNTS) { SubPartitioner subPartitioner = new SubPartitioner(subpartitionCount); - for (String key : keys) { + for (byte[] key : keys) { int assign1 = subPartitioner.partitionFor(key); int assign2 = subPartitioner.partitionFor(key); assertEquals(String.format("[%d] assign of %s", subpartitionCount, key), diff --git a/testing/src/main/java/com/linecorp/decaton/testing/TestUtils.java b/testing/src/main/java/com/linecorp/decaton/testing/TestUtils.java index 16c6b659..cd066261 100644 --- a/testing/src/main/java/com/linecorp/decaton/testing/TestUtils.java +++ b/testing/src/main/java/com/linecorp/decaton/testing/TestUtils.java @@ -29,27 +29,24 @@ import java.util.concurrent.locks.LockSupport; import java.util.function.BooleanSupplier; import java.util.function.Consumer; -import java.util.function.Supplier; -import com.linecorp.decaton.processor.runtime.ProcessorProperties; -import com.linecorp.decaton.processor.runtime.Property; -import com.linecorp.decaton.processor.runtime.PropertySupplier; -import com.linecorp.decaton.processor.runtime.StaticPropertySupplier; -import com.linecorp.decaton.processor.runtime.SubscriptionStateListener; import org.apache.kafka.clients.consumer.ConsumerConfig; import org.apache.kafka.clients.producer.KafkaProducer; import org.apache.kafka.clients.producer.Producer; import org.apache.kafka.clients.producer.ProducerConfig; +import org.apache.kafka.common.serialization.ByteArraySerializer; import com.google.protobuf.MessageLite; import com.linecorp.decaton.client.DecatonClient; -import com.linecorp.decaton.client.kafka.PrintableAsciiStringSerializer; import com.linecorp.decaton.client.kafka.ProtocolBuffersKafkaSerializer; import com.linecorp.decaton.common.Serializer; -import com.linecorp.decaton.processor.runtime.SubscriptionStateListener.State; import com.linecorp.decaton.processor.runtime.ProcessorSubscription; +import com.linecorp.decaton.processor.runtime.Property; +import com.linecorp.decaton.processor.runtime.StaticPropertySupplier; import com.linecorp.decaton.processor.runtime.SubscriptionBuilder; +import com.linecorp.decaton.processor.runtime.SubscriptionStateListener; +import com.linecorp.decaton.processor.runtime.SubscriptionStateListener.State; import com.linecorp.decaton.protobuf.ProtocolBuffersSerializer; import com.linecorp.decaton.protocol.Decaton.DecatonTaskRequest; @@ -112,9 +109,9 @@ public static DecatonClient client(String topic, * @param bootstrapServers bootstrap servers to connect * @return {@link Producer} instance with preset configurations */ - public static Producer producer(String bootstrapServers) { + public static Producer producer(String bootstrapServers) { return new KafkaProducer<>(defaultProducerProps(bootstrapServers), - new PrintableAsciiStringSerializer(), + new ByteArraySerializer(), new ProtocolBuffersKafkaSerializer<>()); } diff --git a/testing/src/main/java/com/linecorp/decaton/testing/processor/ProcessOrdering.java b/testing/src/main/java/com/linecorp/decaton/testing/processor/ProcessOrdering.java index a7607622..de5d15d7 100644 --- a/testing/src/main/java/com/linecorp/decaton/testing/processor/ProcessOrdering.java +++ b/testing/src/main/java/com/linecorp/decaton/testing/processor/ProcessOrdering.java @@ -21,6 +21,7 @@ import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; +import java.nio.ByteBuffer; import java.util.ArrayDeque; import java.util.ArrayList; import java.util.Deque; @@ -33,26 +34,26 @@ public class ProcessOrdering implements ProcessingGuarantee { private final Map taskToOffset = new HashMap<>(); - private final Map> producedRecords = new HashMap<>(); - private final Map> processedRecords = new HashMap<>(); + private final Map> producedRecords = new HashMap<>(); + private final Map> processedRecords = new HashMap<>(); @Override public synchronized void onProduce(ProducedRecord record) { taskToOffset.put(record.task(), record.offset()); - producedRecords.computeIfAbsent(record.key(), + producedRecords.computeIfAbsent(ByteBuffer.wrap(record.key()), key -> new ArrayList<>()).add(record.task()); } @Override public synchronized void onProcess(TaskMetadata metadata, ProcessedRecord record) { - processedRecords.computeIfAbsent(record.key(), + processedRecords.computeIfAbsent(ByteBuffer.wrap(record.key()), key -> new ArrayList<>()).add(record.task()); } @Override public void doAssert() { - for (Entry> entry : producedRecords.entrySet()) { - String key = entry.getKey(); + for (Entry> entry : producedRecords.entrySet()) { + final ByteBuffer key = entry.getKey(); List produced = entry.getValue(); List processed = processedRecords.get(key); diff --git a/testing/src/main/java/com/linecorp/decaton/testing/processor/ProcessedRecord.java b/testing/src/main/java/com/linecorp/decaton/testing/processor/ProcessedRecord.java index 6f15f831..8fad8907 100644 --- a/testing/src/main/java/com/linecorp/decaton/testing/processor/ProcessedRecord.java +++ b/testing/src/main/java/com/linecorp/decaton/testing/processor/ProcessedRecord.java @@ -30,7 +30,7 @@ public class ProcessedRecord { /** * Key of the task */ - String key; + byte[] key; /** * Processed task */ diff --git a/testing/src/main/java/com/linecorp/decaton/testing/processor/ProcessorTestSuite.java b/testing/src/main/java/com/linecorp/decaton/testing/processor/ProcessorTestSuite.java index 2bfc6663..78a48ffd 100644 --- a/testing/src/main/java/com/linecorp/decaton/testing/processor/ProcessorTestSuite.java +++ b/testing/src/main/java/com/linecorp/decaton/testing/processor/ProcessorTestSuite.java @@ -18,6 +18,7 @@ import static com.linecorp.decaton.testing.TestUtils.DEFINITELY_TOO_SLOW; +import java.nio.charset.StandardCharsets; import java.util.Arrays; import java.util.EnumSet; import java.util.HashMap; @@ -95,7 +96,7 @@ public class ProcessorTestSuite { private final Set semantics; private final SubscriptionStatesListener statesListener; private final TracingProvider tracingProvider; - private final Function> producerSupplier; + private final Function> producerSupplier; private final TaskExtractor customTaskExtractor; private static final int DEFAULT_NUM_TASKS = 10000; @@ -153,7 +154,7 @@ public static class Builder { * Expected use case: * supply a producer which adds tracing id to each message to test tracing-functionality in e2e */ - private Function> producerSupplier = TestUtils::producer; + private Function> producerSupplier = TestUtils::producer; /** * Supply custom {@link TaskExtractor} to be used to extract a task. */ @@ -221,7 +222,7 @@ public void run() throws InterruptedException, ExecutionException, TimeoutExcept CountDownLatch rollingRestartLatch = new CountDownLatch(numTasks / 2); ProcessorSubscription[] subscriptions = new ProcessorSubscription[NUM_SUBSCRIPTION_INSTANCES]; - try (Producer producer = producerSupplier.apply(rule.bootstrapServers())) { + try (Producer producer = producerSupplier.apply(rule.bootstrapServers())) { ConcurrentMap retryOffsets = new ConcurrentHashMap<>(); for (int i = 0; i < subscriptions.length; i++) { subscriptions[i] = newSubscription(i, topic, Optional.of(rollingRestartLatch), retryOffsets); @@ -351,7 +352,7 @@ private void awaitAllOffsetsCommitted(Map producedOffsets) * @return A CompletableFuture of Map, which holds partition as the key and max offset as the value */ private CompletableFuture> produceTasks( - Producer producer, + Producer producer, String topic, Consumer onProduce) { @SuppressWarnings("unchecked") @@ -360,7 +361,7 @@ private CompletableFuture> produceTasks( TestTaskSerializer serializer = new TestTaskSerializer(); for (int i = 0; i < produceFutures.length; i++) { TestTask task = new TestTask(String.valueOf(i)); - String key = String.valueOf(i % NUM_KEYS); + byte[] key = String.valueOf(i % NUM_KEYS).getBytes(StandardCharsets.UTF_8); TaskMetadataProto taskMetadata = TaskMetadataProto.newBuilder() .setTimestampMillis(System.currentTimeMillis()) @@ -372,7 +373,7 @@ private CompletableFuture> produceTasks( .setMetadata(taskMetadata) .setSerializedTask(ByteString.copyFrom(serializer.serialize(task))) .build(); - ProducerRecord record = + ProducerRecord record = new ProducerRecord<>(topic, null, taskMetadata.getTimestampMillis(), key, request); CompletableFuture future = new CompletableFuture<>(); produceFutures[i] = future; @@ -403,17 +404,17 @@ private CompletableFuture> produceTasks( }); } - private static class InterceptingProducer extends ProducerAdaptor { + private static class InterceptingProducer extends ProducerAdaptor { private final Consumer interceptor; - InterceptingProducer(Producer delegate, + InterceptingProducer(Producer delegate, Consumer interceptor) { super(delegate); this.interceptor = interceptor; } @Override - public Future send(ProducerRecord record, + public Future send(ProducerRecord record, Callback callback) { return super.send(record, (meta, e) -> { if (meta != null) { diff --git a/testing/src/main/java/com/linecorp/decaton/testing/processor/ProducedRecord.java b/testing/src/main/java/com/linecorp/decaton/testing/processor/ProducedRecord.java index 3dec4a55..75ed832a 100644 --- a/testing/src/main/java/com/linecorp/decaton/testing/processor/ProducedRecord.java +++ b/testing/src/main/java/com/linecorp/decaton/testing/processor/ProducedRecord.java @@ -31,7 +31,7 @@ public class ProducedRecord { /** * Key of the task */ - String key; + byte[] key; /** * Topic partition the record was sent to */ diff --git a/testing/src/main/java/com/linecorp/decaton/testing/processor/SerialProcessing.java b/testing/src/main/java/com/linecorp/decaton/testing/processor/SerialProcessing.java index ebd70131..149c72ec 100644 --- a/testing/src/main/java/com/linecorp/decaton/testing/processor/SerialProcessing.java +++ b/testing/src/main/java/com/linecorp/decaton/testing/processor/SerialProcessing.java @@ -19,6 +19,7 @@ import static org.hamcrest.MatcherAssert.assertThat; import static org.hamcrest.Matchers.lessThan; +import java.nio.ByteBuffer; import java.util.ArrayList; import java.util.Comparator; import java.util.HashMap; @@ -29,7 +30,7 @@ import com.linecorp.decaton.processor.TaskMetadata; public class SerialProcessing implements ProcessingGuarantee { - private final Map> records = new HashMap<>(); + private final Map> records = new HashMap<>(); @Override public void onProduce(ProducedRecord record) { @@ -38,14 +39,14 @@ public void onProduce(ProducedRecord record) { @Override public synchronized void onProcess(TaskMetadata metadata, ProcessedRecord record) { - records.computeIfAbsent(record.key(), + records.computeIfAbsent(ByteBuffer.wrap(record.key()), key -> new ArrayList<>()).add(record); } @Override public void doAssert() { // Checks there's no overlap between two consecutive records' processing time - for (Entry> entry : records.entrySet()) { + for (Entry> entry : records.entrySet()) { List perKeyRecords = entry.getValue(); perKeyRecords.sort(Comparator.comparingLong(ProcessedRecord::startTimeNanos)); diff --git a/testing/src/main/java/com/linecorp/decaton/testing/processor/TestTracingProducer.java b/testing/src/main/java/com/linecorp/decaton/testing/processor/TestTracingProducer.java index 96736c51..7f9cfd8d 100644 --- a/testing/src/main/java/com/linecorp/decaton/testing/processor/TestTracingProducer.java +++ b/testing/src/main/java/com/linecorp/decaton/testing/processor/TestTracingProducer.java @@ -29,12 +29,12 @@ import com.linecorp.decaton.processor.tracing.TestTracingProvider; import com.linecorp.decaton.protocol.Decaton.DecatonTaskRequest; -public class TestTracingProducer extends ProducerAdaptor { - public TestTracingProducer(Producer delegate) { +public class TestTracingProducer extends ProducerAdaptor { + public TestTracingProducer(Producer delegate) { super(delegate); } - private static void propagateCurrentTrace(ProducerRecord record) { + private static void propagateCurrentTrace(ProducerRecord record) { String traceId = TestTracingProvider.getCurrentTraceId(); if (null == traceId) { traceId = "trace-" + UUID.randomUUID(); @@ -45,13 +45,13 @@ private static void propagateCurrentTrace(ProducerRecord send(ProducerRecord record) { + public Future send(ProducerRecord record) { propagateCurrentTrace(record); return super.send(record); } @Override - public Future send(ProducerRecord record, Callback callback) { + public Future send(ProducerRecord record, Callback callback) { propagateCurrentTrace(record); return super.send(record, callback); } From 33d026e188525766ae4e27e4fa45a935075b513b Mon Sep 17 00:00:00 2001 From: Koki Kato Date: Mon, 13 Jun 2022 18:57:07 +0900 Subject: [PATCH 03/27] Use String for DecatonTask key in DecatonClient --- .../decaton/client/DecatonClient.java | 10 ++++---- .../client/internal/DecatonClientImpl.java | 17 ++++++++----- .../decaton/client/DecatonClientTest.java | 8 +++--- .../internal/DecatonClientImplTest.java | 25 +++++++++---------- .../decaton/processor/RateLimiterTest.java | 13 +++++----- 5 files changed, 38 insertions(+), 35 deletions(-) diff --git a/client/src/main/java/com/linecorp/decaton/client/DecatonClient.java b/client/src/main/java/com/linecorp/decaton/client/DecatonClient.java index b7a70999..9af75e69 100644 --- a/client/src/main/java/com/linecorp/decaton/client/DecatonClient.java +++ b/client/src/main/java/com/linecorp/decaton/client/DecatonClient.java @@ -38,7 +38,7 @@ public interface DecatonClient extends AutoCloseable { * * @return a {@link CompletableFuture} which represents the result of task put. */ - CompletableFuture put(byte[] key, T task); + CompletableFuture put(String key, T task); /** * Put a task onto associated decaton queue with specifying arbitrary timestamp. @@ -47,7 +47,7 @@ public interface DecatonClient extends AutoCloseable { * @param timestamp milliseconds precision timestamp which is to be used to set timestamp of task metadata. * @return a {@link CompletableFuture} which represents the result of task put. */ - CompletableFuture put(byte[] key, T task, long timestamp); + CompletableFuture put(String key, T task, long timestamp); /** * Put a task onto associated decaton queue with specifying some fields of task metadata. @@ -56,7 +56,7 @@ public interface DecatonClient extends AutoCloseable { * @param overrideTaskMetadata taskMetaData which can be set by users and used for event publish. * @return a {@link CompletableFuture} which represents the result of task put. */ - CompletableFuture put(byte[] key, T task, TaskMetadata overrideTaskMetadata); + CompletableFuture put(String key, T task, TaskMetadata overrideTaskMetadata); /** * Put a task onto associated decaton queue. @@ -77,7 +77,7 @@ public interface DecatonClient extends AutoCloseable { * * @return a {@link CompletableFuture} which represents the result of task put. */ - CompletableFuture put(byte[] key, T task, Consumer errorCallback); + CompletableFuture put(String key, T task, Consumer errorCallback); /** * Put a task onto associated decaton queue with specifying arbitrary timestamp. @@ -100,7 +100,7 @@ public interface DecatonClient extends AutoCloseable { * * @return a {@link CompletableFuture} which represents the result of task put. */ - default CompletableFuture put(byte[] key, T task, long timestamp, + default CompletableFuture put(String key, T task, long timestamp, Consumer errorCallback) { CompletableFuture result = put(key, task, timestamp); result.exceptionally(e -> { diff --git a/client/src/main/java/com/linecorp/decaton/client/internal/DecatonClientImpl.java b/client/src/main/java/com/linecorp/decaton/client/internal/DecatonClientImpl.java index f19b4205..72c54783 100644 --- a/client/src/main/java/com/linecorp/decaton/client/internal/DecatonClientImpl.java +++ b/client/src/main/java/com/linecorp/decaton/client/internal/DecatonClientImpl.java @@ -26,11 +26,14 @@ import com.linecorp.decaton.client.DecatonClient; import com.linecorp.decaton.client.KafkaProducerSupplier; import com.linecorp.decaton.client.PutTaskResult; +import com.linecorp.decaton.client.kafka.PrintableAsciiStringSerializer; import com.linecorp.decaton.common.Serializer; import com.linecorp.decaton.protocol.Decaton.DecatonTaskRequest; import com.linecorp.decaton.protocol.Decaton.TaskMetadataProto; public class DecatonClientImpl implements DecatonClient { + private static final org.apache.kafka.common.serialization.Serializer keySerializer = new PrintableAsciiStringSerializer(); + private final String topic; private final Serializer serializer; private final DecatonTaskProducer producer; private final String applicationId; @@ -44,6 +47,7 @@ public class DecatonClientImpl implements DecatonClient { Properties producerConfig, KafkaProducerSupplier producerSupplier, Supplier timestampSupplier) { + this.topic = topic; this.serializer = serializer; this.applicationId = applicationId; this.instanceId = instanceId; @@ -62,7 +66,7 @@ public DecatonClientImpl(String topic, } @Override - public CompletableFuture put(byte[] key, T task, long timestamp) { + public CompletableFuture put(String key, T task, long timestamp) { TaskMetadataProto taskMetadata = TaskMetadataProto.newBuilder() .setTimestampMillis(timestamp) .setSourceApplicationId(applicationId) @@ -73,17 +77,17 @@ public CompletableFuture put(byte[] key, T task, long timestamp) } @Override - public CompletableFuture put(byte[] key, T task, TaskMetadata overrideTaskMetadata) { + public CompletableFuture put(String key, T task, TaskMetadata overrideTaskMetadata) { return put(key, task, convertToTaskMetadataProto(overrideTaskMetadata)); } @Override - public CompletableFuture put(byte[] key, T task) { + public CompletableFuture put(String key, T task) { return put(key, task, timestampSupplier.get()); } @Override - public CompletableFuture put(byte[] key, T task, Consumer errorCallback) { + public CompletableFuture put(String key, T task, Consumer errorCallback) { return put(key, task, timestampSupplier.get(), errorCallback); } @@ -92,7 +96,8 @@ public void close() throws Exception { producer.close(); } - private CompletableFuture put(byte[] key, T task, TaskMetadataProto taskMetadataProto) { + private CompletableFuture put(String key, T task, TaskMetadataProto taskMetadataProto) { + byte[] serializedKey = keySerializer.serialize(topic, key); byte[] serializedTask = serializer.serialize(task); DecatonTaskRequest request = @@ -101,7 +106,7 @@ private CompletableFuture put(byte[] key, T task, TaskMetadataPro .setSerializedTask(ByteString.copyFrom(serializedTask)) .build(); - return producer.sendRequest(key, request); + return producer.sendRequest(serializedKey, request); } private TaskMetadataProto convertToTaskMetadataProto(TaskMetadata overrideTaskMetadata) { diff --git a/client/src/test/java/com/linecorp/decaton/client/DecatonClientTest.java b/client/src/test/java/com/linecorp/decaton/client/DecatonClientTest.java index f036d745..b0c1fe43 100644 --- a/client/src/test/java/com/linecorp/decaton/client/DecatonClientTest.java +++ b/client/src/test/java/com/linecorp/decaton/client/DecatonClientTest.java @@ -45,22 +45,22 @@ public class DecatonClientTest { @Spy private final DecatonClient decaton = new DecatonClient() { @Override - public CompletableFuture put(byte[] key, HelloTask task) { + public CompletableFuture put(String key, HelloTask task) { return null; } @Override - public CompletableFuture put(byte[] key, HelloTask task, long timestamp) { + public CompletableFuture put(String key, HelloTask task, long timestamp) { return null; } @Override - public CompletableFuture put(byte[] key, HelloTask task, TaskMetadata overrideTaskMetadata) { + public CompletableFuture put(String key, HelloTask task, TaskMetadata overrideTaskMetadata) { return null; } @Override - public CompletableFuture put(byte[] key, HelloTask task, + public CompletableFuture put(String key, HelloTask task, Consumer errorCallback) { return null; } diff --git a/client/src/test/java/com/linecorp/decaton/client/internal/DecatonClientImplTest.java b/client/src/test/java/com/linecorp/decaton/client/internal/DecatonClientImplTest.java index 81891f7f..1899a4d0 100644 --- a/client/src/test/java/com/linecorp/decaton/client/internal/DecatonClientImplTest.java +++ b/client/src/test/java/com/linecorp/decaton/client/internal/DecatonClientImplTest.java @@ -25,7 +25,6 @@ import static org.mockito.Mockito.times; import static org.mockito.Mockito.verify; -import java.nio.charset.StandardCharsets; import java.util.Properties; import java.util.function.Supplier; @@ -76,7 +75,7 @@ APPLICATION_ID, INSTANCE_ID, new Properties(), public void testTimestampFieldSetInternally() { doReturn(1234L).when(timestampSupplier).get(); - client.put("key".getBytes(StandardCharsets.UTF_8), HelloTask.getDefaultInstance()); + client.put("key", HelloTask.getDefaultInstance()); verify(producer, times(1)).send(captor.capture(), any(Callback.class)); ProducerRecord record = captor.getValue(); @@ -88,7 +87,7 @@ public void testTimestampFieldSetInternally() { public void testTimestampFieldSetInternallyWithCallback() { doReturn(1234L).when(timestampSupplier).get(); - client.put("key".getBytes(StandardCharsets.UTF_8), HelloTask.getDefaultInstance(), ignored -> {}); + client.put("key", HelloTask.getDefaultInstance(), ignored -> {}); verify(producer, times(1)).send(captor.capture(), any(Callback.class)); ProducerRecord record = captor.getValue(); @@ -100,7 +99,7 @@ public void testTimestampFieldSetInternallyWithCallback() { public void testTimestampFieldSetExternally() { doReturn(1234L).when(timestampSupplier).get(); - client.put("key".getBytes(StandardCharsets.UTF_8), HelloTask.getDefaultInstance(), 5678); + client.put("key", HelloTask.getDefaultInstance(), 5678); verify(producer, times(1)).send(captor.capture(), any(Callback.class)); ProducerRecord record = captor.getValue(); @@ -112,7 +111,7 @@ public void testTimestampFieldSetExternally() { public void testTimestampFieldSetExternallyWithCallback() { doReturn(1234L).when(timestampSupplier).get(); - client.put("key".getBytes(StandardCharsets.UTF_8), HelloTask.getDefaultInstance(), 5678, ignored -> {}); + client.put("key", HelloTask.getDefaultInstance(), 5678, ignored -> {}); verify(producer, times(1)).send(captor.capture(), any(Callback.class)); ProducerRecord record = captor.getValue(); @@ -124,10 +123,10 @@ public void testTimestampFieldSetExternallyWithCallback() { public void testTaskMetaDataSetExternally() { doReturn(1234L).when(timestampSupplier).get(); - client.put("key".getBytes(StandardCharsets.UTF_8), HelloTask.getDefaultInstance(), TaskMetadata.builder() - .timestamp(5678L) - .scheduledTime(6912L) - .build()); + client.put("key", HelloTask.getDefaultInstance(), TaskMetadata.builder() + .timestamp(5678L) + .scheduledTime(6912L) + .build()); verifyAndAssertTaskMetadata(5678L, 6912L); } @@ -136,9 +135,9 @@ public void testTaskMetaDataSetExternally() { public void testWithScheduledTimeSetExternally() { doReturn(1234L).when(timestampSupplier).get(); - client.put("key".getBytes(StandardCharsets.UTF_8), HelloTask.getDefaultInstance(), TaskMetadata.builder() - .scheduledTime(181234L) - .build()); + client.put("key", HelloTask.getDefaultInstance(), TaskMetadata.builder() + .scheduledTime(181234L) + .build()); verifyAndAssertTaskMetadata(1234L, 181234L); } @@ -147,7 +146,7 @@ public void testWithScheduledTimeSetExternally() { public void testWithEmptyTaskMetaDataSetExternally() { doReturn(1234L).when(timestampSupplier).get(); - client.put("key".getBytes(StandardCharsets.UTF_8), HelloTask.getDefaultInstance(), TaskMetadata.builder().build()); + client.put("key", HelloTask.getDefaultInstance(), TaskMetadata.builder().build()); verify(producer, times(1)).send(captor.capture(), any(Callback.class)); ProducerRecord record = captor.getValue(); diff --git a/processor/src/it/java/com/linecorp/decaton/processor/RateLimiterTest.java b/processor/src/it/java/com/linecorp/decaton/processor/RateLimiterTest.java index 7b5bde6f..ff3cf490 100644 --- a/processor/src/it/java/com/linecorp/decaton/processor/RateLimiterTest.java +++ b/processor/src/it/java/com/linecorp/decaton/processor/RateLimiterTest.java @@ -19,7 +19,6 @@ import static org.junit.Assert.assertEquals; import java.nio.ByteBuffer; -import java.nio.charset.StandardCharsets; import java.util.Collections; import java.util.HashSet; import java.util.Set; @@ -59,16 +58,16 @@ public void tearDown() { @Test(timeout = 30000) public void testPropertyDynamicSwitch() throws Exception { - Set keys = new HashSet<>(); + Set keys = new HashSet<>(); for (int i = 0; i < 10000; i++) { - keys.add(ByteBuffer.wrap(("key" + i).getBytes(StandardCharsets.UTF_8))); + keys.add("key" + i); } - Set processedKeys = Collections.synchronizedSet(new HashSet<>()); + Set processedKeys = Collections.synchronizedSet(new HashSet<>()); CountDownLatch processLatch = new CountDownLatch(keys.size()); DecatonProcessor processor = (context, task) -> { - processedKeys.add(context.key()); + processedKeys.add(ByteBuffer.wrap(context.key())); processLatch.countDown(); }; @@ -83,11 +82,11 @@ public void testPropertyDynamicSwitch() throws Exception { DecatonClient client = TestUtils.client(topicName, rule.bootstrapServers())) { int count = 0; - for (ByteBuffer key : keys) { + for (String key : keys) { if (++count % 1000 == 0) { rateProp.set((long) count / 10); } - client.put(key.array(), HelloTask.getDefaultInstance()); + client.put(key, HelloTask.getDefaultInstance()); } processLatch.await(); } From 198033e671ff0acfe8d705af04282d1d1f329502 Mon Sep 17 00:00:00 2001 From: Koki Kato Date: Fri, 17 Jun 2022 12:27:02 +0900 Subject: [PATCH 04/27] Remove TODO comment about eliminating array copy --- .../decaton/processor/runtime/internal/SubPartitioner.java | 1 - 1 file changed, 1 deletion(-) diff --git a/processor/src/main/java/com/linecorp/decaton/processor/runtime/internal/SubPartitioner.java b/processor/src/main/java/com/linecorp/decaton/processor/runtime/internal/SubPartitioner.java index 2128a404..44740c74 100644 --- a/processor/src/main/java/com/linecorp/decaton/processor/runtime/internal/SubPartitioner.java +++ b/processor/src/main/java/com/linecorp/decaton/processor/runtime/internal/SubPartitioner.java @@ -41,7 +41,6 @@ public int partitionFor(byte[] key) { // Here just by adding few bytes to the key we can "shift" hashing of the key and // can get back better distribution again in murmur2 result to evenly distribute keys // for subpartitions. - // TODO: Eliminate array copy here final ByteBuffer bb = ByteBuffer.allocate(key.length + 2); bb.put((byte) 's'); bb.put((byte) ':'); From 1dfb7a17cba70eb88dffcecfa0745bee66c3f86c Mon Sep 17 00:00:00 2001 From: Koki Kato Date: Fri, 17 Jun 2022 12:38:02 +0900 Subject: [PATCH 05/27] Add and use TestUtils.producer(String, Serializer, Serializer) --- .../processor/ArbitraryTopicTypeTest.java | 41 ++----------------- .../linecorp/decaton/testing/TestUtils.java | 22 +++++++++- 2 files changed, 23 insertions(+), 40 deletions(-) diff --git a/processor/src/it/java/com/linecorp/decaton/processor/ArbitraryTopicTypeTest.java b/processor/src/it/java/com/linecorp/decaton/processor/ArbitraryTopicTypeTest.java index acf4dac4..3626f827 100644 --- a/processor/src/it/java/com/linecorp/decaton/processor/ArbitraryTopicTypeTest.java +++ b/processor/src/it/java/com/linecorp/decaton/processor/ArbitraryTopicTypeTest.java @@ -17,14 +17,11 @@ package com.linecorp.decaton.processor; import java.time.Duration; -import java.util.Properties; -import java.util.concurrent.CompletableFuture; import java.util.concurrent.CountDownLatch; import java.util.function.Consumer; -import org.apache.kafka.clients.producer.KafkaProducer; +import org.apache.kafka.clients.producer.Producer; import org.apache.kafka.clients.producer.ProducerRecord; -import org.apache.kafka.clients.producer.RecordMetadata; import org.apache.kafka.common.serialization.Deserializer; import org.apache.kafka.common.serialization.LongDeserializer; import org.apache.kafka.common.serialization.LongSerializer; @@ -68,38 +65,6 @@ public void tearDown() { rule.admin().deleteTopics(true, topic, retryTopic); } - private static final class TestProducer implements AutoCloseable { - private final KafkaProducer producer; - private final String topic; - - private TestProducer(String bootstrapServers, String topic, Serializer keySerializer, Serializer valueSerializer) { - final Properties properties = TestUtils.defaultProducerProps(bootstrapServers); - - this.topic = topic; - producer = new KafkaProducer<>(properties, keySerializer, valueSerializer); - } - - public CompletableFuture put(K key, V value) { - final ProducerRecord record = new ProducerRecord<>(topic, key, value); - - final CompletableFuture result = new CompletableFuture<>(); - producer.send(record, (metadata, exception) -> { - if (exception == null) { - result.complete(metadata); - } else { - result.completeExceptionally(exception); - } - }); - - return result; - } - - @Override - public void close() throws Exception { - producer.close(); - } - } - private static final class TestTaskExtractor implements TaskExtractor { private final String topic; private final Deserializer deserializer; @@ -136,8 +101,8 @@ private void testRetryWithKeyValue( })).enableRetry(retryConfig); try (ProcessorSubscription subscription = TestUtils.subscription(rule.bootstrapServers(), builderConfigurer); - TestProducer producer = new TestProducer<>(rule.bootstrapServers(), topic, keySerializer, valueSerializer)) { - producer.put(key, value); + Producer producer = TestUtils.producer(rule.bootstrapServers(), keySerializer, valueSerializer)) { + producer.send(new ProducerRecord<>(topic, key, value)); processLatch.await(); } } diff --git a/testing/src/main/java/com/linecorp/decaton/testing/TestUtils.java b/testing/src/main/java/com/linecorp/decaton/testing/TestUtils.java index cd066261..cf7fd846 100644 --- a/testing/src/main/java/com/linecorp/decaton/testing/TestUtils.java +++ b/testing/src/main/java/com/linecorp/decaton/testing/TestUtils.java @@ -105,14 +105,32 @@ public static DecatonClient client(String topic, /** * A helper to instantiate {@link Producer} with preset configurations + * and default serializers * * @param bootstrapServers bootstrap servers to connect * @return {@link Producer} instance with preset configurations */ public static Producer producer(String bootstrapServers) { + return producer(bootstrapServers, + new ByteArraySerializer(), + new ProtocolBuffersKafkaSerializer<>()); + } + + /** + * A helper to instantiate {@link Producer} with preset configurations + * and custom key/value serializers + * + * @param bootstrapServers bootstrap servers to connect + * @param keySerializer serializer to serialize key into bytes + * @param valueSerializer serializer to serialize value into bytes + * @return {@link Producer} instance with preset configurations + */ + public static Producer producer(String bootstrapServers, + org.apache.kafka.common.serialization.Serializer keySerializer, + org.apache.kafka.common.serialization.Serializer valueSerializer) { return new KafkaProducer<>(defaultProducerProps(bootstrapServers), - new ByteArraySerializer(), - new ProtocolBuffersKafkaSerializer<>()); + keySerializer, + valueSerializer); } /** From d79abd2662000ccb36ebcf2b65e4153f70e95154 Mon Sep 17 00:00:00 2001 From: Koki Kato Date: Fri, 17 Jun 2022 12:46:05 +0900 Subject: [PATCH 06/27] Replace redundant ArbitraryTopicTypeTest test case --- .../processor/ArbitraryTopicTypeTest.java | 17 +++++++++-------- 1 file changed, 9 insertions(+), 8 deletions(-) diff --git a/processor/src/it/java/com/linecorp/decaton/processor/ArbitraryTopicTypeTest.java b/processor/src/it/java/com/linecorp/decaton/processor/ArbitraryTopicTypeTest.java index 3626f827..8e98e855 100644 --- a/processor/src/it/java/com/linecorp/decaton/processor/ArbitraryTopicTypeTest.java +++ b/processor/src/it/java/com/linecorp/decaton/processor/ArbitraryTopicTypeTest.java @@ -16,23 +16,24 @@ package com.linecorp.decaton.processor; +import java.nio.charset.StandardCharsets; import java.time.Duration; import java.util.concurrent.CountDownLatch; import java.util.function.Consumer; import org.apache.kafka.clients.producer.Producer; import org.apache.kafka.clients.producer.ProducerRecord; +import org.apache.kafka.common.serialization.ByteArrayDeserializer; +import org.apache.kafka.common.serialization.ByteArraySerializer; import org.apache.kafka.common.serialization.Deserializer; import org.apache.kafka.common.serialization.LongDeserializer; import org.apache.kafka.common.serialization.LongSerializer; import org.apache.kafka.common.serialization.Serializer; -import org.apache.kafka.common.serialization.StringDeserializer; import org.junit.After; import org.junit.Before; import org.junit.ClassRule; import org.junit.Test; -import com.linecorp.decaton.client.kafka.PrintableAsciiStringSerializer; import com.linecorp.decaton.processor.runtime.DecatonTask; import com.linecorp.decaton.processor.runtime.ProcessorSubscription; import com.linecorp.decaton.processor.runtime.ProcessorsBuilder; @@ -108,13 +109,13 @@ private void testRetryWithKeyValue( } @Test(timeout = 30000) - public void testPrintableAsciiStringKeyValue() throws Exception { + public void testBytesKeyValue() throws Exception { testRetryWithKeyValue( - new PrintableAsciiStringSerializer(), - "abc", - new PrintableAsciiStringSerializer(), - new StringDeserializer(), - "value" + new ByteArraySerializer(), + "key".getBytes(StandardCharsets.UTF_8), + new ByteArraySerializer(), + new ByteArrayDeserializer(), + "value".getBytes(StandardCharsets.UTF_8) ); } From ae91f5642a7fa8443b83d1a8044fc268c46ff959 Mon Sep 17 00:00:00 2001 From: Koki Kato Date: Fri, 17 Jun 2022 12:57:54 +0900 Subject: [PATCH 07/27] Clarify only String keys are supported by CONFIG_IGNORE_KEYS --- .../decaton/processor/runtime/ProcessorProperties.java | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/processor/src/main/java/com/linecorp/decaton/processor/runtime/ProcessorProperties.java b/processor/src/main/java/com/linecorp/decaton/processor/runtime/ProcessorProperties.java index 7d4609ca..94fe34ff 100644 --- a/processor/src/main/java/com/linecorp/decaton/processor/runtime/ProcessorProperties.java +++ b/processor/src/main/java/com/linecorp/decaton/processor/runtime/ProcessorProperties.java @@ -43,6 +43,10 @@ public class ProcessorProperties extends AbstractDecatonProperties { /** * List of keys of task to skip processing. * + * Note that this property accepts only String keys, while Decaton consumer supports consuming + * keys of arbitrary type. This means that records with non-String keys may just pass through + * this filter. + * * Reloadable: yes */ public static final PropertyDefinition> CONFIG_IGNORE_KEYS = From f99151016652001a44be3595af2834fc95b18e21 Mon Sep 17 00:00:00 2001 From: Koki Kato Date: Mon, 20 Jun 2022 16:26:41 +0900 Subject: [PATCH 08/27] Display byte keys with a given key formatter For LoggingContext to print record keys in bytes as human-readable String, this commit introduces a new interface `KeyFormatter` that translates keys from bytes into `String`. This interface has a canonical implementation that reads the byte key as a UTF-8 byte sequence. For other places, this commit just prevents the byte keys from being printed, or prints it as String if it's inside a test case. --- .../decaton/processor/LoggingContext.java | 9 ++++--- .../processor/formatter/KeyFormatter.java | 26 +++++++++++++++++++ .../processor/runtime/ProcessorsBuilder.java | 14 +++++++++- .../runtime/internal/ProcessPipeline.java | 19 +++++++++----- .../internal/ProcessingContextImpl.java | 12 ++++++--- .../runtime/internal/Processors.java | 9 +++++-- .../runtime/internal/TaskRequest.java | 1 + .../processors/CompactionProcessorTest.java | 3 ++- .../runtime/internal/ProcessPipelineTest.java | 4 ++- .../internal/ProcessingContextImplTest.java | 7 ++--- .../runtime/internal/ProcessorsTest.java | 3 ++- .../testing/processor/ProcessedRecord.java | 2 ++ .../testing/processor/SerialProcessing.java | 4 ++- 13 files changed, 88 insertions(+), 25 deletions(-) create mode 100644 processor/src/main/java/com/linecorp/decaton/processor/formatter/KeyFormatter.java diff --git a/processor/src/main/java/com/linecorp/decaton/processor/LoggingContext.java b/processor/src/main/java/com/linecorp/decaton/processor/LoggingContext.java index 7e9b74f4..a182ad88 100644 --- a/processor/src/main/java/com/linecorp/decaton/processor/LoggingContext.java +++ b/processor/src/main/java/com/linecorp/decaton/processor/LoggingContext.java @@ -18,6 +18,7 @@ import org.slf4j.MDC; +import com.linecorp.decaton.processor.formatter.KeyFormatter; import com.linecorp.decaton.processor.runtime.internal.TaskRequest; /** @@ -40,11 +41,11 @@ public class LoggingContext implements AutoCloseable { private final boolean enabled; - public LoggingContext(boolean enabled, String subscriptionId, TaskRequest request, TaskMetadata metadata) { + public LoggingContext(boolean enabled, String subscriptionId, TaskRequest request, TaskMetadata metadata, KeyFormatter keyFormatter) { this.enabled = enabled; if (enabled) { MDC.put(METADATA_KEY, metadata.toString()); - MDC.put(TASK_KEY, String.valueOf(request.key())); + MDC.put(TASK_KEY, keyFormatter.format(request.key())); MDC.put(SUBSCRIPTION_ID_KEY, subscriptionId); MDC.put(OFFSET_KEY, String.valueOf(request.recordOffset())); MDC.put(TOPIC_KEY, request.topicPartition().topic()); @@ -52,8 +53,8 @@ public LoggingContext(boolean enabled, String subscriptionId, TaskRequest reques } } - public LoggingContext(String subscriptionId, TaskRequest request, TaskMetadata metadata) { - this(true, subscriptionId, request, metadata); + public LoggingContext(String subscriptionId, TaskRequest request, TaskMetadata metadata, KeyFormatter keyFormatter) { + this(true, subscriptionId, request, metadata, keyFormatter); } @Override diff --git a/processor/src/main/java/com/linecorp/decaton/processor/formatter/KeyFormatter.java b/processor/src/main/java/com/linecorp/decaton/processor/formatter/KeyFormatter.java new file mode 100644 index 00000000..4fea5856 --- /dev/null +++ b/processor/src/main/java/com/linecorp/decaton/processor/formatter/KeyFormatter.java @@ -0,0 +1,26 @@ +/* + * Copyright 2020 LINE Corporation + * + * LINE Corporation licenses this file to you under the Apache License, + * version 2.0 (the "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at: + * + * https://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the + * License for the specific language governing permissions and limitations + * under the License. + */ + +package com.linecorp.decaton.processor.formatter; + +import java.nio.charset.StandardCharsets; + +@FunctionalInterface +public interface KeyFormatter { + KeyFormatter DEFAULT = key -> new String(key, StandardCharsets.UTF_8); + + String format(byte[] key); +} diff --git a/processor/src/main/java/com/linecorp/decaton/processor/runtime/ProcessorsBuilder.java b/processor/src/main/java/com/linecorp/decaton/processor/runtime/ProcessorsBuilder.java index be18d249..c843b467 100644 --- a/processor/src/main/java/com/linecorp/decaton/processor/runtime/ProcessorsBuilder.java +++ b/processor/src/main/java/com/linecorp/decaton/processor/runtime/ProcessorsBuilder.java @@ -22,6 +22,7 @@ import com.linecorp.decaton.common.Deserializer; import com.linecorp.decaton.processor.DecatonProcessor; +import com.linecorp.decaton.processor.formatter.KeyFormatter; import com.linecorp.decaton.processor.runtime.internal.DecatonProcessorSupplierImpl; import com.linecorp.decaton.processor.runtime.internal.DefaultTaskExtractor; import com.linecorp.decaton.processor.runtime.internal.Processors; @@ -42,6 +43,7 @@ public class ProcessorsBuilder { private final TaskExtractor retryTaskExtractor; private final List> suppliers; + private KeyFormatter keyFormatter = KeyFormatter.DEFAULT; public ProcessorsBuilder(String topic, TaskExtractor taskExtractor, TaskExtractor retryTaskExtractor) { this.topic = topic; @@ -130,7 +132,17 @@ public ProcessorsBuilder thenProcess(DecatonProcessor processor) { return thenProcess(new DecatonProcessorSupplierImpl<>(() -> processor, ProcessorScope.PROVIDED)); } + /** + * Configure a key formatter that translates record keys from bytes into String for logging purpose. + * @param keyFormatter a {@link KeyFormatter}, function that maps bytes into String + * @return updated instance of {@link ProcessorsBuilder}. + */ + public ProcessorsBuilder setKeyFormatter(KeyFormatter keyFormatter) { + this.keyFormatter = keyFormatter; + return this; + } + public Processors build(DecatonProcessorSupplier retryProcessorSupplier) { - return new Processors<>(suppliers, retryProcessorSupplier, taskExtractor, retryTaskExtractor); + return new Processors<>(suppliers, retryProcessorSupplier, taskExtractor, retryTaskExtractor, keyFormatter); } } diff --git a/processor/src/main/java/com/linecorp/decaton/processor/runtime/internal/ProcessPipeline.java b/processor/src/main/java/com/linecorp/decaton/processor/runtime/internal/ProcessPipeline.java index 740adcba..4d3c8385 100644 --- a/processor/src/main/java/com/linecorp/decaton/processor/runtime/internal/ProcessPipeline.java +++ b/processor/src/main/java/com/linecorp/decaton/processor/runtime/internal/ProcessPipeline.java @@ -21,16 +21,17 @@ import java.util.Collections; import java.util.List; -import com.linecorp.decaton.processor.DecatonProcessor; import com.linecorp.decaton.processor.Completion; -import com.linecorp.decaton.processor.runtime.DecatonTask; +import com.linecorp.decaton.processor.DecatonProcessor; +import com.linecorp.decaton.processor.LoggingContext; import com.linecorp.decaton.processor.ProcessingContext; -import com.linecorp.decaton.processor.runtime.ProcessorProperties; -import com.linecorp.decaton.processor.runtime.TaskExtractor; +import com.linecorp.decaton.processor.formatter.KeyFormatter; import com.linecorp.decaton.processor.metrics.Metrics; import com.linecorp.decaton.processor.metrics.Metrics.ProcessMetrics; import com.linecorp.decaton.processor.metrics.Metrics.TaskMetrics; -import com.linecorp.decaton.processor.LoggingContext; +import com.linecorp.decaton.processor.runtime.DecatonTask; +import com.linecorp.decaton.processor.runtime.ProcessorProperties; +import com.linecorp.decaton.processor.runtime.TaskExtractor; import com.linecorp.decaton.processor.runtime.internal.Utils.Timer; import lombok.extern.slf4j.Slf4j; @@ -41,6 +42,7 @@ public class ProcessPipeline implements AutoCloseable { private final List> processors; private final DecatonProcessor retryProcessor; private final TaskExtractor taskExtractor; + private final KeyFormatter keyFormatter; private final ExecutionScheduler scheduler; private final TaskMetrics taskMetrics; private final ProcessMetrics processMetrics; @@ -51,6 +53,7 @@ public class ProcessPipeline implements AutoCloseable { List> processors, DecatonProcessor retryProcessor, TaskExtractor taskExtractor, + KeyFormatter keyFormatter, ExecutionScheduler scheduler, Metrics metrics, Clock clock) { @@ -58,6 +61,7 @@ public class ProcessPipeline implements AutoCloseable { this.processors = Collections.unmodifiableList(processors); this.retryProcessor = retryProcessor; this.taskExtractor = taskExtractor; + this.keyFormatter = keyFormatter; this.scheduler = scheduler; this.clock = clock; @@ -69,9 +73,10 @@ public ProcessPipeline(ThreadScope scope, List> processors, DecatonProcessor retryProcessor, TaskExtractor taskExtractor, + KeyFormatter keyFormatter, ExecutionScheduler scheduler, Metrics metrics) { - this(scope, processors, retryProcessor, taskExtractor, scheduler, metrics, Clock.systemDefaultZone()); + this(scope, processors, retryProcessor, taskExtractor, keyFormatter, scheduler, metrics, Clock.systemDefaultZone()); } public void scheduleThenProcess(TaskRequest request) throws InterruptedException { @@ -119,7 +124,7 @@ DecatonTask extract(TaskRequest request) { Completion process(TaskRequest request, DecatonTask task) throws InterruptedException { ProcessingContext context = new ProcessingContextImpl<>(scope.subscriptionId(), request, task, processors, retryProcessor, - scope.props()); + keyFormatter, scope.props()); Timer timer = Utils.timer(); Completion processResult; diff --git a/processor/src/main/java/com/linecorp/decaton/processor/runtime/internal/ProcessingContextImpl.java b/processor/src/main/java/com/linecorp/decaton/processor/runtime/internal/ProcessingContextImpl.java index 89cc8402..47dacc23 100644 --- a/processor/src/main/java/com/linecorp/decaton/processor/runtime/internal/ProcessingContextImpl.java +++ b/processor/src/main/java/com/linecorp/decaton/processor/runtime/internal/ProcessingContextImpl.java @@ -23,12 +23,13 @@ import org.apache.kafka.common.header.Headers; +import com.linecorp.decaton.processor.Completion; +import com.linecorp.decaton.processor.Completion.TimeoutChoice; import com.linecorp.decaton.processor.DecatonProcessor; import com.linecorp.decaton.processor.LoggingContext; import com.linecorp.decaton.processor.ProcessingContext; import com.linecorp.decaton.processor.TaskMetadata; -import com.linecorp.decaton.processor.Completion; -import com.linecorp.decaton.processor.Completion.TimeoutChoice; +import com.linecorp.decaton.processor.formatter.KeyFormatter; import com.linecorp.decaton.processor.runtime.DecatonTask; import com.linecorp.decaton.processor.runtime.ProcessorProperties; import com.linecorp.decaton.processor.tracing.TracingProvider.ProcessorTraceHandle; @@ -44,6 +45,7 @@ public class ProcessingContextImpl implements ProcessingContext { private final DecatonTask task; private final List> downstreams; private final DecatonProcessor retryQueueingProcessor; + private final KeyFormatter keyFormatter; private final ProcessorProperties props; private final AtomicReference deferredCompletion; @@ -52,12 +54,14 @@ public ProcessingContextImpl(String subscriptionId, DecatonTask task, List> downstreams, DecatonProcessor retryQueueingProcessor, + KeyFormatter keyFormatter, ProcessorProperties props) { this.subscriptionId = subscriptionId; this.request = request; this.task = task; this.downstreams = Collections.unmodifiableList(downstreams); this.retryQueueingProcessor = retryQueueingProcessor; + this.keyFormatter = keyFormatter; this.props = props; deferredCompletion = new AtomicReference<>(); } @@ -80,7 +84,7 @@ public Headers headers() { @Override public LoggingContext loggingContext() { boolean enabled = props.get(ProcessorProperties.CONFIG_LOGGING_MDC_ENABLED).value(); - return new LoggingContext(enabled, subscriptionId, request, task.metadata()); + return new LoggingContext(enabled, subscriptionId, request, task.metadata(), keyFormatter); } @Override @@ -114,7 +118,7 @@ private

Completion pushDownStream(List> downstreams, P t "Exception from tracing", NoopTrace.INSTANCE); ProcessingContextImpl

nextContext = new ProcessingContextImpl<>( subscriptionId, request, task, downstreams.subList(1, downstreams.size()), - retryQueueingProcessor, props); + retryQueueingProcessor, keyFormatter, props); CompletionImpl completion; try { diff --git a/processor/src/main/java/com/linecorp/decaton/processor/runtime/internal/Processors.java b/processor/src/main/java/com/linecorp/decaton/processor/runtime/internal/Processors.java index da2aabe7..a5e63ae4 100644 --- a/processor/src/main/java/com/linecorp/decaton/processor/runtime/internal/Processors.java +++ b/processor/src/main/java/com/linecorp/decaton/processor/runtime/internal/Processors.java @@ -18,6 +18,7 @@ import java.util.Collections; import java.util.List; +import java.util.function.Function; import java.util.stream.Collectors; import org.apache.kafka.common.TopicPartition; @@ -25,6 +26,7 @@ import org.slf4j.LoggerFactory; import com.linecorp.decaton.processor.DecatonProcessor; +import com.linecorp.decaton.processor.formatter.KeyFormatter; import com.linecorp.decaton.processor.runtime.TaskExtractor; import com.linecorp.decaton.processor.metrics.Metrics; import com.linecorp.decaton.processor.runtime.DecatonProcessorSupplier; @@ -36,15 +38,18 @@ public class Processors { private final DecatonProcessorSupplier retryProcessorSupplier; private final TaskExtractor taskExtractor; private final TaskExtractor retryTaskExtractor; + private final KeyFormatter keyFormatter; public Processors(List> suppliers, DecatonProcessorSupplier retryProcessorSupplier, TaskExtractor taskExtractor, - TaskExtractor retryTaskExtractor) { + TaskExtractor retryTaskExtractor, + KeyFormatter keyFormatter) { this.suppliers = Collections.unmodifiableList(suppliers); this.retryProcessorSupplier = retryProcessorSupplier; this.taskExtractor = taskExtractor; this.retryTaskExtractor = retryTaskExtractor; + this.keyFormatter = keyFormatter; } private DecatonProcessor retryProcessor(ThreadScope scope) { @@ -78,7 +83,7 @@ public ProcessPipeline newPipeline(ThreadScope scope, scope.threadId())) .collect(Collectors.toList()); logger.info("Creating partition processor core: {}", scope); - return new ProcessPipeline<>(scope, processors, retryProcessor, taskExtractor, scheduler, metrics); + return new ProcessPipeline<>(scope, processors, retryProcessor, taskExtractor, keyFormatter, scheduler, metrics); } catch (RuntimeException e) { // If exception occurred in the middle of instantiating processors, we have to make sure // all the previously created processors are destroyed before bubbling up the exception. diff --git a/processor/src/main/java/com/linecorp/decaton/processor/runtime/internal/TaskRequest.java b/processor/src/main/java/com/linecorp/decaton/processor/runtime/internal/TaskRequest.java index 64a013e7..4cc195cb 100644 --- a/processor/src/main/java/com/linecorp/decaton/processor/runtime/internal/TaskRequest.java +++ b/processor/src/main/java/com/linecorp/decaton/processor/runtime/internal/TaskRequest.java @@ -32,6 +32,7 @@ public class TaskRequest { private final TopicPartition topicPartition; private final long recordOffset; private final OffsetState offsetState; + @ToString.Exclude private final byte[] key; @ToString.Exclude private final Headers headers; diff --git a/processor/src/test/java/com/linecorp/decaton/processor/processors/CompactionProcessorTest.java b/processor/src/test/java/com/linecorp/decaton/processor/processors/CompactionProcessorTest.java index 44ee2134..9ec4a0a2 100644 --- a/processor/src/test/java/com/linecorp/decaton/processor/processors/CompactionProcessorTest.java +++ b/processor/src/test/java/com/linecorp/decaton/processor/processors/CompactionProcessorTest.java @@ -49,6 +49,7 @@ import com.linecorp.decaton.processor.DecatonProcessor; import com.linecorp.decaton.processor.ProcessingContext; import com.linecorp.decaton.processor.TaskMetadata; +import com.linecorp.decaton.processor.formatter.KeyFormatter; import com.linecorp.decaton.processor.processors.CompactionProcessor.CompactChoice; import com.linecorp.decaton.processor.processors.CompactionProcessor.CompactingTask; import com.linecorp.decaton.processor.runtime.DecatonTask; @@ -106,7 +107,7 @@ private TaskInput put(DecatonProcessor processor, ProcessingContext context = spy(new ProcessingContextImpl<>("subscription", request, task, Arrays.asList(processor, downstream), null, - ProcessorProperties.builder().build())); + KeyFormatter.DEFAULT, ProcessorProperties.builder().build())); if (beforeProcess != null) { beforeProcess.accept(taskData, context); diff --git a/processor/src/test/java/com/linecorp/decaton/processor/runtime/internal/ProcessPipelineTest.java b/processor/src/test/java/com/linecorp/decaton/processor/runtime/internal/ProcessPipelineTest.java index 07f67473..8bd03b6a 100644 --- a/processor/src/test/java/com/linecorp/decaton/processor/runtime/internal/ProcessPipelineTest.java +++ b/processor/src/test/java/com/linecorp/decaton/processor/runtime/internal/ProcessPipelineTest.java @@ -53,6 +53,7 @@ import com.linecorp.decaton.processor.DeferredCompletion; import com.linecorp.decaton.processor.ProcessingContext; import com.linecorp.decaton.processor.TaskMetadata; +import com.linecorp.decaton.processor.formatter.KeyFormatter; import com.linecorp.decaton.processor.metrics.Metrics; import com.linecorp.decaton.processor.runtime.DecatonTask; import com.linecorp.decaton.processor.runtime.DynamicProperty; @@ -118,7 +119,8 @@ public void setUp() { completionTimeoutMsProp.set(100L); doReturn(10L).when(clock).millis(); pipeline = spy(new ProcessPipeline<>( - scope, Collections.singletonList(processorMock), null, extractorMock, schedulerMock, METRICS, clock)); + scope, Collections.singletonList(processorMock), null, extractorMock, KeyFormatter.DEFAULT, + schedulerMock, METRICS, clock)); } @Test diff --git a/processor/src/test/java/com/linecorp/decaton/processor/runtime/internal/ProcessingContextImplTest.java b/processor/src/test/java/com/linecorp/decaton/processor/runtime/internal/ProcessingContextImplTest.java index d40f616a..30064216 100644 --- a/processor/src/test/java/com/linecorp/decaton/processor/runtime/internal/ProcessingContextImplTest.java +++ b/processor/src/test/java/com/linecorp/decaton/processor/runtime/internal/ProcessingContextImplTest.java @@ -56,6 +56,7 @@ import com.linecorp.decaton.processor.TaskMetadata; import com.linecorp.decaton.processor.Completion; import com.linecorp.decaton.processor.Completion.TimeoutChoice; +import com.linecorp.decaton.processor.formatter.KeyFormatter; import com.linecorp.decaton.processor.runtime.DecatonTask; import com.linecorp.decaton.processor.runtime.ProcessorProperties; import com.linecorp.decaton.processor.tracing.TestTraceHandle; @@ -116,7 +117,7 @@ private static ProcessingContextImpl context(RecordTraceHandle traceH DecatonTask task = new DecatonTask<>( TaskMetadata.fromProto(REQUEST.getMetadata()), TASK, TASK.toByteArray()); return new ProcessingContextImpl<>("subscription", request, task, Arrays.asList(processors), - null, ProcessorProperties.builder().build()); + null, KeyFormatter.DEFAULT, ProcessorProperties.builder().build()); } private static void safeAwait(CountDownLatch latch) { @@ -368,7 +369,7 @@ public void process(ProcessingContext context, byte[] task) ProcessingContextImpl context = spy(new ProcessingContextImpl<>("subscription", request, task, - Collections.emptyList(), retryProcessor, + Collections.emptyList(), retryProcessor, KeyFormatter.DEFAULT, ProcessorProperties.builder().build())); Completion retryComp = context.retry(); @@ -421,7 +422,7 @@ public void process(ProcessingContext context, byte[] task) ProcessingContextImpl context = spy(new ProcessingContextImpl<>("subscription", request, task, - Arrays.asList(processor), retryProcessor, + Arrays.asList(processor), retryProcessor, KeyFormatter.DEFAULT, ProcessorProperties.builder().build())); Completion comp = context.push(new byte[0]); diff --git a/processor/src/test/java/com/linecorp/decaton/processor/runtime/internal/ProcessorsTest.java b/processor/src/test/java/com/linecorp/decaton/processor/runtime/internal/ProcessorsTest.java index 9f6ab6b7..04636d40 100644 --- a/processor/src/test/java/com/linecorp/decaton/processor/runtime/internal/ProcessorsTest.java +++ b/processor/src/test/java/com/linecorp/decaton/processor/runtime/internal/ProcessorsTest.java @@ -35,6 +35,7 @@ import org.mockito.junit.MockitoJUnit; import org.mockito.junit.MockitoRule; +import com.linecorp.decaton.processor.formatter.KeyFormatter; import com.linecorp.decaton.processor.runtime.ProcessorProperties; import com.linecorp.decaton.processor.runtime.DecatonProcessorSupplier; import com.linecorp.decaton.processor.tracing.internal.NoopTracingProvider; @@ -69,7 +70,7 @@ public void testCleanupPartiallyInitializedProcessors() { Processors processors = new Processors<>( suppliers, null, new DefaultTaskExtractor<>(bytes -> HelloTask.getDefaultInstance()), - null); + null, KeyFormatter.DEFAULT); doThrow(new RuntimeException("exception")).when(suppliers.get(2)).getProcessor(any(), any(), anyInt()); diff --git a/testing/src/main/java/com/linecorp/decaton/testing/processor/ProcessedRecord.java b/testing/src/main/java/com/linecorp/decaton/testing/processor/ProcessedRecord.java index 8fad8907..9e44539f 100644 --- a/testing/src/main/java/com/linecorp/decaton/testing/processor/ProcessedRecord.java +++ b/testing/src/main/java/com/linecorp/decaton/testing/processor/ProcessedRecord.java @@ -18,6 +18,7 @@ import com.linecorp.decaton.processor.DecatonProcessor; +import lombok.ToString; import lombok.Value; import lombok.experimental.Accessors; @@ -30,6 +31,7 @@ public class ProcessedRecord { /** * Key of the task */ + @ToString.Exclude byte[] key; /** * Processed task diff --git a/testing/src/main/java/com/linecorp/decaton/testing/processor/SerialProcessing.java b/testing/src/main/java/com/linecorp/decaton/testing/processor/SerialProcessing.java index 149c72ec..b8c2cdd3 100644 --- a/testing/src/main/java/com/linecorp/decaton/testing/processor/SerialProcessing.java +++ b/testing/src/main/java/com/linecorp/decaton/testing/processor/SerialProcessing.java @@ -20,6 +20,7 @@ import static org.hamcrest.Matchers.lessThan; import java.nio.ByteBuffer; +import java.nio.charset.StandardCharsets; import java.util.ArrayList; import java.util.Comparator; import java.util.HashMap; @@ -54,7 +55,8 @@ public void doAssert() { ProcessedRecord prev = perKeyRecords.get(i - 1); ProcessedRecord current = perKeyRecords.get(i); - assertThat("Process time shouldn't overlap. key: " + entry.getKey(), + String key = StandardCharsets.UTF_8.decode(entry.getKey()).toString(); + assertThat("Process time shouldn't overlap. key: " + key, prev.endTimeNanos(), lessThan(current.startTimeNanos())); } } From ec75b24010334c389c09455c59c79ac6ba898542 Mon Sep 17 00:00:00 2001 From: Koki Kato Date: Mon, 20 Jun 2022 16:31:55 +0900 Subject: [PATCH 09/27] Add final to KafkaClusterRule field in ArbitraryTopicTypeTest --- .../com/linecorp/decaton/processor/ArbitraryTopicTypeTest.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/processor/src/it/java/com/linecorp/decaton/processor/ArbitraryTopicTypeTest.java b/processor/src/it/java/com/linecorp/decaton/processor/ArbitraryTopicTypeTest.java index 8e98e855..60d183c0 100644 --- a/processor/src/it/java/com/linecorp/decaton/processor/ArbitraryTopicTypeTest.java +++ b/processor/src/it/java/com/linecorp/decaton/processor/ArbitraryTopicTypeTest.java @@ -50,7 +50,7 @@ */ public class ArbitraryTopicTypeTest { @ClassRule - public static KafkaClusterRule rule = new KafkaClusterRule(); + public static final KafkaClusterRule rule = new KafkaClusterRule(); private String topic; private String retryTopic; From 8c90fe593e52823371d2260278bb6b48e889b314 Mon Sep 17 00:00:00 2001 From: Koki Kato Date: Mon, 20 Jun 2022 17:32:54 +0900 Subject: [PATCH 10/27] Introduce TaskKey; a wrapper for keys in bytes --- .../processor/CoreFunctionalityTest.java | 10 ++-- .../decaton/processor/RateLimiterTest.java | 6 +- .../decaton/processor/ProcessingContext.java | 3 +- .../processor/formatter/KeyFormatter.java | 6 +- .../processors/CompactionProcessor.java | 21 ++++--- .../runtime/ProcessorSubscription.java | 3 +- .../DecatonTaskRetryQueueingProcessor.java | 2 +- .../internal/ProcessingContextImpl.java | 2 +- .../runtime/internal/SubPartitioner.java | 7 ++- .../processor/runtime/internal/TaskKey.java | 57 +++++++++++++++++++ .../runtime/internal/TaskRequest.java | 5 +- .../processors/CompactionProcessorTest.java | 3 +- ...DecatonTaskRetryQueueingProcessorTest.java | 6 +- .../runtime/internal/ProcessPipelineTest.java | 3 +- .../internal/ProcessingContextImplTest.java | 6 +- .../runtime/internal/SubPartitionerTest.java | 18 +++--- .../testing/processor/ProcessOrdering.java | 15 +++-- .../testing/processor/ProcessedRecord.java | 5 +- .../testing/processor/ProcessorTestSuite.java | 7 ++- .../testing/processor/ProducedRecord.java | 4 +- .../testing/processor/SerialProcessing.java | 13 ++--- 21 files changed, 131 insertions(+), 71 deletions(-) create mode 100644 processor/src/main/java/com/linecorp/decaton/processor/runtime/internal/TaskKey.java diff --git a/processor/src/it/java/com/linecorp/decaton/processor/CoreFunctionalityTest.java b/processor/src/it/java/com/linecorp/decaton/processor/CoreFunctionalityTest.java index 416bec65..dc0acfe4 100644 --- a/processor/src/it/java/com/linecorp/decaton/processor/CoreFunctionalityTest.java +++ b/processor/src/it/java/com/linecorp/decaton/processor/CoreFunctionalityTest.java @@ -18,7 +18,6 @@ import static org.junit.Assert.assertTrue; -import java.nio.ByteBuffer; import java.util.ArrayList; import java.util.HashMap; import java.util.List; @@ -35,6 +34,7 @@ import com.linecorp.decaton.processor.runtime.ProcessorScope; import com.linecorp.decaton.processor.runtime.Property; import com.linecorp.decaton.processor.runtime.StaticPropertySupplier; +import com.linecorp.decaton.processor.runtime.internal.TaskKey; import com.linecorp.decaton.testing.KafkaClusterRule; import com.linecorp.decaton.testing.RandomRule; import com.linecorp.decaton.testing.processor.ProcessedRecord; @@ -154,17 +154,17 @@ public void testSingleThreadProcessing() throws Exception { // Note that this processing semantics is not be considered as Decaton specification which users can rely on. // Rather, this is just a expected behavior based on current implementation when we set concurrency to 1. ProcessingGuarantee noDuplicates = new ProcessingGuarantee() { - private final Map> produced = new HashMap<>(); - private final Map> processed = new HashMap<>(); + private final Map> produced = new HashMap<>(); + private final Map> processed = new HashMap<>(); @Override public synchronized void onProduce(ProducedRecord record) { - produced.computeIfAbsent(ByteBuffer.wrap(record.key()), key -> new ArrayList<>()).add(record.task()); + produced.computeIfAbsent(record.key(), key -> new ArrayList<>()).add(record.task()); } @Override public synchronized void onProcess(TaskMetadata metadata, ProcessedRecord record) { - processed.computeIfAbsent(ByteBuffer.wrap(record.key()), key -> new ArrayList<>()).add(record.task()); + processed.computeIfAbsent(record.key(), key -> new ArrayList<>()).add(record.task()); } @Override diff --git a/processor/src/it/java/com/linecorp/decaton/processor/RateLimiterTest.java b/processor/src/it/java/com/linecorp/decaton/processor/RateLimiterTest.java index ff3cf490..aa3c6e88 100644 --- a/processor/src/it/java/com/linecorp/decaton/processor/RateLimiterTest.java +++ b/processor/src/it/java/com/linecorp/decaton/processor/RateLimiterTest.java @@ -18,7 +18,6 @@ import static org.junit.Assert.assertEquals; -import java.nio.ByteBuffer; import java.util.Collections; import java.util.HashSet; import java.util.Set; @@ -35,6 +34,7 @@ import com.linecorp.decaton.processor.runtime.ProcessorSubscription; import com.linecorp.decaton.processor.runtime.ProcessorsBuilder; import com.linecorp.decaton.processor.runtime.StaticPropertySupplier; +import com.linecorp.decaton.processor.runtime.internal.TaskKey; import com.linecorp.decaton.protobuf.ProtocolBuffersDeserializer; import com.linecorp.decaton.protocol.Sample.HelloTask; import com.linecorp.decaton.testing.KafkaClusterRule; @@ -63,11 +63,11 @@ public void testPropertyDynamicSwitch() throws Exception { for (int i = 0; i < 10000; i++) { keys.add("key" + i); } - Set processedKeys = Collections.synchronizedSet(new HashSet<>()); + Set processedKeys = Collections.synchronizedSet(new HashSet<>()); CountDownLatch processLatch = new CountDownLatch(keys.size()); DecatonProcessor processor = (context, task) -> { - processedKeys.add(ByteBuffer.wrap(context.key())); + processedKeys.add(context.key()); processLatch.countDown(); }; diff --git a/processor/src/main/java/com/linecorp/decaton/processor/ProcessingContext.java b/processor/src/main/java/com/linecorp/decaton/processor/ProcessingContext.java index 8eeeb4ce..0fa0c548 100644 --- a/processor/src/main/java/com/linecorp/decaton/processor/ProcessingContext.java +++ b/processor/src/main/java/com/linecorp/decaton/processor/ProcessingContext.java @@ -26,6 +26,7 @@ import com.linecorp.decaton.processor.runtime.ProcessorProperties; import com.linecorp.decaton.processor.runtime.RetryConfig; import com.linecorp.decaton.processor.runtime.SubscriptionBuilder; +import com.linecorp.decaton.processor.runtime.internal.TaskKey; public interface ProcessingContext { /** @@ -40,7 +41,7 @@ public interface ProcessingContext { * @return the key associated to the task now being processed. can be null if key isn't supplied for the * task. */ - byte[] key(); + TaskKey key(); /** * Returns the {@link Headers} which is associated to the source {@link ConsumerRecord} of the task. diff --git a/processor/src/main/java/com/linecorp/decaton/processor/formatter/KeyFormatter.java b/processor/src/main/java/com/linecorp/decaton/processor/formatter/KeyFormatter.java index 4fea5856..301d533c 100644 --- a/processor/src/main/java/com/linecorp/decaton/processor/formatter/KeyFormatter.java +++ b/processor/src/main/java/com/linecorp/decaton/processor/formatter/KeyFormatter.java @@ -18,9 +18,11 @@ import java.nio.charset.StandardCharsets; +import com.linecorp.decaton.processor.runtime.internal.TaskKey; + @FunctionalInterface public interface KeyFormatter { - KeyFormatter DEFAULT = key -> new String(key, StandardCharsets.UTF_8); + KeyFormatter DEFAULT = key -> new String(key.array(), StandardCharsets.UTF_8); - String format(byte[] key); + String format(TaskKey key); } diff --git a/processor/src/main/java/com/linecorp/decaton/processor/processors/CompactionProcessor.java b/processor/src/main/java/com/linecorp/decaton/processor/processors/CompactionProcessor.java index eaa872b8..ce5b876a 100644 --- a/processor/src/main/java/com/linecorp/decaton/processor/processors/CompactionProcessor.java +++ b/processor/src/main/java/com/linecorp/decaton/processor/processors/CompactionProcessor.java @@ -16,7 +16,6 @@ package com.linecorp.decaton.processor.processors; -import java.nio.ByteBuffer; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentMap; import java.util.concurrent.ScheduledExecutorService; @@ -27,10 +26,11 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import com.linecorp.decaton.processor.Completion; import com.linecorp.decaton.processor.DecatonProcessor; import com.linecorp.decaton.processor.ProcessingContext; import com.linecorp.decaton.processor.metrics.Metrics; -import com.linecorp.decaton.processor.Completion; +import com.linecorp.decaton.processor.runtime.internal.TaskKey; import io.micrometer.core.instrument.Counter; import lombok.AccessLevel; @@ -85,7 +85,7 @@ public enum CompactChoice { } private final ScheduledExecutorService executor; - private final ConcurrentMap> windowedTasks = new ConcurrentHashMap<>(); + private final ConcurrentMap> windowedTasks = new ConcurrentHashMap<>(); private final BiFunction, CompactingTask, CompactChoice> compactor; private final long lingerMillis; @@ -146,8 +146,8 @@ public CompactionProcessor( this(lingerMillis, compactor, null); } - private void flush(byte[] key) throws InterruptedException { - CompactingTask task = windowedTasks.remove(ByteBuffer.wrap(key)); + private void flush(TaskKey key) throws InterruptedException { + CompactingTask task = windowedTasks.remove(key); if (task == null) { return; } @@ -156,10 +156,9 @@ private void flush(byte[] key) throws InterruptedException { // visible for testing Runnable flushTask(ProcessingContext context) { - byte[] key = context.key(); return () -> { try { - flush(key); + flush(context.key()); } catch (InterruptedException e) { logger.error("interrupted while flushing compacted task result", e); } catch (RuntimeException e) { @@ -183,15 +182,15 @@ private void scheduleFlush(ProcessingContext context) { @Override public void process(ProcessingContext context, T task) throws InterruptedException { - byte[] key = context.key(); CompactingTask newTask = new CompactingTask<>(context.deferCompletion(), context, task); + TaskKey key = context.key(); // Even though we do this read and following updates in separate operation, race condition can't be // happened because tasks are guaranteed to be serialized by it's key, so simultaneous processing // of tasks sharing the same key won't be happen. - CompactingTask prevTask = windowedTasks.get(ByteBuffer.wrap(key)); + CompactingTask prevTask = windowedTasks.get(key); if (prevTask == null) { - windowedTasks.put(ByteBuffer.wrap(key), newTask); + windowedTasks.put(key, newTask); scheduleFlush(context); return; } @@ -206,7 +205,7 @@ public void process(ProcessingContext context, T task) throws InterruptedExce case PICK_RIGHT: case PICK_EITHER: // Newer task has larger offset. We want to forward consumed offset. // Update the last task with new one. - Object oldEntry = windowedTasks.put(ByteBuffer.wrap(key), newTask); + Object oldEntry = windowedTasks.put(key, newTask); if (oldEntry == null) { // By race condition, there is a chance that the scheduled flush for preceding task just // got fired right after this method checked the key's existence at the beginning of this diff --git a/processor/src/main/java/com/linecorp/decaton/processor/runtime/ProcessorSubscription.java b/processor/src/main/java/com/linecorp/decaton/processor/runtime/ProcessorSubscription.java index ad368666..1a6029cc 100644 --- a/processor/src/main/java/com/linecorp/decaton/processor/runtime/ProcessorSubscription.java +++ b/processor/src/main/java/com/linecorp/decaton/processor/runtime/ProcessorSubscription.java @@ -47,6 +47,7 @@ import com.linecorp.decaton.processor.runtime.internal.PartitionContexts; import com.linecorp.decaton.processor.runtime.internal.Processors; import com.linecorp.decaton.processor.runtime.internal.SubscriptionScope; +import com.linecorp.decaton.processor.runtime.internal.TaskKey; import com.linecorp.decaton.processor.runtime.internal.TaskRequest; import com.linecorp.decaton.processor.runtime.internal.Utils; import com.linecorp.decaton.processor.runtime.internal.Utils.Timer; @@ -120,7 +121,7 @@ public void receive(ConsumerRecord record) { if (blacklistedKeysFilter.shouldTake(record)) { TaskRequest taskRequest = - new TaskRequest(tp, record.offset(), offsetState, record.key(), + new TaskRequest(tp, record.offset(), offsetState, new TaskKey(record.key()), record.headers(), trace, record.value()); context.addRequest(taskRequest); } else { diff --git a/processor/src/main/java/com/linecorp/decaton/processor/runtime/internal/DecatonTaskRetryQueueingProcessor.java b/processor/src/main/java/com/linecorp/decaton/processor/runtime/internal/DecatonTaskRetryQueueingProcessor.java index d2e4bf73..81245133 100644 --- a/processor/src/main/java/com/linecorp/decaton/processor/runtime/internal/DecatonTaskRetryQueueingProcessor.java +++ b/processor/src/main/java/com/linecorp/decaton/processor/runtime/internal/DecatonTaskRetryQueueingProcessor.java @@ -64,7 +64,7 @@ public void process(ProcessingContext context, byte[] serializedTask) .setSerializedTask(ByteString.copyFrom(serializedTask)) .build(); - CompletableFuture future = producer.sendRequest(context.key(), request); + CompletableFuture future = producer.sendRequest(context.key().array(), request); future.whenComplete((r, e) -> { if (e == null) { metrics.retryQueuedTasks.increment(); diff --git a/processor/src/main/java/com/linecorp/decaton/processor/runtime/internal/ProcessingContextImpl.java b/processor/src/main/java/com/linecorp/decaton/processor/runtime/internal/ProcessingContextImpl.java index 47dacc23..61bafa70 100644 --- a/processor/src/main/java/com/linecorp/decaton/processor/runtime/internal/ProcessingContextImpl.java +++ b/processor/src/main/java/com/linecorp/decaton/processor/runtime/internal/ProcessingContextImpl.java @@ -72,7 +72,7 @@ public TaskMetadata metadata() { } @Override - public byte[] key() { + public TaskKey key() { return request.key(); } diff --git a/processor/src/main/java/com/linecorp/decaton/processor/runtime/internal/SubPartitioner.java b/processor/src/main/java/com/linecorp/decaton/processor/runtime/internal/SubPartitioner.java index 44740c74..93dcf425 100644 --- a/processor/src/main/java/com/linecorp/decaton/processor/runtime/internal/SubPartitioner.java +++ b/processor/src/main/java/com/linecorp/decaton/processor/runtime/internal/SubPartitioner.java @@ -32,7 +32,7 @@ private static int toPositive(int number) { return number & 2147483647; } - public int partitionFor(byte[] key) { + public int partitionFor(TaskKey key) { if (key == null) { return toPositive((int) monotonicValueSupplier.getAndIncrement()) % bound; } else { @@ -41,10 +41,11 @@ public int partitionFor(byte[] key) { // Here just by adding few bytes to the key we can "shift" hashing of the key and // can get back better distribution again in murmur2 result to evenly distribute keys // for subpartitions. - final ByteBuffer bb = ByteBuffer.allocate(key.length + 2); + final byte[] keyArray = key.array(); + final ByteBuffer bb = ByteBuffer.allocate(keyArray.length + 2); bb.put((byte) 's'); bb.put((byte) ':'); - bb.put(key); + bb.put(keyArray); int hash = org.apache.kafka.common.utils.Utils.murmur2(bb.array()); return toPositive(hash) % bound; diff --git a/processor/src/main/java/com/linecorp/decaton/processor/runtime/internal/TaskKey.java b/processor/src/main/java/com/linecorp/decaton/processor/runtime/internal/TaskKey.java new file mode 100644 index 00000000..33c73161 --- /dev/null +++ b/processor/src/main/java/com/linecorp/decaton/processor/runtime/internal/TaskKey.java @@ -0,0 +1,57 @@ +/* + * Copyright 2020 LINE Corporation + * + * LINE Corporation licenses this file to you under the Apache License, + * version 2.0 (the "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at: + * + * https://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the + * License for the specific language governing permissions and limitations + * under the License. + */ + +package com.linecorp.decaton.processor.runtime.internal; + +import java.nio.charset.StandardCharsets; +import java.util.Arrays; + +public class TaskKey { + private static final byte[] EMPTY = new byte[0]; + private final byte[] key; + private int hash; + + public TaskKey(byte[] key) { + this.key = key == null ? EMPTY : key; + } + + public byte[] array() { + return key; + } + + @Override + public boolean equals(Object o) { + if (this == o) {return true;} + if (o == null || getClass() != o.getClass()) {return false;} + + final TaskKey taskKey = (TaskKey) o; + + return Arrays.equals(key, taskKey.key); + } + + @Override + public int hashCode() { + if (hash == 0 && key.length > 0) { + hash = Arrays.hashCode(key); + } + return hash; + } + + @Override + public String toString() { + return "TaskKey{key=" + new String(key, StandardCharsets.UTF_8) + '}'; + } +} diff --git a/processor/src/main/java/com/linecorp/decaton/processor/runtime/internal/TaskRequest.java b/processor/src/main/java/com/linecorp/decaton/processor/runtime/internal/TaskRequest.java index 4cc195cb..8b0e41aa 100644 --- a/processor/src/main/java/com/linecorp/decaton/processor/runtime/internal/TaskRequest.java +++ b/processor/src/main/java/com/linecorp/decaton/processor/runtime/internal/TaskRequest.java @@ -32,8 +32,7 @@ public class TaskRequest { private final TopicPartition topicPartition; private final long recordOffset; private final OffsetState offsetState; - @ToString.Exclude - private final byte[] key; + private final TaskKey key; @ToString.Exclude private final Headers headers; @ToString.Exclude @@ -44,7 +43,7 @@ public class TaskRequest { public TaskRequest(TopicPartition topicPartition, long recordOffset, OffsetState offsetState, - byte[] key, + TaskKey key, Headers headers, RecordTraceHandle trace, byte[] rawRequestBytes) { diff --git a/processor/src/test/java/com/linecorp/decaton/processor/processors/CompactionProcessorTest.java b/processor/src/test/java/com/linecorp/decaton/processor/processors/CompactionProcessorTest.java index 9ec4a0a2..8a4f9b6a 100644 --- a/processor/src/test/java/com/linecorp/decaton/processor/processors/CompactionProcessorTest.java +++ b/processor/src/test/java/com/linecorp/decaton/processor/processors/CompactionProcessorTest.java @@ -55,6 +55,7 @@ import com.linecorp.decaton.processor.runtime.DecatonTask; import com.linecorp.decaton.processor.runtime.ProcessorProperties; import com.linecorp.decaton.processor.runtime.internal.ProcessingContextImpl; +import com.linecorp.decaton.processor.runtime.internal.TaskKey; import com.linecorp.decaton.processor.runtime.internal.TaskRequest; import com.linecorp.decaton.processor.tracing.internal.NoopTracingProvider.NoopTrace; import com.linecorp.decaton.protocol.Sample.HelloTask; @@ -103,7 +104,7 @@ private TaskInput put(DecatonProcessor processor, taskData, taskData.toByteArray()); TaskRequest request = new TaskRequest( - new TopicPartition("topic", 1), 1, null, name.getBytes(StandardCharsets.UTF_8), null, NoopTrace.INSTANCE, null); + new TopicPartition("topic", 1), 1, null, new TaskKey(name.getBytes(StandardCharsets.UTF_8)), null, NoopTrace.INSTANCE, null); ProcessingContext context = spy(new ProcessingContextImpl<>("subscription", request, task, Arrays.asList(processor, downstream), null, diff --git a/processor/src/test/java/com/linecorp/decaton/processor/runtime/internal/DecatonTaskRetryQueueingProcessorTest.java b/processor/src/test/java/com/linecorp/decaton/processor/runtime/internal/DecatonTaskRetryQueueingProcessorTest.java index 0feb46f6..86167293 100644 --- a/processor/src/test/java/com/linecorp/decaton/processor/runtime/internal/DecatonTaskRetryQueueingProcessorTest.java +++ b/processor/src/test/java/com/linecorp/decaton/processor/runtime/internal/DecatonTaskRetryQueueingProcessorTest.java @@ -77,13 +77,13 @@ public void setUp() { processor = new DecatonTaskRetryQueueingProcessor(scope, producer); doReturn(CompletableFuture.completedFuture(null)).when(producer).sendRequest(any(), any()); doReturn(new CompletionImpl()).when(context).deferCompletion(); - doReturn("key".getBytes(StandardCharsets.UTF_8)).when(context).key(); + doReturn(new TaskKey("key".getBytes(StandardCharsets.UTF_8))).when(context).key(); doReturn(TaskMetadata.builder().build()).when(context).metadata(); } @Test public void testRetryRequest() throws InterruptedException { - byte[] key = "key".getBytes(StandardCharsets.UTF_8); + TaskKey key = new TaskKey("key".getBytes(StandardCharsets.UTF_8)); TaskMetadata meta = TaskMetadata.builder() .sourceApplicationId("unit-test") @@ -101,7 +101,7 @@ public void testRetryRequest() throws InterruptedException { processor.process(context, task.toByteArray()); ArgumentCaptor captor = ArgumentCaptor.forClass(DecatonTaskRequest.class); - verify(producer, times(1)).sendRequest(eq(key), captor.capture()); + verify(producer, times(1)).sendRequest(eq(key.array()), captor.capture()); DecatonTaskRequest request = captor.getValue(); assertEquals(task.toByteString(), request.getSerializedTask()); diff --git a/processor/src/test/java/com/linecorp/decaton/processor/runtime/internal/ProcessPipelineTest.java b/processor/src/test/java/com/linecorp/decaton/processor/runtime/internal/ProcessPipelineTest.java index 8bd03b6a..a165ef5d 100644 --- a/processor/src/test/java/com/linecorp/decaton/processor/runtime/internal/ProcessPipelineTest.java +++ b/processor/src/test/java/com/linecorp/decaton/processor/runtime/internal/ProcessPipelineTest.java @@ -94,7 +94,8 @@ public class ProcessPipelineTest { private static TaskRequest taskRequest() { return new TaskRequest( - new TopicPartition("topic", 1), 1, new OffsetState(1234), "TEST".getBytes(StandardCharsets.UTF_8), null, NoopTrace.INSTANCE, REQUEST.toByteArray()); + new TopicPartition("topic", 1), 1, new OffsetState(1234), + new TaskKey("TEST".getBytes(StandardCharsets.UTF_8)), null, NoopTrace.INSTANCE, REQUEST.toByteArray()); } @Rule diff --git a/processor/src/test/java/com/linecorp/decaton/processor/runtime/internal/ProcessingContextImplTest.java b/processor/src/test/java/com/linecorp/decaton/processor/runtime/internal/ProcessingContextImplTest.java index 30064216..50a81d1e 100644 --- a/processor/src/test/java/com/linecorp/decaton/processor/runtime/internal/ProcessingContextImplTest.java +++ b/processor/src/test/java/com/linecorp/decaton/processor/runtime/internal/ProcessingContextImplTest.java @@ -112,7 +112,7 @@ private static void terminateExecutor(ExecutorService executor) throws Interrupt private static ProcessingContextImpl context(RecordTraceHandle traceHandle, DecatonProcessor... processors) { TaskRequest request = new TaskRequest( - new TopicPartition("topic", 1), 1, null, "TEST".getBytes(StandardCharsets.UTF_8), + new TopicPartition("topic", 1), 1, null, new TaskKey("TEST".getBytes(StandardCharsets.UTF_8)), null, traceHandle, REQUEST.toByteArray()); DecatonTask task = new DecatonTask<>( TaskMetadata.fromProto(REQUEST.getMetadata()), TASK, TASK.toByteArray()); @@ -363,7 +363,7 @@ public void process(ProcessingContext context, byte[] task) } }); TaskRequest request = new TaskRequest( - new TopicPartition("topic", 1), 1, null, "TEST".getBytes(StandardCharsets.UTF_8), null, null, REQUEST.toByteArray()); + new TopicPartition("topic", 1), 1, null, new TaskKey("TEST".getBytes(StandardCharsets.UTF_8)), null, null, REQUEST.toByteArray()); DecatonTask task = new DecatonTask<>( TaskMetadata.fromProto(REQUEST.getMetadata()), TASK.toByteArray(), TASK.toByteArray()); @@ -407,7 +407,7 @@ public void process(ProcessingContext context, byte[] task) } }); TaskRequest request = new TaskRequest( - new TopicPartition("topic", 1), 1, null, "TEST".getBytes(StandardCharsets.UTF_8), null, null, REQUEST.toByteArray()); + new TopicPartition("topic", 1), 1, null, new TaskKey("TEST".getBytes(StandardCharsets.UTF_8)), null, null, REQUEST.toByteArray()); DecatonTask task = new DecatonTask<>( TaskMetadata.fromProto(REQUEST.getMetadata()), TASK.toByteArray(), TASK.toByteArray()); diff --git a/processor/src/test/java/com/linecorp/decaton/processor/runtime/internal/SubPartitionerTest.java b/processor/src/test/java/com/linecorp/decaton/processor/runtime/internal/SubPartitionerTest.java index 25fc9a2e..095a122f 100644 --- a/processor/src/test/java/com/linecorp/decaton/processor/runtime/internal/SubPartitionerTest.java +++ b/processor/src/test/java/com/linecorp/decaton/processor/runtime/internal/SubPartitionerTest.java @@ -28,20 +28,18 @@ import org.junit.Before; import org.junit.Test; -import com.linecorp.decaton.processor.runtime.internal.SubPartitioner; - public class SubPartitionerTest { static final int DIST_KEYS_COUNT = 10000; static final int[] PARTITION_COUNTS = { 1, 8, 16, 32, 64, 96 }; static final int[] SUBPARTITION_COUNTS = { 10, 17, 32, 64 }; static final double TARGET_STDDEV_RATIO = 0.05; - static final byte[][] keys = new byte[DIST_KEYS_COUNT][]; + static final TaskKey[] keys = new TaskKey[DIST_KEYS_COUNT]; @Before public void setUp() { for (int i = 0; i < keys.length; i++) { - keys[i] = String.valueOf(i).getBytes(StandardCharsets.UTF_8); + keys[i] = new TaskKey(String.valueOf(i).getBytes(StandardCharsets.UTF_8)); } } @@ -55,14 +53,14 @@ private static double stddev(int[] counts) { @Test public void testEvenlyDistributedSelection() { for (int partitionCount : PARTITION_COUNTS) { - List> partitions = new ArrayList<>(partitionCount); + List> partitions = new ArrayList<>(partitionCount); for (int i = 0; i < partitionCount; i++) { partitions.add(new ArrayList<>()); } - for (byte[] key : keys) { + for (TaskKey key : keys) { // This is the way used to determine partition in Kafka's DefaultPartitioner - int partition = (Utils.murmur2(key) & 2147483647) % partitionCount; + int partition = (Utils.murmur2(key.array()) & 2147483647) % partitionCount; partitions.get(partition).add(key); } @@ -72,10 +70,10 @@ public void testEvenlyDistributedSelection() { partitionCount, partStddev, Arrays.toString(partCounts)); for (int subpartitionCount : SUBPARTITION_COUNTS) { - for (List partition : partitions) { + for (List partition : partitions) { int[] counts = new int[subpartitionCount]; SubPartitioner subPartitioner = new SubPartitioner(counts.length); - for (byte[] key : partition) { + for (TaskKey key : partition) { int subPartition = subPartitioner.partitionFor(key); counts[subPartition]++; } @@ -97,7 +95,7 @@ public void testEvenlyDistributedSelection() { public void testConsistentSelectionForSameKeys() { for (int subpartitionCount : SUBPARTITION_COUNTS) { SubPartitioner subPartitioner = new SubPartitioner(subpartitionCount); - for (byte[] key : keys) { + for (TaskKey key : keys) { int assign1 = subPartitioner.partitionFor(key); int assign2 = subPartitioner.partitionFor(key); assertEquals(String.format("[%d] assign of %s", subpartitionCount, key), diff --git a/testing/src/main/java/com/linecorp/decaton/testing/processor/ProcessOrdering.java b/testing/src/main/java/com/linecorp/decaton/testing/processor/ProcessOrdering.java index de5d15d7..239a995b 100644 --- a/testing/src/main/java/com/linecorp/decaton/testing/processor/ProcessOrdering.java +++ b/testing/src/main/java/com/linecorp/decaton/testing/processor/ProcessOrdering.java @@ -21,7 +21,6 @@ import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; -import java.nio.ByteBuffer; import java.util.ArrayDeque; import java.util.ArrayList; import java.util.Deque; @@ -31,31 +30,31 @@ import java.util.Map.Entry; import com.linecorp.decaton.processor.TaskMetadata; +import com.linecorp.decaton.processor.runtime.internal.TaskKey; public class ProcessOrdering implements ProcessingGuarantee { private final Map taskToOffset = new HashMap<>(); - private final Map> producedRecords = new HashMap<>(); - private final Map> processedRecords = new HashMap<>(); + private final Map> producedRecords = new HashMap<>(); + private final Map> processedRecords = new HashMap<>(); @Override public synchronized void onProduce(ProducedRecord record) { taskToOffset.put(record.task(), record.offset()); - producedRecords.computeIfAbsent(ByteBuffer.wrap(record.key()), + producedRecords.computeIfAbsent(record.key(), key -> new ArrayList<>()).add(record.task()); } @Override public synchronized void onProcess(TaskMetadata metadata, ProcessedRecord record) { - processedRecords.computeIfAbsent(ByteBuffer.wrap(record.key()), + processedRecords.computeIfAbsent(record.key(), key -> new ArrayList<>()).add(record.task()); } @Override public void doAssert() { - for (Entry> entry : producedRecords.entrySet()) { - final ByteBuffer key = entry.getKey(); + for (Entry> entry : producedRecords.entrySet()) { List produced = entry.getValue(); - List processed = processedRecords.get(key); + List processed = processedRecords.get(entry.getKey()); assertNotNull(processed); assertOrdering(taskToOffset, produced, processed); diff --git a/testing/src/main/java/com/linecorp/decaton/testing/processor/ProcessedRecord.java b/testing/src/main/java/com/linecorp/decaton/testing/processor/ProcessedRecord.java index 9e44539f..05991686 100644 --- a/testing/src/main/java/com/linecorp/decaton/testing/processor/ProcessedRecord.java +++ b/testing/src/main/java/com/linecorp/decaton/testing/processor/ProcessedRecord.java @@ -17,8 +17,8 @@ package com.linecorp.decaton.testing.processor; import com.linecorp.decaton.processor.DecatonProcessor; +import com.linecorp.decaton.processor.runtime.internal.TaskKey; -import lombok.ToString; import lombok.Value; import lombok.experimental.Accessors; @@ -31,8 +31,7 @@ public class ProcessedRecord { /** * Key of the task */ - @ToString.Exclude - byte[] key; + TaskKey key; /** * Processed task */ diff --git a/testing/src/main/java/com/linecorp/decaton/testing/processor/ProcessorTestSuite.java b/testing/src/main/java/com/linecorp/decaton/testing/processor/ProcessorTestSuite.java index 78a48ffd..bd144800 100644 --- a/testing/src/main/java/com/linecorp/decaton/testing/processor/ProcessorTestSuite.java +++ b/testing/src/main/java/com/linecorp/decaton/testing/processor/ProcessorTestSuite.java @@ -58,6 +58,7 @@ import com.linecorp.decaton.processor.runtime.RetryConfig.RetryConfigBuilder; import com.linecorp.decaton.processor.runtime.SubscriptionStateListener; import com.linecorp.decaton.processor.runtime.TaskExtractor; +import com.linecorp.decaton.processor.runtime.internal.TaskKey; import com.linecorp.decaton.processor.tracing.TracingProvider; import com.linecorp.decaton.protocol.Decaton.DecatonTaskRequest; import com.linecorp.decaton.protocol.Decaton.TaskMetadataProto; @@ -170,6 +171,7 @@ public Builder excludeSemantics(GuaranteeType... guarantees) { } return this; } + /** * Include additional semantics in assertion. * Feature-specific processing guarantee will be injected through this method @@ -191,7 +193,8 @@ public ProcessorTestSuite build() { semantics.addAll(customSemantics); if (statesListener == null) { - statesListener = (id, state) -> {}; + statesListener = (id, state) -> { + }; } return new ProcessorTestSuite(rule, @@ -381,7 +384,7 @@ private CompletableFuture> produceTasks( producer.send(record, (metadata, exception) -> { if (exception == null) { future.complete(metadata); - onProduce.accept(new ProducedRecord(key, + onProduce.accept(new ProducedRecord(new TaskKey(key), new TopicPartition(metadata.topic(), metadata.partition()), metadata.offset(), diff --git a/testing/src/main/java/com/linecorp/decaton/testing/processor/ProducedRecord.java b/testing/src/main/java/com/linecorp/decaton/testing/processor/ProducedRecord.java index 75ed832a..dc6b28a9 100644 --- a/testing/src/main/java/com/linecorp/decaton/testing/processor/ProducedRecord.java +++ b/testing/src/main/java/com/linecorp/decaton/testing/processor/ProducedRecord.java @@ -19,6 +19,8 @@ import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.header.Headers; +import com.linecorp.decaton.processor.runtime.internal.TaskKey; + import lombok.Value; import lombok.experimental.Accessors; @@ -31,7 +33,7 @@ public class ProducedRecord { /** * Key of the task */ - byte[] key; + TaskKey key; /** * Topic partition the record was sent to */ diff --git a/testing/src/main/java/com/linecorp/decaton/testing/processor/SerialProcessing.java b/testing/src/main/java/com/linecorp/decaton/testing/processor/SerialProcessing.java index b8c2cdd3..c59ee492 100644 --- a/testing/src/main/java/com/linecorp/decaton/testing/processor/SerialProcessing.java +++ b/testing/src/main/java/com/linecorp/decaton/testing/processor/SerialProcessing.java @@ -19,8 +19,6 @@ import static org.hamcrest.MatcherAssert.assertThat; import static org.hamcrest.Matchers.lessThan; -import java.nio.ByteBuffer; -import java.nio.charset.StandardCharsets; import java.util.ArrayList; import java.util.Comparator; import java.util.HashMap; @@ -29,9 +27,10 @@ import java.util.Map.Entry; import com.linecorp.decaton.processor.TaskMetadata; +import com.linecorp.decaton.processor.runtime.internal.TaskKey; public class SerialProcessing implements ProcessingGuarantee { - private final Map> records = new HashMap<>(); + private final Map> records = new HashMap<>(); @Override public void onProduce(ProducedRecord record) { @@ -40,14 +39,13 @@ public void onProduce(ProducedRecord record) { @Override public synchronized void onProcess(TaskMetadata metadata, ProcessedRecord record) { - records.computeIfAbsent(ByteBuffer.wrap(record.key()), - key -> new ArrayList<>()).add(record); + records.computeIfAbsent(record.key(), key -> new ArrayList<>()).add(record); } @Override public void doAssert() { // Checks there's no overlap between two consecutive records' processing time - for (Entry> entry : records.entrySet()) { + for (Entry> entry : records.entrySet()) { List perKeyRecords = entry.getValue(); perKeyRecords.sort(Comparator.comparingLong(ProcessedRecord::startTimeNanos)); @@ -55,8 +53,7 @@ public void doAssert() { ProcessedRecord prev = perKeyRecords.get(i - 1); ProcessedRecord current = perKeyRecords.get(i); - String key = StandardCharsets.UTF_8.decode(entry.getKey()).toString(); - assertThat("Process time shouldn't overlap. key: " + key, + assertThat("Process time shouldn't overlap. key: " + entry.getKey(), prev.endTimeNanos(), lessThan(current.startTimeNanos())); } } From fc63284865f8fcf2b0401d9d617850f6e278bcc7 Mon Sep 17 00:00:00 2001 From: Koki Kato Date: Mon, 20 Jun 2022 17:40:21 +0900 Subject: [PATCH 11/27] Use TaskKey for BlacklistedKeysFilter To avoid charset decoding step. --- .../runtime/ProcessorSubscription.java | 5 +++-- .../internal/BlacklistedKeysFilter.java | 18 ++++++++---------- 2 files changed, 11 insertions(+), 12 deletions(-) diff --git a/processor/src/main/java/com/linecorp/decaton/processor/runtime/ProcessorSubscription.java b/processor/src/main/java/com/linecorp/decaton/processor/runtime/ProcessorSubscription.java index 1a6029cc..ae89de15 100644 --- a/processor/src/main/java/com/linecorp/decaton/processor/runtime/ProcessorSubscription.java +++ b/processor/src/main/java/com/linecorp/decaton/processor/runtime/ProcessorSubscription.java @@ -119,9 +119,10 @@ public void receive(ConsumerRecord record) { } }); - if (blacklistedKeysFilter.shouldTake(record)) { + TaskKey key = new TaskKey(record.key()); + if (blacklistedKeysFilter.shouldTake(key)) { TaskRequest taskRequest = - new TaskRequest(tp, record.offset(), offsetState, new TaskKey(record.key()), + new TaskRequest(tp, record.offset(), offsetState, key, record.headers(), trace, record.value()); context.addRequest(taskRequest); } else { diff --git a/processor/src/main/java/com/linecorp/decaton/processor/runtime/internal/BlacklistedKeysFilter.java b/processor/src/main/java/com/linecorp/decaton/processor/runtime/internal/BlacklistedKeysFilter.java index c2ac193c..9c9a294e 100644 --- a/processor/src/main/java/com/linecorp/decaton/processor/runtime/internal/BlacklistedKeysFilter.java +++ b/processor/src/main/java/com/linecorp/decaton/processor/runtime/internal/BlacklistedKeysFilter.java @@ -19,10 +19,9 @@ import static com.linecorp.decaton.processor.runtime.ProcessorProperties.CONFIG_IGNORE_KEYS; import java.nio.charset.StandardCharsets; -import java.util.HashSet; import java.util.Set; +import java.util.stream.Collectors; -import org.apache.kafka.clients.consumer.ConsumerRecord; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -31,25 +30,24 @@ public class BlacklistedKeysFilter { private static final Logger logger = LoggerFactory.getLogger(BlacklistedKeysFilter.class); - private volatile Set ignoreKeys; + private volatile Set ignoreKeys; public BlacklistedKeysFilter(ProcessorProperties props) { props.get(CONFIG_IGNORE_KEYS) - .listen((oldValue, newValue) -> ignoreKeys = new HashSet<>(newValue)); + .listen((oldValue, newValue) -> ignoreKeys = + newValue.stream().map(key -> new TaskKey(key.getBytes(StandardCharsets.UTF_8))).collect(Collectors.toSet()) + ); } - public boolean shouldTake(ConsumerRecord record) { - final byte[] key = record.key(); + public boolean shouldTake(TaskKey key) { if (key == null) { return true; } - final String stringKey = new String(key, StandardCharsets.UTF_8); - // Preceding isEmpty() check is for reducing tiny overhead applied for each contains() by calling // Object#hashCode. Since ignoreKeys should be empty for most cases.. - if (!ignoreKeys.isEmpty() && ignoreKeys.contains(stringKey)) { - logger.debug("Ignore task which has key configured to ignore: {}", stringKey); + if (!ignoreKeys.isEmpty() && ignoreKeys.contains(key)) { + logger.debug("Ignore task which has key configured to ignore: {}", key); return false; } From 4f26f2e538be60dd066c4e6f409d2f705cc1bdf6 Mon Sep 17 00:00:00 2001 From: Koki Kato Date: Tue, 21 Jun 2022 14:29:49 +0900 Subject: [PATCH 12/27] Revert "Use TaskKey for BlacklistedKeysFilter" This reverts commit fc63284865f8fcf2b0401d9d617850f6e278bcc7. --- .../runtime/ProcessorSubscription.java | 5 ++--- .../internal/BlacklistedKeysFilter.java | 18 ++++++++++-------- 2 files changed, 12 insertions(+), 11 deletions(-) diff --git a/processor/src/main/java/com/linecorp/decaton/processor/runtime/ProcessorSubscription.java b/processor/src/main/java/com/linecorp/decaton/processor/runtime/ProcessorSubscription.java index ae89de15..1a6029cc 100644 --- a/processor/src/main/java/com/linecorp/decaton/processor/runtime/ProcessorSubscription.java +++ b/processor/src/main/java/com/linecorp/decaton/processor/runtime/ProcessorSubscription.java @@ -119,10 +119,9 @@ public void receive(ConsumerRecord record) { } }); - TaskKey key = new TaskKey(record.key()); - if (blacklistedKeysFilter.shouldTake(key)) { + if (blacklistedKeysFilter.shouldTake(record)) { TaskRequest taskRequest = - new TaskRequest(tp, record.offset(), offsetState, key, + new TaskRequest(tp, record.offset(), offsetState, new TaskKey(record.key()), record.headers(), trace, record.value()); context.addRequest(taskRequest); } else { diff --git a/processor/src/main/java/com/linecorp/decaton/processor/runtime/internal/BlacklistedKeysFilter.java b/processor/src/main/java/com/linecorp/decaton/processor/runtime/internal/BlacklistedKeysFilter.java index 9c9a294e..c2ac193c 100644 --- a/processor/src/main/java/com/linecorp/decaton/processor/runtime/internal/BlacklistedKeysFilter.java +++ b/processor/src/main/java/com/linecorp/decaton/processor/runtime/internal/BlacklistedKeysFilter.java @@ -19,9 +19,10 @@ import static com.linecorp.decaton.processor.runtime.ProcessorProperties.CONFIG_IGNORE_KEYS; import java.nio.charset.StandardCharsets; +import java.util.HashSet; import java.util.Set; -import java.util.stream.Collectors; +import org.apache.kafka.clients.consumer.ConsumerRecord; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -30,24 +31,25 @@ public class BlacklistedKeysFilter { private static final Logger logger = LoggerFactory.getLogger(BlacklistedKeysFilter.class); - private volatile Set ignoreKeys; + private volatile Set ignoreKeys; public BlacklistedKeysFilter(ProcessorProperties props) { props.get(CONFIG_IGNORE_KEYS) - .listen((oldValue, newValue) -> ignoreKeys = - newValue.stream().map(key -> new TaskKey(key.getBytes(StandardCharsets.UTF_8))).collect(Collectors.toSet()) - ); + .listen((oldValue, newValue) -> ignoreKeys = new HashSet<>(newValue)); } - public boolean shouldTake(TaskKey key) { + public boolean shouldTake(ConsumerRecord record) { + final byte[] key = record.key(); if (key == null) { return true; } + final String stringKey = new String(key, StandardCharsets.UTF_8); + // Preceding isEmpty() check is for reducing tiny overhead applied for each contains() by calling // Object#hashCode. Since ignoreKeys should be empty for most cases.. - if (!ignoreKeys.isEmpty() && ignoreKeys.contains(key)) { - logger.debug("Ignore task which has key configured to ignore: {}", key); + if (!ignoreKeys.isEmpty() && ignoreKeys.contains(stringKey)) { + logger.debug("Ignore task which has key configured to ignore: {}", stringKey); return false; } From 3994ebcbd46aa4923ed6f662b542c74a49194e52 Mon Sep 17 00:00:00 2001 From: Koki Kato Date: Tue, 21 Jun 2022 14:29:58 +0900 Subject: [PATCH 13/27] Revert "Introduce TaskKey; a wrapper for keys in bytes" This reverts commit 8c90fe593e52823371d2260278bb6b48e889b314. --- .../processor/CoreFunctionalityTest.java | 10 ++-- .../decaton/processor/RateLimiterTest.java | 6 +- .../decaton/processor/ProcessingContext.java | 3 +- .../processor/formatter/KeyFormatter.java | 6 +- .../processors/CompactionProcessor.java | 21 +++---- .../runtime/ProcessorSubscription.java | 3 +- .../DecatonTaskRetryQueueingProcessor.java | 2 +- .../internal/ProcessingContextImpl.java | 2 +- .../runtime/internal/SubPartitioner.java | 7 +-- .../processor/runtime/internal/TaskKey.java | 57 ------------------- .../runtime/internal/TaskRequest.java | 5 +- .../processors/CompactionProcessorTest.java | 3 +- ...DecatonTaskRetryQueueingProcessorTest.java | 6 +- .../runtime/internal/ProcessPipelineTest.java | 3 +- .../internal/ProcessingContextImplTest.java | 6 +- .../runtime/internal/SubPartitionerTest.java | 18 +++--- .../testing/processor/ProcessOrdering.java | 15 ++--- .../testing/processor/ProcessedRecord.java | 5 +- .../testing/processor/ProcessorTestSuite.java | 7 +-- .../testing/processor/ProducedRecord.java | 4 +- .../testing/processor/SerialProcessing.java | 13 +++-- 21 files changed, 71 insertions(+), 131 deletions(-) delete mode 100644 processor/src/main/java/com/linecorp/decaton/processor/runtime/internal/TaskKey.java diff --git a/processor/src/it/java/com/linecorp/decaton/processor/CoreFunctionalityTest.java b/processor/src/it/java/com/linecorp/decaton/processor/CoreFunctionalityTest.java index dc0acfe4..416bec65 100644 --- a/processor/src/it/java/com/linecorp/decaton/processor/CoreFunctionalityTest.java +++ b/processor/src/it/java/com/linecorp/decaton/processor/CoreFunctionalityTest.java @@ -18,6 +18,7 @@ import static org.junit.Assert.assertTrue; +import java.nio.ByteBuffer; import java.util.ArrayList; import java.util.HashMap; import java.util.List; @@ -34,7 +35,6 @@ import com.linecorp.decaton.processor.runtime.ProcessorScope; import com.linecorp.decaton.processor.runtime.Property; import com.linecorp.decaton.processor.runtime.StaticPropertySupplier; -import com.linecorp.decaton.processor.runtime.internal.TaskKey; import com.linecorp.decaton.testing.KafkaClusterRule; import com.linecorp.decaton.testing.RandomRule; import com.linecorp.decaton.testing.processor.ProcessedRecord; @@ -154,17 +154,17 @@ public void testSingleThreadProcessing() throws Exception { // Note that this processing semantics is not be considered as Decaton specification which users can rely on. // Rather, this is just a expected behavior based on current implementation when we set concurrency to 1. ProcessingGuarantee noDuplicates = new ProcessingGuarantee() { - private final Map> produced = new HashMap<>(); - private final Map> processed = new HashMap<>(); + private final Map> produced = new HashMap<>(); + private final Map> processed = new HashMap<>(); @Override public synchronized void onProduce(ProducedRecord record) { - produced.computeIfAbsent(record.key(), key -> new ArrayList<>()).add(record.task()); + produced.computeIfAbsent(ByteBuffer.wrap(record.key()), key -> new ArrayList<>()).add(record.task()); } @Override public synchronized void onProcess(TaskMetadata metadata, ProcessedRecord record) { - processed.computeIfAbsent(record.key(), key -> new ArrayList<>()).add(record.task()); + processed.computeIfAbsent(ByteBuffer.wrap(record.key()), key -> new ArrayList<>()).add(record.task()); } @Override diff --git a/processor/src/it/java/com/linecorp/decaton/processor/RateLimiterTest.java b/processor/src/it/java/com/linecorp/decaton/processor/RateLimiterTest.java index aa3c6e88..ff3cf490 100644 --- a/processor/src/it/java/com/linecorp/decaton/processor/RateLimiterTest.java +++ b/processor/src/it/java/com/linecorp/decaton/processor/RateLimiterTest.java @@ -18,6 +18,7 @@ import static org.junit.Assert.assertEquals; +import java.nio.ByteBuffer; import java.util.Collections; import java.util.HashSet; import java.util.Set; @@ -34,7 +35,6 @@ import com.linecorp.decaton.processor.runtime.ProcessorSubscription; import com.linecorp.decaton.processor.runtime.ProcessorsBuilder; import com.linecorp.decaton.processor.runtime.StaticPropertySupplier; -import com.linecorp.decaton.processor.runtime.internal.TaskKey; import com.linecorp.decaton.protobuf.ProtocolBuffersDeserializer; import com.linecorp.decaton.protocol.Sample.HelloTask; import com.linecorp.decaton.testing.KafkaClusterRule; @@ -63,11 +63,11 @@ public void testPropertyDynamicSwitch() throws Exception { for (int i = 0; i < 10000; i++) { keys.add("key" + i); } - Set processedKeys = Collections.synchronizedSet(new HashSet<>()); + Set processedKeys = Collections.synchronizedSet(new HashSet<>()); CountDownLatch processLatch = new CountDownLatch(keys.size()); DecatonProcessor processor = (context, task) -> { - processedKeys.add(context.key()); + processedKeys.add(ByteBuffer.wrap(context.key())); processLatch.countDown(); }; diff --git a/processor/src/main/java/com/linecorp/decaton/processor/ProcessingContext.java b/processor/src/main/java/com/linecorp/decaton/processor/ProcessingContext.java index 0fa0c548..8eeeb4ce 100644 --- a/processor/src/main/java/com/linecorp/decaton/processor/ProcessingContext.java +++ b/processor/src/main/java/com/linecorp/decaton/processor/ProcessingContext.java @@ -26,7 +26,6 @@ import com.linecorp.decaton.processor.runtime.ProcessorProperties; import com.linecorp.decaton.processor.runtime.RetryConfig; import com.linecorp.decaton.processor.runtime.SubscriptionBuilder; -import com.linecorp.decaton.processor.runtime.internal.TaskKey; public interface ProcessingContext { /** @@ -41,7 +40,7 @@ public interface ProcessingContext { * @return the key associated to the task now being processed. can be null if key isn't supplied for the * task. */ - TaskKey key(); + byte[] key(); /** * Returns the {@link Headers} which is associated to the source {@link ConsumerRecord} of the task. diff --git a/processor/src/main/java/com/linecorp/decaton/processor/formatter/KeyFormatter.java b/processor/src/main/java/com/linecorp/decaton/processor/formatter/KeyFormatter.java index 301d533c..4fea5856 100644 --- a/processor/src/main/java/com/linecorp/decaton/processor/formatter/KeyFormatter.java +++ b/processor/src/main/java/com/linecorp/decaton/processor/formatter/KeyFormatter.java @@ -18,11 +18,9 @@ import java.nio.charset.StandardCharsets; -import com.linecorp.decaton.processor.runtime.internal.TaskKey; - @FunctionalInterface public interface KeyFormatter { - KeyFormatter DEFAULT = key -> new String(key.array(), StandardCharsets.UTF_8); + KeyFormatter DEFAULT = key -> new String(key, StandardCharsets.UTF_8); - String format(TaskKey key); + String format(byte[] key); } diff --git a/processor/src/main/java/com/linecorp/decaton/processor/processors/CompactionProcessor.java b/processor/src/main/java/com/linecorp/decaton/processor/processors/CompactionProcessor.java index ce5b876a..eaa872b8 100644 --- a/processor/src/main/java/com/linecorp/decaton/processor/processors/CompactionProcessor.java +++ b/processor/src/main/java/com/linecorp/decaton/processor/processors/CompactionProcessor.java @@ -16,6 +16,7 @@ package com.linecorp.decaton.processor.processors; +import java.nio.ByteBuffer; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentMap; import java.util.concurrent.ScheduledExecutorService; @@ -26,11 +27,10 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import com.linecorp.decaton.processor.Completion; import com.linecorp.decaton.processor.DecatonProcessor; import com.linecorp.decaton.processor.ProcessingContext; import com.linecorp.decaton.processor.metrics.Metrics; -import com.linecorp.decaton.processor.runtime.internal.TaskKey; +import com.linecorp.decaton.processor.Completion; import io.micrometer.core.instrument.Counter; import lombok.AccessLevel; @@ -85,7 +85,7 @@ public enum CompactChoice { } private final ScheduledExecutorService executor; - private final ConcurrentMap> windowedTasks = new ConcurrentHashMap<>(); + private final ConcurrentMap> windowedTasks = new ConcurrentHashMap<>(); private final BiFunction, CompactingTask, CompactChoice> compactor; private final long lingerMillis; @@ -146,8 +146,8 @@ public CompactionProcessor( this(lingerMillis, compactor, null); } - private void flush(TaskKey key) throws InterruptedException { - CompactingTask task = windowedTasks.remove(key); + private void flush(byte[] key) throws InterruptedException { + CompactingTask task = windowedTasks.remove(ByteBuffer.wrap(key)); if (task == null) { return; } @@ -156,9 +156,10 @@ private void flush(TaskKey key) throws InterruptedException { // visible for testing Runnable flushTask(ProcessingContext context) { + byte[] key = context.key(); return () -> { try { - flush(context.key()); + flush(key); } catch (InterruptedException e) { logger.error("interrupted while flushing compacted task result", e); } catch (RuntimeException e) { @@ -182,15 +183,15 @@ private void scheduleFlush(ProcessingContext context) { @Override public void process(ProcessingContext context, T task) throws InterruptedException { + byte[] key = context.key(); CompactingTask newTask = new CompactingTask<>(context.deferCompletion(), context, task); - TaskKey key = context.key(); // Even though we do this read and following updates in separate operation, race condition can't be // happened because tasks are guaranteed to be serialized by it's key, so simultaneous processing // of tasks sharing the same key won't be happen. - CompactingTask prevTask = windowedTasks.get(key); + CompactingTask prevTask = windowedTasks.get(ByteBuffer.wrap(key)); if (prevTask == null) { - windowedTasks.put(key, newTask); + windowedTasks.put(ByteBuffer.wrap(key), newTask); scheduleFlush(context); return; } @@ -205,7 +206,7 @@ public void process(ProcessingContext context, T task) throws InterruptedExce case PICK_RIGHT: case PICK_EITHER: // Newer task has larger offset. We want to forward consumed offset. // Update the last task with new one. - Object oldEntry = windowedTasks.put(key, newTask); + Object oldEntry = windowedTasks.put(ByteBuffer.wrap(key), newTask); if (oldEntry == null) { // By race condition, there is a chance that the scheduled flush for preceding task just // got fired right after this method checked the key's existence at the beginning of this diff --git a/processor/src/main/java/com/linecorp/decaton/processor/runtime/ProcessorSubscription.java b/processor/src/main/java/com/linecorp/decaton/processor/runtime/ProcessorSubscription.java index 1a6029cc..ad368666 100644 --- a/processor/src/main/java/com/linecorp/decaton/processor/runtime/ProcessorSubscription.java +++ b/processor/src/main/java/com/linecorp/decaton/processor/runtime/ProcessorSubscription.java @@ -47,7 +47,6 @@ import com.linecorp.decaton.processor.runtime.internal.PartitionContexts; import com.linecorp.decaton.processor.runtime.internal.Processors; import com.linecorp.decaton.processor.runtime.internal.SubscriptionScope; -import com.linecorp.decaton.processor.runtime.internal.TaskKey; import com.linecorp.decaton.processor.runtime.internal.TaskRequest; import com.linecorp.decaton.processor.runtime.internal.Utils; import com.linecorp.decaton.processor.runtime.internal.Utils.Timer; @@ -121,7 +120,7 @@ public void receive(ConsumerRecord record) { if (blacklistedKeysFilter.shouldTake(record)) { TaskRequest taskRequest = - new TaskRequest(tp, record.offset(), offsetState, new TaskKey(record.key()), + new TaskRequest(tp, record.offset(), offsetState, record.key(), record.headers(), trace, record.value()); context.addRequest(taskRequest); } else { diff --git a/processor/src/main/java/com/linecorp/decaton/processor/runtime/internal/DecatonTaskRetryQueueingProcessor.java b/processor/src/main/java/com/linecorp/decaton/processor/runtime/internal/DecatonTaskRetryQueueingProcessor.java index 81245133..d2e4bf73 100644 --- a/processor/src/main/java/com/linecorp/decaton/processor/runtime/internal/DecatonTaskRetryQueueingProcessor.java +++ b/processor/src/main/java/com/linecorp/decaton/processor/runtime/internal/DecatonTaskRetryQueueingProcessor.java @@ -64,7 +64,7 @@ public void process(ProcessingContext context, byte[] serializedTask) .setSerializedTask(ByteString.copyFrom(serializedTask)) .build(); - CompletableFuture future = producer.sendRequest(context.key().array(), request); + CompletableFuture future = producer.sendRequest(context.key(), request); future.whenComplete((r, e) -> { if (e == null) { metrics.retryQueuedTasks.increment(); diff --git a/processor/src/main/java/com/linecorp/decaton/processor/runtime/internal/ProcessingContextImpl.java b/processor/src/main/java/com/linecorp/decaton/processor/runtime/internal/ProcessingContextImpl.java index 61bafa70..47dacc23 100644 --- a/processor/src/main/java/com/linecorp/decaton/processor/runtime/internal/ProcessingContextImpl.java +++ b/processor/src/main/java/com/linecorp/decaton/processor/runtime/internal/ProcessingContextImpl.java @@ -72,7 +72,7 @@ public TaskMetadata metadata() { } @Override - public TaskKey key() { + public byte[] key() { return request.key(); } diff --git a/processor/src/main/java/com/linecorp/decaton/processor/runtime/internal/SubPartitioner.java b/processor/src/main/java/com/linecorp/decaton/processor/runtime/internal/SubPartitioner.java index 93dcf425..44740c74 100644 --- a/processor/src/main/java/com/linecorp/decaton/processor/runtime/internal/SubPartitioner.java +++ b/processor/src/main/java/com/linecorp/decaton/processor/runtime/internal/SubPartitioner.java @@ -32,7 +32,7 @@ private static int toPositive(int number) { return number & 2147483647; } - public int partitionFor(TaskKey key) { + public int partitionFor(byte[] key) { if (key == null) { return toPositive((int) monotonicValueSupplier.getAndIncrement()) % bound; } else { @@ -41,11 +41,10 @@ public int partitionFor(TaskKey key) { // Here just by adding few bytes to the key we can "shift" hashing of the key and // can get back better distribution again in murmur2 result to evenly distribute keys // for subpartitions. - final byte[] keyArray = key.array(); - final ByteBuffer bb = ByteBuffer.allocate(keyArray.length + 2); + final ByteBuffer bb = ByteBuffer.allocate(key.length + 2); bb.put((byte) 's'); bb.put((byte) ':'); - bb.put(keyArray); + bb.put(key); int hash = org.apache.kafka.common.utils.Utils.murmur2(bb.array()); return toPositive(hash) % bound; diff --git a/processor/src/main/java/com/linecorp/decaton/processor/runtime/internal/TaskKey.java b/processor/src/main/java/com/linecorp/decaton/processor/runtime/internal/TaskKey.java deleted file mode 100644 index 33c73161..00000000 --- a/processor/src/main/java/com/linecorp/decaton/processor/runtime/internal/TaskKey.java +++ /dev/null @@ -1,57 +0,0 @@ -/* - * Copyright 2020 LINE Corporation - * - * LINE Corporation licenses this file to you under the Apache License, - * version 2.0 (the "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at: - * - * https://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT - * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the - * License for the specific language governing permissions and limitations - * under the License. - */ - -package com.linecorp.decaton.processor.runtime.internal; - -import java.nio.charset.StandardCharsets; -import java.util.Arrays; - -public class TaskKey { - private static final byte[] EMPTY = new byte[0]; - private final byte[] key; - private int hash; - - public TaskKey(byte[] key) { - this.key = key == null ? EMPTY : key; - } - - public byte[] array() { - return key; - } - - @Override - public boolean equals(Object o) { - if (this == o) {return true;} - if (o == null || getClass() != o.getClass()) {return false;} - - final TaskKey taskKey = (TaskKey) o; - - return Arrays.equals(key, taskKey.key); - } - - @Override - public int hashCode() { - if (hash == 0 && key.length > 0) { - hash = Arrays.hashCode(key); - } - return hash; - } - - @Override - public String toString() { - return "TaskKey{key=" + new String(key, StandardCharsets.UTF_8) + '}'; - } -} diff --git a/processor/src/main/java/com/linecorp/decaton/processor/runtime/internal/TaskRequest.java b/processor/src/main/java/com/linecorp/decaton/processor/runtime/internal/TaskRequest.java index 8b0e41aa..4cc195cb 100644 --- a/processor/src/main/java/com/linecorp/decaton/processor/runtime/internal/TaskRequest.java +++ b/processor/src/main/java/com/linecorp/decaton/processor/runtime/internal/TaskRequest.java @@ -32,7 +32,8 @@ public class TaskRequest { private final TopicPartition topicPartition; private final long recordOffset; private final OffsetState offsetState; - private final TaskKey key; + @ToString.Exclude + private final byte[] key; @ToString.Exclude private final Headers headers; @ToString.Exclude @@ -43,7 +44,7 @@ public class TaskRequest { public TaskRequest(TopicPartition topicPartition, long recordOffset, OffsetState offsetState, - TaskKey key, + byte[] key, Headers headers, RecordTraceHandle trace, byte[] rawRequestBytes) { diff --git a/processor/src/test/java/com/linecorp/decaton/processor/processors/CompactionProcessorTest.java b/processor/src/test/java/com/linecorp/decaton/processor/processors/CompactionProcessorTest.java index 8a4f9b6a..9ec4a0a2 100644 --- a/processor/src/test/java/com/linecorp/decaton/processor/processors/CompactionProcessorTest.java +++ b/processor/src/test/java/com/linecorp/decaton/processor/processors/CompactionProcessorTest.java @@ -55,7 +55,6 @@ import com.linecorp.decaton.processor.runtime.DecatonTask; import com.linecorp.decaton.processor.runtime.ProcessorProperties; import com.linecorp.decaton.processor.runtime.internal.ProcessingContextImpl; -import com.linecorp.decaton.processor.runtime.internal.TaskKey; import com.linecorp.decaton.processor.runtime.internal.TaskRequest; import com.linecorp.decaton.processor.tracing.internal.NoopTracingProvider.NoopTrace; import com.linecorp.decaton.protocol.Sample.HelloTask; @@ -104,7 +103,7 @@ private TaskInput put(DecatonProcessor processor, taskData, taskData.toByteArray()); TaskRequest request = new TaskRequest( - new TopicPartition("topic", 1), 1, null, new TaskKey(name.getBytes(StandardCharsets.UTF_8)), null, NoopTrace.INSTANCE, null); + new TopicPartition("topic", 1), 1, null, name.getBytes(StandardCharsets.UTF_8), null, NoopTrace.INSTANCE, null); ProcessingContext context = spy(new ProcessingContextImpl<>("subscription", request, task, Arrays.asList(processor, downstream), null, diff --git a/processor/src/test/java/com/linecorp/decaton/processor/runtime/internal/DecatonTaskRetryQueueingProcessorTest.java b/processor/src/test/java/com/linecorp/decaton/processor/runtime/internal/DecatonTaskRetryQueueingProcessorTest.java index 86167293..0feb46f6 100644 --- a/processor/src/test/java/com/linecorp/decaton/processor/runtime/internal/DecatonTaskRetryQueueingProcessorTest.java +++ b/processor/src/test/java/com/linecorp/decaton/processor/runtime/internal/DecatonTaskRetryQueueingProcessorTest.java @@ -77,13 +77,13 @@ public void setUp() { processor = new DecatonTaskRetryQueueingProcessor(scope, producer); doReturn(CompletableFuture.completedFuture(null)).when(producer).sendRequest(any(), any()); doReturn(new CompletionImpl()).when(context).deferCompletion(); - doReturn(new TaskKey("key".getBytes(StandardCharsets.UTF_8))).when(context).key(); + doReturn("key".getBytes(StandardCharsets.UTF_8)).when(context).key(); doReturn(TaskMetadata.builder().build()).when(context).metadata(); } @Test public void testRetryRequest() throws InterruptedException { - TaskKey key = new TaskKey("key".getBytes(StandardCharsets.UTF_8)); + byte[] key = "key".getBytes(StandardCharsets.UTF_8); TaskMetadata meta = TaskMetadata.builder() .sourceApplicationId("unit-test") @@ -101,7 +101,7 @@ public void testRetryRequest() throws InterruptedException { processor.process(context, task.toByteArray()); ArgumentCaptor captor = ArgumentCaptor.forClass(DecatonTaskRequest.class); - verify(producer, times(1)).sendRequest(eq(key.array()), captor.capture()); + verify(producer, times(1)).sendRequest(eq(key), captor.capture()); DecatonTaskRequest request = captor.getValue(); assertEquals(task.toByteString(), request.getSerializedTask()); diff --git a/processor/src/test/java/com/linecorp/decaton/processor/runtime/internal/ProcessPipelineTest.java b/processor/src/test/java/com/linecorp/decaton/processor/runtime/internal/ProcessPipelineTest.java index a165ef5d..8bd03b6a 100644 --- a/processor/src/test/java/com/linecorp/decaton/processor/runtime/internal/ProcessPipelineTest.java +++ b/processor/src/test/java/com/linecorp/decaton/processor/runtime/internal/ProcessPipelineTest.java @@ -94,8 +94,7 @@ public class ProcessPipelineTest { private static TaskRequest taskRequest() { return new TaskRequest( - new TopicPartition("topic", 1), 1, new OffsetState(1234), - new TaskKey("TEST".getBytes(StandardCharsets.UTF_8)), null, NoopTrace.INSTANCE, REQUEST.toByteArray()); + new TopicPartition("topic", 1), 1, new OffsetState(1234), "TEST".getBytes(StandardCharsets.UTF_8), null, NoopTrace.INSTANCE, REQUEST.toByteArray()); } @Rule diff --git a/processor/src/test/java/com/linecorp/decaton/processor/runtime/internal/ProcessingContextImplTest.java b/processor/src/test/java/com/linecorp/decaton/processor/runtime/internal/ProcessingContextImplTest.java index 50a81d1e..30064216 100644 --- a/processor/src/test/java/com/linecorp/decaton/processor/runtime/internal/ProcessingContextImplTest.java +++ b/processor/src/test/java/com/linecorp/decaton/processor/runtime/internal/ProcessingContextImplTest.java @@ -112,7 +112,7 @@ private static void terminateExecutor(ExecutorService executor) throws Interrupt private static ProcessingContextImpl context(RecordTraceHandle traceHandle, DecatonProcessor... processors) { TaskRequest request = new TaskRequest( - new TopicPartition("topic", 1), 1, null, new TaskKey("TEST".getBytes(StandardCharsets.UTF_8)), + new TopicPartition("topic", 1), 1, null, "TEST".getBytes(StandardCharsets.UTF_8), null, traceHandle, REQUEST.toByteArray()); DecatonTask task = new DecatonTask<>( TaskMetadata.fromProto(REQUEST.getMetadata()), TASK, TASK.toByteArray()); @@ -363,7 +363,7 @@ public void process(ProcessingContext context, byte[] task) } }); TaskRequest request = new TaskRequest( - new TopicPartition("topic", 1), 1, null, new TaskKey("TEST".getBytes(StandardCharsets.UTF_8)), null, null, REQUEST.toByteArray()); + new TopicPartition("topic", 1), 1, null, "TEST".getBytes(StandardCharsets.UTF_8), null, null, REQUEST.toByteArray()); DecatonTask task = new DecatonTask<>( TaskMetadata.fromProto(REQUEST.getMetadata()), TASK.toByteArray(), TASK.toByteArray()); @@ -407,7 +407,7 @@ public void process(ProcessingContext context, byte[] task) } }); TaskRequest request = new TaskRequest( - new TopicPartition("topic", 1), 1, null, new TaskKey("TEST".getBytes(StandardCharsets.UTF_8)), null, null, REQUEST.toByteArray()); + new TopicPartition("topic", 1), 1, null, "TEST".getBytes(StandardCharsets.UTF_8), null, null, REQUEST.toByteArray()); DecatonTask task = new DecatonTask<>( TaskMetadata.fromProto(REQUEST.getMetadata()), TASK.toByteArray(), TASK.toByteArray()); diff --git a/processor/src/test/java/com/linecorp/decaton/processor/runtime/internal/SubPartitionerTest.java b/processor/src/test/java/com/linecorp/decaton/processor/runtime/internal/SubPartitionerTest.java index 095a122f..25fc9a2e 100644 --- a/processor/src/test/java/com/linecorp/decaton/processor/runtime/internal/SubPartitionerTest.java +++ b/processor/src/test/java/com/linecorp/decaton/processor/runtime/internal/SubPartitionerTest.java @@ -28,18 +28,20 @@ import org.junit.Before; import org.junit.Test; +import com.linecorp.decaton.processor.runtime.internal.SubPartitioner; + public class SubPartitionerTest { static final int DIST_KEYS_COUNT = 10000; static final int[] PARTITION_COUNTS = { 1, 8, 16, 32, 64, 96 }; static final int[] SUBPARTITION_COUNTS = { 10, 17, 32, 64 }; static final double TARGET_STDDEV_RATIO = 0.05; - static final TaskKey[] keys = new TaskKey[DIST_KEYS_COUNT]; + static final byte[][] keys = new byte[DIST_KEYS_COUNT][]; @Before public void setUp() { for (int i = 0; i < keys.length; i++) { - keys[i] = new TaskKey(String.valueOf(i).getBytes(StandardCharsets.UTF_8)); + keys[i] = String.valueOf(i).getBytes(StandardCharsets.UTF_8); } } @@ -53,14 +55,14 @@ private static double stddev(int[] counts) { @Test public void testEvenlyDistributedSelection() { for (int partitionCount : PARTITION_COUNTS) { - List> partitions = new ArrayList<>(partitionCount); + List> partitions = new ArrayList<>(partitionCount); for (int i = 0; i < partitionCount; i++) { partitions.add(new ArrayList<>()); } - for (TaskKey key : keys) { + for (byte[] key : keys) { // This is the way used to determine partition in Kafka's DefaultPartitioner - int partition = (Utils.murmur2(key.array()) & 2147483647) % partitionCount; + int partition = (Utils.murmur2(key) & 2147483647) % partitionCount; partitions.get(partition).add(key); } @@ -70,10 +72,10 @@ public void testEvenlyDistributedSelection() { partitionCount, partStddev, Arrays.toString(partCounts)); for (int subpartitionCount : SUBPARTITION_COUNTS) { - for (List partition : partitions) { + for (List partition : partitions) { int[] counts = new int[subpartitionCount]; SubPartitioner subPartitioner = new SubPartitioner(counts.length); - for (TaskKey key : partition) { + for (byte[] key : partition) { int subPartition = subPartitioner.partitionFor(key); counts[subPartition]++; } @@ -95,7 +97,7 @@ public void testEvenlyDistributedSelection() { public void testConsistentSelectionForSameKeys() { for (int subpartitionCount : SUBPARTITION_COUNTS) { SubPartitioner subPartitioner = new SubPartitioner(subpartitionCount); - for (TaskKey key : keys) { + for (byte[] key : keys) { int assign1 = subPartitioner.partitionFor(key); int assign2 = subPartitioner.partitionFor(key); assertEquals(String.format("[%d] assign of %s", subpartitionCount, key), diff --git a/testing/src/main/java/com/linecorp/decaton/testing/processor/ProcessOrdering.java b/testing/src/main/java/com/linecorp/decaton/testing/processor/ProcessOrdering.java index 239a995b..de5d15d7 100644 --- a/testing/src/main/java/com/linecorp/decaton/testing/processor/ProcessOrdering.java +++ b/testing/src/main/java/com/linecorp/decaton/testing/processor/ProcessOrdering.java @@ -21,6 +21,7 @@ import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; +import java.nio.ByteBuffer; import java.util.ArrayDeque; import java.util.ArrayList; import java.util.Deque; @@ -30,31 +31,31 @@ import java.util.Map.Entry; import com.linecorp.decaton.processor.TaskMetadata; -import com.linecorp.decaton.processor.runtime.internal.TaskKey; public class ProcessOrdering implements ProcessingGuarantee { private final Map taskToOffset = new HashMap<>(); - private final Map> producedRecords = new HashMap<>(); - private final Map> processedRecords = new HashMap<>(); + private final Map> producedRecords = new HashMap<>(); + private final Map> processedRecords = new HashMap<>(); @Override public synchronized void onProduce(ProducedRecord record) { taskToOffset.put(record.task(), record.offset()); - producedRecords.computeIfAbsent(record.key(), + producedRecords.computeIfAbsent(ByteBuffer.wrap(record.key()), key -> new ArrayList<>()).add(record.task()); } @Override public synchronized void onProcess(TaskMetadata metadata, ProcessedRecord record) { - processedRecords.computeIfAbsent(record.key(), + processedRecords.computeIfAbsent(ByteBuffer.wrap(record.key()), key -> new ArrayList<>()).add(record.task()); } @Override public void doAssert() { - for (Entry> entry : producedRecords.entrySet()) { + for (Entry> entry : producedRecords.entrySet()) { + final ByteBuffer key = entry.getKey(); List produced = entry.getValue(); - List processed = processedRecords.get(entry.getKey()); + List processed = processedRecords.get(key); assertNotNull(processed); assertOrdering(taskToOffset, produced, processed); diff --git a/testing/src/main/java/com/linecorp/decaton/testing/processor/ProcessedRecord.java b/testing/src/main/java/com/linecorp/decaton/testing/processor/ProcessedRecord.java index 05991686..9e44539f 100644 --- a/testing/src/main/java/com/linecorp/decaton/testing/processor/ProcessedRecord.java +++ b/testing/src/main/java/com/linecorp/decaton/testing/processor/ProcessedRecord.java @@ -17,8 +17,8 @@ package com.linecorp.decaton.testing.processor; import com.linecorp.decaton.processor.DecatonProcessor; -import com.linecorp.decaton.processor.runtime.internal.TaskKey; +import lombok.ToString; import lombok.Value; import lombok.experimental.Accessors; @@ -31,7 +31,8 @@ public class ProcessedRecord { /** * Key of the task */ - TaskKey key; + @ToString.Exclude + byte[] key; /** * Processed task */ diff --git a/testing/src/main/java/com/linecorp/decaton/testing/processor/ProcessorTestSuite.java b/testing/src/main/java/com/linecorp/decaton/testing/processor/ProcessorTestSuite.java index bd144800..78a48ffd 100644 --- a/testing/src/main/java/com/linecorp/decaton/testing/processor/ProcessorTestSuite.java +++ b/testing/src/main/java/com/linecorp/decaton/testing/processor/ProcessorTestSuite.java @@ -58,7 +58,6 @@ import com.linecorp.decaton.processor.runtime.RetryConfig.RetryConfigBuilder; import com.linecorp.decaton.processor.runtime.SubscriptionStateListener; import com.linecorp.decaton.processor.runtime.TaskExtractor; -import com.linecorp.decaton.processor.runtime.internal.TaskKey; import com.linecorp.decaton.processor.tracing.TracingProvider; import com.linecorp.decaton.protocol.Decaton.DecatonTaskRequest; import com.linecorp.decaton.protocol.Decaton.TaskMetadataProto; @@ -171,7 +170,6 @@ public Builder excludeSemantics(GuaranteeType... guarantees) { } return this; } - /** * Include additional semantics in assertion. * Feature-specific processing guarantee will be injected through this method @@ -193,8 +191,7 @@ public ProcessorTestSuite build() { semantics.addAll(customSemantics); if (statesListener == null) { - statesListener = (id, state) -> { - }; + statesListener = (id, state) -> {}; } return new ProcessorTestSuite(rule, @@ -384,7 +381,7 @@ private CompletableFuture> produceTasks( producer.send(record, (metadata, exception) -> { if (exception == null) { future.complete(metadata); - onProduce.accept(new ProducedRecord(new TaskKey(key), + onProduce.accept(new ProducedRecord(key, new TopicPartition(metadata.topic(), metadata.partition()), metadata.offset(), diff --git a/testing/src/main/java/com/linecorp/decaton/testing/processor/ProducedRecord.java b/testing/src/main/java/com/linecorp/decaton/testing/processor/ProducedRecord.java index dc6b28a9..75ed832a 100644 --- a/testing/src/main/java/com/linecorp/decaton/testing/processor/ProducedRecord.java +++ b/testing/src/main/java/com/linecorp/decaton/testing/processor/ProducedRecord.java @@ -19,8 +19,6 @@ import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.header.Headers; -import com.linecorp.decaton.processor.runtime.internal.TaskKey; - import lombok.Value; import lombok.experimental.Accessors; @@ -33,7 +31,7 @@ public class ProducedRecord { /** * Key of the task */ - TaskKey key; + byte[] key; /** * Topic partition the record was sent to */ diff --git a/testing/src/main/java/com/linecorp/decaton/testing/processor/SerialProcessing.java b/testing/src/main/java/com/linecorp/decaton/testing/processor/SerialProcessing.java index c59ee492..b8c2cdd3 100644 --- a/testing/src/main/java/com/linecorp/decaton/testing/processor/SerialProcessing.java +++ b/testing/src/main/java/com/linecorp/decaton/testing/processor/SerialProcessing.java @@ -19,6 +19,8 @@ import static org.hamcrest.MatcherAssert.assertThat; import static org.hamcrest.Matchers.lessThan; +import java.nio.ByteBuffer; +import java.nio.charset.StandardCharsets; import java.util.ArrayList; import java.util.Comparator; import java.util.HashMap; @@ -27,10 +29,9 @@ import java.util.Map.Entry; import com.linecorp.decaton.processor.TaskMetadata; -import com.linecorp.decaton.processor.runtime.internal.TaskKey; public class SerialProcessing implements ProcessingGuarantee { - private final Map> records = new HashMap<>(); + private final Map> records = new HashMap<>(); @Override public void onProduce(ProducedRecord record) { @@ -39,13 +40,14 @@ public void onProduce(ProducedRecord record) { @Override public synchronized void onProcess(TaskMetadata metadata, ProcessedRecord record) { - records.computeIfAbsent(record.key(), key -> new ArrayList<>()).add(record); + records.computeIfAbsent(ByteBuffer.wrap(record.key()), + key -> new ArrayList<>()).add(record); } @Override public void doAssert() { // Checks there's no overlap between two consecutive records' processing time - for (Entry> entry : records.entrySet()) { + for (Entry> entry : records.entrySet()) { List perKeyRecords = entry.getValue(); perKeyRecords.sort(Comparator.comparingLong(ProcessedRecord::startTimeNanos)); @@ -53,7 +55,8 @@ public void doAssert() { ProcessedRecord prev = perKeyRecords.get(i - 1); ProcessedRecord current = perKeyRecords.get(i); - assertThat("Process time shouldn't overlap. key: " + entry.getKey(), + String key = StandardCharsets.UTF_8.decode(entry.getKey()).toString(); + assertThat("Process time shouldn't overlap. key: " + key, prev.endTimeNanos(), lessThan(current.startTimeNanos())); } } From f034907fe29c2d7f7e4e2658e15b3825f51956ed Mon Sep 17 00:00:00 2001 From: Koki Kato Date: Tue, 21 Jun 2022 14:30:07 +0900 Subject: [PATCH 14/27] Revert "Display byte keys with a given key formatter" This reverts commit f99151016652001a44be3595af2834fc95b18e21. --- .../decaton/processor/LoggingContext.java | 9 +++---- .../processor/formatter/KeyFormatter.java | 26 ------------------- .../processor/runtime/ProcessorsBuilder.java | 14 +--------- .../runtime/internal/ProcessPipeline.java | 19 +++++--------- .../internal/ProcessingContextImpl.java | 12 +++------ .../runtime/internal/Processors.java | 9 ++----- .../runtime/internal/TaskRequest.java | 1 - .../processors/CompactionProcessorTest.java | 3 +-- .../runtime/internal/ProcessPipelineTest.java | 4 +-- .../internal/ProcessingContextImplTest.java | 7 +++-- .../runtime/internal/ProcessorsTest.java | 3 +-- .../testing/processor/ProcessedRecord.java | 2 -- .../testing/processor/SerialProcessing.java | 4 +-- 13 files changed, 25 insertions(+), 88 deletions(-) delete mode 100644 processor/src/main/java/com/linecorp/decaton/processor/formatter/KeyFormatter.java diff --git a/processor/src/main/java/com/linecorp/decaton/processor/LoggingContext.java b/processor/src/main/java/com/linecorp/decaton/processor/LoggingContext.java index a182ad88..7e9b74f4 100644 --- a/processor/src/main/java/com/linecorp/decaton/processor/LoggingContext.java +++ b/processor/src/main/java/com/linecorp/decaton/processor/LoggingContext.java @@ -18,7 +18,6 @@ import org.slf4j.MDC; -import com.linecorp.decaton.processor.formatter.KeyFormatter; import com.linecorp.decaton.processor.runtime.internal.TaskRequest; /** @@ -41,11 +40,11 @@ public class LoggingContext implements AutoCloseable { private final boolean enabled; - public LoggingContext(boolean enabled, String subscriptionId, TaskRequest request, TaskMetadata metadata, KeyFormatter keyFormatter) { + public LoggingContext(boolean enabled, String subscriptionId, TaskRequest request, TaskMetadata metadata) { this.enabled = enabled; if (enabled) { MDC.put(METADATA_KEY, metadata.toString()); - MDC.put(TASK_KEY, keyFormatter.format(request.key())); + MDC.put(TASK_KEY, String.valueOf(request.key())); MDC.put(SUBSCRIPTION_ID_KEY, subscriptionId); MDC.put(OFFSET_KEY, String.valueOf(request.recordOffset())); MDC.put(TOPIC_KEY, request.topicPartition().topic()); @@ -53,8 +52,8 @@ public LoggingContext(boolean enabled, String subscriptionId, TaskRequest reques } } - public LoggingContext(String subscriptionId, TaskRequest request, TaskMetadata metadata, KeyFormatter keyFormatter) { - this(true, subscriptionId, request, metadata, keyFormatter); + public LoggingContext(String subscriptionId, TaskRequest request, TaskMetadata metadata) { + this(true, subscriptionId, request, metadata); } @Override diff --git a/processor/src/main/java/com/linecorp/decaton/processor/formatter/KeyFormatter.java b/processor/src/main/java/com/linecorp/decaton/processor/formatter/KeyFormatter.java deleted file mode 100644 index 4fea5856..00000000 --- a/processor/src/main/java/com/linecorp/decaton/processor/formatter/KeyFormatter.java +++ /dev/null @@ -1,26 +0,0 @@ -/* - * Copyright 2020 LINE Corporation - * - * LINE Corporation licenses this file to you under the Apache License, - * version 2.0 (the "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at: - * - * https://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT - * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the - * License for the specific language governing permissions and limitations - * under the License. - */ - -package com.linecorp.decaton.processor.formatter; - -import java.nio.charset.StandardCharsets; - -@FunctionalInterface -public interface KeyFormatter { - KeyFormatter DEFAULT = key -> new String(key, StandardCharsets.UTF_8); - - String format(byte[] key); -} diff --git a/processor/src/main/java/com/linecorp/decaton/processor/runtime/ProcessorsBuilder.java b/processor/src/main/java/com/linecorp/decaton/processor/runtime/ProcessorsBuilder.java index c843b467..be18d249 100644 --- a/processor/src/main/java/com/linecorp/decaton/processor/runtime/ProcessorsBuilder.java +++ b/processor/src/main/java/com/linecorp/decaton/processor/runtime/ProcessorsBuilder.java @@ -22,7 +22,6 @@ import com.linecorp.decaton.common.Deserializer; import com.linecorp.decaton.processor.DecatonProcessor; -import com.linecorp.decaton.processor.formatter.KeyFormatter; import com.linecorp.decaton.processor.runtime.internal.DecatonProcessorSupplierImpl; import com.linecorp.decaton.processor.runtime.internal.DefaultTaskExtractor; import com.linecorp.decaton.processor.runtime.internal.Processors; @@ -43,7 +42,6 @@ public class ProcessorsBuilder { private final TaskExtractor retryTaskExtractor; private final List> suppliers; - private KeyFormatter keyFormatter = KeyFormatter.DEFAULT; public ProcessorsBuilder(String topic, TaskExtractor taskExtractor, TaskExtractor retryTaskExtractor) { this.topic = topic; @@ -132,17 +130,7 @@ public ProcessorsBuilder thenProcess(DecatonProcessor processor) { return thenProcess(new DecatonProcessorSupplierImpl<>(() -> processor, ProcessorScope.PROVIDED)); } - /** - * Configure a key formatter that translates record keys from bytes into String for logging purpose. - * @param keyFormatter a {@link KeyFormatter}, function that maps bytes into String - * @return updated instance of {@link ProcessorsBuilder}. - */ - public ProcessorsBuilder setKeyFormatter(KeyFormatter keyFormatter) { - this.keyFormatter = keyFormatter; - return this; - } - public Processors build(DecatonProcessorSupplier retryProcessorSupplier) { - return new Processors<>(suppliers, retryProcessorSupplier, taskExtractor, retryTaskExtractor, keyFormatter); + return new Processors<>(suppliers, retryProcessorSupplier, taskExtractor, retryTaskExtractor); } } diff --git a/processor/src/main/java/com/linecorp/decaton/processor/runtime/internal/ProcessPipeline.java b/processor/src/main/java/com/linecorp/decaton/processor/runtime/internal/ProcessPipeline.java index 4d3c8385..740adcba 100644 --- a/processor/src/main/java/com/linecorp/decaton/processor/runtime/internal/ProcessPipeline.java +++ b/processor/src/main/java/com/linecorp/decaton/processor/runtime/internal/ProcessPipeline.java @@ -21,17 +21,16 @@ import java.util.Collections; import java.util.List; -import com.linecorp.decaton.processor.Completion; import com.linecorp.decaton.processor.DecatonProcessor; -import com.linecorp.decaton.processor.LoggingContext; +import com.linecorp.decaton.processor.Completion; +import com.linecorp.decaton.processor.runtime.DecatonTask; import com.linecorp.decaton.processor.ProcessingContext; -import com.linecorp.decaton.processor.formatter.KeyFormatter; +import com.linecorp.decaton.processor.runtime.ProcessorProperties; +import com.linecorp.decaton.processor.runtime.TaskExtractor; import com.linecorp.decaton.processor.metrics.Metrics; import com.linecorp.decaton.processor.metrics.Metrics.ProcessMetrics; import com.linecorp.decaton.processor.metrics.Metrics.TaskMetrics; -import com.linecorp.decaton.processor.runtime.DecatonTask; -import com.linecorp.decaton.processor.runtime.ProcessorProperties; -import com.linecorp.decaton.processor.runtime.TaskExtractor; +import com.linecorp.decaton.processor.LoggingContext; import com.linecorp.decaton.processor.runtime.internal.Utils.Timer; import lombok.extern.slf4j.Slf4j; @@ -42,7 +41,6 @@ public class ProcessPipeline implements AutoCloseable { private final List> processors; private final DecatonProcessor retryProcessor; private final TaskExtractor taskExtractor; - private final KeyFormatter keyFormatter; private final ExecutionScheduler scheduler; private final TaskMetrics taskMetrics; private final ProcessMetrics processMetrics; @@ -53,7 +51,6 @@ public class ProcessPipeline implements AutoCloseable { List> processors, DecatonProcessor retryProcessor, TaskExtractor taskExtractor, - KeyFormatter keyFormatter, ExecutionScheduler scheduler, Metrics metrics, Clock clock) { @@ -61,7 +58,6 @@ public class ProcessPipeline implements AutoCloseable { this.processors = Collections.unmodifiableList(processors); this.retryProcessor = retryProcessor; this.taskExtractor = taskExtractor; - this.keyFormatter = keyFormatter; this.scheduler = scheduler; this.clock = clock; @@ -73,10 +69,9 @@ public ProcessPipeline(ThreadScope scope, List> processors, DecatonProcessor retryProcessor, TaskExtractor taskExtractor, - KeyFormatter keyFormatter, ExecutionScheduler scheduler, Metrics metrics) { - this(scope, processors, retryProcessor, taskExtractor, keyFormatter, scheduler, metrics, Clock.systemDefaultZone()); + this(scope, processors, retryProcessor, taskExtractor, scheduler, metrics, Clock.systemDefaultZone()); } public void scheduleThenProcess(TaskRequest request) throws InterruptedException { @@ -124,7 +119,7 @@ DecatonTask extract(TaskRequest request) { Completion process(TaskRequest request, DecatonTask task) throws InterruptedException { ProcessingContext context = new ProcessingContextImpl<>(scope.subscriptionId(), request, task, processors, retryProcessor, - keyFormatter, scope.props()); + scope.props()); Timer timer = Utils.timer(); Completion processResult; diff --git a/processor/src/main/java/com/linecorp/decaton/processor/runtime/internal/ProcessingContextImpl.java b/processor/src/main/java/com/linecorp/decaton/processor/runtime/internal/ProcessingContextImpl.java index 47dacc23..89cc8402 100644 --- a/processor/src/main/java/com/linecorp/decaton/processor/runtime/internal/ProcessingContextImpl.java +++ b/processor/src/main/java/com/linecorp/decaton/processor/runtime/internal/ProcessingContextImpl.java @@ -23,13 +23,12 @@ import org.apache.kafka.common.header.Headers; -import com.linecorp.decaton.processor.Completion; -import com.linecorp.decaton.processor.Completion.TimeoutChoice; import com.linecorp.decaton.processor.DecatonProcessor; import com.linecorp.decaton.processor.LoggingContext; import com.linecorp.decaton.processor.ProcessingContext; import com.linecorp.decaton.processor.TaskMetadata; -import com.linecorp.decaton.processor.formatter.KeyFormatter; +import com.linecorp.decaton.processor.Completion; +import com.linecorp.decaton.processor.Completion.TimeoutChoice; import com.linecorp.decaton.processor.runtime.DecatonTask; import com.linecorp.decaton.processor.runtime.ProcessorProperties; import com.linecorp.decaton.processor.tracing.TracingProvider.ProcessorTraceHandle; @@ -45,7 +44,6 @@ public class ProcessingContextImpl implements ProcessingContext { private final DecatonTask task; private final List> downstreams; private final DecatonProcessor retryQueueingProcessor; - private final KeyFormatter keyFormatter; private final ProcessorProperties props; private final AtomicReference deferredCompletion; @@ -54,14 +52,12 @@ public ProcessingContextImpl(String subscriptionId, DecatonTask task, List> downstreams, DecatonProcessor retryQueueingProcessor, - KeyFormatter keyFormatter, ProcessorProperties props) { this.subscriptionId = subscriptionId; this.request = request; this.task = task; this.downstreams = Collections.unmodifiableList(downstreams); this.retryQueueingProcessor = retryQueueingProcessor; - this.keyFormatter = keyFormatter; this.props = props; deferredCompletion = new AtomicReference<>(); } @@ -84,7 +80,7 @@ public Headers headers() { @Override public LoggingContext loggingContext() { boolean enabled = props.get(ProcessorProperties.CONFIG_LOGGING_MDC_ENABLED).value(); - return new LoggingContext(enabled, subscriptionId, request, task.metadata(), keyFormatter); + return new LoggingContext(enabled, subscriptionId, request, task.metadata()); } @Override @@ -118,7 +114,7 @@ private

Completion pushDownStream(List> downstreams, P t "Exception from tracing", NoopTrace.INSTANCE); ProcessingContextImpl

nextContext = new ProcessingContextImpl<>( subscriptionId, request, task, downstreams.subList(1, downstreams.size()), - retryQueueingProcessor, keyFormatter, props); + retryQueueingProcessor, props); CompletionImpl completion; try { diff --git a/processor/src/main/java/com/linecorp/decaton/processor/runtime/internal/Processors.java b/processor/src/main/java/com/linecorp/decaton/processor/runtime/internal/Processors.java index a5e63ae4..da2aabe7 100644 --- a/processor/src/main/java/com/linecorp/decaton/processor/runtime/internal/Processors.java +++ b/processor/src/main/java/com/linecorp/decaton/processor/runtime/internal/Processors.java @@ -18,7 +18,6 @@ import java.util.Collections; import java.util.List; -import java.util.function.Function; import java.util.stream.Collectors; import org.apache.kafka.common.TopicPartition; @@ -26,7 +25,6 @@ import org.slf4j.LoggerFactory; import com.linecorp.decaton.processor.DecatonProcessor; -import com.linecorp.decaton.processor.formatter.KeyFormatter; import com.linecorp.decaton.processor.runtime.TaskExtractor; import com.linecorp.decaton.processor.metrics.Metrics; import com.linecorp.decaton.processor.runtime.DecatonProcessorSupplier; @@ -38,18 +36,15 @@ public class Processors { private final DecatonProcessorSupplier retryProcessorSupplier; private final TaskExtractor taskExtractor; private final TaskExtractor retryTaskExtractor; - private final KeyFormatter keyFormatter; public Processors(List> suppliers, DecatonProcessorSupplier retryProcessorSupplier, TaskExtractor taskExtractor, - TaskExtractor retryTaskExtractor, - KeyFormatter keyFormatter) { + TaskExtractor retryTaskExtractor) { this.suppliers = Collections.unmodifiableList(suppliers); this.retryProcessorSupplier = retryProcessorSupplier; this.taskExtractor = taskExtractor; this.retryTaskExtractor = retryTaskExtractor; - this.keyFormatter = keyFormatter; } private DecatonProcessor retryProcessor(ThreadScope scope) { @@ -83,7 +78,7 @@ public ProcessPipeline newPipeline(ThreadScope scope, scope.threadId())) .collect(Collectors.toList()); logger.info("Creating partition processor core: {}", scope); - return new ProcessPipeline<>(scope, processors, retryProcessor, taskExtractor, keyFormatter, scheduler, metrics); + return new ProcessPipeline<>(scope, processors, retryProcessor, taskExtractor, scheduler, metrics); } catch (RuntimeException e) { // If exception occurred in the middle of instantiating processors, we have to make sure // all the previously created processors are destroyed before bubbling up the exception. diff --git a/processor/src/main/java/com/linecorp/decaton/processor/runtime/internal/TaskRequest.java b/processor/src/main/java/com/linecorp/decaton/processor/runtime/internal/TaskRequest.java index 4cc195cb..64a013e7 100644 --- a/processor/src/main/java/com/linecorp/decaton/processor/runtime/internal/TaskRequest.java +++ b/processor/src/main/java/com/linecorp/decaton/processor/runtime/internal/TaskRequest.java @@ -32,7 +32,6 @@ public class TaskRequest { private final TopicPartition topicPartition; private final long recordOffset; private final OffsetState offsetState; - @ToString.Exclude private final byte[] key; @ToString.Exclude private final Headers headers; diff --git a/processor/src/test/java/com/linecorp/decaton/processor/processors/CompactionProcessorTest.java b/processor/src/test/java/com/linecorp/decaton/processor/processors/CompactionProcessorTest.java index 9ec4a0a2..44ee2134 100644 --- a/processor/src/test/java/com/linecorp/decaton/processor/processors/CompactionProcessorTest.java +++ b/processor/src/test/java/com/linecorp/decaton/processor/processors/CompactionProcessorTest.java @@ -49,7 +49,6 @@ import com.linecorp.decaton.processor.DecatonProcessor; import com.linecorp.decaton.processor.ProcessingContext; import com.linecorp.decaton.processor.TaskMetadata; -import com.linecorp.decaton.processor.formatter.KeyFormatter; import com.linecorp.decaton.processor.processors.CompactionProcessor.CompactChoice; import com.linecorp.decaton.processor.processors.CompactionProcessor.CompactingTask; import com.linecorp.decaton.processor.runtime.DecatonTask; @@ -107,7 +106,7 @@ private TaskInput put(DecatonProcessor processor, ProcessingContext context = spy(new ProcessingContextImpl<>("subscription", request, task, Arrays.asList(processor, downstream), null, - KeyFormatter.DEFAULT, ProcessorProperties.builder().build())); + ProcessorProperties.builder().build())); if (beforeProcess != null) { beforeProcess.accept(taskData, context); diff --git a/processor/src/test/java/com/linecorp/decaton/processor/runtime/internal/ProcessPipelineTest.java b/processor/src/test/java/com/linecorp/decaton/processor/runtime/internal/ProcessPipelineTest.java index 8bd03b6a..07f67473 100644 --- a/processor/src/test/java/com/linecorp/decaton/processor/runtime/internal/ProcessPipelineTest.java +++ b/processor/src/test/java/com/linecorp/decaton/processor/runtime/internal/ProcessPipelineTest.java @@ -53,7 +53,6 @@ import com.linecorp.decaton.processor.DeferredCompletion; import com.linecorp.decaton.processor.ProcessingContext; import com.linecorp.decaton.processor.TaskMetadata; -import com.linecorp.decaton.processor.formatter.KeyFormatter; import com.linecorp.decaton.processor.metrics.Metrics; import com.linecorp.decaton.processor.runtime.DecatonTask; import com.linecorp.decaton.processor.runtime.DynamicProperty; @@ -119,8 +118,7 @@ public void setUp() { completionTimeoutMsProp.set(100L); doReturn(10L).when(clock).millis(); pipeline = spy(new ProcessPipeline<>( - scope, Collections.singletonList(processorMock), null, extractorMock, KeyFormatter.DEFAULT, - schedulerMock, METRICS, clock)); + scope, Collections.singletonList(processorMock), null, extractorMock, schedulerMock, METRICS, clock)); } @Test diff --git a/processor/src/test/java/com/linecorp/decaton/processor/runtime/internal/ProcessingContextImplTest.java b/processor/src/test/java/com/linecorp/decaton/processor/runtime/internal/ProcessingContextImplTest.java index 30064216..d40f616a 100644 --- a/processor/src/test/java/com/linecorp/decaton/processor/runtime/internal/ProcessingContextImplTest.java +++ b/processor/src/test/java/com/linecorp/decaton/processor/runtime/internal/ProcessingContextImplTest.java @@ -56,7 +56,6 @@ import com.linecorp.decaton.processor.TaskMetadata; import com.linecorp.decaton.processor.Completion; import com.linecorp.decaton.processor.Completion.TimeoutChoice; -import com.linecorp.decaton.processor.formatter.KeyFormatter; import com.linecorp.decaton.processor.runtime.DecatonTask; import com.linecorp.decaton.processor.runtime.ProcessorProperties; import com.linecorp.decaton.processor.tracing.TestTraceHandle; @@ -117,7 +116,7 @@ private static ProcessingContextImpl context(RecordTraceHandle traceH DecatonTask task = new DecatonTask<>( TaskMetadata.fromProto(REQUEST.getMetadata()), TASK, TASK.toByteArray()); return new ProcessingContextImpl<>("subscription", request, task, Arrays.asList(processors), - null, KeyFormatter.DEFAULT, ProcessorProperties.builder().build()); + null, ProcessorProperties.builder().build()); } private static void safeAwait(CountDownLatch latch) { @@ -369,7 +368,7 @@ public void process(ProcessingContext context, byte[] task) ProcessingContextImpl context = spy(new ProcessingContextImpl<>("subscription", request, task, - Collections.emptyList(), retryProcessor, KeyFormatter.DEFAULT, + Collections.emptyList(), retryProcessor, ProcessorProperties.builder().build())); Completion retryComp = context.retry(); @@ -422,7 +421,7 @@ public void process(ProcessingContext context, byte[] task) ProcessingContextImpl context = spy(new ProcessingContextImpl<>("subscription", request, task, - Arrays.asList(processor), retryProcessor, KeyFormatter.DEFAULT, + Arrays.asList(processor), retryProcessor, ProcessorProperties.builder().build())); Completion comp = context.push(new byte[0]); diff --git a/processor/src/test/java/com/linecorp/decaton/processor/runtime/internal/ProcessorsTest.java b/processor/src/test/java/com/linecorp/decaton/processor/runtime/internal/ProcessorsTest.java index 04636d40..9f6ab6b7 100644 --- a/processor/src/test/java/com/linecorp/decaton/processor/runtime/internal/ProcessorsTest.java +++ b/processor/src/test/java/com/linecorp/decaton/processor/runtime/internal/ProcessorsTest.java @@ -35,7 +35,6 @@ import org.mockito.junit.MockitoJUnit; import org.mockito.junit.MockitoRule; -import com.linecorp.decaton.processor.formatter.KeyFormatter; import com.linecorp.decaton.processor.runtime.ProcessorProperties; import com.linecorp.decaton.processor.runtime.DecatonProcessorSupplier; import com.linecorp.decaton.processor.tracing.internal.NoopTracingProvider; @@ -70,7 +69,7 @@ public void testCleanupPartiallyInitializedProcessors() { Processors processors = new Processors<>( suppliers, null, new DefaultTaskExtractor<>(bytes -> HelloTask.getDefaultInstance()), - null, KeyFormatter.DEFAULT); + null); doThrow(new RuntimeException("exception")).when(suppliers.get(2)).getProcessor(any(), any(), anyInt()); diff --git a/testing/src/main/java/com/linecorp/decaton/testing/processor/ProcessedRecord.java b/testing/src/main/java/com/linecorp/decaton/testing/processor/ProcessedRecord.java index 9e44539f..8fad8907 100644 --- a/testing/src/main/java/com/linecorp/decaton/testing/processor/ProcessedRecord.java +++ b/testing/src/main/java/com/linecorp/decaton/testing/processor/ProcessedRecord.java @@ -18,7 +18,6 @@ import com.linecorp.decaton.processor.DecatonProcessor; -import lombok.ToString; import lombok.Value; import lombok.experimental.Accessors; @@ -31,7 +30,6 @@ public class ProcessedRecord { /** * Key of the task */ - @ToString.Exclude byte[] key; /** * Processed task diff --git a/testing/src/main/java/com/linecorp/decaton/testing/processor/SerialProcessing.java b/testing/src/main/java/com/linecorp/decaton/testing/processor/SerialProcessing.java index b8c2cdd3..149c72ec 100644 --- a/testing/src/main/java/com/linecorp/decaton/testing/processor/SerialProcessing.java +++ b/testing/src/main/java/com/linecorp/decaton/testing/processor/SerialProcessing.java @@ -20,7 +20,6 @@ import static org.hamcrest.Matchers.lessThan; import java.nio.ByteBuffer; -import java.nio.charset.StandardCharsets; import java.util.ArrayList; import java.util.Comparator; import java.util.HashMap; @@ -55,8 +54,7 @@ public void doAssert() { ProcessedRecord prev = perKeyRecords.get(i - 1); ProcessedRecord current = perKeyRecords.get(i); - String key = StandardCharsets.UTF_8.decode(entry.getKey()).toString(); - assertThat("Process time shouldn't overlap. key: " + key, + assertThat("Process time shouldn't overlap. key: " + entry.getKey(), prev.endTimeNanos(), lessThan(current.startTimeNanos())); } } From 601ef688e409d20a93bf51de9121b3f0a06bb3d1 Mon Sep 17 00:00:00 2001 From: Koki Kato Date: Tue, 21 Jun 2022 14:31:57 +0900 Subject: [PATCH 15/27] Print bytes key as UTF-8 in LoggingContext We agreed that it doesn't pay to support printing bytes keys in user-specified format in `LoggingContext`. Rather, we just print the key as UTF-8 sequence regardless of the actual type. --- .../java/com/linecorp/decaton/processor/LoggingContext.java | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/processor/src/main/java/com/linecorp/decaton/processor/LoggingContext.java b/processor/src/main/java/com/linecorp/decaton/processor/LoggingContext.java index 7e9b74f4..60c37be6 100644 --- a/processor/src/main/java/com/linecorp/decaton/processor/LoggingContext.java +++ b/processor/src/main/java/com/linecorp/decaton/processor/LoggingContext.java @@ -16,6 +16,8 @@ package com.linecorp.decaton.processor; +import java.nio.charset.StandardCharsets; + import org.slf4j.MDC; import com.linecorp.decaton.processor.runtime.internal.TaskRequest; @@ -43,8 +45,10 @@ public class LoggingContext implements AutoCloseable { public LoggingContext(boolean enabled, String subscriptionId, TaskRequest request, TaskMetadata metadata) { this.enabled = enabled; if (enabled) { + final String taskKey = request.key() == null ? "" : new String(request.key(), StandardCharsets.UTF_8); + MDC.put(METADATA_KEY, metadata.toString()); - MDC.put(TASK_KEY, String.valueOf(request.key())); + MDC.put(TASK_KEY, taskKey); MDC.put(SUBSCRIPTION_ID_KEY, subscriptionId); MDC.put(OFFSET_KEY, String.valueOf(request.recordOffset())); MDC.put(TOPIC_KEY, request.topicPartition().topic()); From f69c30382ec4aa0a722a5c440e6ebc8d4387d39a Mon Sep 17 00:00:00 2001 From: Koki Kato Date: Tue, 21 Jun 2022 15:33:53 +0900 Subject: [PATCH 16/27] Add HashableKey to replace ByteBuffer usage for Map/Set --- .../processor/CoreFunctionalityTest.java | 9 ++- .../decaton/processor/RateLimiterTest.java | 5 +- .../decaton/processor/HashableKey.java | 62 +++++++++++++++++++ .../processors/CompactionProcessor.java | 16 ++--- .../testing/processor/ProcessOrdering.java | 14 ++--- .../testing/processor/SerialProcessing.java | 8 +-- 6 files changed, 87 insertions(+), 27 deletions(-) create mode 100644 processor/src/main/java/com/linecorp/decaton/processor/HashableKey.java diff --git a/processor/src/it/java/com/linecorp/decaton/processor/CoreFunctionalityTest.java b/processor/src/it/java/com/linecorp/decaton/processor/CoreFunctionalityTest.java index 416bec65..acc6bc8b 100644 --- a/processor/src/it/java/com/linecorp/decaton/processor/CoreFunctionalityTest.java +++ b/processor/src/it/java/com/linecorp/decaton/processor/CoreFunctionalityTest.java @@ -18,7 +18,6 @@ import static org.junit.Assert.assertTrue; -import java.nio.ByteBuffer; import java.util.ArrayList; import java.util.HashMap; import java.util.List; @@ -154,17 +153,17 @@ public void testSingleThreadProcessing() throws Exception { // Note that this processing semantics is not be considered as Decaton specification which users can rely on. // Rather, this is just a expected behavior based on current implementation when we set concurrency to 1. ProcessingGuarantee noDuplicates = new ProcessingGuarantee() { - private final Map> produced = new HashMap<>(); - private final Map> processed = new HashMap<>(); + private final Map> produced = new HashMap<>(); + private final Map> processed = new HashMap<>(); @Override public synchronized void onProduce(ProducedRecord record) { - produced.computeIfAbsent(ByteBuffer.wrap(record.key()), key -> new ArrayList<>()).add(record.task()); + produced.computeIfAbsent(new HashableKey(record.key()), key -> new ArrayList<>()).add(record.task()); } @Override public synchronized void onProcess(TaskMetadata metadata, ProcessedRecord record) { - processed.computeIfAbsent(ByteBuffer.wrap(record.key()), key -> new ArrayList<>()).add(record.task()); + processed.computeIfAbsent(new HashableKey(record.key()), key -> new ArrayList<>()).add(record.task()); } @Override diff --git a/processor/src/it/java/com/linecorp/decaton/processor/RateLimiterTest.java b/processor/src/it/java/com/linecorp/decaton/processor/RateLimiterTest.java index ff3cf490..6f4cbb35 100644 --- a/processor/src/it/java/com/linecorp/decaton/processor/RateLimiterTest.java +++ b/processor/src/it/java/com/linecorp/decaton/processor/RateLimiterTest.java @@ -18,7 +18,6 @@ import static org.junit.Assert.assertEquals; -import java.nio.ByteBuffer; import java.util.Collections; import java.util.HashSet; import java.util.Set; @@ -63,11 +62,11 @@ public void testPropertyDynamicSwitch() throws Exception { for (int i = 0; i < 10000; i++) { keys.add("key" + i); } - Set processedKeys = Collections.synchronizedSet(new HashSet<>()); + Set processedKeys = Collections.synchronizedSet(new HashSet<>()); CountDownLatch processLatch = new CountDownLatch(keys.size()); DecatonProcessor processor = (context, task) -> { - processedKeys.add(ByteBuffer.wrap(context.key())); + processedKeys.add(new HashableKey(context.key())); processLatch.countDown(); }; diff --git a/processor/src/main/java/com/linecorp/decaton/processor/HashableKey.java b/processor/src/main/java/com/linecorp/decaton/processor/HashableKey.java new file mode 100644 index 00000000..645cc0cf --- /dev/null +++ b/processor/src/main/java/com/linecorp/decaton/processor/HashableKey.java @@ -0,0 +1,62 @@ +/* + * Copyright 2020 LINE Corporation + * + * LINE Corporation licenses this file to you under the Apache License, + * version 2.0 (the "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at: + * + * https://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the + * License for the specific language governing permissions and limitations + * under the License. + */ + +package com.linecorp.decaton.processor; + +import java.nio.charset.StandardCharsets; +import java.util.Arrays; + +/** + * This is a wrapper for record keys in {@code byte[]} to support {@link #equals(Object)} and {@link #hashCode()}, + * so that they can be stored as a key or item of HashMap or HashSet. + * + * It also supports caching the computed hash code like {@link String}. + */ +public class HashableKey { + private final byte[] key; + private int hash; + + public HashableKey(byte[] key) { + this.key = key; + } + + public byte[] getKey() { + return key; + } + + @Override + public boolean equals(Object o) { + if (this == o) {return true;} + if (o == null || getClass() != o.getClass()) {return false;} + + final HashableKey that = (HashableKey) o; + + return Arrays.equals(key, that.key); + } + + @Override + public int hashCode() { + if (hash == 0 && key.length > 0) { + hash = Arrays.hashCode(key); + } + return hash; + } + + @Override + public String toString() { + return "HashableKey{key=" + new String(key, StandardCharsets.UTF_8) + '}'; + } +} diff --git a/processor/src/main/java/com/linecorp/decaton/processor/processors/CompactionProcessor.java b/processor/src/main/java/com/linecorp/decaton/processor/processors/CompactionProcessor.java index eaa872b8..d942264e 100644 --- a/processor/src/main/java/com/linecorp/decaton/processor/processors/CompactionProcessor.java +++ b/processor/src/main/java/com/linecorp/decaton/processor/processors/CompactionProcessor.java @@ -16,7 +16,6 @@ package com.linecorp.decaton.processor.processors; -import java.nio.ByteBuffer; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentMap; import java.util.concurrent.ScheduledExecutorService; @@ -27,10 +26,11 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import com.linecorp.decaton.processor.Completion; import com.linecorp.decaton.processor.DecatonProcessor; +import com.linecorp.decaton.processor.HashableKey; import com.linecorp.decaton.processor.ProcessingContext; import com.linecorp.decaton.processor.metrics.Metrics; -import com.linecorp.decaton.processor.Completion; import io.micrometer.core.instrument.Counter; import lombok.AccessLevel; @@ -85,7 +85,7 @@ public enum CompactChoice { } private final ScheduledExecutorService executor; - private final ConcurrentMap> windowedTasks = new ConcurrentHashMap<>(); + private final ConcurrentMap> windowedTasks = new ConcurrentHashMap<>(); private final BiFunction, CompactingTask, CompactChoice> compactor; private final long lingerMillis; @@ -147,7 +147,7 @@ public CompactionProcessor( } private void flush(byte[] key) throws InterruptedException { - CompactingTask task = windowedTasks.remove(ByteBuffer.wrap(key)); + CompactingTask task = windowedTasks.remove(new HashableKey(key)); if (task == null) { return; } @@ -183,15 +183,15 @@ private void scheduleFlush(ProcessingContext context) { @Override public void process(ProcessingContext context, T task) throws InterruptedException { - byte[] key = context.key(); CompactingTask newTask = new CompactingTask<>(context.deferCompletion(), context, task); // Even though we do this read and following updates in separate operation, race condition can't be // happened because tasks are guaranteed to be serialized by it's key, so simultaneous processing // of tasks sharing the same key won't be happen. - CompactingTask prevTask = windowedTasks.get(ByteBuffer.wrap(key)); + final HashableKey key = new HashableKey(context.key()); + CompactingTask prevTask = windowedTasks.get(key); if (prevTask == null) { - windowedTasks.put(ByteBuffer.wrap(key), newTask); + windowedTasks.put(key, newTask); scheduleFlush(context); return; } @@ -206,7 +206,7 @@ public void process(ProcessingContext context, T task) throws InterruptedExce case PICK_RIGHT: case PICK_EITHER: // Newer task has larger offset. We want to forward consumed offset. // Update the last task with new one. - Object oldEntry = windowedTasks.put(ByteBuffer.wrap(key), newTask); + Object oldEntry = windowedTasks.put(key, newTask); if (oldEntry == null) { // By race condition, there is a chance that the scheduled flush for preceding task just // got fired right after this method checked the key's existence at the beginning of this diff --git a/testing/src/main/java/com/linecorp/decaton/testing/processor/ProcessOrdering.java b/testing/src/main/java/com/linecorp/decaton/testing/processor/ProcessOrdering.java index de5d15d7..3a7c6ea7 100644 --- a/testing/src/main/java/com/linecorp/decaton/testing/processor/ProcessOrdering.java +++ b/testing/src/main/java/com/linecorp/decaton/testing/processor/ProcessOrdering.java @@ -21,7 +21,6 @@ import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; -import java.nio.ByteBuffer; import java.util.ArrayDeque; import java.util.ArrayList; import java.util.Deque; @@ -30,30 +29,31 @@ import java.util.Map; import java.util.Map.Entry; +import com.linecorp.decaton.processor.HashableKey; import com.linecorp.decaton.processor.TaskMetadata; public class ProcessOrdering implements ProcessingGuarantee { private final Map taskToOffset = new HashMap<>(); - private final Map> producedRecords = new HashMap<>(); - private final Map> processedRecords = new HashMap<>(); + private final Map> producedRecords = new HashMap<>(); + private final Map> processedRecords = new HashMap<>(); @Override public synchronized void onProduce(ProducedRecord record) { taskToOffset.put(record.task(), record.offset()); - producedRecords.computeIfAbsent(ByteBuffer.wrap(record.key()), + producedRecords.computeIfAbsent(new HashableKey(record.key()), key -> new ArrayList<>()).add(record.task()); } @Override public synchronized void onProcess(TaskMetadata metadata, ProcessedRecord record) { - processedRecords.computeIfAbsent(ByteBuffer.wrap(record.key()), + processedRecords.computeIfAbsent(new HashableKey(record.key()), key -> new ArrayList<>()).add(record.task()); } @Override public void doAssert() { - for (Entry> entry : producedRecords.entrySet()) { - final ByteBuffer key = entry.getKey(); + for (Entry> entry : producedRecords.entrySet()) { + final HashableKey key = entry.getKey(); List produced = entry.getValue(); List processed = processedRecords.get(key); diff --git a/testing/src/main/java/com/linecorp/decaton/testing/processor/SerialProcessing.java b/testing/src/main/java/com/linecorp/decaton/testing/processor/SerialProcessing.java index 149c72ec..b073e31f 100644 --- a/testing/src/main/java/com/linecorp/decaton/testing/processor/SerialProcessing.java +++ b/testing/src/main/java/com/linecorp/decaton/testing/processor/SerialProcessing.java @@ -19,7 +19,6 @@ import static org.hamcrest.MatcherAssert.assertThat; import static org.hamcrest.Matchers.lessThan; -import java.nio.ByteBuffer; import java.util.ArrayList; import java.util.Comparator; import java.util.HashMap; @@ -27,10 +26,11 @@ import java.util.Map; import java.util.Map.Entry; +import com.linecorp.decaton.processor.HashableKey; import com.linecorp.decaton.processor.TaskMetadata; public class SerialProcessing implements ProcessingGuarantee { - private final Map> records = new HashMap<>(); + private final Map> records = new HashMap<>(); @Override public void onProduce(ProducedRecord record) { @@ -39,14 +39,14 @@ public void onProduce(ProducedRecord record) { @Override public synchronized void onProcess(TaskMetadata metadata, ProcessedRecord record) { - records.computeIfAbsent(ByteBuffer.wrap(record.key()), + records.computeIfAbsent(new HashableKey(record.key()), key -> new ArrayList<>()).add(record); } @Override public void doAssert() { // Checks there's no overlap between two consecutive records' processing time - for (Entry> entry : records.entrySet()) { + for (Entry> entry : records.entrySet()) { List perKeyRecords = entry.getValue(); perKeyRecords.sort(Comparator.comparingLong(ProcessedRecord::startTimeNanos)); From cd07084fd39350d5761b4274221cc716d937cf08 Mon Sep 17 00:00:00 2001 From: Koki Kato Date: Tue, 21 Jun 2022 15:43:02 +0900 Subject: [PATCH 17/27] Exclude byte[] key from toString target --- .../decaton/processor/runtime/internal/TaskRequest.java | 1 + .../com/linecorp/decaton/testing/processor/ProcessedRecord.java | 2 ++ .../com/linecorp/decaton/testing/processor/ProducedRecord.java | 2 ++ 3 files changed, 5 insertions(+) diff --git a/processor/src/main/java/com/linecorp/decaton/processor/runtime/internal/TaskRequest.java b/processor/src/main/java/com/linecorp/decaton/processor/runtime/internal/TaskRequest.java index 64a013e7..4cc195cb 100644 --- a/processor/src/main/java/com/linecorp/decaton/processor/runtime/internal/TaskRequest.java +++ b/processor/src/main/java/com/linecorp/decaton/processor/runtime/internal/TaskRequest.java @@ -32,6 +32,7 @@ public class TaskRequest { private final TopicPartition topicPartition; private final long recordOffset; private final OffsetState offsetState; + @ToString.Exclude private final byte[] key; @ToString.Exclude private final Headers headers; diff --git a/testing/src/main/java/com/linecorp/decaton/testing/processor/ProcessedRecord.java b/testing/src/main/java/com/linecorp/decaton/testing/processor/ProcessedRecord.java index 8fad8907..9e44539f 100644 --- a/testing/src/main/java/com/linecorp/decaton/testing/processor/ProcessedRecord.java +++ b/testing/src/main/java/com/linecorp/decaton/testing/processor/ProcessedRecord.java @@ -18,6 +18,7 @@ import com.linecorp.decaton.processor.DecatonProcessor; +import lombok.ToString; import lombok.Value; import lombok.experimental.Accessors; @@ -30,6 +31,7 @@ public class ProcessedRecord { /** * Key of the task */ + @ToString.Exclude byte[] key; /** * Processed task diff --git a/testing/src/main/java/com/linecorp/decaton/testing/processor/ProducedRecord.java b/testing/src/main/java/com/linecorp/decaton/testing/processor/ProducedRecord.java index 75ed832a..47ab670b 100644 --- a/testing/src/main/java/com/linecorp/decaton/testing/processor/ProducedRecord.java +++ b/testing/src/main/java/com/linecorp/decaton/testing/processor/ProducedRecord.java @@ -19,6 +19,7 @@ import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.header.Headers; +import lombok.ToString; import lombok.Value; import lombok.experimental.Accessors; @@ -31,6 +32,7 @@ public class ProducedRecord { /** * Key of the task */ + @ToString.Exclude byte[] key; /** * Topic partition the record was sent to From 6d3cc6c53ed73db999b8adb96dca99d46df28be1 Mon Sep 17 00:00:00 2001 From: Koki Kato Date: Tue, 21 Jun 2022 15:49:26 +0900 Subject: [PATCH 18/27] Use HashableKey for BlacklistedKeysFilter --- .../internal/BlacklistedKeysFilter.java | 19 ++++++++++--------- 1 file changed, 10 insertions(+), 9 deletions(-) diff --git a/processor/src/main/java/com/linecorp/decaton/processor/runtime/internal/BlacklistedKeysFilter.java b/processor/src/main/java/com/linecorp/decaton/processor/runtime/internal/BlacklistedKeysFilter.java index c2ac193c..df79b35a 100644 --- a/processor/src/main/java/com/linecorp/decaton/processor/runtime/internal/BlacklistedKeysFilter.java +++ b/processor/src/main/java/com/linecorp/decaton/processor/runtime/internal/BlacklistedKeysFilter.java @@ -19,37 +19,38 @@ import static com.linecorp.decaton.processor.runtime.ProcessorProperties.CONFIG_IGNORE_KEYS; import java.nio.charset.StandardCharsets; -import java.util.HashSet; import java.util.Set; +import java.util.stream.Collectors; import org.apache.kafka.clients.consumer.ConsumerRecord; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import com.linecorp.decaton.processor.HashableKey; import com.linecorp.decaton.processor.runtime.ProcessorProperties; public class BlacklistedKeysFilter { private static final Logger logger = LoggerFactory.getLogger(BlacklistedKeysFilter.class); - private volatile Set ignoreKeys; + private volatile Set ignoreKeys; public BlacklistedKeysFilter(ProcessorProperties props) { props.get(CONFIG_IGNORE_KEYS) - .listen((oldValue, newValue) -> ignoreKeys = new HashSet<>(newValue)); + .listen((oldValue, newValue) -> ignoreKeys = + newValue.stream().map(key -> new HashableKey(key.getBytes(StandardCharsets.UTF_8))).collect(Collectors.toSet()) + ); } public boolean shouldTake(ConsumerRecord record) { - final byte[] key = record.key(); - if (key == null) { + if (record.key() == null) { return true; } - final String stringKey = new String(key, StandardCharsets.UTF_8); - + final HashableKey key = new HashableKey(record.key()); // Preceding isEmpty() check is for reducing tiny overhead applied for each contains() by calling // Object#hashCode. Since ignoreKeys should be empty for most cases.. - if (!ignoreKeys.isEmpty() && ignoreKeys.contains(stringKey)) { - logger.debug("Ignore task which has key configured to ignore: {}", stringKey); + if (!ignoreKeys.isEmpty() && ignoreKeys.contains(key)) { + logger.debug("Ignore task which has key configured to ignore: {}", key); return false; } From d2894c4d28c641ae8c0c7a2d7bf56241e8d5f18e Mon Sep 17 00:00:00 2001 From: Koki Kato Date: Tue, 21 Jun 2022 15:53:57 +0900 Subject: [PATCH 19/27] Support null key in HashableKey --- .../java/com/linecorp/decaton/processor/HashableKey.java | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/processor/src/main/java/com/linecorp/decaton/processor/HashableKey.java b/processor/src/main/java/com/linecorp/decaton/processor/HashableKey.java index 645cc0cf..af4bdcd9 100644 --- a/processor/src/main/java/com/linecorp/decaton/processor/HashableKey.java +++ b/processor/src/main/java/com/linecorp/decaton/processor/HashableKey.java @@ -49,7 +49,7 @@ public boolean equals(Object o) { @Override public int hashCode() { - if (hash == 0 && key.length > 0) { + if (hash == 0 && key != null && key.length > 0) { hash = Arrays.hashCode(key); } return hash; @@ -57,6 +57,8 @@ public int hashCode() { @Override public String toString() { - return "HashableKey{key=" + new String(key, StandardCharsets.UTF_8) + '}'; + final String keyStr = key == null ? "null" + : '\"' + new String(key, StandardCharsets.UTF_8) + '\"'; + return "HashableKey{key=" + keyStr + '}'; } } From b9eac225ae18733e87306098fd6cce11cc5fe6cd Mon Sep 17 00:00:00 2001 From: Koki Kato Date: Wed, 22 Jun 2022 12:01:39 +0900 Subject: [PATCH 20/27] Print "null" instead of empty string if key is null --- .../java/com/linecorp/decaton/processor/LoggingContext.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/processor/src/main/java/com/linecorp/decaton/processor/LoggingContext.java b/processor/src/main/java/com/linecorp/decaton/processor/LoggingContext.java index 60c37be6..2a6c0eb1 100644 --- a/processor/src/main/java/com/linecorp/decaton/processor/LoggingContext.java +++ b/processor/src/main/java/com/linecorp/decaton/processor/LoggingContext.java @@ -45,7 +45,7 @@ public class LoggingContext implements AutoCloseable { public LoggingContext(boolean enabled, String subscriptionId, TaskRequest request, TaskMetadata metadata) { this.enabled = enabled; if (enabled) { - final String taskKey = request.key() == null ? "" : new String(request.key(), StandardCharsets.UTF_8); + final String taskKey = request.key() == null ? "null" : new String(request.key(), StandardCharsets.UTF_8); MDC.put(METADATA_KEY, metadata.toString()); MDC.put(TASK_KEY, taskKey); From bf3d42c6c2b76074d0f759f45d5c16beb0c10a71 Mon Sep 17 00:00:00 2001 From: Koki Kato Date: Wed, 22 Jun 2022 12:03:40 +0900 Subject: [PATCH 21/27] Make TestUtils.defaultProducerProps private again --- .../src/main/java/com/linecorp/decaton/testing/TestUtils.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/testing/src/main/java/com/linecorp/decaton/testing/TestUtils.java b/testing/src/main/java/com/linecorp/decaton/testing/TestUtils.java index cf7fd846..959267e7 100644 --- a/testing/src/main/java/com/linecorp/decaton/testing/TestUtils.java +++ b/testing/src/main/java/com/linecorp/decaton/testing/TestUtils.java @@ -58,7 +58,7 @@ private static int sequence() { return sequence.getAndIncrement(); } - public static Properties defaultProducerProps(String bootstrapServers) { + private static Properties defaultProducerProps(String bootstrapServers) { Properties props = new Properties(); props.setProperty(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, bootstrapServers); props.setProperty(ProducerConfig.CLIENT_ID_CONFIG, "test-client-" + sequence()); From 9d24b494083f52bc7b19578009fd1476770ac12d Mon Sep 17 00:00:00 2001 From: Koki Kato Date: Wed, 22 Jun 2022 12:07:49 +0900 Subject: [PATCH 22/27] Redefine HashableKey as HashableByteArray - rename class and fields - make it `final` class - move to decaton.processor.runtime.internal --- .../processor/CoreFunctionalityTest.java | 9 +++--- .../decaton/processor/RateLimiterTest.java | 5 ++-- .../processors/CompactionProcessor.java | 8 +++--- .../internal/BlacklistedKeysFilter.java | 7 ++--- .../internal/HashableByteArray.java} | 28 +++++++++---------- .../testing/processor/ProcessOrdering.java | 14 +++++----- .../testing/processor/SerialProcessing.java | 8 +++--- 7 files changed, 40 insertions(+), 39 deletions(-) rename processor/src/main/java/com/linecorp/decaton/processor/{HashableKey.java => runtime/internal/HashableByteArray.java} (63%) diff --git a/processor/src/it/java/com/linecorp/decaton/processor/CoreFunctionalityTest.java b/processor/src/it/java/com/linecorp/decaton/processor/CoreFunctionalityTest.java index acc6bc8b..ce42b1a1 100644 --- a/processor/src/it/java/com/linecorp/decaton/processor/CoreFunctionalityTest.java +++ b/processor/src/it/java/com/linecorp/decaton/processor/CoreFunctionalityTest.java @@ -34,6 +34,7 @@ import com.linecorp.decaton.processor.runtime.ProcessorScope; import com.linecorp.decaton.processor.runtime.Property; import com.linecorp.decaton.processor.runtime.StaticPropertySupplier; +import com.linecorp.decaton.processor.runtime.internal.HashableByteArray; import com.linecorp.decaton.testing.KafkaClusterRule; import com.linecorp.decaton.testing.RandomRule; import com.linecorp.decaton.testing.processor.ProcessedRecord; @@ -153,17 +154,17 @@ public void testSingleThreadProcessing() throws Exception { // Note that this processing semantics is not be considered as Decaton specification which users can rely on. // Rather, this is just a expected behavior based on current implementation when we set concurrency to 1. ProcessingGuarantee noDuplicates = new ProcessingGuarantee() { - private final Map> produced = new HashMap<>(); - private final Map> processed = new HashMap<>(); + private final Map> produced = new HashMap<>(); + private final Map> processed = new HashMap<>(); @Override public synchronized void onProduce(ProducedRecord record) { - produced.computeIfAbsent(new HashableKey(record.key()), key -> new ArrayList<>()).add(record.task()); + produced.computeIfAbsent(new HashableByteArray(record.key()), key -> new ArrayList<>()).add(record.task()); } @Override public synchronized void onProcess(TaskMetadata metadata, ProcessedRecord record) { - processed.computeIfAbsent(new HashableKey(record.key()), key -> new ArrayList<>()).add(record.task()); + processed.computeIfAbsent(new HashableByteArray(record.key()), key -> new ArrayList<>()).add(record.task()); } @Override diff --git a/processor/src/it/java/com/linecorp/decaton/processor/RateLimiterTest.java b/processor/src/it/java/com/linecorp/decaton/processor/RateLimiterTest.java index 6f4cbb35..467806a5 100644 --- a/processor/src/it/java/com/linecorp/decaton/processor/RateLimiterTest.java +++ b/processor/src/it/java/com/linecorp/decaton/processor/RateLimiterTest.java @@ -34,6 +34,7 @@ import com.linecorp.decaton.processor.runtime.ProcessorSubscription; import com.linecorp.decaton.processor.runtime.ProcessorsBuilder; import com.linecorp.decaton.processor.runtime.StaticPropertySupplier; +import com.linecorp.decaton.processor.runtime.internal.HashableByteArray; import com.linecorp.decaton.protobuf.ProtocolBuffersDeserializer; import com.linecorp.decaton.protocol.Sample.HelloTask; import com.linecorp.decaton.testing.KafkaClusterRule; @@ -62,11 +63,11 @@ public void testPropertyDynamicSwitch() throws Exception { for (int i = 0; i < 10000; i++) { keys.add("key" + i); } - Set processedKeys = Collections.synchronizedSet(new HashSet<>()); + Set processedKeys = Collections.synchronizedSet(new HashSet<>()); CountDownLatch processLatch = new CountDownLatch(keys.size()); DecatonProcessor processor = (context, task) -> { - processedKeys.add(new HashableKey(context.key())); + processedKeys.add(new HashableByteArray(context.key())); processLatch.countDown(); }; diff --git a/processor/src/main/java/com/linecorp/decaton/processor/processors/CompactionProcessor.java b/processor/src/main/java/com/linecorp/decaton/processor/processors/CompactionProcessor.java index d942264e..2659100e 100644 --- a/processor/src/main/java/com/linecorp/decaton/processor/processors/CompactionProcessor.java +++ b/processor/src/main/java/com/linecorp/decaton/processor/processors/CompactionProcessor.java @@ -28,7 +28,7 @@ import com.linecorp.decaton.processor.Completion; import com.linecorp.decaton.processor.DecatonProcessor; -import com.linecorp.decaton.processor.HashableKey; +import com.linecorp.decaton.processor.runtime.internal.HashableByteArray; import com.linecorp.decaton.processor.ProcessingContext; import com.linecorp.decaton.processor.metrics.Metrics; @@ -85,7 +85,7 @@ public enum CompactChoice { } private final ScheduledExecutorService executor; - private final ConcurrentMap> windowedTasks = new ConcurrentHashMap<>(); + private final ConcurrentMap> windowedTasks = new ConcurrentHashMap<>(); private final BiFunction, CompactingTask, CompactChoice> compactor; private final long lingerMillis; @@ -147,7 +147,7 @@ public CompactionProcessor( } private void flush(byte[] key) throws InterruptedException { - CompactingTask task = windowedTasks.remove(new HashableKey(key)); + CompactingTask task = windowedTasks.remove(new HashableByteArray(key)); if (task == null) { return; } @@ -188,7 +188,7 @@ public void process(ProcessingContext context, T task) throws InterruptedExce // Even though we do this read and following updates in separate operation, race condition can't be // happened because tasks are guaranteed to be serialized by it's key, so simultaneous processing // of tasks sharing the same key won't be happen. - final HashableKey key = new HashableKey(context.key()); + final HashableByteArray key = new HashableByteArray(context.key()); CompactingTask prevTask = windowedTasks.get(key); if (prevTask == null) { windowedTasks.put(key, newTask); diff --git a/processor/src/main/java/com/linecorp/decaton/processor/runtime/internal/BlacklistedKeysFilter.java b/processor/src/main/java/com/linecorp/decaton/processor/runtime/internal/BlacklistedKeysFilter.java index df79b35a..657ba12c 100644 --- a/processor/src/main/java/com/linecorp/decaton/processor/runtime/internal/BlacklistedKeysFilter.java +++ b/processor/src/main/java/com/linecorp/decaton/processor/runtime/internal/BlacklistedKeysFilter.java @@ -26,18 +26,17 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import com.linecorp.decaton.processor.HashableKey; import com.linecorp.decaton.processor.runtime.ProcessorProperties; public class BlacklistedKeysFilter { private static final Logger logger = LoggerFactory.getLogger(BlacklistedKeysFilter.class); - private volatile Set ignoreKeys; + private volatile Set ignoreKeys; public BlacklistedKeysFilter(ProcessorProperties props) { props.get(CONFIG_IGNORE_KEYS) .listen((oldValue, newValue) -> ignoreKeys = - newValue.stream().map(key -> new HashableKey(key.getBytes(StandardCharsets.UTF_8))).collect(Collectors.toSet()) + newValue.stream().map(key -> new HashableByteArray(key.getBytes(StandardCharsets.UTF_8))).collect(Collectors.toSet()) ); } @@ -46,7 +45,7 @@ public boolean shouldTake(ConsumerRecord record) { return true; } - final HashableKey key = new HashableKey(record.key()); + final HashableByteArray key = new HashableByteArray(record.key()); // Preceding isEmpty() check is for reducing tiny overhead applied for each contains() by calling // Object#hashCode. Since ignoreKeys should be empty for most cases.. if (!ignoreKeys.isEmpty() && ignoreKeys.contains(key)) { diff --git a/processor/src/main/java/com/linecorp/decaton/processor/HashableKey.java b/processor/src/main/java/com/linecorp/decaton/processor/runtime/internal/HashableByteArray.java similarity index 63% rename from processor/src/main/java/com/linecorp/decaton/processor/HashableKey.java rename to processor/src/main/java/com/linecorp/decaton/processor/runtime/internal/HashableByteArray.java index af4bdcd9..5cf0d4f2 100644 --- a/processor/src/main/java/com/linecorp/decaton/processor/HashableKey.java +++ b/processor/src/main/java/com/linecorp/decaton/processor/runtime/internal/HashableByteArray.java @@ -14,27 +14,27 @@ * under the License. */ -package com.linecorp.decaton.processor; +package com.linecorp.decaton.processor.runtime.internal; import java.nio.charset.StandardCharsets; import java.util.Arrays; /** - * This is a wrapper for record keys in {@code byte[]} to support {@link #equals(Object)} and {@link #hashCode()}, + * This is a wrapper for {@code byte[]} to support {@link #equals(Object)} and {@link #hashCode()}, * so that they can be stored as a key or item of HashMap or HashSet. * * It also supports caching the computed hash code like {@link String}. */ -public class HashableKey { - private final byte[] key; +public final class HashableByteArray { + private final byte[] array; private int hash; - public HashableKey(byte[] key) { - this.key = key; + public HashableByteArray(byte[] array) { + this.array = array; } - public byte[] getKey() { - return key; + public byte[] getArray() { + return array; } @Override @@ -42,23 +42,23 @@ public boolean equals(Object o) { if (this == o) {return true;} if (o == null || getClass() != o.getClass()) {return false;} - final HashableKey that = (HashableKey) o; + final HashableByteArray that = (HashableByteArray) o; - return Arrays.equals(key, that.key); + return Arrays.equals(array, that.array); } @Override public int hashCode() { - if (hash == 0 && key != null && key.length > 0) { - hash = Arrays.hashCode(key); + if (hash == 0 && array != null && array.length > 0) { + hash = Arrays.hashCode(array); } return hash; } @Override public String toString() { - final String keyStr = key == null ? "null" - : '\"' + new String(key, StandardCharsets.UTF_8) + '\"'; + final String keyStr = array == null ? "null" + : '\"' + new String(array, StandardCharsets.UTF_8) + '\"'; return "HashableKey{key=" + keyStr + '}'; } } diff --git a/testing/src/main/java/com/linecorp/decaton/testing/processor/ProcessOrdering.java b/testing/src/main/java/com/linecorp/decaton/testing/processor/ProcessOrdering.java index 3a7c6ea7..2084a029 100644 --- a/testing/src/main/java/com/linecorp/decaton/testing/processor/ProcessOrdering.java +++ b/testing/src/main/java/com/linecorp/decaton/testing/processor/ProcessOrdering.java @@ -29,31 +29,31 @@ import java.util.Map; import java.util.Map.Entry; -import com.linecorp.decaton.processor.HashableKey; +import com.linecorp.decaton.processor.runtime.internal.HashableByteArray; import com.linecorp.decaton.processor.TaskMetadata; public class ProcessOrdering implements ProcessingGuarantee { private final Map taskToOffset = new HashMap<>(); - private final Map> producedRecords = new HashMap<>(); - private final Map> processedRecords = new HashMap<>(); + private final Map> producedRecords = new HashMap<>(); + private final Map> processedRecords = new HashMap<>(); @Override public synchronized void onProduce(ProducedRecord record) { taskToOffset.put(record.task(), record.offset()); - producedRecords.computeIfAbsent(new HashableKey(record.key()), + producedRecords.computeIfAbsent(new HashableByteArray(record.key()), key -> new ArrayList<>()).add(record.task()); } @Override public synchronized void onProcess(TaskMetadata metadata, ProcessedRecord record) { - processedRecords.computeIfAbsent(new HashableKey(record.key()), + processedRecords.computeIfAbsent(new HashableByteArray(record.key()), key -> new ArrayList<>()).add(record.task()); } @Override public void doAssert() { - for (Entry> entry : producedRecords.entrySet()) { - final HashableKey key = entry.getKey(); + for (Entry> entry : producedRecords.entrySet()) { + final HashableByteArray key = entry.getKey(); List produced = entry.getValue(); List processed = processedRecords.get(key); diff --git a/testing/src/main/java/com/linecorp/decaton/testing/processor/SerialProcessing.java b/testing/src/main/java/com/linecorp/decaton/testing/processor/SerialProcessing.java index b073e31f..4896c3f0 100644 --- a/testing/src/main/java/com/linecorp/decaton/testing/processor/SerialProcessing.java +++ b/testing/src/main/java/com/linecorp/decaton/testing/processor/SerialProcessing.java @@ -26,11 +26,11 @@ import java.util.Map; import java.util.Map.Entry; -import com.linecorp.decaton.processor.HashableKey; +import com.linecorp.decaton.processor.runtime.internal.HashableByteArray; import com.linecorp.decaton.processor.TaskMetadata; public class SerialProcessing implements ProcessingGuarantee { - private final Map> records = new HashMap<>(); + private final Map> records = new HashMap<>(); @Override public void onProduce(ProducedRecord record) { @@ -39,14 +39,14 @@ public void onProduce(ProducedRecord record) { @Override public synchronized void onProcess(TaskMetadata metadata, ProcessedRecord record) { - records.computeIfAbsent(new HashableKey(record.key()), + records.computeIfAbsent(new HashableByteArray(record.key()), key -> new ArrayList<>()).add(record); } @Override public void doAssert() { // Checks there's no overlap between two consecutive records' processing time - for (Entry> entry : records.entrySet()) { + for (Entry> entry : records.entrySet()) { List perKeyRecords = entry.getValue(); perKeyRecords.sort(Comparator.comparingLong(ProcessedRecord::startTimeNanos)); From 0c6ee0aadae6625001ee0a278585971ab1ea52c9 Mon Sep 17 00:00:00 2001 From: Koki Kato Date: Wed, 22 Jun 2022 12:12:40 +0900 Subject: [PATCH 23/27] Compute hashCode of bytes preemptively only once --- .../processor/runtime/internal/HashableByteArray.java | 6 ++---- 1 file changed, 2 insertions(+), 4 deletions(-) diff --git a/processor/src/main/java/com/linecorp/decaton/processor/runtime/internal/HashableByteArray.java b/processor/src/main/java/com/linecorp/decaton/processor/runtime/internal/HashableByteArray.java index 5cf0d4f2..55861ca5 100644 --- a/processor/src/main/java/com/linecorp/decaton/processor/runtime/internal/HashableByteArray.java +++ b/processor/src/main/java/com/linecorp/decaton/processor/runtime/internal/HashableByteArray.java @@ -27,10 +27,11 @@ */ public final class HashableByteArray { private final byte[] array; - private int hash; + private final int hash; public HashableByteArray(byte[] array) { this.array = array; + hash = Arrays.hashCode(array); } public byte[] getArray() { @@ -49,9 +50,6 @@ public boolean equals(Object o) { @Override public int hashCode() { - if (hash == 0 && array != null && array.length > 0) { - hash = Arrays.hashCode(array); - } return hash; } From 9d9db8300840bd6917f219e1a60ddb1d46eb9373 Mon Sep 17 00:00:00 2001 From: Koki Kato Date: Wed, 22 Jun 2022 17:53:11 +0900 Subject: [PATCH 24/27] Move HashableByteArray to decaton.processor.internal --- .../com/linecorp/decaton/processor/CoreFunctionalityTest.java | 2 +- .../it/java/com/linecorp/decaton/processor/RateLimiterTest.java | 2 +- .../processor/{runtime => }/internal/HashableByteArray.java | 2 +- .../decaton/processor/processors/CompactionProcessor.java | 2 +- .../processor/runtime/internal/BlacklistedKeysFilter.java | 1 + .../com/linecorp/decaton/testing/processor/ProcessOrdering.java | 2 +- .../linecorp/decaton/testing/processor/SerialProcessing.java | 2 +- 7 files changed, 7 insertions(+), 6 deletions(-) rename processor/src/main/java/com/linecorp/decaton/processor/{runtime => }/internal/HashableByteArray.java (97%) diff --git a/processor/src/it/java/com/linecorp/decaton/processor/CoreFunctionalityTest.java b/processor/src/it/java/com/linecorp/decaton/processor/CoreFunctionalityTest.java index ce42b1a1..707b4308 100644 --- a/processor/src/it/java/com/linecorp/decaton/processor/CoreFunctionalityTest.java +++ b/processor/src/it/java/com/linecorp/decaton/processor/CoreFunctionalityTest.java @@ -34,7 +34,7 @@ import com.linecorp.decaton.processor.runtime.ProcessorScope; import com.linecorp.decaton.processor.runtime.Property; import com.linecorp.decaton.processor.runtime.StaticPropertySupplier; -import com.linecorp.decaton.processor.runtime.internal.HashableByteArray; +import com.linecorp.decaton.processor.internal.HashableByteArray; import com.linecorp.decaton.testing.KafkaClusterRule; import com.linecorp.decaton.testing.RandomRule; import com.linecorp.decaton.testing.processor.ProcessedRecord; diff --git a/processor/src/it/java/com/linecorp/decaton/processor/RateLimiterTest.java b/processor/src/it/java/com/linecorp/decaton/processor/RateLimiterTest.java index 467806a5..3a6e82c2 100644 --- a/processor/src/it/java/com/linecorp/decaton/processor/RateLimiterTest.java +++ b/processor/src/it/java/com/linecorp/decaton/processor/RateLimiterTest.java @@ -34,7 +34,7 @@ import com.linecorp.decaton.processor.runtime.ProcessorSubscription; import com.linecorp.decaton.processor.runtime.ProcessorsBuilder; import com.linecorp.decaton.processor.runtime.StaticPropertySupplier; -import com.linecorp.decaton.processor.runtime.internal.HashableByteArray; +import com.linecorp.decaton.processor.internal.HashableByteArray; import com.linecorp.decaton.protobuf.ProtocolBuffersDeserializer; import com.linecorp.decaton.protocol.Sample.HelloTask; import com.linecorp.decaton.testing.KafkaClusterRule; diff --git a/processor/src/main/java/com/linecorp/decaton/processor/runtime/internal/HashableByteArray.java b/processor/src/main/java/com/linecorp/decaton/processor/internal/HashableByteArray.java similarity index 97% rename from processor/src/main/java/com/linecorp/decaton/processor/runtime/internal/HashableByteArray.java rename to processor/src/main/java/com/linecorp/decaton/processor/internal/HashableByteArray.java index 55861ca5..dd5b1823 100644 --- a/processor/src/main/java/com/linecorp/decaton/processor/runtime/internal/HashableByteArray.java +++ b/processor/src/main/java/com/linecorp/decaton/processor/internal/HashableByteArray.java @@ -14,7 +14,7 @@ * under the License. */ -package com.linecorp.decaton.processor.runtime.internal; +package com.linecorp.decaton.processor.internal; import java.nio.charset.StandardCharsets; import java.util.Arrays; diff --git a/processor/src/main/java/com/linecorp/decaton/processor/processors/CompactionProcessor.java b/processor/src/main/java/com/linecorp/decaton/processor/processors/CompactionProcessor.java index 2659100e..0aabc7ef 100644 --- a/processor/src/main/java/com/linecorp/decaton/processor/processors/CompactionProcessor.java +++ b/processor/src/main/java/com/linecorp/decaton/processor/processors/CompactionProcessor.java @@ -28,7 +28,7 @@ import com.linecorp.decaton.processor.Completion; import com.linecorp.decaton.processor.DecatonProcessor; -import com.linecorp.decaton.processor.runtime.internal.HashableByteArray; +import com.linecorp.decaton.processor.internal.HashableByteArray; import com.linecorp.decaton.processor.ProcessingContext; import com.linecorp.decaton.processor.metrics.Metrics; diff --git a/processor/src/main/java/com/linecorp/decaton/processor/runtime/internal/BlacklistedKeysFilter.java b/processor/src/main/java/com/linecorp/decaton/processor/runtime/internal/BlacklistedKeysFilter.java index 657ba12c..afd1a51c 100644 --- a/processor/src/main/java/com/linecorp/decaton/processor/runtime/internal/BlacklistedKeysFilter.java +++ b/processor/src/main/java/com/linecorp/decaton/processor/runtime/internal/BlacklistedKeysFilter.java @@ -26,6 +26,7 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import com.linecorp.decaton.processor.internal.HashableByteArray; import com.linecorp.decaton.processor.runtime.ProcessorProperties; public class BlacklistedKeysFilter { diff --git a/testing/src/main/java/com/linecorp/decaton/testing/processor/ProcessOrdering.java b/testing/src/main/java/com/linecorp/decaton/testing/processor/ProcessOrdering.java index 2084a029..1b57b649 100644 --- a/testing/src/main/java/com/linecorp/decaton/testing/processor/ProcessOrdering.java +++ b/testing/src/main/java/com/linecorp/decaton/testing/processor/ProcessOrdering.java @@ -29,7 +29,7 @@ import java.util.Map; import java.util.Map.Entry; -import com.linecorp.decaton.processor.runtime.internal.HashableByteArray; +import com.linecorp.decaton.processor.internal.HashableByteArray; import com.linecorp.decaton.processor.TaskMetadata; public class ProcessOrdering implements ProcessingGuarantee { diff --git a/testing/src/main/java/com/linecorp/decaton/testing/processor/SerialProcessing.java b/testing/src/main/java/com/linecorp/decaton/testing/processor/SerialProcessing.java index 4896c3f0..08a099fc 100644 --- a/testing/src/main/java/com/linecorp/decaton/testing/processor/SerialProcessing.java +++ b/testing/src/main/java/com/linecorp/decaton/testing/processor/SerialProcessing.java @@ -26,7 +26,7 @@ import java.util.Map; import java.util.Map.Entry; -import com.linecorp.decaton.processor.runtime.internal.HashableByteArray; +import com.linecorp.decaton.processor.internal.HashableByteArray; import com.linecorp.decaton.processor.TaskMetadata; public class SerialProcessing implements ProcessingGuarantee { From a288fc6eec9b263b61c832e14c44f7f6a57e5f62 Mon Sep 17 00:00:00 2001 From: Koki Kato Date: Wed, 22 Jun 2022 18:04:02 +0900 Subject: [PATCH 25/27] Introduce ByteArrays.toString for reuse --- .../decaton/processor/LoggingContext.java | 7 ++-- .../processor/internal/ByteArrays.java | 34 +++++++++++++++++++ .../processor/internal/HashableByteArray.java | 5 +-- 3 files changed, 37 insertions(+), 9 deletions(-) create mode 100644 processor/src/main/java/com/linecorp/decaton/processor/internal/ByteArrays.java diff --git a/processor/src/main/java/com/linecorp/decaton/processor/LoggingContext.java b/processor/src/main/java/com/linecorp/decaton/processor/LoggingContext.java index 2a6c0eb1..aba9e774 100644 --- a/processor/src/main/java/com/linecorp/decaton/processor/LoggingContext.java +++ b/processor/src/main/java/com/linecorp/decaton/processor/LoggingContext.java @@ -16,10 +16,9 @@ package com.linecorp.decaton.processor; -import java.nio.charset.StandardCharsets; - import org.slf4j.MDC; +import com.linecorp.decaton.processor.internal.ByteArrays; import com.linecorp.decaton.processor.runtime.internal.TaskRequest; /** @@ -45,10 +44,8 @@ public class LoggingContext implements AutoCloseable { public LoggingContext(boolean enabled, String subscriptionId, TaskRequest request, TaskMetadata metadata) { this.enabled = enabled; if (enabled) { - final String taskKey = request.key() == null ? "null" : new String(request.key(), StandardCharsets.UTF_8); - MDC.put(METADATA_KEY, metadata.toString()); - MDC.put(TASK_KEY, taskKey); + MDC.put(TASK_KEY, ByteArrays.toString(request.key())); MDC.put(SUBSCRIPTION_ID_KEY, subscriptionId); MDC.put(OFFSET_KEY, String.valueOf(request.recordOffset())); MDC.put(TOPIC_KEY, request.topicPartition().topic()); diff --git a/processor/src/main/java/com/linecorp/decaton/processor/internal/ByteArrays.java b/processor/src/main/java/com/linecorp/decaton/processor/internal/ByteArrays.java new file mode 100644 index 00000000..09927b50 --- /dev/null +++ b/processor/src/main/java/com/linecorp/decaton/processor/internal/ByteArrays.java @@ -0,0 +1,34 @@ +/* + * Copyright 2020 LINE Corporation + * + * LINE Corporation licenses this file to you under the Apache License, + * version 2.0 (the "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at: + * + * https://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the + * License for the specific language governing permissions and limitations + * under the License. + */ + +package com.linecorp.decaton.processor.internal; + +import java.nio.charset.StandardCharsets; + +/** + * A collection of utility methods for {@code byte[]}, which aim to be used internally. + */ +public final class ByteArrays { + public static String toString(byte[] array) { + if (array == null) { + return "null"; + } + // Quote here to differentiate null vs array of {'n', 'u', 'l', 'l'} + return '"' + new String(array, StandardCharsets.UTF_8) + '"'; + } + + private ByteArrays() {} +} diff --git a/processor/src/main/java/com/linecorp/decaton/processor/internal/HashableByteArray.java b/processor/src/main/java/com/linecorp/decaton/processor/internal/HashableByteArray.java index dd5b1823..50b876dd 100644 --- a/processor/src/main/java/com/linecorp/decaton/processor/internal/HashableByteArray.java +++ b/processor/src/main/java/com/linecorp/decaton/processor/internal/HashableByteArray.java @@ -16,7 +16,6 @@ package com.linecorp.decaton.processor.internal; -import java.nio.charset.StandardCharsets; import java.util.Arrays; /** @@ -55,8 +54,6 @@ public int hashCode() { @Override public String toString() { - final String keyStr = array == null ? "null" - : '\"' + new String(array, StandardCharsets.UTF_8) + '\"'; - return "HashableKey{key=" + keyStr + '}'; + return "HashableKey{key=" + ByteArrays.toString(array) + '}'; } } From 6f693768012a9d1e4e0d01efe9d68d198b9e0157 Mon Sep 17 00:00:00 2001 From: Koki Kato Date: Thu, 23 Jun 2022 11:57:12 +0900 Subject: [PATCH 26/27] Do not surround decoded array with quotes for consistency --- .../com/linecorp/decaton/processor/internal/ByteArrays.java | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/processor/src/main/java/com/linecorp/decaton/processor/internal/ByteArrays.java b/processor/src/main/java/com/linecorp/decaton/processor/internal/ByteArrays.java index 09927b50..db521cde 100644 --- a/processor/src/main/java/com/linecorp/decaton/processor/internal/ByteArrays.java +++ b/processor/src/main/java/com/linecorp/decaton/processor/internal/ByteArrays.java @@ -26,8 +26,7 @@ public static String toString(byte[] array) { if (array == null) { return "null"; } - // Quote here to differentiate null vs array of {'n', 'u', 'l', 'l'} - return '"' + new String(array, StandardCharsets.UTF_8) + '"'; + return new String(array, StandardCharsets.UTF_8); } private ByteArrays() {} From 46ce23c4300731e32138e87ec80c533525b34108 Mon Sep 17 00:00:00 2001 From: Koki Kato Date: Mon, 27 Jun 2022 10:57:10 +0900 Subject: [PATCH 27/27] LINELINT-911 Remove unused HashableByteArray#getArray --- .../decaton/processor/internal/HashableByteArray.java | 4 ---- 1 file changed, 4 deletions(-) diff --git a/processor/src/main/java/com/linecorp/decaton/processor/internal/HashableByteArray.java b/processor/src/main/java/com/linecorp/decaton/processor/internal/HashableByteArray.java index 50b876dd..56a372f8 100644 --- a/processor/src/main/java/com/linecorp/decaton/processor/internal/HashableByteArray.java +++ b/processor/src/main/java/com/linecorp/decaton/processor/internal/HashableByteArray.java @@ -33,10 +33,6 @@ public HashableByteArray(byte[] array) { hash = Arrays.hashCode(array); } - public byte[] getArray() { - return array; - } - @Override public boolean equals(Object o) { if (this == o) {return true;}