[ 
https://issues.apache.org/jira/browse/HIVE-23069?focusedWorklogId=457132&page=com.atlassian.jira.plugin.system.issuetabpanels:worklog-tabpanel#worklog-457132
 ]

ASF GitHub Bot logged work on HIVE-23069:
-----------------------------------------

                Author: ASF GitHub Bot
            Created on: 10/Jul/20 12:35
            Start Date: 10/Jul/20 12:35
    Worklog Time Spent: 10m 
      Work Description: pkumarsinha commented on a change in pull request #1225:
URL: https://github.com/apache/hive/pull/1225#discussion_r452815017



##########
File path: 
ql/src/java/org/apache/hadoop/hive/ql/exec/repl/util/FileListStreamer.java
##########
@@ -0,0 +1,137 @@
+/*
+ * 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.exec.repl.util;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hive.ql.parse.SemanticException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.BufferedWriter;
+import java.io.Closeable;
+import java.io.IOException;
+import java.io.OutputStreamWriter;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.TimeUnit;
+
+public class FileListStreamer extends Thread implements Closeable {
+  private static final Logger LOG = 
LoggerFactory.getLogger(FileListStreamer.class);
+  private static final long TIMEOUT_IN_SECS = 5L;
+  private volatile boolean stop;
+  private final LinkedBlockingQueue<String> cache;
+  private Path backingFile;
+  private Configuration conf;
+  private BufferedWriter backingFileWriter;
+  private volatile boolean valid = true;
+  private volatile boolean asyncMode = false;
+  private final Object COMPLETION_LOCK = new Object();
+  private volatile boolean completed = false;
+
+
+
+  public FileListStreamer(LinkedBlockingQueue<String> cache, Path backingFile, 
Configuration conf) throws IOException {
+    this.cache = cache;
+    this.backingFile = backingFile;
+    this.conf = conf;
+    init();
+  }
+
+  private void init() throws IOException {
+    FileSystem fs = FileSystem.get(backingFile.toUri(), conf);
+    backingFileWriter = new BufferedWriter(new 
OutputStreamWriter(fs.create(backingFile, !asyncMode)));
+    LOG.info("Initialized a file based store to save a list at: {}, 
ayncMode:{}", backingFile, asyncMode);
+  }
+
+  public boolean isValid() {
+    return valid;
+  }
+
+  @Override
+  public void close() throws IOException {
+    if (!asyncMode) {
+      closeBackingFile();
+      return;
+    }
+    stop = true;
+    synchronized (COMPLETION_LOCK) {
+      while (!completed && isValid()) {
+        try {
+          COMPLETION_LOCK.wait(TimeUnit.SECONDS.toMillis(TIMEOUT_IN_SECS));
+        } catch (InterruptedException e) {
+        }
+      }
+    }
+    if (!isValid()) {

Review comment:
       No, it can't be moved above as this ensures the correctness of the 
consumption of the remaining entries from the cache.




----------------------------------------------------------------
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:
us...@infra.apache.org


Issue Time Tracking
-------------------

    Worklog Id:     (was: 457132)
    Time Spent: 2h 10m  (was: 2h)

> Memory efficient iterator should be used during replication.
> ------------------------------------------------------------
>
>                 Key: HIVE-23069
>                 URL: https://issues.apache.org/jira/browse/HIVE-23069
>             Project: Hive
>          Issue Type: Improvement
>            Reporter: Pravin Sinha
>            Assignee: Pravin Sinha
>            Priority: Major
>              Labels: pull-request-available
>         Attachments: HIVE-23069.01.patch
>
>          Time Spent: 2h 10m
>  Remaining Estimate: 0h
>
> Currently the iterator used while copying table data is memory based. In case 
> of a database with very large number of table/partitions, such iterator may 
> cause HS2 process to go OOM.
> Also introduces a config option to run data copy tasks during repl load 
> operation.



--
This message was sent by Atlassian Jira
(v8.3.4#803005)

Reply via email to