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

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

                Author: ASF GitHub Bot
            Created on: 26/Oct/20 17:32
            Start Date: 26/Oct/20 17:32
    Worklog Time Spent: 10m 
      Work Description: mustafaiman commented on a change in pull request #1577:
URL: https://github.com/apache/hive/pull/1577#discussion_r512144358



##########
File path: 
ql/src/java/org/apache/hadoop/hive/ql/cleanup/EventualCleanupService.java
##########
@@ -0,0 +1,145 @@
+/*
+ * 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.cleanup;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.util.concurrent.ThreadFactoryBuilder;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.ThreadFactory;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+public class EventualCleanupService implements CleanupService {
+  private final int threadCount;
+  private final int queueSize;
+  private final ThreadFactory factory;
+  private final Logger LOG = 
LoggerFactory.getLogger(EventualCleanupService.class.getName());
+  private final AtomicBoolean isRunning = new AtomicBoolean(true);
+  private final BlockingQueue<AsyncDeleteAction> deleteActions;
+  private ExecutorService cleanerExecutorService;
+
+  public EventualCleanupService(int threadCount, int queueSize) {
+    if (queueSize < threadCount) {
+      throw new IllegalArgumentException("Queue size should be greater or 
equal to thread count. Queue size: "
+          + queueSize + ", thread count: " + threadCount);
+    }
+    this.factory = new 
ThreadFactoryBuilder().setDaemon(true).setNameFormat("EventualCleanupService 
thread %d").build();
+    this.threadCount = threadCount;
+    this.queueSize = queueSize;
+    this.deleteActions = new LinkedBlockingQueue<>(queueSize);
+  }
+
+  @Override
+  public synchronized void start() {
+    if (cleanerExecutorService != null) {
+      LOG.debug("EventualCleanupService is already running.");
+      return;
+    }
+    cleanerExecutorService = Executors.newFixedThreadPool(threadCount, 
factory);
+    for (int i = 0; i < threadCount; i++) {
+      cleanerExecutorService.submit(new CleanupRunnable());

Review comment:
       CleanupService is one per HiveServer, not per session. So there will be 
N threads running all the time (actually blocked on blocking queue when there 
is no cleanup to be done, so not scheduled when unncessary).




----------------------------------------------------------------
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: 504820)
    Time Spent: 2h  (was: 1h 50m)

> Move scratchdir cleanup to background
> -------------------------------------
>
>                 Key: HIVE-24270
>                 URL: https://issues.apache.org/jira/browse/HIVE-24270
>             Project: Hive
>          Issue Type: Improvement
>            Reporter: Mustafa Iman
>            Assignee: Mustafa Iman
>            Priority: Major
>              Labels: pull-request-available
>          Time Spent: 2h
>  Remaining Estimate: 0h
>
> In cloud environment, scratchdir cleaning at the end of the query may take 
> long time. This causes client to hang up to 1 minute even after the results 
> were streamed back. During this time client just waits for cleanup to finish. 
> Cleanup can take place in the background in HiveServer.



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

Reply via email to