-
Notifications
You must be signed in to change notification settings - Fork 2.5k
[HUDI-1483] Support async clustering for deltastreamer and Spark streaming #3142
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Changes from all commits
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
| Original file line number | Diff line number | Diff line change |
|---|---|---|
| @@ -0,0 +1,100 @@ | ||
| /* | ||
| * 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.AbstractClusteringClient; | ||
| import org.apache.hudi.client.AbstractHoodieWriteClient; | ||
| 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 java.io.IOException; | ||
| import java.util.concurrent.CompletableFuture; | ||
| import java.util.concurrent.ExecutorService; | ||
| import java.util.concurrent.Executors; | ||
| import java.util.stream.IntStream; | ||
|
|
||
| /** | ||
| * Async clustering service that runs in a separate thread. | ||
| * Currently, only one clustering thread is allowed to run at any time. | ||
| */ | ||
| public abstract class AsyncClusteringService extends HoodieAsyncService { | ||
|
|
||
| private static final long serialVersionUID = 1L; | ||
| private static final Logger LOG = LogManager.getLogger(AsyncClusteringService.class); | ||
|
|
||
codope marked this conversation as resolved.
Outdated
Show resolved
Hide resolved
|
||
| private final int maxConcurrentClustering; | ||
| private transient AbstractClusteringClient clusteringClient; | ||
|
|
||
| public AsyncClusteringService(AbstractHoodieWriteClient writeClient) { | ||
| this(writeClient, false); | ||
| } | ||
|
|
||
| public AsyncClusteringService(AbstractHoodieWriteClient writeClient, boolean runInDaemonMode) { | ||
| super(runInDaemonMode); | ||
| this.clusteringClient = createClusteringClient(writeClient); | ||
| this.maxConcurrentClustering = 1; | ||
| } | ||
|
|
||
| protected abstract AbstractClusteringClient createClusteringClient(AbstractHoodieWriteClient client); | ||
|
|
||
| /** | ||
| * Start clustering service. | ||
| */ | ||
| @Override | ||
| protected Pair<CompletableFuture, ExecutorService> startService() { | ||
| ExecutorService executor = Executors.newFixedThreadPool(maxConcurrentClustering, | ||
| r -> { | ||
| Thread t = new Thread(r, "async_clustering_thread"); | ||
| t.setDaemon(isRunInDaemonMode()); | ||
| return t; | ||
| }); | ||
|
|
||
| return Pair.of(CompletableFuture.allOf(IntStream.range(0, maxConcurrentClustering).mapToObj(i -> CompletableFuture.supplyAsync(() -> { | ||
| try { | ||
| while (!isShutdownRequested()) { | ||
| final HoodieInstant instant = fetchNextAsyncServiceInstant(); | ||
| if (null != instant) { | ||
| LOG.info("Starting clustering for instant " + instant); | ||
| clusteringClient.cluster(instant); | ||
| LOG.info("Finished clustering for instant " + instant); | ||
| } | ||
| } | ||
| LOG.info("Clustering executor shutting down properly"); | ||
| } catch (InterruptedException ie) { | ||
| LOG.warn("Clustering executor got interrupted exception! Stopping", ie); | ||
| } catch (IOException e) { | ||
| LOG.error("Clustering executor failed", e); | ||
| throw new HoodieIOException(e.getMessage(), e); | ||
| } | ||
| return true; | ||
| }, executor)).toArray(CompletableFuture[]::new)), executor); | ||
| } | ||
|
|
||
| /** | ||
| * Update the write client to be used for clustering. | ||
| */ | ||
| public synchronized void updateWriteClient(AbstractHoodieWriteClient writeClient) { | ||
codope marked this conversation as resolved.
Outdated
Show resolved
Hide resolved
|
||
| this.clusteringClient.updateWriteClient(writeClient); | ||
| } | ||
| } | ||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
| @@ -0,0 +1,55 @@ | ||
| /* | ||
| * 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.model.HoodieRecordPayload; | ||
| import org.apache.hudi.common.table.timeline.HoodieInstant; | ||
|
|
||
| import java.io.IOException; | ||
| import java.io.Serializable; | ||
|
|
||
| /** | ||
| * Client will run one round of clustering. | ||
| */ | ||
| public abstract class AbstractClusteringClient<T extends HoodieRecordPayload, I, K, O> implements Serializable { | ||
codope marked this conversation as resolved.
Outdated
Show resolved
Hide resolved
|
||
|
|
||
| private static final long serialVersionUID = 1L; | ||
|
|
||
| protected transient AbstractHoodieWriteClient<T, I, K, O> clusteringClient; | ||
|
|
||
codope marked this conversation as resolved.
Outdated
Show resolved
Hide resolved
|
||
| public AbstractClusteringClient(AbstractHoodieWriteClient<T, I, K, O> clusteringClient) { | ||
| this.clusteringClient = clusteringClient; | ||
| } | ||
|
|
||
| /** | ||
| * Run clustering for the instant. | ||
| * @param instant | ||
| * @throws IOException | ||
| */ | ||
| public abstract void cluster(HoodieInstant instant) throws IOException; | ||
|
||
|
|
||
| /** | ||
| * Update the write client used by async clustering. | ||
| * @param writeClient | ||
| */ | ||
| public void updateWriteClient(AbstractHoodieWriteClient<T, I, K, O> writeClient) { | ||
| this.clusteringClient = writeClient; | ||
| } | ||
| } | ||
Uh oh!
There was an error while loading. Please reload this page.