You are viewing a plain text version of this content. The canonical link for it is here.
Posted to gitbox@hive.apache.org by GitBox <gi...@apache.org> on 2022/08/03 01:48:51 UTC

[GitHub] [hive] subhasisgorai commented on a diff in pull request #3444: AS[]: Dump Tables in parallel

subhasisgorai commented on code in PR #3444:
URL: https://github.com/apache/hive/pull/3444#discussion_r935595737


##########
ql/src/java/org/apache/hadoop/hive/ql/parse/repl/dump/DumpExporter.java:
##########
@@ -0,0 +1,95 @@
+/*
+ * 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.hadoop.hive.ql.parse.repl.dump;
+
+import com.google.common.util.concurrent.ThreadFactoryBuilder;
+import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.metastore.ThreadPool;
+import org.apache.hadoop.hive.ql.metadata.HiveException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.LinkedList;
+import java.util.List;
+import java.util.concurrent.*;
+
+public class DumpExporter {

Review Comment:
   +1 to @pudidic's comment, this is a wrapper, a pure wrapper around an executor service, it does not contain any exporter logic. So the Class name should change and be clear and intuitive.
   Since this is not going to be a very generic purpose executor service and each task(Runnable) will have the same priority/importance the chances of abusing the thread pool is feeble. While I agree that the object lifecycle management by the caller could be error-prone and result in nondesirable side effects. Hence the design may need a second thought. 



##########
common/src/java/org/apache/hadoop/hive/conf/HiveConf.java:
##########
@@ -528,6 +528,9 @@ public static enum ConfVars {
             + "task increment that would cross the specified limit."),
     REPL_PARTITIONS_DUMP_PARALLELISM("hive.repl.partitions.dump.parallelism",100,
         "Number of threads that will be used to dump partition data information during repl dump."),
+    REPL_EXPORT_DUMP_PARALLELISM("hive.repl.export.dump.parallelism", 0,
+            "Number of threads that will be used to dump tables and partitions data information during repl dump." +
+                    "Supersedes 'hive.repl.partitions.dump.parallelism' if set greater than 0"),

Review Comment:
   If this conf takes precedence over the hive.repl.partitions.dump.parallelism, we should probably document this over there as well. Otherwise, this can lead to confusion.



##########
ql/src/java/org/apache/hadoop/hive/ql/parse/repl/dump/PartitionExport.java:
##########
@@ -73,78 +75,95 @@ class PartitionExport {
     this.nThreads = hiveConf.getIntVar(HiveConf.ConfVars.REPL_PARTITIONS_DUMP_PARALLELISM);
     this.queue = new ArrayBlockingQueue<>(2 * nThreads);
     this.callersSession = SessionState.get();
+    this.dumpExporter = dumpExporter;
   }
 
   List<DataCopyPath> write(final ReplicationSpec forReplicationSpec, boolean isExportTask,
                                    FileList fileList, boolean dataCopyAtLoad)
           throws InterruptedException, HiveException {
-    List<Future<?>> futures = new LinkedList<>();
     List<DataCopyPath> managedTableCopyPaths = new LinkedList<>();
-    ExecutorService producer = Executors.newFixedThreadPool(1,
-        new ThreadFactoryBuilder().setNameFormat("partition-submitter-thread-%d").build());
-    futures.add(producer.submit(() -> {
-      SessionState.setCurrentSessionState(callersSession);
-      for (Partition partition : partitionIterable) {
-        try {
-          queue.put(partition);
-        } catch (InterruptedException e) {
-          throw new RuntimeException(
-              "Error while queuing up the partitions for export of data files", e);
+    if (dumpExporter.isSingleThreadedDumpExporter()) {
+      List<Future<?>> futures = new LinkedList<>();
+      ExecutorService producer = Executors.newFixedThreadPool(1,
+              new ThreadFactoryBuilder().setNameFormat("partition-submitter-thread-%d").build());
+      futures.add(producer.submit(() -> {
+        SessionState.setCurrentSessionState(callersSession);
+        for (Partition partition : partitionIterable) {
+          try {
+            queue.put(partition);
+          } catch (InterruptedException e) {
+            throw new RuntimeException(
+                    "Error while queuing up the partitions for export of data files", e);
+          }
         }
-      }
-    }));
-    producer.shutdown();
+      }));
+      producer.shutdown();
 
-    ThreadFactory namingThreadFactory =
-        new ThreadFactoryBuilder().setNameFormat("partition-dump-thread-%d").build();
-    ExecutorService consumer = Executors.newFixedThreadPool(nThreads, namingThreadFactory);
+      ThreadFactory namingThreadFactory =
+              new ThreadFactoryBuilder().setNameFormat("partition-dump-thread-%d").build();
+      ExecutorService consumer = Executors.newFixedThreadPool(nThreads, namingThreadFactory);
 
-    while (!producer.isTerminated() || !queue.isEmpty()) {
+      while (!producer.isTerminated() || !queue.isEmpty()) {
       /*
       This is removed using a poll because there can be a case where there partitions iterator is empty
       but because both the producer and consumer are started simultaneously the while loop will execute
       because producer is not terminated but it wont produce anything so queue will be empty and then we
       should only wait for a specific time before continuing, as the next loop cycle will fail.
        */
-      Partition partition = queue.poll(1, TimeUnit.SECONDS);
-      if (partition == null) {
-        continue;
+        Partition partition = queue.poll(1, TimeUnit.SECONDS);
+        if (partition == null) {
+          continue;
+        }
+        LOG.debug("scheduling partition dump {}", partition.getName());
+        Runnable r = () -> {
+          dumpPartition(partition, forReplicationSpec, isExportTask, fileList, dataCopyAtLoad);
+        };
+        futures.add(consumer.submit(r));
       }
-      LOG.debug("scheduling partition dump {}", partition.getName());
-      futures.add(consumer.submit(() -> {
-        String partitionName = partition.getName();
-        String threadName = Thread.currentThread().getName();
-        LOG.debug("Thread: {}, start partition dump {}", threadName, partitionName);
+      consumer.shutdown();
+      for (Future<?> future : futures) {
         try {
-          // Data Copy in case of ExportTask or when dataCopyAtLoad is true
-          List<Path> dataPathList = Utils.getDataPathList(partition.getDataLocation(),
-                  forReplicationSpec, hiveConf);
-          Path rootDataDumpDir = isExportTask
-                  ? paths.partitionMetadataExportDir(partitionName) : paths.partitionDataExportDir(partitionName);
-          new FileOperations(dataPathList, rootDataDumpDir, distCpDoAsUser, hiveConf, mmCtx)
-                  .export(isExportTask, dataCopyAtLoad);
-          Path dataDumpDir = new Path(paths.dataExportRootDir(), partitionName);
-          LOG.debug("Thread: {}, finish partition dump {}", threadName, partitionName);
-          if (!(isExportTask || dataCopyAtLoad)) {
-            fileList.add(new DataCopyPath(forReplicationSpec, partition.getDataLocation(),
-                    dataDumpDir).convertToString());
-          }
+          future.get();
         } catch (Exception e) {
-          throw new RuntimeException(e.getMessage(), e);
+          LOG.error("failed", e.getCause());
+          throw new HiveException(e.getCause().getMessage(), e.getCause());
         }
-      }));
-    }
-    consumer.shutdown();
-    for (Future<?> future : futures) {
-      try {
-        future.get();
-      } catch (Exception e) {
-        LOG.error("failed", e.getCause());
-        throw new HiveException(e.getCause().getMessage(), e.getCause());
+      }
+      // may be drive this via configuration as well.
+      consumer.awaitTermination(Long.MAX_VALUE, TimeUnit.SECONDS);

Review Comment:
   Is this needed here?



##########
ql/src/java/org/apache/hadoop/hive/ql/parse/repl/dump/TableExport.java:
##########
@@ -160,31 +160,50 @@ private void writeMetaData(PartitionIterable partitions) throws SemanticExceptio
     }
   }
 
-  private void writeData(PartitionIterable partitions, boolean isExportTask, FileList fileList, boolean dataCopyAtLoad)
+  private void writeData(DumpExporter dumpExporter, PartitionIterable partitions, boolean isExportTask, FileList fileList, boolean dataCopyAtLoad)
           throws SemanticException {
     try {
       if (tableSpec.tableHandle.isPartitioned()) {
         if (partitions == null) {
           throw new IllegalStateException("partitions cannot be null for partitionTable :"
               + tableSpec.getTableName().getTable());
         }
-        new PartitionExport(paths, partitions, distCpDoAsUser, conf, mmCtx).write(replicationSpec, isExportTask,
+        new PartitionExport(paths, partitions, distCpDoAsUser, conf, mmCtx, dumpExporter).write(replicationSpec, isExportTask,
                 fileList, dataCopyAtLoad);
       } else {
-        List<Path> dataPathList = Utils.getDataPathList(tableSpec.tableHandle.getDataLocation(),
-                replicationSpec, conf);
-        if (!(isExportTask || dataCopyAtLoad)) {
-          fileList.add(new DataCopyPath(replicationSpec, tableSpec.tableHandle.getDataLocation(),
-                  paths.dataExportDir()).convertToString());
+        if (dumpExporter.isSingleThreadedDumpExporter()) {

Review Comment:
   If DumpExporter instantiated with the no-args constructor, it executes the Runnable on the calling thread itself, correct me if my understanding is wrong.
   If that's the case, why do we need this if-else check?



##########
ql/src/java/org/apache/hadoop/hive/ql/parse/repl/dump/DumpExporter.java:
##########
@@ -0,0 +1,95 @@
+/*
+ * 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.hadoop.hive.ql.parse.repl.dump;
+
+import com.google.common.util.concurrent.ThreadFactoryBuilder;
+import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.metastore.ThreadPool;
+import org.apache.hadoop.hive.ql.metadata.HiveException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.LinkedList;
+import java.util.List;
+import java.util.concurrent.*;
+
+public class DumpExporter {
+  private ExecutorService execService;
+  private List<Future<?>> futures;
+
+  private static final Logger LOG = LoggerFactory.getLogger(DumpExporter.class);
+
+  public DumpExporter() {
+    execService = null;
+    futures = null;
+  }
+
+  public DumpExporter(HiveConf hiveConf) {
+    final int nDumpThreads = hiveConf.getIntVar(HiveConf.ConfVars.REPL_EXPORT_DUMP_PARALLELISM);
+    LOG.debug("DumpExporter created with thread pool size {} ", nDumpThreads);
+    if (nDumpThreads != 0) {
+      ThreadFactory namingThreadFactory = new ThreadFactoryBuilder().setNameFormat("TableAndPartition-dump-thread-%d").build();
+      execService = Executors.newFixedThreadPool(nDumpThreads, namingThreadFactory);
+      futures = new LinkedList<>();
+    } else {
+      execService = null;
+      futures = null;
+    }
+  }
+
+  public static DumpExporter getSingleThreadedDumpExporter() {
+    return new DumpExporter();
+  }
+
+  public boolean isSingleThreadedDumpExporter() {
+    return execService == null && futures == null;
+  }
+
+  public void submit(Runnable r) {
+    if (execService != null && futures != null) {
+      futures.add(execService.submit(r));
+    } else {
+      r.run();
+    }
+  }
+
+  public void finish() throws HiveException {
+    if (!isSingleThreadedDumpExporter()) {
+      execService.shutdown();
+      for (Future<?> future : futures) {
+        try {
+          future.get();
+        } catch (Exception e) {
+          LOG.error("DumpExporter task failed with reason: {} ", e.getCause());
+          throw new HiveException(e.getMessage(), e);
+        }
+      }
+    }
+  }
+
+  public long getTotalTaskEverExecuted() {
+    if (!isSingleThreadedDumpExporter()) {
+      ThreadPoolExecutor executor = (ThreadPoolExecutor) execService;

Review Comment:
   +1



##########
ql/src/java/org/apache/hadoop/hive/ql/parse/repl/dump/DumpExporter.java:
##########
@@ -0,0 +1,95 @@
+/*
+ * 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.hadoop.hive.ql.parse.repl.dump;
+
+import com.google.common.util.concurrent.ThreadFactoryBuilder;
+import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.metastore.ThreadPool;
+import org.apache.hadoop.hive.ql.metadata.HiveException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.LinkedList;
+import java.util.List;
+import java.util.concurrent.*;
+
+public class DumpExporter {
+  private ExecutorService execService;
+  private List<Future<?>> futures;
+
+  private static final Logger LOG = LoggerFactory.getLogger(DumpExporter.class);
+
+  public DumpExporter() {
+    execService = null;
+    futures = null;
+  }
+
+  public DumpExporter(HiveConf hiveConf) {
+    final int nDumpThreads = hiveConf.getIntVar(HiveConf.ConfVars.REPL_EXPORT_DUMP_PARALLELISM);
+    LOG.debug("DumpExporter created with thread pool size {} ", nDumpThreads);
+    if (nDumpThreads != 0) {
+      ThreadFactory namingThreadFactory = new ThreadFactoryBuilder().setNameFormat("TableAndPartition-dump-thread-%d").build();
+      execService = Executors.newFixedThreadPool(nDumpThreads, namingThreadFactory);
+      futures = new LinkedList<>();
+    } else {
+      execService = null;
+      futures = null;
+    }
+  }
+
+  public static DumpExporter getSingleThreadedDumpExporter() {
+    return new DumpExporter();
+  }
+
+  public boolean isSingleThreadedDumpExporter() {
+    return execService == null && futures == null;
+  }
+
+  public void submit(Runnable r) {
+    if (execService != null && futures != null) {
+      futures.add(execService.submit(r));
+    } else {
+      r.run();
+    }
+  }
+
+  public void finish() throws HiveException {
+    if (!isSingleThreadedDumpExporter()) {
+      execService.shutdown();
+      for (Future<?> future : futures) {
+        try {
+          future.get();
+        } catch (Exception e) {
+          LOG.error("DumpExporter task failed with reason: {} ", e.getCause());
+          throw new HiveException(e.getMessage(), e);
+        }

Review Comment:
   The compact and elegant way to achieve this could be leveraging ExecutorService#awaitTermination https://docs.oracle.com/javase/8/docs/api/java/util/concurrent/ExecutorService.html#awaitTermination-long-java.util.concurrent.TimeUnit-



##########
ql/src/java/org/apache/hadoop/hive/ql/exec/ExportTask.java:
##########
@@ -50,7 +52,9 @@ public int execute() {
       work.acidPostProcess(db);
       TableExport tableExport = new TableExport(exportPaths, work.getTableSpec(),
           work.getReplicationSpec(), db, null, conf, work.getMmContext());
-      tableExport.write(true, null, false);
+      DumpExporter dumpExporter = DumpExporter.getSingleThreadedDumpExporter();
+      tableExport.write(dumpExporter, true, null, false);
+      dumpExporter.finish();

Review Comment:
   Thinking of exceptional situations, while I still need to understand the failure handling - should we invoke the finish from the 'finally' block?



##########
ql/src/java/org/apache/hadoop/hive/ql/parse/repl/dump/PartitionExport.java:
##########
@@ -73,78 +75,95 @@ class PartitionExport {
     this.nThreads = hiveConf.getIntVar(HiveConf.ConfVars.REPL_PARTITIONS_DUMP_PARALLELISM);
     this.queue = new ArrayBlockingQueue<>(2 * nThreads);
     this.callersSession = SessionState.get();
+    this.dumpExporter = dumpExporter;
   }
 
   List<DataCopyPath> write(final ReplicationSpec forReplicationSpec, boolean isExportTask,
                                    FileList fileList, boolean dataCopyAtLoad)
           throws InterruptedException, HiveException {
-    List<Future<?>> futures = new LinkedList<>();
     List<DataCopyPath> managedTableCopyPaths = new LinkedList<>();
-    ExecutorService producer = Executors.newFixedThreadPool(1,
-        new ThreadFactoryBuilder().setNameFormat("partition-submitter-thread-%d").build());
-    futures.add(producer.submit(() -> {
-      SessionState.setCurrentSessionState(callersSession);
-      for (Partition partition : partitionIterable) {
-        try {
-          queue.put(partition);
-        } catch (InterruptedException e) {
-          throw new RuntimeException(
-              "Error while queuing up the partitions for export of data files", e);
+    if (dumpExporter.isSingleThreadedDumpExporter()) {

Review Comment:
   Why this check here? 
   If I understood your design correctly, we can merely pass the Runnable to the DumpExporter instantiated with the no-args constructor. We can better organize the code to avoid duplication.
   Please let me know if I am missing something.



##########
ql/src/java/org/apache/hadoop/hive/ql/parse/repl/dump/PartitionExport.java:
##########
@@ -73,78 +75,95 @@ class PartitionExport {
     this.nThreads = hiveConf.getIntVar(HiveConf.ConfVars.REPL_PARTITIONS_DUMP_PARALLELISM);
     this.queue = new ArrayBlockingQueue<>(2 * nThreads);
     this.callersSession = SessionState.get();
+    this.dumpExporter = dumpExporter;
   }
 
   List<DataCopyPath> write(final ReplicationSpec forReplicationSpec, boolean isExportTask,
                                    FileList fileList, boolean dataCopyAtLoad)
           throws InterruptedException, HiveException {
-    List<Future<?>> futures = new LinkedList<>();
     List<DataCopyPath> managedTableCopyPaths = new LinkedList<>();
-    ExecutorService producer = Executors.newFixedThreadPool(1,
-        new ThreadFactoryBuilder().setNameFormat("partition-submitter-thread-%d").build());
-    futures.add(producer.submit(() -> {
-      SessionState.setCurrentSessionState(callersSession);
-      for (Partition partition : partitionIterable) {
-        try {
-          queue.put(partition);
-        } catch (InterruptedException e) {
-          throw new RuntimeException(
-              "Error while queuing up the partitions for export of data files", e);
+    if (dumpExporter.isSingleThreadedDumpExporter()) {
+      List<Future<?>> futures = new LinkedList<>();
+      ExecutorService producer = Executors.newFixedThreadPool(1,
+              new ThreadFactoryBuilder().setNameFormat("partition-submitter-thread-%d").build());
+      futures.add(producer.submit(() -> {
+        SessionState.setCurrentSessionState(callersSession);
+        for (Partition partition : partitionIterable) {
+          try {
+            queue.put(partition);
+          } catch (InterruptedException e) {
+            throw new RuntimeException(
+                    "Error while queuing up the partitions for export of data files", e);
+          }
         }
-      }
-    }));
-    producer.shutdown();
+      }));
+      producer.shutdown();
 
-    ThreadFactory namingThreadFactory =
-        new ThreadFactoryBuilder().setNameFormat("partition-dump-thread-%d").build();
-    ExecutorService consumer = Executors.newFixedThreadPool(nThreads, namingThreadFactory);
+      ThreadFactory namingThreadFactory =
+              new ThreadFactoryBuilder().setNameFormat("partition-dump-thread-%d").build();
+      ExecutorService consumer = Executors.newFixedThreadPool(nThreads, namingThreadFactory);
 
-    while (!producer.isTerminated() || !queue.isEmpty()) {
+      while (!producer.isTerminated() || !queue.isEmpty()) {

Review Comment:
   are these space(s) intentional?



##########
ql/src/java/org/apache/hadoop/hive/ql/exec/repl/ReplDumpTask.java:
##########
@@ -1289,6 +1294,7 @@ Long bootStrapDump(Path dumpRoot, DumpMetaData dmd, Path cmRoot, Hive hiveDb)
               tableList.add(tblName);
             }
           }
+          dumpExporter.finish();

Review Comment:
   Same observation as earlier, please see if we should move this within a 'finally' block.



##########
ql/src/java/org/apache/hadoop/hive/ql/parse/repl/dump/DumpExporter.java:
##########
@@ -0,0 +1,95 @@
+/*
+ * 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.hadoop.hive.ql.parse.repl.dump;
+
+import com.google.common.util.concurrent.ThreadFactoryBuilder;
+import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.metastore.ThreadPool;
+import org.apache.hadoop.hive.ql.metadata.HiveException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.LinkedList;
+import java.util.List;
+import java.util.concurrent.*;
+
+public class DumpExporter {
+  private ExecutorService execService;
+  private List<Future<?>> futures;
+
+  private static final Logger LOG = LoggerFactory.getLogger(DumpExporter.class);
+
+  public DumpExporter() {
+    execService = null;
+    futures = null;
+  }
+
+  public DumpExporter(HiveConf hiveConf) {
+    final int nDumpThreads = hiveConf.getIntVar(HiveConf.ConfVars.REPL_EXPORT_DUMP_PARALLELISM);
+    LOG.debug("DumpExporter created with thread pool size {} ", nDumpThreads);
+    if (nDumpThreads != 0) {
+      ThreadFactory namingThreadFactory = new ThreadFactoryBuilder().setNameFormat("TableAndPartition-dump-thread-%d").build();
+      execService = Executors.newFixedThreadPool(nDumpThreads, namingThreadFactory);
+      futures = new LinkedList<>();
+    } else {
+      execService = null;
+      futures = null;
+    }
+  }
+
+  public static DumpExporter getSingleThreadedDumpExporter() {

Review Comment:
   This is confusing - IMO, if you call the function as SingleThreaded still it carries the notion of being asynchronous which is not the case here. Here the execution of runnable happens on the calling thread only in a synchronous manner.



##########
ql/src/java/org/apache/hadoop/hive/ql/parse/repl/dump/DumpExporter.java:
##########
@@ -0,0 +1,95 @@
+/*
+ * 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.hadoop.hive.ql.parse.repl.dump;
+
+import com.google.common.util.concurrent.ThreadFactoryBuilder;
+import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.metastore.ThreadPool;
+import org.apache.hadoop.hive.ql.metadata.HiveException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.LinkedList;
+import java.util.List;
+import java.util.concurrent.*;
+
+public class DumpExporter {

Review Comment:
   +1 to @pudidic's observation, please document well all the classes and methods. I noticed especially the documentation is poor for the replication code, we should not add up.



##########
ql/src/java/org/apache/hadoop/hive/ql/parse/repl/dump/PartitionExport.java:
##########
@@ -73,78 +75,95 @@ class PartitionExport {
     this.nThreads = hiveConf.getIntVar(HiveConf.ConfVars.REPL_PARTITIONS_DUMP_PARALLELISM);
     this.queue = new ArrayBlockingQueue<>(2 * nThreads);
     this.callersSession = SessionState.get();
+    this.dumpExporter = dumpExporter;
   }
 
   List<DataCopyPath> write(final ReplicationSpec forReplicationSpec, boolean isExportTask,
                                    FileList fileList, boolean dataCopyAtLoad)
           throws InterruptedException, HiveException {
-    List<Future<?>> futures = new LinkedList<>();
     List<DataCopyPath> managedTableCopyPaths = new LinkedList<>();
-    ExecutorService producer = Executors.newFixedThreadPool(1,
-        new ThreadFactoryBuilder().setNameFormat("partition-submitter-thread-%d").build());
-    futures.add(producer.submit(() -> {
-      SessionState.setCurrentSessionState(callersSession);
-      for (Partition partition : partitionIterable) {
-        try {
-          queue.put(partition);
-        } catch (InterruptedException e) {
-          throw new RuntimeException(
-              "Error while queuing up the partitions for export of data files", e);
+    if (dumpExporter.isSingleThreadedDumpExporter()) {
+      List<Future<?>> futures = new LinkedList<>();
+      ExecutorService producer = Executors.newFixedThreadPool(1,
+              new ThreadFactoryBuilder().setNameFormat("partition-submitter-thread-%d").build());
+      futures.add(producer.submit(() -> {
+        SessionState.setCurrentSessionState(callersSession);
+        for (Partition partition : partitionIterable) {
+          try {
+            queue.put(partition);
+          } catch (InterruptedException e) {
+            throw new RuntimeException(
+                    "Error while queuing up the partitions for export of data files", e);
+          }
         }
-      }
-    }));
-    producer.shutdown();
+      }));
+      producer.shutdown();
 
-    ThreadFactory namingThreadFactory =
-        new ThreadFactoryBuilder().setNameFormat("partition-dump-thread-%d").build();
-    ExecutorService consumer = Executors.newFixedThreadPool(nThreads, namingThreadFactory);
+      ThreadFactory namingThreadFactory =
+              new ThreadFactoryBuilder().setNameFormat("partition-dump-thread-%d").build();
+      ExecutorService consumer = Executors.newFixedThreadPool(nThreads, namingThreadFactory);
 
-    while (!producer.isTerminated() || !queue.isEmpty()) {
+      while (!producer.isTerminated() || !queue.isEmpty()) {
       /*
       This is removed using a poll because there can be a case where there partitions iterator is empty
       but because both the producer and consumer are started simultaneously the while loop will execute
       because producer is not terminated but it wont produce anything so queue will be empty and then we
       should only wait for a specific time before continuing, as the next loop cycle will fail.
        */
-      Partition partition = queue.poll(1, TimeUnit.SECONDS);
-      if (partition == null) {
-        continue;
+        Partition partition = queue.poll(1, TimeUnit.SECONDS);
+        if (partition == null) {
+          continue;
+        }
+        LOG.debug("scheduling partition dump {}", partition.getName());
+        Runnable r = () -> {
+          dumpPartition(partition, forReplicationSpec, isExportTask, fileList, dataCopyAtLoad);
+        };
+        futures.add(consumer.submit(r));
       }
-      LOG.debug("scheduling partition dump {}", partition.getName());
-      futures.add(consumer.submit(() -> {
-        String partitionName = partition.getName();
-        String threadName = Thread.currentThread().getName();
-        LOG.debug("Thread: {}, start partition dump {}", threadName, partitionName);
+      consumer.shutdown();
+      for (Future<?> future : futures) {
         try {
-          // Data Copy in case of ExportTask or when dataCopyAtLoad is true
-          List<Path> dataPathList = Utils.getDataPathList(partition.getDataLocation(),
-                  forReplicationSpec, hiveConf);
-          Path rootDataDumpDir = isExportTask
-                  ? paths.partitionMetadataExportDir(partitionName) : paths.partitionDataExportDir(partitionName);
-          new FileOperations(dataPathList, rootDataDumpDir, distCpDoAsUser, hiveConf, mmCtx)
-                  .export(isExportTask, dataCopyAtLoad);
-          Path dataDumpDir = new Path(paths.dataExportRootDir(), partitionName);
-          LOG.debug("Thread: {}, finish partition dump {}", threadName, partitionName);
-          if (!(isExportTask || dataCopyAtLoad)) {
-            fileList.add(new DataCopyPath(forReplicationSpec, partition.getDataLocation(),
-                    dataDumpDir).convertToString());
-          }
+          future.get();

Review Comment:
   Can we leverage ExecutorService#awaitTermination here?



##########
ql/src/java/org/apache/hadoop/hive/ql/exec/repl/ReplDumpTask.java:
##########
@@ -978,6 +980,8 @@ private Long incrementalDump(Path dumpRoot, DumpMetaData dmd, Path cmRoot, Hive
               LOG.debug(te.getMessage());
             }
           }
+          dumpExporter.finish();

Review Comment:
   Same comment as above, can we make sure this gets executed irrespective of whatever exceptions we will probably encounter?



##########
ql/src/java/org/apache/hadoop/hive/ql/parse/repl/dump/PartitionExport.java:
##########
@@ -73,78 +75,95 @@ class PartitionExport {
     this.nThreads = hiveConf.getIntVar(HiveConf.ConfVars.REPL_PARTITIONS_DUMP_PARALLELISM);
     this.queue = new ArrayBlockingQueue<>(2 * nThreads);
     this.callersSession = SessionState.get();
+    this.dumpExporter = dumpExporter;
   }
 
   List<DataCopyPath> write(final ReplicationSpec forReplicationSpec, boolean isExportTask,
                                    FileList fileList, boolean dataCopyAtLoad)
           throws InterruptedException, HiveException {
-    List<Future<?>> futures = new LinkedList<>();
     List<DataCopyPath> managedTableCopyPaths = new LinkedList<>();
-    ExecutorService producer = Executors.newFixedThreadPool(1,
-        new ThreadFactoryBuilder().setNameFormat("partition-submitter-thread-%d").build());
-    futures.add(producer.submit(() -> {
-      SessionState.setCurrentSessionState(callersSession);
-      for (Partition partition : partitionIterable) {
-        try {
-          queue.put(partition);
-        } catch (InterruptedException e) {
-          throw new RuntimeException(
-              "Error while queuing up the partitions for export of data files", e);
+    if (dumpExporter.isSingleThreadedDumpExporter()) {
+      List<Future<?>> futures = new LinkedList<>();
+      ExecutorService producer = Executors.newFixedThreadPool(1,
+              new ThreadFactoryBuilder().setNameFormat("partition-submitter-thread-%d").build());
+      futures.add(producer.submit(() -> {
+        SessionState.setCurrentSessionState(callersSession);
+        for (Partition partition : partitionIterable) {
+          try {
+            queue.put(partition);
+          } catch (InterruptedException e) {
+            throw new RuntimeException(
+                    "Error while queuing up the partitions for export of data files", e);
+          }
         }
-      }
-    }));
-    producer.shutdown();
+      }));
+      producer.shutdown();
 
-    ThreadFactory namingThreadFactory =
-        new ThreadFactoryBuilder().setNameFormat("partition-dump-thread-%d").build();
-    ExecutorService consumer = Executors.newFixedThreadPool(nThreads, namingThreadFactory);
+      ThreadFactory namingThreadFactory =
+              new ThreadFactoryBuilder().setNameFormat("partition-dump-thread-%d").build();

Review Comment:
   What is the change here?



##########
ql/src/java/org/apache/hadoop/hive/ql/parse/repl/dump/DumpExporter.java:
##########
@@ -0,0 +1,95 @@
+/*
+ * 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.hadoop.hive.ql.parse.repl.dump;
+
+import com.google.common.util.concurrent.ThreadFactoryBuilder;
+import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.metastore.ThreadPool;
+import org.apache.hadoop.hive.ql.metadata.HiveException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.LinkedList;
+import java.util.List;
+import java.util.concurrent.*;

Review Comment:
   +1, please don't use wildcard imports and maintain a proper import order.



-- 
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: gitbox-unsubscribe@hive.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org