[kafka] branch trunk updated: KAFKA-12960: Enforcing strict retention time for WindowStore and Sess… (#11211)

2022-10-12 Thread showuon
This is an automated email from the ASF dual-hosted git repository.

showuon pushed a commit to branch trunk
in repository https://gitbox.apache.org/repos/asf/kafka.git


The following commit(s) were added to refs/heads/trunk by this push:
 new 07c10024890 KAFKA-12960: Enforcing strict retention time for 
WindowStore and Sess… (#11211)
07c10024890 is described below

commit 07c10024890c0761ba47ccd4d6301e8101d8c8de
Author: vamossagar12 
AuthorDate: Thu Oct 13 07:09:58 2022 +0530

KAFKA-12960: Enforcing strict retention time for WindowStore and Sess… 
(#11211)

WindowedStore and SessionStore do not implement a strict retention time in 
general. We should consider to make retention time strict: even if we still 
have some record in the store (due to the segmented implementation), we might 
want to filter expired records on-read. This might benefit PAPI users.

This PR, adds the filtering behaviour in the Metered store so that, it gets 
automatically applied for cases when a custom state store is implemented

Reviewer: Luke Chen , A. Sophie Blee-Goldman 
, Matthias J. Sax 
---
 ...stractDualSchemaRocksDBSegmentedBytesStore.java |  47 -
 .../AbstractRocksDBSegmentedBytesStore.java|  75 ++--
 ...tractRocksDBTimeOrderedSegmentedBytesStore.java |  70 +---
 .../internals/RocksDBSegmentedBytesStore.java  |   2 +-
 .../RocksDBTimestampedSegmentedBytesStore.java |   2 +-
 .../TimeWindowedKStreamIntegrationTest.java|  66 ---
 .../KStreamSlidingWindowAggregateTest.java |  18 +-
 .../internals/KStreamWindowAggregateTest.java  |  61 ---
 .../internals/SessionWindowedKStreamImplTest.java  |  63 +--
 .../internals/TimeWindowedKStreamImplTest.java | 121 +
 ...ctDualSchemaRocksDBSegmentedBytesStoreTest.java | 190 +
 .../AbstractRocksDBSegmentedBytesStoreTest.java| 163 +-
 .../internals/AbstractWindowBytesStoreTest.java|   4 +-
 .../CachingPersistentWindowStoreTest.java  |   3 +-
 .../state/internals/MeteredSessionStoreTest.java   |  49 ++
 .../MeteredTimestampedWindowStoreTest.java |   1 +
 .../state/internals/MeteredWindowStoreTest.java|  14 ++
 .../state/internals/RocksDBSessionStoreTest.java   |  93 ++
 .../state/internals/RocksDBWindowStoreTest.java| 143 
 19 files changed, 835 insertions(+), 350 deletions(-)

diff --git 
a/streams/src/main/java/org/apache/kafka/streams/state/internals/AbstractDualSchemaRocksDBSegmentedBytesStore.java
 
b/streams/src/main/java/org/apache/kafka/streams/state/internals/AbstractDualSchemaRocksDBSegmentedBytesStore.java
index 95c1d8d8c81..b446a52eb5f 100644
--- 
a/streams/src/main/java/org/apache/kafka/streams/state/internals/AbstractDualSchemaRocksDBSegmentedBytesStore.java
+++ 
b/streams/src/main/java/org/apache/kafka/streams/state/internals/AbstractDualSchemaRocksDBSegmentedBytesStore.java
@@ -52,6 +52,7 @@ public abstract class 
AbstractDualSchemaRocksDBSegmentedBytesStore segments;
 protected final KeySchema baseKeySchema;
 protected final Optional indexKeySchema;
+private final long retentionPeriod;
 
 
 protected ProcessorContext context;
@@ -66,22 +67,27 @@ public abstract class 
AbstractDualSchemaRocksDBSegmentedBytesStore 
indexKeySchema,
- final AbstractSegments 
segments) {
+ final AbstractSegments 
segments,
+ final long retentionPeriod) {
 this.name = name;
 this.baseKeySchema = baseKeySchema;
 this.indexKeySchema = indexKeySchema;
 this.segments = segments;
+this.retentionPeriod = retentionPeriod;
 }
 
 @Override
 public KeyValueIterator all() {
+
+final long actualFrom = getActualFrom(0, baseKeySchema instanceof 
PrefixedWindowKeySchemas.TimeFirstWindowKeySchema);
+
 final List searchSpace = segments.allSegments(true);
-final Bytes from = baseKeySchema.lowerRange(null, 0);
+final Bytes from = baseKeySchema.lowerRange(null, actualFrom);
 final Bytes to = baseKeySchema.upperRange(null, Long.MAX_VALUE);
 
 return new SegmentIterator<>(
 searchSpace.iterator(),
-baseKeySchema.hasNextCondition(null, null, 0, Long.MAX_VALUE, 
true),
+baseKeySchema.hasNextCondition(null, null, actualFrom, 
Long.MAX_VALUE, true),
 from,
 to,
 true);
@@ -89,13 +95,16 @@ public abstract class 
AbstractDualSchemaRocksDBSegmentedBytesStore backwardAll() {
+
+final long actualFrom = getActualFrom(0, baseKeySchema instanceof 
PrefixedWindowKeySchemas.TimeFirstWindowKeySchema);
+
 final List searchSpace = segments.allSegments(false);
-final Bytes from = baseKeySchema.lowerRange(null, 0);
+final Bytes from = 

[kafka] branch trunk updated: Kafka Streams Threading P1: Add Interface for new TaskManager and TaskExecutor (#12737)

2022-10-12 Thread guozhang
This is an automated email from the ASF dual-hosted git repository.

guozhang pushed a commit to branch trunk
in repository https://gitbox.apache.org/repos/asf/kafka.git


The following commit(s) were added to refs/heads/trunk by this push:
 new 69059b5f288 Kafka Streams Threading P1: Add Interface for new 
TaskManager and TaskExecutor (#12737)
69059b5f288 is described below

commit 69059b5f288ef5b6496953ba1d77b2570d8b3142
Author: Guozhang Wang 
AuthorDate: Wed Oct 12 16:33:13 2022 -0700

Kafka Streams Threading P1: Add Interface for new TaskManager and 
TaskExecutor (#12737)

The interfaces (and their future impls) are added under the 
processor/internals/tasks package, to distinguish with the existing old classes:

1. TaskExecutor is the interface for a processor thread. It takes at most 
one task to process at a given time from the task manager. When being asked 
from the task manager to un-assign the current processing task, it will stop 
processing and give the task back to task manager.
2. TaskManager schedules all the active tasks to assign to TaskExecutors. 
Specifically: 1) when a task executor ask it for an unassigned task to process 
(assignNextTask), it will return the available task based on its scheduling 
algorithm. 2) when the task manager decides to commit (all) tasks, or when a 
rebalance event requires it to modify the maintained active tasks (via 
onAssignment), it will lock all the tasks that are going to be closed / 
committed, asking the TaskExecutor to gi [...]

Reviewers: John Roesler , Anna Sophie Blee-Goldman 

---
 .../processor/internals/tasks/TaskExecutor.java|  57 
 .../processor/internals/tasks/TaskManager.java | 100 +
 2 files changed, 157 insertions(+)

diff --git 
a/streams/src/main/java/org/apache/kafka/streams/processor/internals/tasks/TaskExecutor.java
 
b/streams/src/main/java/org/apache/kafka/streams/processor/internals/tasks/TaskExecutor.java
new file mode 100644
index 000..ead1fb8179e
--- /dev/null
+++ 
b/streams/src/main/java/org/apache/kafka/streams/processor/internals/tasks/TaskExecutor.java
@@ -0,0 +1,57 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.streams.processor.internals.tasks;
+
+import org.apache.kafka.common.KafkaFuture;
+import org.apache.kafka.streams.processor.internals.ReadOnlyTask;
+import org.apache.kafka.streams.processor.internals.StreamTask;
+
+import java.time.Duration;
+
+public interface TaskExecutor {
+
+/**
+ * Starts the task processor.
+ */
+void start();
+
+/**
+ * Shuts down the task processor updater.
+ *
+ * @param timeout duration how long to wait until the state updater is 
shut down
+ *
+ * @throws
+ * org.apache.kafka.streams.errors.StreamsException if the state 
updater thread cannot shutdown within the timeout
+ */
+void shutdown(final Duration timeout);
+
+/**
+ * Get the current assigned processing task. The task returned is 
read-only and cannot be modified.
+ *
+ * @return the current processing task
+ */
+ReadOnlyTask currentTask();
+
+/**
+ * Unassign the current processing task from the task processor and give 
it back to the state manager.
+ *
+ * The paused task must be flushed since it may be committed or closed by 
the task manager next.
+ *
+ * This method does not block, instead a future is returned.
+ */
+KafkaFuture unassign();
+}
diff --git 
a/streams/src/main/java/org/apache/kafka/streams/processor/internals/tasks/TaskManager.java
 
b/streams/src/main/java/org/apache/kafka/streams/processor/internals/tasks/TaskManager.java
new file mode 100644
index 000..e9929714aca
--- /dev/null
+++ 
b/streams/src/main/java/org/apache/kafka/streams/processor/internals/tasks/TaskManager.java
@@ -0,0 +1,100 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this 

[kafka] branch trunk updated (62914129c7d -> 4b30ed8e47a)

2022-10-12 Thread jgus
This is an automated email from the ASF dual-hosted git repository.

jgus pushed a change to branch trunk
in repository https://gitbox.apache.org/repos/asf/kafka.git


from 62914129c7d KAFKA-14099 - Fix request logging in connect (#12434)
 add 4b30ed8e47a MINOR: Fix incorrect example in feature command help 
(#12732)

No new revisions were added by this update.

Summary of changes:
 core/src/main/scala/kafka/admin/FeatureCommand.scala | 5 ++---
 1 file changed, 2 insertions(+), 3 deletions(-)



[kafka] branch trunk updated: KAFKA-14099 - Fix request logging in connect (#12434)

2022-10-12 Thread cegerton
This is an automated email from the ASF dual-hosted git repository.

cegerton pushed a commit to branch trunk
in repository https://gitbox.apache.org/repos/asf/kafka.git


The following commit(s) were added to refs/heads/trunk by this push:
 new 62914129c7d KAFKA-14099 - Fix request logging in connect (#12434)
62914129c7d is described below

commit 62914129c7dbf856a7702f33ed456109c6980b0c
Author: Alexandre Garnier 
AuthorDate: Wed Oct 12 16:28:55 2022 +0200

KAFKA-14099 - Fix request logging in connect (#12434)

Reviewers: Chris Egerton 
---
 build.gradle   |   1 +
 checkstyle/import-control.xml  |   1 +
 .../kafka/connect/runtime/rest/RestServer.java |  16 +--
 .../kafka/connect/runtime/rest/RestServerTest.java | 137 -
 .../unit/kafka/utils/LogCaptureAppender.scala  |   6 +
 5 files changed, 94 insertions(+), 67 deletions(-)

diff --git a/build.gradle b/build.gradle
index 642c904d766..7cf6cc44b1c 100644
--- a/build.gradle
+++ b/build.gradle
@@ -1281,6 +1281,7 @@ project(':clients') {
 testRuntimeOnly libs.jacksonJDK8Datatypes
 testImplementation libs.jose4j
 testImplementation libs.jacksonJaxrsJsonProvider
+testImplementation libs.log4j
 
 generator project(':generator')
   }
diff --git a/checkstyle/import-control.xml b/checkstyle/import-control.xml
index cad049935b5..b2959d9ae6b 100644
--- a/checkstyle/import-control.xml
+++ b/checkstyle/import-control.xml
@@ -573,6 +573,7 @@
 
 
 
+
 
   
 
diff --git 
a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/rest/RestServer.java
 
b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/rest/RestServer.java
index 3c89ddb55fc..baa9b041520 100644
--- 
a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/rest/RestServer.java
+++ 
b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/rest/RestServer.java
@@ -41,8 +41,6 @@ import org.eclipse.jetty.server.Server;
 import org.eclipse.jetty.server.ServerConnector;
 import org.eclipse.jetty.server.Slf4jRequestLogWriter;
 import org.eclipse.jetty.server.handler.ContextHandlerCollection;
-import org.eclipse.jetty.server.handler.DefaultHandler;
-import org.eclipse.jetty.server.handler.RequestLogHandler;
 import org.eclipse.jetty.server.handler.StatisticsHandler;
 import org.eclipse.jetty.servlet.FilterHolder;
 import org.eclipse.jetty.servlet.ServletContextHandler;
@@ -190,6 +188,11 @@ public class RestServer {
 public void initializeServer() {
 log.info("Initializing REST server");
 
+Slf4jRequestLogWriter slf4jRequestLogWriter = new 
Slf4jRequestLogWriter();
+
slf4jRequestLogWriter.setLoggerName(RestServer.class.getCanonicalName());
+CustomRequestLog requestLog = new 
CustomRequestLog(slf4jRequestLogWriter, CustomRequestLog.EXTENDED_NCSA_FORMAT + 
" %{ms}T");
+jettyServer.setRequestLog(requestLog);
+
 /* Needed for graceful shutdown as per `setStopTimeout` documentation 
*/
 StatisticsHandler statsHandler = new StatisticsHandler();
 statsHandler.setHandler(handlers);
@@ -284,15 +287,6 @@ public class RestServer {
 configureHttpResponsHeaderFilter(context);
 }
 
-RequestLogHandler requestLogHandler = new RequestLogHandler();
-Slf4jRequestLogWriter slf4jRequestLogWriter = new 
Slf4jRequestLogWriter();
-
slf4jRequestLogWriter.setLoggerName(RestServer.class.getCanonicalName());
-CustomRequestLog requestLog = new 
CustomRequestLog(slf4jRequestLogWriter, CustomRequestLog.EXTENDED_NCSA_FORMAT + 
" %{ms}T");
-requestLogHandler.setRequestLog(requestLog);
-
-contextHandlers.add(new DefaultHandler());
-contextHandlers.add(requestLogHandler);
-
 handlers.setHandlers(contextHandlers.toArray(new Handler[0]));
 try {
 context.start();
diff --git 
a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/rest/RestServerTest.java
 
b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/rest/RestServerTest.java
index 2aedb32cdf3..82f9e0395e3 100644
--- 
a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/rest/RestServerTest.java
+++ 
b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/rest/RestServerTest.java
@@ -16,10 +16,12 @@
  */
 package org.apache.kafka.connect.runtime.rest;
 
+import kafka.utils.LogCaptureAppender;
 import com.fasterxml.jackson.core.type.TypeReference;
 import com.fasterxml.jackson.databind.ObjectMapper;
 import org.apache.http.HttpHost;
 import org.apache.http.HttpRequest;
+import org.apache.http.HttpResponse;
 import org.apache.http.client.methods.CloseableHttpResponse;
 import org.apache.http.client.methods.HttpGet;
 import org.apache.http.client.methods.HttpOptions;
@@ -44,8 +46,11 @@ import org.slf4j.LoggerFactory;
 import javax.ws.rs.core.MediaType;
 import 

[kafka] branch trunk updated: KAFKA-12965 - Graceful clean up of task error metrics (#10910)

2022-10-12 Thread cegerton
This is an automated email from the ASF dual-hosted git repository.

cegerton pushed a commit to branch trunk
in repository https://gitbox.apache.org/repos/asf/kafka.git


The following commit(s) were added to refs/heads/trunk by this push:
 new a6b60e7cf38 KAFKA-12965 - Graceful clean up of task error metrics 
(#10910)
a6b60e7cf38 is described below

commit a6b60e7cf3860be7a19708a2b762e6bd507885d7
Author: Ramesh 
AuthorDate: Wed Oct 12 08:59:52 2022 -0500

KAFKA-12965 - Graceful clean up of task error metrics (#10910)

Reviewers: Chris Egerton 
---
 .../kafka/connect/runtime/AbstractWorkerSourceTask.java   |  4 +++-
 .../connect/runtime/ExactlyOnceWorkerSourceTask.java  |  4 +++-
 .../java/org/apache/kafka/connect/runtime/Worker.java |  6 +++---
 .../org/apache/kafka/connect/runtime/WorkerSinkTask.java  |  4 +++-
 .../apache/kafka/connect/runtime/WorkerSourceTask.java|  4 +++-
 .../java/org/apache/kafka/connect/runtime/WorkerTask.java |  9 -
 .../connect/runtime/errors/ErrorHandlingMetrics.java  | 15 ++-
 .../connect/runtime/AbstractWorkerSourceTaskTest.java |  4 +++-
 .../kafka/connect/runtime/ErrorHandlingTaskTest.java  |  4 ++--
 .../connect/runtime/ExactlyOnceWorkerSourceTaskTest.java  |  4 +++-
 .../apache/kafka/connect/runtime/WorkerSinkTaskTest.java  |  5 -
 .../kafka/connect/runtime/WorkerSinkTaskThreadedTest.java |  4 +++-
 .../kafka/connect/runtime/WorkerSourceTaskTest.java   |  4 +++-
 .../org/apache/kafka/connect/runtime/WorkerTaskTest.java  | 13 +++--
 14 files changed, 62 insertions(+), 22 deletions(-)

diff --git 
a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/AbstractWorkerSourceTask.java
 
b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/AbstractWorkerSourceTask.java
index 693ef510f1a..671e9aefa7f 100644
--- 
a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/AbstractWorkerSourceTask.java
+++ 
b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/AbstractWorkerSourceTask.java
@@ -36,6 +36,7 @@ import org.apache.kafka.connect.errors.ConnectException;
 import org.apache.kafka.connect.errors.RetriableException;
 import org.apache.kafka.connect.header.Header;
 import org.apache.kafka.connect.header.Headers;
+import org.apache.kafka.connect.runtime.errors.ErrorHandlingMetrics;
 import org.apache.kafka.connect.runtime.errors.RetryWithToleranceOperator;
 import org.apache.kafka.connect.runtime.errors.Stage;
 import org.apache.kafka.connect.runtime.errors.ToleranceType;
@@ -217,13 +218,14 @@ public abstract class AbstractWorkerSourceTask extends 
WorkerTask {
ConnectorOffsetBackingStore offsetStore,
WorkerConfig workerConfig,
ConnectMetrics connectMetrics,
+   ErrorHandlingMetrics errorMetrics,
ClassLoader loader,
Time time,
RetryWithToleranceOperator 
retryWithToleranceOperator,
StatusBackingStore statusBackingStore,
Executor closeExecutor) {
 
-super(id, statusListener, initialState, loader, connectMetrics,
+super(id, statusListener, initialState, loader, connectMetrics, 
errorMetrics,
 retryWithToleranceOperator, time, statusBackingStore);
 
 this.workerConfig = workerConfig;
diff --git 
a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/ExactlyOnceWorkerSourceTask.java
 
b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/ExactlyOnceWorkerSourceTask.java
index 931917b9e15..428ad2fb07b 100644
--- 
a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/ExactlyOnceWorkerSourceTask.java
+++ 
b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/ExactlyOnceWorkerSourceTask.java
@@ -27,6 +27,7 @@ import org.apache.kafka.common.metrics.stats.Min;
 import org.apache.kafka.common.utils.Time;
 import org.apache.kafka.common.utils.Utils;
 import org.apache.kafka.connect.errors.ConnectException;
+import org.apache.kafka.connect.runtime.errors.ErrorHandlingMetrics;
 import org.apache.kafka.connect.runtime.errors.RetryWithToleranceOperator;
 import org.apache.kafka.connect.source.SourceRecord;
 import org.apache.kafka.connect.source.SourceTask;
@@ -85,6 +86,7 @@ class ExactlyOnceWorkerSourceTask extends 
AbstractWorkerSourceTask {
WorkerConfig workerConfig,
ClusterConfigState configState,
ConnectMetrics connectMetrics,
+   ErrorHandlingMetrics errorMetrics,
ClassLoader loader,
Time time,