vinothchandar commented on a change in pull request #1752:
URL: https://github.com/apache/hudi/pull/1752#discussion_r446628169



##########
File path: hudi-spark/src/test/java/HoodieJavaStreamingApp.java
##########
@@ -68,7 +74,7 @@
   private String tableName = "hoodie_test";
 
   @Parameter(names = {"--table-type", "-t"}, description = "One of 
COPY_ON_WRITE or MERGE_ON_READ")
-  private String tableType = HoodieTableType.MERGE_ON_READ.name();
+  private String tableType = HoodieTableType.COPY_ON_WRITE.name();

Review comment:
       why move to COW?

##########
File path: hudi-integ-test/src/test/java/org/apache/hudi/integ/ITTestBase.java
##########
@@ -58,6 +58,7 @@
   protected static final String PRESTO_COORDINATOR = "/presto-coordinator-1";
   protected static final String HOODIE_WS_ROOT = "/var/hoodie/ws";
   protected static final String HOODIE_JAVA_APP = HOODIE_WS_ROOT + 
"/hudi-spark/run_hoodie_app.sh";
+  protected static final String HOODIE_JAVA_STREAMING_APP = HOODIE_WS_ROOT + 
"/hudi-spark/run_hoodie_streaming_app.sh";

Review comment:
       more importantly, we should also renable the test in `TestDataSource` 

##########
File path: hudi-spark/src/main/scala/org/apache/hudi/HoodieStreamingSink.scala
##########
@@ -38,46 +50,65 @@ class HoodieStreamingSink(sqlContext: SQLContext,
   private val retryIntervalMs = 
options(DataSourceWriteOptions.STREAMING_RETRY_INTERVAL_MS_OPT_KEY).toLong
   private val ignoreFailedBatch = 
options(DataSourceWriteOptions.STREAMING_IGNORE_FAILED_BATCH_OPT_KEY).toBoolean
 
+  private var isAsyncCompactorServiceShutdownAbnormally = false

Review comment:
       
https://jaceklaskowski.gitbooks.io/spark-structured-streaming/spark-sql-streaming-StreamingQueryManager.html
 seems like there are some listeners we can exploit to know of a StreamingQuery?

##########
File path: 
hudi-client/src/main/java/org/apache/hudi/async/AsyncCompactService.java
##########
@@ -0,0 +1,163 @@
+/*
+ * 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.hudi.async;
+
+import org.apache.hudi.client.Compactor;
+import org.apache.hudi.client.HoodieWriteClient;
+import org.apache.hudi.common.table.timeline.HoodieInstant;
+import org.apache.hudi.common.util.collection.Pair;
+import org.apache.hudi.exception.HoodieIOException;
+import org.apache.log4j.LogManager;
+import org.apache.log4j.Logger;
+import org.apache.spark.api.java.JavaSparkContext;
+
+import java.io.IOException;
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.locks.Condition;
+import java.util.concurrent.locks.ReentrantLock;
+import java.util.stream.IntStream;
+
+/**
+ * Async Compactor Service that runs in separate thread. Currently, only one 
compactor is allowed to run at any time.
+ */
+public class AsyncCompactService extends AbstractAsyncService {
+
+  private static final long serialVersionUID = 1L;
+  private static final Logger LOG = 
LogManager.getLogger(AsyncCompactService.class);
+
+  /**
+   * This is the job pool used by async compaction.
+   * In case of deltastreamer, Spark job scheduling configs are automatically 
set.
+   * As the configs needs to be set before spark context is initiated, it is 
not
+   * automated for Structured Streaming.
+   * https://spark.apache.org/docs/latest/job-scheduling.html
+   */
+  public static final String COMPACT_POOL_NAME = "hoodiecompact";
+
+  private final int maxConcurrentCompaction;
+  private transient Compactor compactor;
+  private transient JavaSparkContext jssc;
+  private transient BlockingQueue<HoodieInstant> pendingCompactions = new 
LinkedBlockingQueue<>();
+  private transient ReentrantLock queueLock = new ReentrantLock();
+  private transient Condition consumed = queueLock.newCondition();
+
+  public AsyncCompactService(JavaSparkContext jssc, HoodieWriteClient client) {
+    this.jssc = jssc;
+    this.compactor = new Compactor(client, jssc);
+    this.maxConcurrentCompaction = 1;
+  }
+
+  /**
+   * Enqueues new Pending compaction.
+   */
+  public void enqueuePendingCompaction(HoodieInstant instant) {
+    pendingCompactions.add(instant);
+  }
+
+  /**
+   * Wait till outstanding pending compactions reduces to the passed in value.
+   *
+   * @param numPendingCompactions Maximum pending compactions allowed
+   * @throws InterruptedException
+   */
+  public void waitTillPendingCompactionsReducesTo(int numPendingCompactions) 
throws InterruptedException {
+    try {
+      queueLock.lock();
+      while (!isShutdown() && (pendingCompactions.size() > 
numPendingCompactions)) {
+        consumed.await();
+      }
+    } finally {
+      queueLock.unlock();
+    }
+  }
+
+  /**
+   * Fetch Next pending compaction if available.
+   *
+   * @return
+   * @throws InterruptedException
+   */
+  private HoodieInstant fetchNextCompactionInstant() throws 
InterruptedException {
+    LOG.info("Compactor waiting for next instant for compaction upto 60 
seconds");
+    HoodieInstant instant = pendingCompactions.poll(10, TimeUnit.SECONDS);
+    if (instant != null) {
+      try {
+        queueLock.lock();
+        // Signal waiting thread
+        consumed.signal();
+      } finally {
+        queueLock.unlock();
+      }
+    }
+    return instant;
+  }
+
+  /**
+   * Start Compaction Service.
+   */
+  @Override
+  protected Pair<CompletableFuture, ExecutorService> startService() {
+    ExecutorService executor = 
Executors.newFixedThreadPool(maxConcurrentCompaction,
+        r -> new Thread(r, "async_compact_thread"));
+    return Pair.of(CompletableFuture.allOf(IntStream.range(0, 
maxConcurrentCompaction).mapToObj(i -> CompletableFuture.supplyAsync(() -> {
+      try {
+        // Set Compactor Pool Name for allowing users to prioritize compaction
+        LOG.info("Setting Spark Pool name for compaction to " + 
COMPACT_POOL_NAME);
+        jssc.setLocalProperty("spark.scheduler.pool", COMPACT_POOL_NAME);
+
+        while (!isShutdownRequested()) {
+          final HoodieInstant instant = fetchNextCompactionInstant();
+
+          if (null != instant) {
+            LOG.info("Starting Compaction for instant " + instant);
+            compactor.compact(instant);
+            LOG.info("Finished Compaction for instant " + instant);
+          }
+
+          if (shouldStopCompactor()) {
+            return true;
+          }
+        }
+        LOG.info("Compactor shutting down properly!!");
+      } catch (InterruptedException ie) {
+        LOG.warn("Compactor executor thread got interrupted exception. 
Stopping", ie);
+      } catch (IOException e) {
+        LOG.error("Compactor executor failed", e);
+        throw new HoodieIOException(e.getMessage(), e);
+      }
+      return true;
+    }, executor)).toArray(CompletableFuture[]::new)), executor);
+  }
+
+  /**
+   * Spark Structured Streaming Sink implementation do not have mechanism to 
know when the stream is shutdown.

Review comment:
       move comments that refer to a sub-class impl to that class itself?

##########
File path: hudi-spark/src/main/scala/org/apache/hudi/HoodieStreamingSink.scala
##########
@@ -111,12 +143,64 @@ class HoodieStreamingSink(sqlContext: SQLContext,
 
   @annotation.tailrec
   private def retry[T](n: Int, waitInMillis: Long)(fn: => Try[T]): Try[T] = {
+    lastStartBatchTimeNanos = System.nanoTime()
     fn match {
-      case x: util.Success[T] => x
+      case x: Success[T] =>
+        lastEndBatchTimeNanos = System.nanoTime()
+        x
       case _ if n > 1 =>
         Thread.sleep(waitInMillis)
+        lastEndBatchTimeNanos = System.nanoTime()
         retry(n - 1, waitInMillis * 2)(fn)
-      case f => f
+      case f =>
+        lastEndBatchTimeNanos = System.nanoTime()
+        reset(false)
+        f
+    }
+  }
+
+  protected def triggerAsyncCompactor(client: 
HoodieWriteClient[HoodieRecordPayload[Nothing]]): Unit = {
+    log.info("Triggering Async compaction !!")
+    if (null == asyncCompactorService) {
+      asyncCompactorService = new SparkStreamingAsyncCompactService(new 
JavaSparkContext(sqlContext.sparkContext),
+        client, new SparkStreamingWriterActivityDetector(new 
Supplier[lang.Long] {
+          override def get(): lang.Long = lastStartBatchTimeNanos
+        }, new Supplier[lang.Long] {
+          override def get(): lang.Long = lastEndBatchTimeNanos
+        }, 10))
+      asyncCompactorService.start(new Function[java.lang.Boolean, 
java.lang.Boolean] {
+        override def apply(errored: lang.Boolean): lang.Boolean = {
+          log.info(s"Async Compactor shutdown. Errored ? $errored")
+          isAsyncCompactorServiceShutdownAbnormally = errored
+          reset(false)
+          log.info("Done resetting write client.")
+          true
+        }
+      })
+
+      // Add Shutdown Hook
+      Runtime.getRuntime.addShutdownHook(new Thread(new Runnable {

Review comment:
       this alone should be good enough to prevent the jvm from not hanging 
during exit? do we really need the laststart/lastend logic?

##########
File path: 
hudi-spark/src/main/java/org/apache/hudi/async/SparkStreamingWriterActivityDetector.java
##########
@@ -0,0 +1,77 @@
+/*
+ * 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.hudi.async;
+
+import java.util.function.Supplier;
+import org.apache.log4j.LogManager;
+import org.apache.log4j.Logger;
+
+/**
+ * This class is used to detect activity of spark streaming writer. THis is 
used to decide if HoodieWriteClient
+ * and async compactor needs to be closed. Spark Structured Streaming do not 
have explicit API on the Sink side to
+ * determine if the stream is done. In this absence, async compactor 
proactively checks with the sink if it is
+ * active. If there is no activity for sufficient period, async compactor 
shuts down. If the sink was indeed active,
+ * a subsequent batch will re-trigger async compaction.
+ */
+public class SparkStreamingWriterActivityDetector {
+
+  private static final long serialVersionUID = 1L;
+  private static final Logger LOG = 
LogManager.getLogger(SparkStreamingWriterActivityDetector.class);
+
+  private final Supplier<Long> lastStartBatchNanoTimeSupplier;
+  private final Supplier<Long> lastEndBatchNanoTimeSupplier;
+  private final long sinkInactivityTimeoutSecs;
+
+  private static final long SECS_TO_NANOS = 1000000000L;
+
+  public SparkStreamingWriterActivityDetector(
+      Supplier<Long> lastStartBatchNanoTimeSupplier, Supplier<Long> 
lastEndBatchNanoTimeSupplier,
+      long sinkInactivityTimeoutSecs) {
+    this.lastStartBatchNanoTimeSupplier = lastStartBatchNanoTimeSupplier;
+    this.lastEndBatchNanoTimeSupplier = lastEndBatchNanoTimeSupplier;
+    this.sinkInactivityTimeoutSecs = sinkInactivityTimeoutSecs;
+  }
+
+  /**
+   * Detects if spark streaming write is still active based on time.
+   * @return
+   */
+  public boolean hasRecentlyWritten() {
+    long lastStartBatchTime = lastStartBatchNanoTimeSupplier.get();
+    long lastEndBatchTime = lastEndBatchNanoTimeSupplier.get();
+
+    LOG.info("Checking if compactor needs to be stopped. "
+        + "lastStartBatchTime=" + lastStartBatchTime + ", lastEndBatchTime=" + 
lastEndBatchTime
+        + ", CurrTime=" + System.nanoTime());
+
+    if (lastEndBatchTime - lastStartBatchTime < 0) {
+      LOG.info("End Batch Time (" + lastEndBatchTime + ") is less than Start 
Batch Time (" + lastStartBatchTime + ")"
+          + "Sink is running. So, no need to stop");
+      return true;
+    }
+
+    long currTime = System.nanoTime();
+    long elapsedTimeSecs = Double.valueOf(Math.ceil(1.0 * (currTime - 
lastEndBatchTime) / SECS_TO_NANOS)).longValue();
+    if (elapsedTimeSecs > sinkInactivityTimeoutSecs) {
+      LOG.warn("Streaming Sink has been idle for " + elapsedTimeSecs + " 
seconds");

Review comment:
       this does not mean there is no work for compaction right? 

##########
File path: hudi-spark/src/main/scala/org/apache/hudi/HoodieStreamingSink.scala
##########
@@ -111,12 +143,64 @@ class HoodieStreamingSink(sqlContext: SQLContext,
 
   @annotation.tailrec
   private def retry[T](n: Int, waitInMillis: Long)(fn: => Try[T]): Try[T] = {
+    lastStartBatchTimeNanos = System.nanoTime()
     fn match {
-      case x: util.Success[T] => x
+      case x: Success[T] =>
+        lastEndBatchTimeNanos = System.nanoTime()
+        x
       case _ if n > 1 =>
         Thread.sleep(waitInMillis)
+        lastEndBatchTimeNanos = System.nanoTime()
         retry(n - 1, waitInMillis * 2)(fn)
-      case f => f
+      case f =>
+        lastEndBatchTimeNanos = System.nanoTime()
+        reset(false)
+        f
+    }
+  }
+
+  protected def triggerAsyncCompactor(client: 
HoodieWriteClient[HoodieRecordPayload[Nothing]]): Unit = {
+    log.info("Triggering Async compaction !!")
+    if (null == asyncCompactorService) {
+      asyncCompactorService = new SparkStreamingAsyncCompactService(new 
JavaSparkContext(sqlContext.sparkContext),
+        client, new SparkStreamingWriterActivityDetector(new 
Supplier[lang.Long] {
+          override def get(): lang.Long = lastStartBatchTimeNanos
+        }, new Supplier[lang.Long] {
+          override def get(): lang.Long = lastEndBatchTimeNanos
+        }, 10))
+      asyncCompactorService.start(new Function[java.lang.Boolean, 
java.lang.Boolean] {
+        override def apply(errored: lang.Boolean): lang.Boolean = {
+          log.info(s"Async Compactor shutdown. Errored ? $errored")
+          isAsyncCompactorServiceShutdownAbnormally = errored
+          reset(false)
+          log.info("Done resetting write client.")
+          true
+        }
+      })
+
+      // Add Shutdown Hook
+      Runtime.getRuntime.addShutdownHook(new Thread(new Runnable {
+        override def run(): Unit = reset(true)
+      }))
+
+      // First time, scan .hoodie folder and get all pending compactions
+      val metaClient = new 
HoodieTableMetaClient(sqlContext.sparkContext.hadoopConfiguration,

Review comment:
       Seems like this will happen each trigger/ not just first time?

##########
File path: hudi-spark/src/main/scala/org/apache/hudi/HoodieStreamingSink.scala
##########
@@ -38,46 +50,65 @@ class HoodieStreamingSink(sqlContext: SQLContext,
   private val retryIntervalMs = 
options(DataSourceWriteOptions.STREAMING_RETRY_INTERVAL_MS_OPT_KEY).toLong
   private val ignoreFailedBatch = 
options(DataSourceWriteOptions.STREAMING_IGNORE_FAILED_BATCH_OPT_KEY).toBoolean
 
+  private var isAsyncCompactorServiceShutdownAbnormally = false
+
   private val mode =
     if (outputMode == OutputMode.Append()) {
       SaveMode.Append
     } else {
       SaveMode.Overwrite
     }
 
-  override def addBatch(batchId: Long, data: DataFrame): Unit = {
+  private var asyncCompactorService : AsyncCompactService = _
+  private var writeClient : 
Option[HoodieWriteClient[HoodieRecordPayload[Nothing]]] = Option.empty
+  private var lastStartBatchTimeNanos : lang.Long = System.nanoTime()
+  private var lastEndBatchTimeNanos : lang.Long = System.nanoTime()
+
+  override def addBatch(batchId: Long, data: DataFrame): Unit = 
this.synchronized {
+    if (isAsyncCompactorServiceShutdownAbnormally)  {
+      throw new IllegalStateException("Async Compaction shutdown unexpectedly")
+    }
+
+    lastStartBatchTimeNanos = System.nanoTime()
+
     retry(retryCnt, retryIntervalMs)(
       Try(
         HoodieSparkSqlWriter.write(
-          sqlContext,
-          mode,
-          options,
-          data)
+          sqlContext, mode, options, data, writeClient, 
Some(triggerAsyncCompactor))

Review comment:
       just confirming that reuse of writeClient across batches is fine.. 




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


Reply via email to