[HUDI-575] Spark Streaming with async compaction support (#1752)
This commit is contained in:
committed by
GitHub
parent
61e027fadd
commit
7a2429f5ba
@@ -48,9 +48,16 @@ public abstract class AbstractAsyncService implements Serializable {
|
||||
private transient ExecutorService executor;
|
||||
// Future tracking delta-sync/compaction
|
||||
private transient CompletableFuture future;
|
||||
// Run in daemon mode
|
||||
private final boolean runInDaemonMode;
|
||||
|
||||
protected AbstractAsyncService() {
|
||||
this(false);
|
||||
}
|
||||
|
||||
protected AbstractAsyncService(boolean runInDaemonMode) {
|
||||
shutdownRequested = false;
|
||||
this.runInDaemonMode = runInDaemonMode;
|
||||
}
|
||||
|
||||
protected boolean isShutdownRequested() {
|
||||
@@ -129,7 +136,11 @@ public abstract class AbstractAsyncService implements Serializable {
|
||||
*/
|
||||
private void monitorThreads(Function<Boolean, Boolean> onShutdownCallback) {
|
||||
LOG.info("Submitting monitor thread !!");
|
||||
Executors.newSingleThreadExecutor().submit(() -> {
|
||||
Executors.newSingleThreadExecutor(r -> {
|
||||
Thread t = new Thread(r, "Monitor Thread");
|
||||
t.setDaemon(isRunInDaemonMode());
|
||||
return t;
|
||||
}).submit(() -> {
|
||||
boolean error = false;
|
||||
try {
|
||||
LOG.info("Monitoring thread(s) !!");
|
||||
@@ -137,18 +148,21 @@ public abstract class AbstractAsyncService implements Serializable {
|
||||
} catch (ExecutionException ex) {
|
||||
LOG.error("Monitor noticed one or more threads failed. Requesting graceful shutdown of other threads", ex);
|
||||
error = true;
|
||||
shutdown(false);
|
||||
} catch (InterruptedException ie) {
|
||||
LOG.error("Got interrupted Monitoring threads", ie);
|
||||
error = true;
|
||||
shutdown(false);
|
||||
} finally {
|
||||
// Mark as shutdown
|
||||
shutdown = true;
|
||||
if (null != onShutdownCallback) {
|
||||
onShutdownCallback.apply(error);
|
||||
}
|
||||
shutdown(false);
|
||||
}
|
||||
});
|
||||
}
|
||||
|
||||
public boolean isRunInDaemonMode() {
|
||||
return runInDaemonMode;
|
||||
}
|
||||
}
|
||||
|
||||
@@ -0,0 +1,161 @@
|
||||
/*
|
||||
* 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.
|
||||
*/
|
||||
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, client, false);
|
||||
}
|
||||
|
||||
public AsyncCompactService(JavaSparkContext jssc, HoodieWriteClient client, boolean runInDaemonMode) {
|
||||
super(runInDaemonMode);
|
||||
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 -> {
|
||||
Thread t = new Thread(r, "async_compact_thread");
|
||||
t.setDaemon(isRunInDaemonMode());
|
||||
return t;
|
||||
});
|
||||
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);
|
||||
}
|
||||
}
|
||||
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);
|
||||
}
|
||||
|
||||
|
||||
/**
|
||||
* Check whether compactor thread needs to be stopped.
|
||||
* @return
|
||||
*/
|
||||
protected boolean shouldStopCompactor() {
|
||||
return false;
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,62 @@
|
||||
/*
|
||||
* 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.client;
|
||||
|
||||
import org.apache.hudi.common.table.timeline.HoodieInstant;
|
||||
import org.apache.hudi.common.util.Option;
|
||||
import org.apache.hudi.exception.HoodieException;
|
||||
|
||||
import org.apache.log4j.LogManager;
|
||||
import org.apache.log4j.Logger;
|
||||
import org.apache.spark.api.java.JavaRDD;
|
||||
import org.apache.spark.api.java.JavaSparkContext;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.io.Serializable;
|
||||
|
||||
/**
|
||||
* Run one round of compaction.
|
||||
*/
|
||||
public class Compactor implements Serializable {
|
||||
|
||||
private static final long serialVersionUID = 1L;
|
||||
private static final Logger LOG = LogManager.getLogger(Compactor.class);
|
||||
|
||||
private transient HoodieWriteClient compactionClient;
|
||||
private transient JavaSparkContext jssc;
|
||||
|
||||
public Compactor(HoodieWriteClient compactionClient, JavaSparkContext jssc) {
|
||||
this.jssc = jssc;
|
||||
this.compactionClient = compactionClient;
|
||||
}
|
||||
|
||||
public void compact(HoodieInstant instant) throws IOException {
|
||||
LOG.info("Compactor executing compaction " + instant);
|
||||
JavaRDD<WriteStatus> res = compactionClient.compact(instant.getTimestamp());
|
||||
long numWriteErrors = res.collect().stream().filter(WriteStatus::hasErrors).count();
|
||||
if (numWriteErrors != 0) {
|
||||
// We treat even a single error in compaction as fatal
|
||||
LOG.error("Compaction for instant (" + instant + ") failed with write errors. Errors :" + numWriteErrors);
|
||||
throw new HoodieException(
|
||||
"Compaction for instant (" + instant + ") failed with write errors. Errors :" + numWriteErrors);
|
||||
}
|
||||
// Commit compaction
|
||||
compactionClient.commitCompaction(instant.getTimestamp(), res, Option.empty());
|
||||
}
|
||||
}
|
||||
Reference in New Issue
Block a user