Skip to content
Merged
Show file tree
Hide file tree
Changes from 4 commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -711,12 +711,30 @@ public void rollbackInflightCompaction(HoodieInstant inflightInstant, HoodieTabl
table.getActiveTimeline().revertCompactionInflightToRequested(inflightInstant);
}

/**
* Get inflight time line exclude compaction and clustering.
* @param table
* @return
*/
private HoodieTimeline getInflightTimelineExcludeCompactionAndClustering(HoodieTable<T, I, K, O> table) {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@lw309637554 Can you add this API to the HoodieTimeline ? You can name it filterPendingExcludingCompactionAndReplace for now.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@n3nash Hello, about this API in timeline or as a util have discussed with @satishkotha . Finally decided to put it here. Because here just need to exclude clustering of replace, not insert_overwrite of replace. Also check clustering need use metaclient, but timeline do not have metaclient object.

HoodieTimeline inflightTimelineWithReplaceCommit = table.getMetaClient().getCommitsTimeline().filterPendingExcludingCompaction();
HoodieTimeline inflightTimelineExcludeClusteringCommit = inflightTimelineWithReplaceCommit.filter(instant -> {
if (instant.getAction().equals(HoodieTimeline.REPLACE_COMMIT_ACTION)) {
Option<Pair<HoodieInstant, HoodieClusteringPlan>> instantPlan = ClusteringUtils.getClusteringPlan(table.getMetaClient(), instant);
return !instantPlan.isPresent();
} else {
return true;
}
});
return inflightTimelineExcludeClusteringCommit;
}

/**
* Cleanup all pending commits.
*/
private void rollbackPendingCommits() {
HoodieTable<T, I, K, O> table = createTable(config, hadoopConf);
HoodieTimeline inflightTimeline = table.getMetaClient().getCommitsTimeline().filterPendingExcludingCompaction();
HoodieTimeline inflightTimeline = getInflightTimelineExcludeCompactionAndClustering(table);
List<String> commits = inflightTimeline.getReverseOrderedInstants().map(HoodieInstant::getTimestamp)
.collect(Collectors.toList());
for (String commit : commits) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -152,7 +152,18 @@ private Schema getTableAvroSchemaFromDataFile() throws Exception {
* @throws Exception
*/
public Schema getTableAvroSchema() throws Exception {
Option<Schema> schemaFromCommitMetadata = getTableSchemaFromCommitMetadata(true);
return getTableAvroSchema(true);
}

/**
* Gets schema for a hoodie table in Avro format, can choice if include metadata fields.
*
* @param includeMetadataFields choice if include metadata fields
* @return Avro schema for this table
* @throws Exception
*/
public Schema getTableAvroSchema(boolean includeMetadataFields) throws Exception {
Option<Schema> schemaFromCommitMetadata = getTableSchemaFromCommitMetadata(includeMetadataFields);
return schemaFromCommitMetadata.isPresent() ? schemaFromCommitMetadata.get() : getTableAvroSchemaFromDataFile();
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -109,7 +109,7 @@ public interface HoodieTimeline extends Serializable {
/**
* Filter this timeline to just include the in-flights excluding compaction instants.
*
* @return New instance of HoodieTimeline with just in-flights excluding compaction inflights
* @return New instance of HoodieTimeline with just in-flights excluding compaction instants
*/
HoodieTimeline filterPendingExcludingCompaction();

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -204,9 +204,10 @@ class TestStructuredStreaming extends HoodieClientTestBase {
// check have schedule clustering and clustering file group to one
waitTillHasCompletedReplaceInstant(destPath, 120, 5)
metaClient.reloadActiveTimeline()
assertEquals(1, getLatestFileGroupsFileId.size)
assertEquals(1, getLatestFileGroupsFileId(HoodieTestDataGenerator.DEFAULT_FIRST_PARTITION_PATH).size)
}
structuredStreamingForTestClusteringRunner(sourcePath, destPath, true, checkClusteringResult)
structuredStreamingForTestClusteringRunner(sourcePath, destPath, true,
HoodieTestDataGenerator.DEFAULT_FIRST_PARTITION_PATH, checkClusteringResult)
}

@Test
Expand All @@ -219,21 +220,21 @@ class TestStructuredStreaming extends HoodieClientTestBase {
override def execute(): Unit = {
waitTillHasCompletedReplaceInstant(destPath, 120, 5)
}
}
, "Should have replace commit completed")
}, msg)
println(msg)
}
structuredStreamingForTestClusteringRunner(sourcePath, destPath, false, checkClusteringResult)
structuredStreamingForTestClusteringRunner(sourcePath, destPath, false,
HoodieTestDataGenerator.DEFAULT_FIRST_PARTITION_PATH, checkClusteringResult)
}

def structuredStreamingForTestClusteringRunner(sourcePath: String, destPath: String,
isInlineClustering: Boolean, checkClusteringResult: String => Unit): Unit = {
isInlineClustering: Boolean, partitionOfRecords: String, checkClusteringResult: String => Unit): Unit = {
// First insert of data
val records1 = recordsToStrings(dataGen.generateInsertsForPartition("000", 100, HoodieTestDataGenerator.DEFAULT_FIRST_PARTITION_PATH)).toList
val records1 = recordsToStrings(dataGen.generateInsertsForPartition("000", 100, partitionOfRecords)).toList
val inputDF1 = spark.read.json(spark.sparkContext.parallelize(records1, 2))

// Second insert of data
val records2 = recordsToStrings(dataGen.generateInsertsForPartition("001", 100, HoodieTestDataGenerator.DEFAULT_FIRST_PARTITION_PATH)).toList
val records2 = recordsToStrings(dataGen.generateInsertsForPartition("001", 100, partitionOfRecords)).toList
val inputDF2 = spark.read.json(spark.sparkContext.parallelize(records2, 2))

val hudiOptions = getInlineClusteringOpts(isInlineClustering.toString, "2", 100)
Expand All @@ -252,7 +253,7 @@ class TestStructuredStreaming extends HoodieClientTestBase {

// check have more than one file group
this.metaClient = new HoodieTableMetaClient(fs.getConf, destPath, true)
assertTrue(getLatestFileGroupsFileId().size > 1)
assertTrue(getLatestFileGroupsFileId(partitionOfRecords).size > 1)

// check clustering result
checkClusteringResult(destPath)
Expand All @@ -265,10 +266,10 @@ class TestStructuredStreaming extends HoodieClientTestBase {
Await.result(Future.sequence(Seq(f1, f2)), Duration.Inf)
}

private def getLatestFileGroupsFileId():Array[String] = {
private def getLatestFileGroupsFileId(partition: String):Array[String] = {
getHoodieTableFileSystemView(metaClient, metaClient.getActiveTimeline,
HoodieTestTable.of(metaClient).listAllBaseFiles())
tableView.getLatestFileSlices(HoodieTestDataGenerator.DEFAULT_FIRST_PARTITION_PATH)
tableView.getLatestFileSlices(partition)
.toArray().map(slice => slice.asInstanceOf[FileSlice].getFileGroupId.getFileId)
}

Expand All @@ -283,7 +284,7 @@ class TestStructuredStreaming extends HoodieClientTestBase {
this.metaClient.reloadActiveTimeline()
val completeReplaceSize = this.metaClient.getActiveTimeline.getCompletedReplaceTimeline().getInstants.toArray.size
println("completeReplaceSize:" + completeReplaceSize)
if(completeReplaceSize > 0) {
if (completeReplaceSize > 0) {
success = true
}
} catch {
Expand All @@ -293,7 +294,7 @@ class TestStructuredStreaming extends HoodieClientTestBase {
Thread.sleep(sleepSecsAfterEachRun * 1000)
currTime = System.currentTimeMillis
}
if (!success) throw new IllegalStateException("Timed-out waiting for " + " have completed replace instant appear in " + tablePath)
if (!success) throw new IllegalStateException("Timed-out waiting for completing replace instant appear in " + tablePath)
}

}
Original file line number Diff line number Diff line change
@@ -0,0 +1,159 @@
/*
* 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.utilities;

import com.beust.jcommander.JCommander;
import com.beust.jcommander.Parameter;
import org.apache.avro.Schema;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hudi.client.SparkRDDWriteClient;
import org.apache.hudi.client.WriteStatus;
import org.apache.hudi.common.config.TypedProperties;
import org.apache.hudi.common.fs.FSUtils;
import org.apache.hudi.common.table.HoodieTableMetaClient;
import org.apache.hudi.common.table.TableSchemaResolver;
import org.apache.hudi.common.util.Option;
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.Serializable;
import java.util.ArrayList;
import java.util.List;

public class HoodieClusteringJob {

private static final Logger LOG = LogManager.getLogger(HoodieClusteringJob.class);
private final Config cfg;
private transient FileSystem fs;
private TypedProperties props;
private final JavaSparkContext jsc;

public HoodieClusteringJob(JavaSparkContext jsc, Config cfg) {
this.cfg = cfg;
this.jsc = jsc;
this.props = cfg.propsFilePath == null
? UtilHelpers.buildProperties(cfg.configs)
: readConfigFromFileSystem(jsc, cfg);
}

private TypedProperties readConfigFromFileSystem(JavaSparkContext jsc, Config cfg) {
final FileSystem fs = FSUtils.getFs(cfg.basePath, jsc.hadoopConfiguration());

return UtilHelpers
.readConfig(fs, new Path(cfg.propsFilePath), cfg.configs)
.getConfig();
}

public static class Config implements Serializable {
@Parameter(names = {"--base-path", "-sp"}, description = "Base path for the table", required = true)
public String basePath = null;
@Parameter(names = {"--table-name", "-tn"}, description = "Table name", required = true)
public String tableName = null;
@Parameter(names = {"--instant-time", "-it"}, description = "Clustering Instant time", required = true)
public String clusteringInstantTime = null;
@Parameter(names = {"--parallelism", "-pl"}, description = "Parallelism for hoodie insert", required = false)
public int parallelism = 1;
@Parameter(names = {"--spark-master", "-ms"}, description = "Spark master", required = false)
public String sparkMaster = null;
@Parameter(names = {"--spark-memory", "-sm"}, description = "spark memory to use", required = true)
public String sparkMemory = null;
@Parameter(names = {"--retry", "-rt"}, description = "number of retries", required = false)
public int retry = 0;

@Parameter(names = {"--schedule", "-sc"}, description = "Schedule clustering")
public Boolean runSchedule = false;
@Parameter(names = {"--help", "-h"}, help = true)
public Boolean help = false;

@Parameter(names = {"--props"}, description = "path to properties file on localfs or dfs, with configurations for "
+ "hoodie client for clustering")
public String propsFilePath = null;

@Parameter(names = {"--hoodie-conf"}, description = "Any configuration that can be set in the properties file "
+ "(using the CLI parameter \"--props\") can also be passed command line using this parameter. This can be repeated",
splitter = IdentitySplitter.class)
public List<String> configs = new ArrayList<>();
}

public static void main(String[] args) {
final Config cfg = new Config();
JCommander cmd = new JCommander(cfg, null, args);
if (cfg.help || args.length == 0) {
cmd.usage();
System.exit(1);
}
final JavaSparkContext jsc = UtilHelpers.buildSparkContext("clustering-" + cfg.tableName, cfg.sparkMaster, cfg.sparkMemory);
HoodieClusteringJob clusteringJob = new HoodieClusteringJob(jsc, cfg);
int result = clusteringJob.cluster(cfg.retry);
String resultMsg = String.format("Clustering with basePath: %s, tableName: %s, runSchedule: %s, clusteringInstantTime: %s",
cfg.basePath, cfg.tableName, cfg.runSchedule, cfg.clusteringInstantTime);
if (result == -1) {
LOG.error(resultMsg + " failed");
} else {
LOG.info(resultMsg + " success");
}
jsc.stop();
}

public int cluster(int retry) {
this.fs = FSUtils.getFs(cfg.basePath, jsc.hadoopConfiguration());
int ret = -1;
try {
do {
if (cfg.runSchedule) {
LOG.info("Do schedule");
ret = doSchedule(jsc);
} else {
LOG.info("Do cluster");
ret = doCluster(jsc);
}
} while (ret != 0 && retry-- > 0);
} catch (Throwable t) {
LOG.error("Cluster failed", t);
}
return ret;
}

private String getSchemaFromLatestInstant() throws Exception {
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), cfg.basePath, true);
TableSchemaResolver schemaUtil = new TableSchemaResolver(metaClient);
Schema schema = schemaUtil.getTableAvroSchema(false);
return schema.toString();
}

private int doCluster(JavaSparkContext jsc) throws Exception {
String schemaStr = getSchemaFromLatestInstant();
SparkRDDWriteClient client =
UtilHelpers.createHoodieClient(jsc, cfg.basePath, schemaStr, cfg.parallelism, Option.empty(), props);
JavaRDD<WriteStatus> writeResponse =
(JavaRDD<WriteStatus>) client.cluster(cfg.clusteringInstantTime, true).getWriteStatuses();
return UtilHelpers.handleErrors(jsc, cfg.clusteringInstantTime, writeResponse);
}

private int doSchedule(JavaSparkContext jsc) throws Exception {
String schemaStr = getSchemaFromLatestInstant();
SparkRDDWriteClient client =
UtilHelpers.createHoodieClient(jsc, cfg.basePath, schemaStr, cfg.parallelism, Option.empty(), props);
return client.scheduleClusteringAtInstant(cfg.clusteringInstantTime, Option.empty()) ? 0 : -1;
}

}
Original file line number Diff line number Diff line change
Expand Up @@ -296,7 +296,9 @@ public Pair<SchemaProvider, Pair<String, JavaRDD<HoodieRecord>>> readFromSource(
// Retrieve the previous round checkpoints, if any
Option<String> resumeCheckpointStr = Option.empty();
if (commitTimelineOpt.isPresent()) {
Option<HoodieInstant> lastCommit = commitTimelineOpt.get().lastInstant();
// TODO: now not support replace action HUDI-1500
Option<HoodieInstant> lastCommit = commitTimelineOpt.get()
.filter(instant -> !instant.getAction().equals(HoodieTimeline.REPLACE_COMMIT_ACTION)).lastInstant();
if (lastCommit.isPresent()) {
HoodieCommitMetadata commitMetadata = HoodieCommitMetadata
.fromBytes(commitTimelineOpt.get().getInstantDetails(lastCommit.get()).get(), HoodieCommitMetadata.class);
Expand Down
Loading