You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@pulsar.apache.org by GitBox <gi...@apache.org> on 2021/01/18 09:17:58 UTC

[GitHub] [pulsar] congbobo184 opened a new pull request #9229: [Transaction] Transaction timeout implementation.

congbobo184 opened a new pull request #9229:
URL: https://github.com/apache/pulsar/pull/9229


   ## Motivation
   in order to handle the transaction timeout.
   
   when aborting and committing finish we should change the status in transaction coordinator.
   ## implement
   1. add the transaction timeout tracker factory
   2. add the transaction timeout tracker
   3. use HashedWheelTimer to implement it.
   ### Verifying this change
   Add the tests for it
   
   Does this pull request potentially affect one of the following parts:
   If yes was chosen, please highlight the changes
   
   Dependencies (does it add or upgrade a dependency): (no)
   The public API: (no)
   The schema: (no)
   The default values of configurations: (no)
   The wire protocol: (no)
   The rest endpoints: (no)
   The admin cli options: (no)
   Anything that affects deployment: (no)
   
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [pulsar] congbobo184 commented on a change in pull request #9229: [Transaction] Transaction timeout implementation.

Posted by GitBox <gi...@apache.org>.
congbobo184 commented on a change in pull request #9229:
URL: https://github.com/apache/pulsar/pull/9229#discussion_r560705392



##########
File path: pulsar-transaction/coordinator/src/main/java/org/apache/pulsar/transaction/coordinator/TransactionTimeoutTracker.java
##########
@@ -0,0 +1,62 @@
+/**
+ * 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.pulsar.transaction.coordinator;
+
+import com.google.common.annotations.Beta;
+
+import java.util.concurrent.CompletableFuture;
+
+/**
+ * Represent the tracker for the timeout of the transaction.
+ */
+@Beta
+public interface TransactionTimeoutTracker extends AutoCloseable {
+
+    /**
+     * Add a txnID to the tracker.
+     *
+     * @param sequenceId
+     *            the sequenceId
+     * @param timeout
+     *            the absolute timestamp for transaction timeout
+     *
+     * @return true if the transaction was added to the tracker or false if had timed out
+     */
+    CompletableFuture<Boolean> addTransaction(long sequenceId, long timeout);
+
+    /**
+     * When replay the log, add the txnMeta to timer task queue.
+     *
+     * @param sequenceId
+     *            the sequenceId
+     * @param timeout
+     *            the absolute timestamp for transaction timeout
+     */
+    void replayAddTransaction(long sequenceId, long timeout);

Review comment:
       transaction log not reply complete, we can't start timer to abort transaction.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [pulsar] congbobo184 commented on pull request #9229: [Transaction] Transaction timeout implementation.

Posted by GitBox <gi...@apache.org>.
congbobo184 commented on pull request #9229:
URL: https://github.com/apache/pulsar/pull/9229#issuecomment-765105707


   /pulsarbot run-failure-checks


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [pulsar] congbobo184 commented on a change in pull request #9229: [Transaction] Transaction timeout implementation.

Posted by GitBox <gi...@apache.org>.
congbobo184 commented on a change in pull request #9229:
URL: https://github.com/apache/pulsar/pull/9229#discussion_r560702633



##########
File path: pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/timeout/TransactionTimeoutTrackerImpl.java
##########
@@ -0,0 +1,120 @@
+/**
+ * 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.pulsar.broker.transaction.timeout;
+
+import io.netty.util.Timeout;
+import io.netty.util.Timer;
+import io.netty.util.TimerTask;
+import java.time.Clock;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.TimeUnit;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.pulsar.broker.TransactionMetadataStoreService;
+import org.apache.pulsar.client.api.transaction.TxnID;
+import org.apache.pulsar.common.api.proto.TxnAction;
+import org.apache.pulsar.common.util.collections.TripleLongPriorityQueue;
+import org.apache.pulsar.transaction.coordinator.TransactionTimeoutTracker;
+
+/**
+ * An timer-task implementation of {@link TransactionTimeoutTracker}.
+ */
+@Slf4j
+public class TransactionTimeoutTrackerImpl implements TransactionTimeoutTracker, TimerTask {
+
+    private final Timer timer;
+    private final TripleLongPriorityQueue priorityQueue = new TripleLongPriorityQueue();
+    private final long tickTimeMillis;
+    private final Clock clock;
+    private final static long BASE_OF_MILLIS_TO_SECOND = 1000L;
+    private Timeout currentTimeout;
+    private final static long INITIAL_TIMEOUT = 1L;
+    private long nowTaskTimeoutTime = INITIAL_TIMEOUT;

Review comment:
       The timeout may wait time longer than the new transaction timeout time, so we should cancel the current timeout and create a timeout wait time is the new transaction timeout time.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [pulsar] congbobo184 commented on a change in pull request #9229: [Transaction] Transaction timeout implementation.

Posted by GitBox <gi...@apache.org>.
congbobo184 commented on a change in pull request #9229:
URL: https://github.com/apache/pulsar/pull/9229#discussion_r560678757



##########
File path: pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/timeout/TransactionTimeoutTrackerFactoryImpl.java
##########
@@ -0,0 +1,68 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pulsar.broker.transaction.timeout;
+
+import io.netty.util.HashedWheelTimer;
+import io.netty.util.Timer;
+import io.netty.util.concurrent.DefaultThreadFactory;
+import java.util.concurrent.TimeUnit;
+import org.apache.pulsar.broker.TransactionMetadataStoreService;
+import org.apache.pulsar.transaction.coordinator.TransactionCoordinatorID;
+import org.apache.pulsar.transaction.coordinator.TransactionTimeoutTracker;
+import org.apache.pulsar.transaction.coordinator.TransactionTimeoutTrackerFactory;
+
+/**
+ * An timeout tracker factory implementation of {@link TransactionTimeoutTrackerFactory}.
+ */
+public class TransactionTimeoutTrackerFactoryImpl implements TransactionTimeoutTrackerFactory {
+
+    private Timer timer;
+
+    private static final long tickTimeMillis = 1L;

Review comment:
       this tick time is for one timeOut, the timer is use by default.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [pulsar] congbobo184 commented on pull request #9229: [Transaction] Transaction timeout implementation.

Posted by GitBox <gi...@apache.org>.
congbobo184 commented on pull request #9229:
URL: https://github.com/apache/pulsar/pull/9229#issuecomment-764445852


   /pulsarbot run-failure-checks


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [pulsar] congbobo184 commented on pull request #9229: [Transaction] Transaction timeout implementation.

Posted by GitBox <gi...@apache.org>.
congbobo184 commented on pull request #9229:
URL: https://github.com/apache/pulsar/pull/9229#issuecomment-763571939


   /pulsarbot run-failure-checks


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [pulsar] congbobo184 commented on a change in pull request #9229: [Transaction] Transaction timeout implementation.

Posted by GitBox <gi...@apache.org>.
congbobo184 commented on a change in pull request #9229:
URL: https://github.com/apache/pulsar/pull/9229#discussion_r560705872



##########
File path: pulsar-transaction/coordinator/src/main/java/org/apache/pulsar/transaction/coordinator/TransactionTimeoutTracker.java
##########
@@ -0,0 +1,62 @@
+/**
+ * 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.pulsar.transaction.coordinator;
+
+import com.google.common.annotations.Beta;
+
+import java.util.concurrent.CompletableFuture;
+
+/**
+ * Represent the tracker for the timeout of the transaction.
+ */
+@Beta
+public interface TransactionTimeoutTracker extends AutoCloseable {
+
+    /**
+     * Add a txnID to the tracker.
+     *
+     * @param sequenceId
+     *            the sequenceId
+     * @param timeout
+     *            the absolute timestamp for transaction timeout
+     *
+     * @return true if the transaction was added to the tracker or false if had timed out
+     */
+    CompletableFuture<Boolean> addTransaction(long sequenceId, long timeout);
+
+    /**
+     * When replay the log, add the txnMeta to timer task queue.
+     *
+     * @param sequenceId
+     *            the sequenceId
+     * @param timeout
+     *            the absolute timestamp for transaction timeout
+     */
+    void replayAddTransaction(long sequenceId, long timeout);
+
+    /**
+     * When replay the log finished, we need to start the tracker.
+     */
+    void start();

Review comment:
       return void, we also need to async?




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [pulsar] congbobo184 removed a comment on pull request #9229: [Transaction] Transaction timeout implementation.

Posted by GitBox <gi...@apache.org>.
congbobo184 removed a comment on pull request #9229:
URL: https://github.com/apache/pulsar/pull/9229#issuecomment-764399043


   /pulsarbot run-failure-checks


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [pulsar] congbobo184 commented on pull request #9229: [Transaction] Transaction timeout implementation.

Posted by GitBox <gi...@apache.org>.
congbobo184 commented on pull request #9229:
URL: https://github.com/apache/pulsar/pull/9229#issuecomment-763527655


   /pulsarbot run-failure-checks


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [pulsar] congbobo184 commented on pull request #9229: [Transaction] Transaction timeout implementation.

Posted by GitBox <gi...@apache.org>.
congbobo184 commented on pull request #9229:
URL: https://github.com/apache/pulsar/pull/9229#issuecomment-764399043






----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [pulsar] congbobo184 commented on pull request #9229: [Transaction] Transaction timeout implementation.

Posted by GitBox <gi...@apache.org>.
congbobo184 commented on pull request #9229:
URL: https://github.com/apache/pulsar/pull/9229#issuecomment-764448604


   /pulsarbot run-failure-checks


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [pulsar] codelipenghui merged pull request #9229: [Transaction] Transaction timeout implementation.

Posted by GitBox <gi...@apache.org>.
codelipenghui merged pull request #9229:
URL: https://github.com/apache/pulsar/pull/9229


   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [pulsar] congbobo184 commented on a change in pull request #9229: [Transaction] Transaction timeout implementation.

Posted by GitBox <gi...@apache.org>.
congbobo184 commented on a change in pull request #9229:
URL: https://github.com/apache/pulsar/pull/9229#discussion_r560697145



##########
File path: pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/timeout/TransactionTimeoutTrackerImpl.java
##########
@@ -0,0 +1,120 @@
+/**
+ * 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.pulsar.broker.transaction.timeout;
+
+import io.netty.util.Timeout;
+import io.netty.util.Timer;
+import io.netty.util.TimerTask;
+import java.time.Clock;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.TimeUnit;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.pulsar.broker.TransactionMetadataStoreService;
+import org.apache.pulsar.client.api.transaction.TxnID;
+import org.apache.pulsar.common.api.proto.TxnAction;
+import org.apache.pulsar.common.util.collections.TripleLongPriorityQueue;
+import org.apache.pulsar.transaction.coordinator.TransactionTimeoutTracker;
+
+/**
+ * An timer-task implementation of {@link TransactionTimeoutTracker}.
+ */
+@Slf4j
+public class TransactionTimeoutTrackerImpl implements TransactionTimeoutTracker, TimerTask {
+
+    private final Timer timer;
+    private final TripleLongPriorityQueue priorityQueue = new TripleLongPriorityQueue();
+    private final long tickTimeMillis;
+    private final Clock clock;
+    private final static long BASE_OF_MILLIS_TO_SECOND = 1000L;
+    private Timeout currentTimeout;
+    private final static long INITIAL_TIMEOUT = 1L;
+    private long nowTaskTimeoutTime = INITIAL_TIMEOUT;
+    private final long tcId;
+    private final TransactionMetadataStoreService transactionMetadataStoreService;
+
+    TransactionTimeoutTrackerImpl(long tcId, Timer timer, long tickTimeMillis,
+                                  TransactionMetadataStoreService transactionMetadataStoreService) {
+        this.tcId = tcId;
+        this.transactionMetadataStoreService = transactionMetadataStoreService;
+        this.timer = timer;
+        this.tickTimeMillis  = tickTimeMillis;
+        this.clock = Clock.systemUTC();
+    }
+
+    @Override
+    public CompletableFuture<Boolean> addTransaction(long sequenceId, long timeout) {
+        if (timeout < tickTimeMillis) {
+            this.transactionMetadataStoreService.endTransaction(new TxnID(priorityQueue.peekN2(),
+                    priorityQueue.peekN3()), TxnAction.ABORT_VALUE);
+            return CompletableFuture.completedFuture(false);
+        }
+        synchronized (this){
+            long nowTime = clock.millis() / BASE_OF_MILLIS_TO_SECOND;

Review comment:
       Transaction timeout unit is second.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [pulsar] congbobo184 commented on pull request #9229: [Transaction] Transaction timeout implementation.

Posted by GitBox <gi...@apache.org>.
congbobo184 commented on pull request #9229:
URL: https://github.com/apache/pulsar/pull/9229#issuecomment-764399043


   /pulsarbot run-failure-checks


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [pulsar] codelipenghui commented on a change in pull request #9229: [Transaction] Transaction timeout implementation.

Posted by GitBox <gi...@apache.org>.
codelipenghui commented on a change in pull request #9229:
URL: https://github.com/apache/pulsar/pull/9229#discussion_r560177164



##########
File path: pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/timeout/TransactionTimeoutTrackerFactoryImpl.java
##########
@@ -0,0 +1,68 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pulsar.broker.transaction.timeout;
+
+import io.netty.util.HashedWheelTimer;
+import io.netty.util.Timer;
+import io.netty.util.concurrent.DefaultThreadFactory;
+import java.util.concurrent.TimeUnit;
+import org.apache.pulsar.broker.TransactionMetadataStoreService;
+import org.apache.pulsar.transaction.coordinator.TransactionCoordinatorID;
+import org.apache.pulsar.transaction.coordinator.TransactionTimeoutTracker;
+import org.apache.pulsar.transaction.coordinator.TransactionTimeoutTrackerFactory;
+
+/**
+ * An timeout tracker factory implementation of {@link TransactionTimeoutTrackerFactory}.
+ */
+public class TransactionTimeoutTrackerFactoryImpl implements TransactionTimeoutTrackerFactory {
+
+    private Timer timer;
+
+    private static final long tickTimeMillis = 1L;

Review comment:
       Use 1 millis as the default tick duration will lead to high CPU workload, it's better to use the default tick duration 100ms of the HashedWheelTimer

##########
File path: pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/timeout/TransactionTimeoutTrackerFactoryImpl.java
##########
@@ -0,0 +1,68 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pulsar.broker.transaction.timeout;
+
+import io.netty.util.HashedWheelTimer;
+import io.netty.util.Timer;
+import io.netty.util.concurrent.DefaultThreadFactory;
+import java.util.concurrent.TimeUnit;
+import org.apache.pulsar.broker.TransactionMetadataStoreService;
+import org.apache.pulsar.transaction.coordinator.TransactionCoordinatorID;
+import org.apache.pulsar.transaction.coordinator.TransactionTimeoutTracker;
+import org.apache.pulsar.transaction.coordinator.TransactionTimeoutTrackerFactory;
+
+/**
+ * An timeout tracker factory implementation of {@link TransactionTimeoutTrackerFactory}.
+ */
+public class TransactionTimeoutTrackerFactoryImpl implements TransactionTimeoutTrackerFactory {
+
+    private Timer timer;
+
+    private static final long tickTimeMillis = 1L;
+
+    private final TransactionMetadataStoreService transactionMetadataStoreService;
+
+    public TransactionTimeoutTrackerFactoryImpl(TransactionMetadataStoreService transactionMetadataStoreService) {
+        this.transactionMetadataStoreService = transactionMetadataStoreService;
+    }
+
+    @Override
+    public void initialize() {
+        this.timer = new HashedWheelTimer(new DefaultThreadFactory("transaction-timeout-tracker"),

Review comment:
       Please consider reusing the external timer such as the timer that the broker used? This will saving CPU workload.

##########
File path: pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/timeout/TransactionTimeoutTrackerImpl.java
##########
@@ -0,0 +1,120 @@
+/**
+ * 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.pulsar.broker.transaction.timeout;
+
+import io.netty.util.Timeout;
+import io.netty.util.Timer;
+import io.netty.util.TimerTask;
+import java.time.Clock;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.TimeUnit;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.pulsar.broker.TransactionMetadataStoreService;
+import org.apache.pulsar.client.api.transaction.TxnID;
+import org.apache.pulsar.common.api.proto.TxnAction;
+import org.apache.pulsar.common.util.collections.TripleLongPriorityQueue;
+import org.apache.pulsar.transaction.coordinator.TransactionTimeoutTracker;
+
+/**
+ * An timer-task implementation of {@link TransactionTimeoutTracker}.
+ */
+@Slf4j
+public class TransactionTimeoutTrackerImpl implements TransactionTimeoutTracker, TimerTask {
+
+    private final Timer timer;
+    private final TripleLongPriorityQueue priorityQueue = new TripleLongPriorityQueue();
+    private final long tickTimeMillis;
+    private final Clock clock;
+    private final static long BASE_OF_MILLIS_TO_SECOND = 1000L;
+    private Timeout currentTimeout;
+    private final static long INITIAL_TIMEOUT = 1L;
+    private long nowTaskTimeoutTime = INITIAL_TIMEOUT;
+    private final long tcId;
+    private final TransactionMetadataStoreService transactionMetadataStoreService;
+
+    TransactionTimeoutTrackerImpl(long tcId, Timer timer, long tickTimeMillis,
+                                  TransactionMetadataStoreService transactionMetadataStoreService) {
+        this.tcId = tcId;
+        this.transactionMetadataStoreService = transactionMetadataStoreService;
+        this.timer = timer;
+        this.tickTimeMillis  = tickTimeMillis;
+        this.clock = Clock.systemUTC();
+    }
+
+    @Override
+    public CompletableFuture<Boolean> addTransaction(long sequenceId, long timeout) {
+        if (timeout < tickTimeMillis) {
+            this.transactionMetadataStoreService.endTransaction(new TxnID(priorityQueue.peekN2(),
+                    priorityQueue.peekN3()), TxnAction.ABORT_VALUE);
+            return CompletableFuture.completedFuture(false);
+        }
+        synchronized (this){
+            long nowTime = clock.millis() / BASE_OF_MILLIS_TO_SECOND;
+            priorityQueue.add(timeout + nowTime, tcId, sequenceId);
+            long nowTransactionTimeoutTime = nowTime + timeout;
+            if (nowTaskTimeoutTime == INITIAL_TIMEOUT) {
+                currentTimeout = timer.newTimeout(this, timeout, TimeUnit.SECONDS);
+                nowTaskTimeoutTime = nowTransactionTimeoutTime;
+            } else if (nowTaskTimeoutTime > nowTransactionTimeoutTime) {
+                currentTimeout.cancel();
+                currentTimeout = timer.newTimeout(this, timeout, TimeUnit.SECONDS);
+                nowTaskTimeoutTime = nowTransactionTimeoutTime;
+            }
+        }
+        return CompletableFuture.completedFuture(false);
+    }
+
+    @Override
+    public void replayAddTransaction(long sequenceId, long timeout) {
+        long nowTime = clock.millis() / BASE_OF_MILLIS_TO_SECOND;
+        priorityQueue.add(timeout + nowTime, tcId, sequenceId);
+    }
+
+    @Override
+    public void start() {
+        run(null);
+    }
+
+    @Override
+    public void close() {
+        priorityQueue.close();
+        this.close();
+    }
+
+    @Override
+    public void run(Timeout timeout) {
+        synchronized (this){
+            while (!priorityQueue.isEmpty()){
+                long timeoutTime = priorityQueue.peekN1();
+                long nowTime = clock.millis() / BASE_OF_MILLIS_TO_SECOND;
+                if (timeoutTime < nowTime){
+                    transactionMetadataStoreService.endTransaction(new TxnID(priorityQueue.peekN2(),

Review comment:
       How to handle the transaction is committed?  we don't need to abort a transaction that already committed or aborted.

##########
File path: pulsar-broker/src/test/java/org/apache/pulsar/broker/service/TransactionMetadataStoreServiceTest.java
##########
@@ -123,4 +130,92 @@ public void testAddAckedPartitionToTxn() throws ExecutionException, InterruptedE
         transactionMetadataStoreService.removeTransactionMetadataStore(TransactionCoordinatorID.get(0));
         Assert.assertEquals(transactionMetadataStoreService.getStores().size(), 0);
     }
+
+    @Test
+    public void testTimeoutTracker() throws Exception {
+        pulsar.getTransactionMetadataStoreService().addTransactionMetadataStore(TransactionCoordinatorID.get(0));
+        Awaitility.await().atMost(2000, TimeUnit.MILLISECONDS)
+                .until(() -> pulsar.getTransactionMetadataStoreService()
+                        .getStores().get(TransactionCoordinatorID.get(0)) != null);
+        MLTransactionMetadataStore transactionMetadataStore =
+                (MLTransactionMetadataStore) pulsar.getTransactionMetadataStoreService()
+                        .getStores().get(TransactionCoordinatorID.get(0));
+        Method method = TransactionMetadataStoreState.class.getDeclaredMethod("checkIfReady");
+        method.setAccessible(true);
+        Awaitility.await().atMost(1000, TimeUnit.MILLISECONDS)
+                .until(() -> (Boolean) method.invoke(transactionMetadataStore));
+        Field field = MLTransactionMetadataStore.class.getDeclaredField("txnMetaMap");
+        field.setAccessible(true);
+        ConcurrentMap<TxnID, Pair<TxnMeta, List<Position>>> txnMap =
+                (ConcurrentMap<TxnID, Pair<TxnMeta, List<Position>>>) field.get(transactionMetadataStore);
+        for (int i = 0; i < 1000; i ++) {
+            transactionMetadataStore.newTransaction(5).get();
+        }
+
+        txnMap.forEach((txnID, txnMetaListPair) -> {
+            Assert.assertEquals(txnMetaListPair.getLeft().status(), TxnStatus.OPEN);
+        });
+        Awaitility.await().atLeast(5000, TimeUnit.MICROSECONDS).atMost(10000, TimeUnit.MILLISECONDS)
+                .until(() -> txnMap.size() == 0);
+    }
+
+    @Test
+    public void testTimeoutTrackerMultiThreading() throws Exception {
+        pulsar.getTransactionMetadataStoreService().addTransactionMetadataStore(TransactionCoordinatorID.get(0));
+        Awaitility.await().atMost(2000, TimeUnit.MILLISECONDS)
+                .until(() -> pulsar.getTransactionMetadataStoreService()
+                        .getStores().get(TransactionCoordinatorID.get(0)) != null);
+        MLTransactionMetadataStore transactionMetadataStore =
+                (MLTransactionMetadataStore) pulsar.getTransactionMetadataStoreService()
+                        .getStores().get(TransactionCoordinatorID.get(0));
+        Method method = TransactionMetadataStoreState.class.getDeclaredMethod("checkIfReady");
+        method.setAccessible(true);
+        Awaitility.await().atMost(1000, TimeUnit.MILLISECONDS)
+                .until(() -> (Boolean) method.invoke(transactionMetadataStore));
+        Field field = MLTransactionMetadataStore.class.getDeclaredField("txnMetaMap");
+        field.setAccessible(true);
+        ConcurrentMap<TxnID, Pair<TxnMeta, List<Position>>> txnMap =
+                (ConcurrentMap<TxnID, Pair<TxnMeta, List<Position>>>) field.get(transactionMetadataStore);
+        new Thread(() -> {
+            for (int i = 0; i < 100; i ++) {
+                try {
+                    transactionMetadataStore.newTransaction(1);
+                } catch (Exception e) {
+                    //no operation
+                }
+            }
+        }).start();
+
+        new Thread(() -> {
+            for (int i = 0; i < 100; i ++) {
+                try {
+                    transactionMetadataStore.newTransaction(3);
+                } catch (Exception e) {
+                    //no operation
+                }
+            }
+        }).start();
+
+        new Thread(() -> {
+            for (int i = 0; i < 100; i ++) {
+                try {
+                    transactionMetadataStore.newTransaction(2);
+                } catch (Exception e) {
+                    //no operation
+                }
+            }
+        }).start();
+
+        new Thread(() -> {
+            for (int i = 0; i < 100; i ++) {
+                try {
+                    transactionMetadataStore.newTransaction(10);
+                } catch (Exception e) {
+                    //no operation
+                }
+            }
+        }).start();

Review comment:
       use for i ?

##########
File path: pulsar-transaction/coordinator/src/main/java/org/apache/pulsar/transaction/coordinator/TransactionTimeoutTracker.java
##########
@@ -0,0 +1,62 @@
+/**
+ * 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.pulsar.transaction.coordinator;
+
+import com.google.common.annotations.Beta;
+
+import java.util.concurrent.CompletableFuture;
+
+/**
+ * Represent the tracker for the timeout of the transaction.
+ */
+@Beta
+public interface TransactionTimeoutTracker extends AutoCloseable {
+
+    /**
+     * Add a txnID to the tracker.
+     *
+     * @param sequenceId
+     *            the sequenceId
+     * @param timeout
+     *            the absolute timestamp for transaction timeout
+     *
+     * @return true if the transaction was added to the tracker or false if had timed out
+     */
+    CompletableFuture<Boolean> addTransaction(long sequenceId, long timeout);
+
+    /**
+     * When replay the log, add the txnMeta to timer task queue.
+     *
+     * @param sequenceId
+     *            the sequenceId
+     * @param timeout
+     *            the absolute timestamp for transaction timeout
+     */
+    void replayAddTransaction(long sequenceId, long timeout);

Review comment:
       Why can't use addTransaction?

##########
File path: pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/timeout/TransactionTimeoutTrackerImpl.java
##########
@@ -0,0 +1,120 @@
+/**
+ * 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.pulsar.broker.transaction.timeout;
+
+import io.netty.util.Timeout;
+import io.netty.util.Timer;
+import io.netty.util.TimerTask;
+import java.time.Clock;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.TimeUnit;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.pulsar.broker.TransactionMetadataStoreService;
+import org.apache.pulsar.client.api.transaction.TxnID;
+import org.apache.pulsar.common.api.proto.TxnAction;
+import org.apache.pulsar.common.util.collections.TripleLongPriorityQueue;
+import org.apache.pulsar.transaction.coordinator.TransactionTimeoutTracker;
+
+/**
+ * An timer-task implementation of {@link TransactionTimeoutTracker}.
+ */
+@Slf4j
+public class TransactionTimeoutTrackerImpl implements TransactionTimeoutTracker, TimerTask {
+
+    private final Timer timer;
+    private final TripleLongPriorityQueue priorityQueue = new TripleLongPriorityQueue();
+    private final long tickTimeMillis;
+    private final Clock clock;
+    private final static long BASE_OF_MILLIS_TO_SECOND = 1000L;
+    private Timeout currentTimeout;
+    private final static long INITIAL_TIMEOUT = 1L;
+    private long nowTaskTimeoutTime = INITIAL_TIMEOUT;
+    private final long tcId;
+    private final TransactionMetadataStoreService transactionMetadataStoreService;
+
+    TransactionTimeoutTrackerImpl(long tcId, Timer timer, long tickTimeMillis,
+                                  TransactionMetadataStoreService transactionMetadataStoreService) {
+        this.tcId = tcId;
+        this.transactionMetadataStoreService = transactionMetadataStoreService;
+        this.timer = timer;
+        this.tickTimeMillis  = tickTimeMillis;
+        this.clock = Clock.systemUTC();
+    }
+
+    @Override
+    public CompletableFuture<Boolean> addTransaction(long sequenceId, long timeout) {
+        if (timeout < tickTimeMillis) {
+            this.transactionMetadataStoreService.endTransaction(new TxnID(priorityQueue.peekN2(),
+                    priorityQueue.peekN3()), TxnAction.ABORT_VALUE);
+            return CompletableFuture.completedFuture(false);
+        }
+        synchronized (this){
+            long nowTime = clock.millis() / BASE_OF_MILLIS_TO_SECOND;
+            priorityQueue.add(timeout + nowTime, tcId, sequenceId);
+            long nowTransactionTimeoutTime = nowTime + timeout;
+            if (nowTaskTimeoutTime == INITIAL_TIMEOUT) {
+                currentTimeout = timer.newTimeout(this, timeout, TimeUnit.SECONDS);
+                nowTaskTimeoutTime = nowTransactionTimeoutTime;
+            } else if (nowTaskTimeoutTime > nowTransactionTimeoutTime) {
+                currentTimeout.cancel();
+                currentTimeout = timer.newTimeout(this, timeout, TimeUnit.SECONDS);
+                nowTaskTimeoutTime = nowTransactionTimeoutTime;
+            }
+        }
+        return CompletableFuture.completedFuture(false);
+    }
+
+    @Override
+    public void replayAddTransaction(long sequenceId, long timeout) {
+        long nowTime = clock.millis() / BASE_OF_MILLIS_TO_SECOND;
+        priorityQueue.add(timeout + nowTime, tcId, sequenceId);
+    }
+
+    @Override
+    public void start() {
+        run(null);
+    }
+
+    @Override
+    public void close() {
+        priorityQueue.close();
+        this.close();

Review comment:
       should cancel the timeout?

##########
File path: pulsar-transaction/coordinator/src/main/java/org/apache/pulsar/transaction/coordinator/TransactionTimeoutTracker.java
##########
@@ -0,0 +1,62 @@
+/**
+ * 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.pulsar.transaction.coordinator;
+
+import com.google.common.annotations.Beta;
+
+import java.util.concurrent.CompletableFuture;
+
+/**
+ * Represent the tracker for the timeout of the transaction.
+ */
+@Beta
+public interface TransactionTimeoutTracker extends AutoCloseable {
+
+    /**
+     * Add a txnID to the tracker.
+     *
+     * @param sequenceId
+     *            the sequenceId
+     * @param timeout
+     *            the absolute timestamp for transaction timeout
+     *
+     * @return true if the transaction was added to the tracker or false if had timed out
+     */
+    CompletableFuture<Boolean> addTransaction(long sequenceId, long timeout);
+
+    /**
+     * When replay the log, add the txnMeta to timer task queue.
+     *
+     * @param sequenceId
+     *            the sequenceId
+     * @param timeout
+     *            the absolute timestamp for transaction timeout
+     */
+    void replayAddTransaction(long sequenceId, long timeout);
+
+    /**
+     * When replay the log finished, we need to start the tracker.
+     */
+    void start();

Review comment:
       I think the start method does not only used by transactions replay?  Is there any problem that call start first then replay the transactions?

##########
File path: pulsar-transaction/coordinator/src/main/java/org/apache/pulsar/transaction/coordinator/TransactionTimeoutTracker.java
##########
@@ -0,0 +1,62 @@
+/**
+ * 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.pulsar.transaction.coordinator;
+
+import com.google.common.annotations.Beta;
+
+import java.util.concurrent.CompletableFuture;
+
+/**
+ * Represent the tracker for the timeout of the transaction.
+ */
+@Beta
+public interface TransactionTimeoutTracker extends AutoCloseable {
+
+    /**
+     * Add a txnID to the tracker.
+     *
+     * @param sequenceId
+     *            the sequenceId
+     * @param timeout
+     *            the absolute timestamp for transaction timeout
+     *
+     * @return true if the transaction was added to the tracker or false if had timed out
+     */
+    CompletableFuture<Boolean> addTransaction(long sequenceId, long timeout);
+
+    /**
+     * When replay the log, add the txnMeta to timer task queue.
+     *
+     * @param sequenceId
+     *            the sequenceId
+     * @param timeout
+     *            the absolute timestamp for transaction timeout
+     */
+    void replayAddTransaction(long sequenceId, long timeout);
+
+    /**
+     * When replay the log finished, we need to start the tracker.
+     */
+    void start();

Review comment:
       BTW, we can only define all method async. 

##########
File path: pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/timeout/TransactionTimeoutTrackerImpl.java
##########
@@ -0,0 +1,120 @@
+/**
+ * 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.pulsar.broker.transaction.timeout;
+
+import io.netty.util.Timeout;
+import io.netty.util.Timer;
+import io.netty.util.TimerTask;
+import java.time.Clock;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.TimeUnit;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.pulsar.broker.TransactionMetadataStoreService;
+import org.apache.pulsar.client.api.transaction.TxnID;
+import org.apache.pulsar.common.api.proto.TxnAction;
+import org.apache.pulsar.common.util.collections.TripleLongPriorityQueue;
+import org.apache.pulsar.transaction.coordinator.TransactionTimeoutTracker;
+
+/**
+ * An timer-task implementation of {@link TransactionTimeoutTracker}.
+ */
+@Slf4j
+public class TransactionTimeoutTrackerImpl implements TransactionTimeoutTracker, TimerTask {
+
+    private final Timer timer;
+    private final TripleLongPriorityQueue priorityQueue = new TripleLongPriorityQueue();
+    private final long tickTimeMillis;
+    private final Clock clock;
+    private final static long BASE_OF_MILLIS_TO_SECOND = 1000L;
+    private Timeout currentTimeout;
+    private final static long INITIAL_TIMEOUT = 1L;
+    private long nowTaskTimeoutTime = INITIAL_TIMEOUT;

Review comment:
       Could you please give more details about these 2 fields?




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [pulsar] congbobo184 commented on pull request #9229: [Transaction] Transaction timeout implementation.

Posted by GitBox <gi...@apache.org>.
congbobo184 commented on pull request #9229:
URL: https://github.com/apache/pulsar/pull/9229#issuecomment-765028441


   /pulsarbot run-failure-checks


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [pulsar] codelipenghui merged pull request #9229: [Transaction] Transaction timeout implementation.

Posted by GitBox <gi...@apache.org>.
codelipenghui merged pull request #9229:
URL: https://github.com/apache/pulsar/pull/9229


   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [pulsar] congbobo184 removed a comment on pull request #9229: [Transaction] Transaction timeout implementation.

Posted by GitBox <gi...@apache.org>.
congbobo184 removed a comment on pull request #9229:
URL: https://github.com/apache/pulsar/pull/9229#issuecomment-764445852






----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [pulsar] congbobo184 commented on pull request #9229: [Transaction] Transaction timeout implementation.

Posted by GitBox <gi...@apache.org>.
congbobo184 commented on pull request #9229:
URL: https://github.com/apache/pulsar/pull/9229#issuecomment-765129949


   /pulsarbot run-failure-checks


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [pulsar] congbobo184 removed a comment on pull request #9229: [Transaction] Transaction timeout implementation.

Posted by GitBox <gi...@apache.org>.
congbobo184 removed a comment on pull request #9229:
URL: https://github.com/apache/pulsar/pull/9229#issuecomment-763571939


   /pulsarbot run-failure-checks


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [pulsar] congbobo184 commented on pull request #9229: [Transaction] Transaction timeout implementation.

Posted by GitBox <gi...@apache.org>.
congbobo184 commented on pull request #9229:
URL: https://github.com/apache/pulsar/pull/9229#issuecomment-764540031


   /pulsarbot run-failure-checks


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [pulsar] congbobo184 removed a comment on pull request #9229: [Transaction] Transaction timeout implementation.

Posted by GitBox <gi...@apache.org>.
congbobo184 removed a comment on pull request #9229:
URL: https://github.com/apache/pulsar/pull/9229#issuecomment-763571939






----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [pulsar] congbobo184 commented on a change in pull request #9229: [Transaction] Transaction timeout implementation.

Posted by GitBox <gi...@apache.org>.
congbobo184 commented on a change in pull request #9229:
URL: https://github.com/apache/pulsar/pull/9229#discussion_r560702974



##########
File path: pulsar-broker/src/test/java/org/apache/pulsar/broker/service/TransactionMetadataStoreServiceTest.java
##########
@@ -123,4 +130,92 @@ public void testAddAckedPartitionToTxn() throws ExecutionException, InterruptedE
         transactionMetadataStoreService.removeTransactionMetadataStore(TransactionCoordinatorID.get(0));
         Assert.assertEquals(transactionMetadataStoreService.getStores().size(), 0);
     }
+
+    @Test
+    public void testTimeoutTracker() throws Exception {
+        pulsar.getTransactionMetadataStoreService().addTransactionMetadataStore(TransactionCoordinatorID.get(0));
+        Awaitility.await().atMost(2000, TimeUnit.MILLISECONDS)
+                .until(() -> pulsar.getTransactionMetadataStoreService()
+                        .getStores().get(TransactionCoordinatorID.get(0)) != null);
+        MLTransactionMetadataStore transactionMetadataStore =
+                (MLTransactionMetadataStore) pulsar.getTransactionMetadataStoreService()
+                        .getStores().get(TransactionCoordinatorID.get(0));
+        Method method = TransactionMetadataStoreState.class.getDeclaredMethod("checkIfReady");
+        method.setAccessible(true);
+        Awaitility.await().atMost(1000, TimeUnit.MILLISECONDS)
+                .until(() -> (Boolean) method.invoke(transactionMetadataStore));
+        Field field = MLTransactionMetadataStore.class.getDeclaredField("txnMetaMap");
+        field.setAccessible(true);
+        ConcurrentMap<TxnID, Pair<TxnMeta, List<Position>>> txnMap =
+                (ConcurrentMap<TxnID, Pair<TxnMeta, List<Position>>>) field.get(transactionMetadataStore);
+        for (int i = 0; i < 1000; i ++) {
+            transactionMetadataStore.newTransaction(5).get();
+        }
+
+        txnMap.forEach((txnID, txnMetaListPair) -> {
+            Assert.assertEquals(txnMetaListPair.getLeft().status(), TxnStatus.OPEN);
+        });
+        Awaitility.await().atLeast(5000, TimeUnit.MICROSECONDS).atMost(10000, TimeUnit.MILLISECONDS)
+                .until(() -> txnMap.size() == 0);
+    }
+
+    @Test
+    public void testTimeoutTrackerMultiThreading() throws Exception {
+        pulsar.getTransactionMetadataStoreService().addTransactionMetadataStore(TransactionCoordinatorID.get(0));
+        Awaitility.await().atMost(2000, TimeUnit.MILLISECONDS)
+                .until(() -> pulsar.getTransactionMetadataStoreService()
+                        .getStores().get(TransactionCoordinatorID.get(0)) != null);
+        MLTransactionMetadataStore transactionMetadataStore =
+                (MLTransactionMetadataStore) pulsar.getTransactionMetadataStoreService()
+                        .getStores().get(TransactionCoordinatorID.get(0));
+        Method method = TransactionMetadataStoreState.class.getDeclaredMethod("checkIfReady");
+        method.setAccessible(true);
+        Awaitility.await().atMost(1000, TimeUnit.MILLISECONDS)
+                .until(() -> (Boolean) method.invoke(transactionMetadataStore));
+        Field field = MLTransactionMetadataStore.class.getDeclaredField("txnMetaMap");
+        field.setAccessible(true);
+        ConcurrentMap<TxnID, Pair<TxnMeta, List<Position>>> txnMap =
+                (ConcurrentMap<TxnID, Pair<TxnMeta, List<Position>>>) field.get(transactionMetadataStore);
+        new Thread(() -> {
+            for (int i = 0; i < 100; i ++) {
+                try {
+                    transactionMetadataStore.newTransaction(1);
+                } catch (Exception e) {
+                    //no operation
+                }
+            }
+        }).start();
+
+        new Thread(() -> {
+            for (int i = 0; i < 100; i ++) {
+                try {
+                    transactionMetadataStore.newTransaction(3);
+                } catch (Exception e) {
+                    //no operation
+                }
+            }
+        }).start();
+
+        new Thread(() -> {
+            for (int i = 0; i < 100; i ++) {
+                try {
+                    transactionMetadataStore.newTransaction(2);
+                } catch (Exception e) {
+                    //no operation
+                }
+            }
+        }).start();
+
+        new Thread(() -> {
+            for (int i = 0; i < 100; i ++) {
+                try {
+                    transactionMetadataStore.newTransaction(10);
+                } catch (Exception e) {
+                    //no operation
+                }
+            }
+        }).start();

Review comment:
       in order to new 100 transaction, we also change the for to while.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [pulsar] codelipenghui commented on a change in pull request #9229: [Transaction] Transaction timeout implementation.

Posted by GitBox <gi...@apache.org>.
codelipenghui commented on a change in pull request #9229:
URL: https://github.com/apache/pulsar/pull/9229#discussion_r560187664



##########
File path: pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/timeout/TransactionTimeoutTrackerImpl.java
##########
@@ -0,0 +1,120 @@
+/**
+ * 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.pulsar.broker.transaction.timeout;
+
+import io.netty.util.Timeout;
+import io.netty.util.Timer;
+import io.netty.util.TimerTask;
+import java.time.Clock;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.TimeUnit;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.pulsar.broker.TransactionMetadataStoreService;
+import org.apache.pulsar.client.api.transaction.TxnID;
+import org.apache.pulsar.common.api.proto.TxnAction;
+import org.apache.pulsar.common.util.collections.TripleLongPriorityQueue;
+import org.apache.pulsar.transaction.coordinator.TransactionTimeoutTracker;
+
+/**
+ * An timer-task implementation of {@link TransactionTimeoutTracker}.
+ */
+@Slf4j
+public class TransactionTimeoutTrackerImpl implements TransactionTimeoutTracker, TimerTask {
+
+    private final Timer timer;
+    private final TripleLongPriorityQueue priorityQueue = new TripleLongPriorityQueue();
+    private final long tickTimeMillis;
+    private final Clock clock;
+    private final static long BASE_OF_MILLIS_TO_SECOND = 1000L;
+    private Timeout currentTimeout;
+    private final static long INITIAL_TIMEOUT = 1L;
+    private long nowTaskTimeoutTime = INITIAL_TIMEOUT;
+    private final long tcId;
+    private final TransactionMetadataStoreService transactionMetadataStoreService;
+
+    TransactionTimeoutTrackerImpl(long tcId, Timer timer, long tickTimeMillis,
+                                  TransactionMetadataStoreService transactionMetadataStoreService) {
+        this.tcId = tcId;
+        this.transactionMetadataStoreService = transactionMetadataStoreService;
+        this.timer = timer;
+        this.tickTimeMillis  = tickTimeMillis;
+        this.clock = Clock.systemUTC();
+    }
+
+    @Override
+    public CompletableFuture<Boolean> addTransaction(long sequenceId, long timeout) {
+        if (timeout < tickTimeMillis) {
+            this.transactionMetadataStoreService.endTransaction(new TxnID(priorityQueue.peekN2(),
+                    priorityQueue.peekN3()), TxnAction.ABORT_VALUE);
+            return CompletableFuture.completedFuture(false);
+        }
+        synchronized (this){
+            long nowTime = clock.millis() / BASE_OF_MILLIS_TO_SECOND;

Review comment:
       Why not use millis directly?




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [pulsar] congbobo184 commented on pull request #9229: [Transaction] Transaction timeout implementation.

Posted by GitBox <gi...@apache.org>.
congbobo184 commented on pull request #9229:
URL: https://github.com/apache/pulsar/pull/9229#issuecomment-763581182


   /pulsarbot run-failure-checks


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org