You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@inlong.apache.org by zi...@apache.org on 2022/08/04 06:09:51 UTC

[inlong] branch master updated: [INLONG-5169][Sort] Fix race condition issue of HBaseSinkFunction metric collection (#5170)

This is an automated email from the ASF dual-hosted git repository.

zirui pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/inlong.git


The following commit(s) were added to refs/heads/master by this push:
     new 089ea1ed7 [INLONG-5169][Sort] Fix race condition issue of HBaseSinkFunction metric collection (#5170)
089ea1ed7 is described below

commit 089ea1ed74c7819dbc23d60b1aa4bae1b21d4816
Author: Biao Liu <mm...@gmail.com>
AuthorDate: Thu Aug 4 14:09:44 2022 +0800

    [INLONG-5169][Sort] Fix race condition issue of HBaseSinkFunction metric collection (#5170)
---
 .../inlong/sort/base/metric/ThreadSafeCounter.java | 60 +++++++++++++++++++
 .../sort/base/metric/ThreadSafeCounterTest.java    | 67 ++++++++++++++++++++++
 .../inlong/sort/hbase/sink/HBaseSinkFunction.java  | 13 +++--
 3 files changed, 136 insertions(+), 4 deletions(-)

diff --git a/inlong-sort/sort-connectors/connector-base/src/main/java/org/apache/inlong/sort/base/metric/ThreadSafeCounter.java b/inlong-sort/sort-connectors/connector-base/src/main/java/org/apache/inlong/sort/base/metric/ThreadSafeCounter.java
new file mode 100644
index 000000000..7a7366654
--- /dev/null
+++ b/inlong-sort/sort-connectors/connector-base/src/main/java/org/apache/inlong/sort/base/metric/ThreadSafeCounter.java
@@ -0,0 +1,60 @@
+/*
+ * 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.inlong.sort.base.metric;
+
+import org.apache.flink.metrics.Counter;
+
+import java.util.concurrent.atomic.AtomicLong;
+
+/**
+ * A thread-safe implementation of {@link Counter}.
+ */
+public class ThreadSafeCounter implements Counter {
+
+    private final AtomicLong count;
+
+    public ThreadSafeCounter() {
+        count = new AtomicLong(0);
+    }
+
+    @Override
+    public void inc() {
+        count.incrementAndGet();
+    }
+
+    @Override
+    public void inc(long n) {
+        count.addAndGet(n);
+    }
+
+    @Override
+    public void dec() {
+        count.decrementAndGet();
+    }
+
+    @Override
+    public void dec(long n) {
+        count.addAndGet(-n);
+    }
+
+    @Override
+    public long getCount() {
+        return count.get();
+    }
+}
diff --git a/inlong-sort/sort-connectors/connector-base/src/test/java/org/apache/inlong/sort/base/metric/ThreadSafeCounterTest.java b/inlong-sort/sort-connectors/connector-base/src/test/java/org/apache/inlong/sort/base/metric/ThreadSafeCounterTest.java
new file mode 100644
index 000000000..abc08e4d9
--- /dev/null
+++ b/inlong-sort/sort-connectors/connector-base/src/test/java/org/apache/inlong/sort/base/metric/ThreadSafeCounterTest.java
@@ -0,0 +1,67 @@
+/*
+ * 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.inlong.sort.base.metric;
+
+import org.junit.Test;
+
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+
+import static org.junit.Assert.assertEquals;
+
+/**
+ * Unit testing of {@link ThreadSafeCounter}.
+ */
+public class ThreadSafeCounterTest {
+
+    @Test
+    public void testCounting() {
+        final ThreadSafeCounter counter = new ThreadSafeCounter();
+        counter.inc();
+        assertEquals(1, counter.getCount());
+        counter.inc(100);
+        assertEquals(101, counter.getCount());
+        counter.dec();
+        assertEquals(100, counter.getCount());
+        counter.dec(99);
+        assertEquals(1, counter.getCount());
+    }
+
+    @Test(timeout = 10000)
+    public void testMultipleThreadCounting() throws Exception {
+        final ThreadSafeCounter counter = new ThreadSafeCounter();
+        final int threadCount = 3;
+        final int incTimes = 100;
+        final ExecutorService executorService = Executors.newFixedThreadPool(threadCount);
+        final CountDownLatch finished = new CountDownLatch(threadCount);
+        for (int i = 0; i < threadCount; i++) {
+            executorService.submit(() -> {
+                for (int j = 0; j < incTimes; j++) {
+                    counter.inc();
+                }
+                finished.countDown();
+            });
+        }
+        finished.await();
+        assertEquals(threadCount * incTimes, counter.getCount());
+
+        executorService.shutdown();
+    }
+}
diff --git a/inlong-sort/sort-connectors/hbase/src/main/java/org/apache/inlong/sort/hbase/sink/HBaseSinkFunction.java b/inlong-sort/sort-connectors/hbase/src/main/java/org/apache/inlong/sort/hbase/sink/HBaseSinkFunction.java
index 10d92db09..0550393ab 100644
--- a/inlong-sort/sort-connectors/hbase/src/main/java/org/apache/inlong/sort/hbase/sink/HBaseSinkFunction.java
+++ b/inlong-sort/sort-connectors/hbase/src/main/java/org/apache/inlong/sort/hbase/sink/HBaseSinkFunction.java
@@ -38,6 +38,7 @@ import org.apache.hadoop.hbase.client.Connection;
 import org.apache.hadoop.hbase.client.ConnectionFactory;
 import org.apache.hadoop.hbase.client.RetriesExhaustedWithDetailsException;
 import org.apache.inlong.sort.base.metric.SinkMetricData;
+import org.apache.inlong.sort.base.metric.ThreadSafeCounter;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -131,10 +132,14 @@ public class HBaseSinkFunction<T> extends RichSinkFunction<T>
                 String groupId = inLongMetricArray[0];
                 String streamId = inLongMetricArray[1];
                 String nodeId = inLongMetricArray[2];
-                sinkMetricData.registerMetricsForDirtyBytes(groupId, streamId, nodeId, DIRTY_BYTES);
-                sinkMetricData.registerMetricsForDirtyRecords(groupId, streamId, nodeId, DIRTY_RECORDS);
-                sinkMetricData.registerMetricsForNumBytesOut(groupId, streamId, nodeId, NUM_BYTES_OUT);
-                sinkMetricData.registerMetricsForNumRecordsOut(groupId, streamId, nodeId, NUM_RECORDS_OUT);
+                sinkMetricData.registerMetricsForDirtyBytes(groupId, streamId, nodeId, DIRTY_BYTES,
+                    new ThreadSafeCounter());
+                sinkMetricData.registerMetricsForDirtyRecords(groupId, streamId, nodeId, DIRTY_RECORDS,
+                    new ThreadSafeCounter());
+                sinkMetricData.registerMetricsForNumBytesOut(groupId, streamId, nodeId, NUM_BYTES_OUT,
+                    new ThreadSafeCounter());
+                sinkMetricData.registerMetricsForNumRecordsOut(groupId, streamId, nodeId, NUM_RECORDS_OUT,
+                    new ThreadSafeCounter());
                 sinkMetricData.registerMetricsForNumBytesOutPerSecond(groupId, streamId, nodeId,
                         NUM_BYTES_OUT_PER_SECOND);
                 sinkMetricData.registerMetricsForNumRecordsOutPerSecond(groupId, streamId, nodeId,