1
0

[HUDI-1483] Support async clustering for deltastreamer and Spark streaming (#3142)

- Integrate async clustering service with HoodieDeltaStreamer and HoodieStreamingSink
- Added methods in HoodieAsyncService to reuse code
This commit is contained in:
Sagar Sumit
2021-07-12 00:13:38 +05:30
committed by GitHub
parent 9b01d2a045
commit 5804ad8e32
23 changed files with 710 additions and 112 deletions

View File

@@ -0,0 +1,39 @@
/*
* 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.client.HoodieSparkClusteringClient;
/**
* Async clustering service for Spark datasource.
*/
public class SparkAsyncClusteringService extends AsyncClusteringService {
public SparkAsyncClusteringService(AbstractHoodieWriteClient writeClient) {
super(writeClient);
}
@Override
protected AbstractClusteringClient createClusteringClient(AbstractHoodieWriteClient client) {
return new HoodieSparkClusteringClient(client);
}
}

View File

@@ -0,0 +1,56 @@
/*
* 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.HoodieKey;
import org.apache.hudi.common.model.HoodieRecord;
import org.apache.hudi.common.model.HoodieRecordPayload;
import org.apache.hudi.common.table.timeline.HoodieInstant;
import org.apache.log4j.LogManager;
import org.apache.log4j.Logger;
import org.apache.spark.api.java.JavaRDD;
import java.io.IOException;
/**
* Async clustering client for Spark datasource.
*/
public class HoodieSparkClusteringClient<T extends HoodieRecordPayload> extends
AbstractClusteringClient<T, JavaRDD<HoodieRecord<T>>, JavaRDD<HoodieKey>, JavaRDD<WriteStatus>> {
private static final Logger LOG = LogManager.getLogger(HoodieSparkClusteringClient.class);
public HoodieSparkClusteringClient(
AbstractHoodieWriteClient<T, JavaRDD<HoodieRecord<T>>, JavaRDD<HoodieKey>, JavaRDD<WriteStatus>> clusteringClient) {
super(clusteringClient);
}
@Override
public void cluster(HoodieInstant instant) throws IOException {
LOG.info("Executing clustering instance " + instant);
SparkRDDWriteClient<T> writeClient = (SparkRDDWriteClient<T>) clusteringClient;
JavaRDD<WriteStatus> res = writeClient.cluster(instant.getTimestamp(), true).getWriteStatuses();
if (res != null && res.collect().stream().anyMatch(WriteStatus::hasErrors)) {
// TODO: Should we treat this fatal and throw exception?
LOG.error("Clustering for instant (" + instant + ") failed with write errors");
}
}
}

View File

@@ -58,13 +58,20 @@ public class SparkClusteringPlanActionExecutor<T extends HoodieRecordPayload> ex
int commitsSinceLastClustering = table.getActiveTimeline().getCommitsTimeline().filterCompletedInstants()
.findInstantsAfter(lastClusteringInstant.map(HoodieInstant::getTimestamp).orElse("0"), Integer.MAX_VALUE)
.countInstants();
if (config.getInlineClusterMaxCommits() > commitsSinceLastClustering) {
LOG.info("Not scheduling clustering as only " + commitsSinceLastClustering
if (config.inlineClusteringEnabled() && config.getInlineClusterMaxCommits() > commitsSinceLastClustering) {
LOG.info("Not scheduling inline clustering as only " + commitsSinceLastClustering
+ " commits was found since last clustering " + lastClusteringInstant + ". Waiting for "
+ config.getInlineClusterMaxCommits());
return Option.empty();
}
if (config.isAsyncClusteringEnabled() && config.getAsyncClusterMaxCommits() > commitsSinceLastClustering) {
LOG.info("Not scheduling async clustering as only " + commitsSinceLastClustering
+ " commits was found since last clustering " + lastClusteringInstant + ". Waiting for "
+ config.getAsyncClusterMaxCommits());
return Option.empty();
}
LOG.info("Generating clustering plan for table " + config.getBasePath());
ClusteringPlanStrategy strategy = (ClusteringPlanStrategy)
ReflectionUtils.loadClass(config.getClusteringPlanStrategyClass(), table, context, config);