Himanshu-g81 commented on code in PR #2188:
URL: https://github.com/apache/phoenix/pull/2188#discussion_r2160358987


##########
phoenix-core-server/src/main/java/org/apache/phoenix/replication/reader/ReplicationLogProcessor.java:
##########
@@ -0,0 +1,288 @@
+/*
+ * 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.phoenix.replication.reader;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Future;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.HBaseConfiguration;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.AsyncConnection;
+import org.apache.hadoop.hbase.client.AsyncTable;
+import org.apache.hadoop.hbase.client.ConnectionFactory;
+import org.apache.hadoop.hbase.client.Mutation;
+import org.apache.hadoop.hbase.client.RetriesExhaustedException;
+import org.apache.hadoop.hbase.util.FutureUtils;
+import org.apache.phoenix.replication.log.LogFile;
+import org.apache.phoenix.replication.log.LogFileReader;
+import org.apache.phoenix.replication.log.LogFileReaderContext;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class ReplicationLogProcessor {
+
+    private static final Logger LOG = 
LoggerFactory.getLogger(ReplicationLogProcessor.class);
+
+    /**
+     * The maximum count of mutations to process in single batch while reading 
replication log file
+     */
+    public static final String REPLICATION_STANDBY_LOG_REPLAY_BATCH_SIZE =
+            "phoenix.replication.log.standby.replay.batch.size";
+
+    /**
+     * The default batch size for reading the replication log file.
+     * Assuming each log record to be 10 KB (un-compressed) and allow at max 
64 MB of
+     * in-memory records to be processed
+     */
+    public static final int DEFAULT_REPLICATION_STANDBY_LOG_REPLAY_BATCH_SIZE 
= 6400;
+
+    /**
+     * The maximum number of retries for HBase client operations while 
applying the mutations
+     */
+    public static final String REPLICATION_STANDBY_HBASE_CLIENT_RETRIES_COUNT =
+            "phoenix.replication.standby.hbase.client.retries.number";
+
+    /**
+     * The default number of retries for HBase client operations while 
applying the mutations.
+     */
+    public static final int 
DEFAULT_REPLICATION_STANDBY_HBASE_CLIENT_RETRIES_COUNT = 4;
+
+    /**
+     * The timeout for HBase client operations while applying the mutations.
+     */
+    public static final String 
REPLICATION_STANDBY_HBASE_CLIENT_OPERATION_TIMEOUT_MS =
+            "phoenix.replication.standby.hbase.client.operations.timeout";
+
+    /**
+     * The default timeout for HBase client operations while applying the 
mutations.
+     */
+    public static final int 
DEFAULT_REPLICATION_STANDBY_HBASE_CLIENT_OPERATION_TIMEOUT_MS = 10000;
+
+    private final Configuration conf;
+
+    private final ExecutorService executorService;
+
+    /**
+     * This {@link AsyncConnection} is used for handling mutations
+     */
+    private volatile AsyncConnection asyncConnection;
+
+    private final Object asyncConnectionLock = new Object();

Review Comment:
   Sure, I usually keep single lock object per attribute that should be 
explicitly locked before access (instead of taking lock over the object 
itself), which might be useful in case we have more such attributes.
   But in this case since we only have single attribute, using object itself 
would also have same effect. Thanks for pointing this.
   
   > if you add an init()
   I thought if multiple threads at upper layer are using this object and one 
of them closed it, others should not be impacted. It's not expected but just 
added it via getConnection() method to be on safe side.



-- 
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.

To unsubscribe, e-mail: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]

Reply via email to