HUDI-123 Rename code packages/constants to org.apache.hudi (#830)
- Rename com.uber.hoodie to org.apache.hudi - Flag to pass com.uber.hoodie Input formats for hoodie-sync - Works with HUDI demo. - Also tested for backwards compatibility with datasets built by com.uber.hoodie packages - Migration guide : https://cwiki.apache.org/confluence/display/HUDI/Migration+Guide+From+com.uber.hoodie+to+org.apache.hudi
This commit is contained in:
committed by
vinoth chandar
parent
722b6be04a
commit
a4f9d7575f
236
hudi-client/pom.xml
Normal file
236
hudi-client/pom.xml
Normal file
@@ -0,0 +1,236 @@
|
||||
<?xml version="1.0" encoding="UTF-8"?>
|
||||
<!--
|
||||
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.
|
||||
-->
|
||||
<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
|
||||
<parent>
|
||||
<artifactId>hudi</artifactId>
|
||||
<groupId>org.apache.hudi</groupId>
|
||||
<version>0.5.0-SNAPSHOT</version>
|
||||
</parent>
|
||||
<modelVersion>4.0.0</modelVersion>
|
||||
|
||||
<artifactId>hudi-client</artifactId>
|
||||
<packaging>jar</packaging>
|
||||
|
||||
<properties>
|
||||
<notice.dir>${project.basedir}/src/main/resources/META-INF</notice.dir>
|
||||
</properties>
|
||||
|
||||
<build>
|
||||
<plugins>
|
||||
<plugin>
|
||||
<groupId>org.jacoco</groupId>
|
||||
<artifactId>jacoco-maven-plugin</artifactId>
|
||||
</plugin>
|
||||
<plugin>
|
||||
<groupId>org.apache.maven.plugins</groupId>
|
||||
<artifactId>maven-jar-plugin</artifactId>
|
||||
<executions>
|
||||
<execution>
|
||||
<goals>
|
||||
<goal>test-jar</goal>
|
||||
</goals>
|
||||
<phase>test-compile</phase>
|
||||
</execution>
|
||||
</executions>
|
||||
<configuration>
|
||||
<skip>false</skip>
|
||||
</configuration>
|
||||
</plugin>
|
||||
<plugin>
|
||||
<groupId>org.apache.rat</groupId>
|
||||
<artifactId>apache-rat-plugin</artifactId>
|
||||
</plugin>
|
||||
</plugins>
|
||||
|
||||
<resources>
|
||||
<resource>
|
||||
<directory>src/main/resources</directory>
|
||||
</resource>
|
||||
<resource>
|
||||
<directory>src/test/resources</directory>
|
||||
</resource>
|
||||
</resources>
|
||||
</build>
|
||||
|
||||
<dependencies>
|
||||
<dependency>
|
||||
<groupId>org.apache.hudi</groupId>
|
||||
<artifactId>hudi-common</artifactId>
|
||||
<version>${project.version}</version>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.apache.hudi</groupId>
|
||||
<artifactId>hudi-timeline-service</artifactId>
|
||||
<version>${project.version}</version>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.apache.hadoop</groupId>
|
||||
<artifactId>hadoop-hdfs</artifactId>
|
||||
<classifier>tests</classifier>
|
||||
<!-- Need these exclusions to make sure JavaSparkContext can be setup. https://issues.apache.org/jira/browse/SPARK-1693 -->
|
||||
<exclusions>
|
||||
<exclusion>
|
||||
<groupId>org.mortbay.jetty</groupId>
|
||||
<artifactId>*</artifactId>
|
||||
</exclusion>
|
||||
<exclusion>
|
||||
<groupId>javax.servlet.jsp</groupId>
|
||||
<artifactId>*</artifactId>
|
||||
</exclusion>
|
||||
<exclusion>
|
||||
<groupId>javax.servlet</groupId>
|
||||
<artifactId>*</artifactId>
|
||||
</exclusion>
|
||||
</exclusions>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.apache.hadoop</groupId>
|
||||
<artifactId>hadoop-common</artifactId>
|
||||
<classifier>tests</classifier>
|
||||
<exclusions>
|
||||
<exclusion>
|
||||
<groupId>org.mortbay.jetty</groupId>
|
||||
<artifactId>*</artifactId>
|
||||
</exclusion>
|
||||
<exclusion>
|
||||
<groupId>javax.servlet.jsp</groupId>
|
||||
<artifactId>*</artifactId>
|
||||
</exclusion>
|
||||
<exclusion>
|
||||
<groupId>javax.servlet</groupId>
|
||||
<artifactId>*</artifactId>
|
||||
</exclusion>
|
||||
</exclusions>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.apache.hudi</groupId>
|
||||
<artifactId>hudi-common</artifactId>
|
||||
<version>${project.version}</version>
|
||||
<classifier>tests</classifier>
|
||||
<type>test-jar</type>
|
||||
<scope>test</scope>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>io.dropwizard.metrics</groupId>
|
||||
<artifactId>metrics-graphite</artifactId>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>io.dropwizard.metrics</groupId>
|
||||
<artifactId>metrics-core</artifactId>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>com.beust</groupId>
|
||||
<artifactId>jcommander</artifactId>
|
||||
<version>1.48</version>
|
||||
</dependency>
|
||||
|
||||
<!-- Parent dependencies -->
|
||||
<dependency>
|
||||
<groupId>log4j</groupId>
|
||||
<artifactId>log4j</artifactId>
|
||||
</dependency>
|
||||
|
||||
<dependency>
|
||||
<groupId>org.apache.hadoop</groupId>
|
||||
<artifactId>hadoop-client</artifactId>
|
||||
<exclusions>
|
||||
<exclusion>
|
||||
<groupId>javax.servlet</groupId>
|
||||
<artifactId>*</artifactId>
|
||||
</exclusion>
|
||||
</exclusions>
|
||||
</dependency>
|
||||
|
||||
<dependency>
|
||||
<groupId>org.apache.parquet</groupId>
|
||||
<artifactId>parquet-avro</artifactId>
|
||||
</dependency>
|
||||
|
||||
<dependency>
|
||||
<groupId>org.apache.parquet</groupId>
|
||||
<artifactId>parquet-hadoop</artifactId>
|
||||
</dependency>
|
||||
|
||||
<dependency>
|
||||
<groupId>com.google.guava</groupId>
|
||||
<artifactId>guava</artifactId>
|
||||
</dependency>
|
||||
|
||||
<dependency>
|
||||
<groupId>org.apache.spark</groupId>
|
||||
<artifactId>spark-core_2.11</artifactId>
|
||||
</dependency>
|
||||
|
||||
<dependency>
|
||||
<groupId>org.apache.spark</groupId>
|
||||
<artifactId>spark-sql_2.11</artifactId>
|
||||
</dependency>
|
||||
|
||||
<dependency>
|
||||
<groupId>${hive.groupid}</groupId>
|
||||
<artifactId>hive-exec</artifactId>
|
||||
<version>${hive.version}</version>
|
||||
<scope>test</scope>
|
||||
</dependency>
|
||||
|
||||
<dependency>
|
||||
<groupId>org.mockito</groupId>
|
||||
<artifactId>mockito-all</artifactId>
|
||||
<version>1.10.19</version>
|
||||
<scope>test</scope>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.apache.hudi</groupId>
|
||||
<artifactId>hudi-hadoop-mr</artifactId>
|
||||
<version>${project.version}</version>
|
||||
<scope>test</scope>
|
||||
</dependency>
|
||||
<!-- Hbase dependencies -->
|
||||
<dependency>
|
||||
<groupId>org.apache.hbase</groupId>
|
||||
<artifactId>hbase-client</artifactId>
|
||||
</dependency>
|
||||
|
||||
<dependency>
|
||||
<groupId>org.htrace</groupId>
|
||||
<artifactId>htrace-core</artifactId>
|
||||
<version>3.0.4</version>
|
||||
</dependency>
|
||||
|
||||
<dependency>
|
||||
<groupId>org.apache.hbase</groupId>
|
||||
<artifactId>hbase-testing-util</artifactId>
|
||||
<version>1.2.3</version>
|
||||
<scope>test</scope>
|
||||
<exclusions>
|
||||
<exclusion>
|
||||
<groupId>org.codehaus.jackson</groupId>
|
||||
<artifactId>jackson-mapper-asl</artifactId>
|
||||
</exclusion>
|
||||
<exclusion>
|
||||
<groupId>org.codehaus.jackson</groupId>
|
||||
<artifactId>jackson-core-asl</artifactId>
|
||||
</exclusion>
|
||||
<exclusion>
|
||||
<groupId>javax.xml.bind</groupId>
|
||||
<artifactId>*</artifactId>
|
||||
</exclusion>
|
||||
</exclusions>
|
||||
</dependency>
|
||||
</dependencies>
|
||||
</project>
|
||||
@@ -0,0 +1,126 @@
|
||||
/*
|
||||
* 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;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.io.Serializable;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hudi.client.embedded.EmbeddedTimelineService;
|
||||
import org.apache.hudi.client.utils.ClientUtils;
|
||||
import org.apache.hudi.common.table.HoodieTableMetaClient;
|
||||
import org.apache.hudi.common.util.FSUtils;
|
||||
import org.apache.hudi.common.util.Option;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
import org.apache.log4j.LogManager;
|
||||
import org.apache.log4j.Logger;
|
||||
import org.apache.spark.api.java.JavaSparkContext;
|
||||
|
||||
/**
|
||||
* Abstract class taking care of holding common member variables (FileSystem, SparkContext, HoodieConfigs)
|
||||
* Also, manages embedded timeline-server if enabled.
|
||||
*/
|
||||
public abstract class AbstractHoodieClient implements Serializable {
|
||||
|
||||
private static final Logger logger = LogManager.getLogger(AbstractHoodieClient.class);
|
||||
|
||||
protected final transient FileSystem fs;
|
||||
protected final transient JavaSparkContext jsc;
|
||||
protected final HoodieWriteConfig config;
|
||||
protected final String basePath;
|
||||
|
||||
/**
|
||||
* Timeline Server has the same lifetime as that of Client.
|
||||
* Any operations done on the same timeline service will be able to take advantage
|
||||
* of the cached file-system view. New completed actions will be synced automatically
|
||||
* in an incremental fashion.
|
||||
*/
|
||||
private transient Option<EmbeddedTimelineService> timelineServer;
|
||||
private final boolean shouldStopTimelineServer;
|
||||
|
||||
protected AbstractHoodieClient(JavaSparkContext jsc, HoodieWriteConfig clientConfig) {
|
||||
this(jsc, clientConfig, Option.empty());
|
||||
}
|
||||
|
||||
protected AbstractHoodieClient(JavaSparkContext jsc, HoodieWriteConfig clientConfig,
|
||||
Option<EmbeddedTimelineService> timelineServer) {
|
||||
this.fs = FSUtils.getFs(clientConfig.getBasePath(), jsc.hadoopConfiguration());
|
||||
this.jsc = jsc;
|
||||
this.basePath = clientConfig.getBasePath();
|
||||
this.config = clientConfig;
|
||||
this.timelineServer = timelineServer;
|
||||
shouldStopTimelineServer = !timelineServer.isPresent();
|
||||
startEmbeddedServerView();
|
||||
}
|
||||
|
||||
/**
|
||||
* Releases any resources used by the client.
|
||||
*/
|
||||
public void close() {
|
||||
stopEmbeddedServerView(true);
|
||||
}
|
||||
|
||||
private synchronized void stopEmbeddedServerView(boolean resetViewStorageConfig) {
|
||||
if (timelineServer.isPresent() && shouldStopTimelineServer) {
|
||||
// Stop only if owner
|
||||
logger.info("Stopping Timeline service !!");
|
||||
timelineServer.get().stop();
|
||||
}
|
||||
|
||||
timelineServer = Option.empty();
|
||||
// Reset Storage Config to Client specified config
|
||||
if (resetViewStorageConfig) {
|
||||
config.resetViewStorageConfig();
|
||||
}
|
||||
}
|
||||
|
||||
private synchronized void startEmbeddedServerView() {
|
||||
if (config.isEmbeddedTimelineServerEnabled()) {
|
||||
if (!timelineServer.isPresent()) {
|
||||
// Run Embedded Timeline Server
|
||||
logger.info("Starting Timeline service !!");
|
||||
timelineServer = Option.of(new EmbeddedTimelineService(jsc.hadoopConfiguration(), jsc.getConf(),
|
||||
config.getClientSpecifiedViewStorageConfig()));
|
||||
try {
|
||||
timelineServer.get().startServer();
|
||||
// Allow executor to find this newly instantiated timeline service
|
||||
config.setViewStorageConfig(timelineServer.get().getRemoteFileSystemViewConfig());
|
||||
} catch (IOException e) {
|
||||
logger.warn("Unable to start timeline service. Proceeding as if embedded server is disabled", e);
|
||||
stopEmbeddedServerView(false);
|
||||
}
|
||||
} else {
|
||||
logger.info("Timeline Server already running. Not restarting the service");
|
||||
}
|
||||
} else {
|
||||
logger.info("Embedded Timeline Server is disabled. Not starting timeline service");
|
||||
}
|
||||
}
|
||||
|
||||
public HoodieWriteConfig getConfig() {
|
||||
return config;
|
||||
}
|
||||
|
||||
public Option<EmbeddedTimelineService> getTimelineServer() {
|
||||
return timelineServer;
|
||||
}
|
||||
|
||||
protected HoodieTableMetaClient createMetaClient(boolean loadActiveTimelineOnLoad) {
|
||||
return ClientUtils.createMetaClient(jsc, config, loadActiveTimelineOnLoad);
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,552 @@
|
||||
/*
|
||||
* 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;
|
||||
|
||||
import static org.apache.hudi.common.table.HoodieTimeline.COMPACTION_ACTION;
|
||||
|
||||
import com.google.common.base.Preconditions;
|
||||
import java.io.FileNotFoundException;
|
||||
import java.io.IOException;
|
||||
import java.io.Serializable;
|
||||
import java.util.ArrayList;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
import java.util.stream.Collectors;
|
||||
import org.apache.hadoop.fs.FileStatus;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hudi.avro.model.HoodieCompactionOperation;
|
||||
import org.apache.hudi.avro.model.HoodieCompactionPlan;
|
||||
import org.apache.hudi.client.embedded.EmbeddedTimelineService;
|
||||
import org.apache.hudi.common.model.CompactionOperation;
|
||||
import org.apache.hudi.common.model.FileSlice;
|
||||
import org.apache.hudi.common.model.HoodieDataFile;
|
||||
import org.apache.hudi.common.model.HoodieFileGroupId;
|
||||
import org.apache.hudi.common.model.HoodieLogFile;
|
||||
import org.apache.hudi.common.table.HoodieTableMetaClient;
|
||||
import org.apache.hudi.common.table.HoodieTimeline;
|
||||
import org.apache.hudi.common.table.log.HoodieLogFormat;
|
||||
import org.apache.hudi.common.table.timeline.HoodieInstant;
|
||||
import org.apache.hudi.common.table.timeline.HoodieInstant.State;
|
||||
import org.apache.hudi.common.table.view.HoodieTableFileSystemView;
|
||||
import org.apache.hudi.common.util.AvroUtils;
|
||||
import org.apache.hudi.common.util.CompactionUtils;
|
||||
import org.apache.hudi.common.util.FSUtils;
|
||||
import org.apache.hudi.common.util.Option;
|
||||
import org.apache.hudi.common.util.collection.Pair;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
import org.apache.hudi.exception.HoodieException;
|
||||
import org.apache.hudi.exception.HoodieIOException;
|
||||
import org.apache.hudi.func.OperationResult;
|
||||
import org.apache.log4j.LogManager;
|
||||
import org.apache.log4j.Logger;
|
||||
import org.apache.spark.api.java.JavaSparkContext;
|
||||
|
||||
/**
|
||||
* Client to perform admin operations related to compaction
|
||||
*/
|
||||
public class CompactionAdminClient extends AbstractHoodieClient {
|
||||
|
||||
private static Logger log = LogManager.getLogger(CompactionAdminClient.class);
|
||||
|
||||
public CompactionAdminClient(JavaSparkContext jsc, String basePath) {
|
||||
super(jsc, HoodieWriteConfig.newBuilder().withPath(basePath).build());
|
||||
}
|
||||
|
||||
public CompactionAdminClient(JavaSparkContext jsc, String basePath,
|
||||
Option<EmbeddedTimelineService> timelineServer) {
|
||||
super(jsc, HoodieWriteConfig.newBuilder().withPath(basePath).build(), timelineServer);
|
||||
}
|
||||
|
||||
/**
|
||||
* Validate all compaction operations in a compaction plan. Verifies the file-slices are consistent with corresponding
|
||||
* compaction operations.
|
||||
*
|
||||
* @param metaClient Hoodie Table Meta Client
|
||||
* @param compactionInstant Compaction Instant
|
||||
*/
|
||||
public List<ValidationOpResult> validateCompactionPlan(HoodieTableMetaClient metaClient,
|
||||
String compactionInstant, int parallelism) throws IOException {
|
||||
HoodieCompactionPlan plan = getCompactionPlan(metaClient, compactionInstant);
|
||||
HoodieTableFileSystemView fsView =
|
||||
new HoodieTableFileSystemView(metaClient, metaClient.getCommitsAndCompactionTimeline());
|
||||
|
||||
if (plan.getOperations() != null) {
|
||||
List<CompactionOperation> ops = plan.getOperations().stream()
|
||||
.map(CompactionOperation::convertFromAvroRecordInstance).collect(Collectors.toList());
|
||||
return jsc.parallelize(ops, parallelism).map(op -> {
|
||||
try {
|
||||
return validateCompactionOperation(metaClient, compactionInstant, op, Option.of(fsView));
|
||||
} catch (IOException e) {
|
||||
throw new HoodieIOException(e.getMessage(), e);
|
||||
}
|
||||
}).collect();
|
||||
}
|
||||
return new ArrayList<>();
|
||||
}
|
||||
|
||||
/**
|
||||
* Un-schedules compaction plan. Remove All compaction operation scheduled and re-arrange delta-files that were
|
||||
* created after the compaction was scheduled.
|
||||
*
|
||||
* This operation MUST be executed with compactions and writer turned OFF.
|
||||
*
|
||||
* @param compactionInstant Compaction Instant
|
||||
* @param skipValidation Skip validation step
|
||||
* @param parallelism Parallelism
|
||||
* @param dryRun Dry Run
|
||||
*/
|
||||
public List<RenameOpResult> unscheduleCompactionPlan(
|
||||
String compactionInstant, boolean skipValidation, int parallelism, boolean dryRun) throws Exception {
|
||||
HoodieTableMetaClient metaClient = createMetaClient(false);
|
||||
List<Pair<HoodieLogFile, HoodieLogFile>> renameActions =
|
||||
getRenamingActionsForUnschedulingCompactionPlan(metaClient, compactionInstant, parallelism,
|
||||
Option.empty(), skipValidation);
|
||||
|
||||
List<RenameOpResult> res =
|
||||
runRenamingOps(metaClient, renameActions, parallelism, dryRun);
|
||||
|
||||
Option<Boolean> success =
|
||||
Option.fromJavaOptional(res.stream().map(r -> (r.isExecuted() && r.isSuccess())).reduce(Boolean::logicalAnd));
|
||||
Option<Boolean> allSuccess = success.isPresent() ? Option.of(success.get()) : Option.empty();
|
||||
|
||||
// Only if all operations are successfully executed
|
||||
if (!dryRun && allSuccess.isPresent() && allSuccess.get()) {
|
||||
// Overwrite compaction request with empty compaction operations
|
||||
HoodieInstant inflight = new HoodieInstant(State.INFLIGHT, COMPACTION_ACTION, compactionInstant);
|
||||
Path inflightPath = new Path(metaClient.getMetaPath(), inflight.getFileName());
|
||||
if (metaClient.getFs().exists(inflightPath)) {
|
||||
// We need to rollback data-files because of this inflight compaction before unscheduling
|
||||
throw new IllegalStateException("Please rollback the inflight compaction before unscheduling");
|
||||
}
|
||||
// Leave the trace in aux folder but delete from metapath.
|
||||
// TODO: Add a rollback instant but for compaction
|
||||
HoodieInstant instant = new HoodieInstant(State.REQUESTED, COMPACTION_ACTION, compactionInstant);
|
||||
boolean deleted = metaClient.getFs().delete(new Path(metaClient.getMetaPath(), instant.getFileName()), false);
|
||||
Preconditions.checkArgument(deleted, "Unable to delete compaction instant.");
|
||||
}
|
||||
return res;
|
||||
}
|
||||
|
||||
/**
|
||||
* Remove a fileId from pending compaction. Removes the associated compaction operation and rename delta-files
|
||||
* that were generated for that file-id after the compaction operation was scheduled.
|
||||
*
|
||||
* This operation MUST be executed with compactions and writer turned OFF.
|
||||
*
|
||||
* @param fgId FileGroupId to be unscheduled
|
||||
* @param skipValidation Skip validation
|
||||
* @param dryRun Dry Run Mode
|
||||
*/
|
||||
public List<RenameOpResult> unscheduleCompactionFileId(HoodieFileGroupId fgId,
|
||||
boolean skipValidation, boolean dryRun) throws Exception {
|
||||
HoodieTableMetaClient metaClient = createMetaClient(false);
|
||||
List<Pair<HoodieLogFile, HoodieLogFile>> renameActions =
|
||||
getRenamingActionsForUnschedulingCompactionForFileId(metaClient, fgId,
|
||||
Option.empty(), skipValidation);
|
||||
|
||||
List<RenameOpResult> res = runRenamingOps(metaClient, renameActions, 1, dryRun);
|
||||
|
||||
if (!dryRun && !res.isEmpty() && res.get(0).isExecuted() && res.get(0).isSuccess()) {
|
||||
// Ready to remove this file-Id from compaction request
|
||||
Pair<String, HoodieCompactionOperation> compactionOperationWithInstant =
|
||||
CompactionUtils.getAllPendingCompactionOperations(metaClient).get(fgId);
|
||||
HoodieCompactionPlan plan = CompactionUtils
|
||||
.getCompactionPlan(metaClient, compactionOperationWithInstant.getKey());
|
||||
List<HoodieCompactionOperation> newOps = plan.getOperations().stream()
|
||||
.filter(op -> (!op.getFileId().equals(fgId.getFileId()))
|
||||
&& (!op.getPartitionPath().equals(fgId.getPartitionPath()))).collect(Collectors.toList());
|
||||
HoodieCompactionPlan newPlan =
|
||||
HoodieCompactionPlan.newBuilder().setOperations(newOps).setExtraMetadata(plan.getExtraMetadata()).build();
|
||||
HoodieInstant inflight = new HoodieInstant(State.INFLIGHT, COMPACTION_ACTION,
|
||||
compactionOperationWithInstant.getLeft());
|
||||
Path inflightPath = new Path(metaClient.getMetaPath(), inflight.getFileName());
|
||||
if (metaClient.getFs().exists(inflightPath)) {
|
||||
// revert if in inflight state
|
||||
metaClient.getActiveTimeline().revertCompactionInflightToRequested(inflight);
|
||||
}
|
||||
metaClient.getActiveTimeline().saveToCompactionRequested(
|
||||
new HoodieInstant(State.REQUESTED, COMPACTION_ACTION, compactionOperationWithInstant.getLeft()),
|
||||
AvroUtils.serializeCompactionPlan(newPlan));
|
||||
}
|
||||
return res;
|
||||
}
|
||||
|
||||
/**
|
||||
* Renames delta files to make file-slices consistent with the timeline as dictated by Hoodie metadata.
|
||||
* Use when compaction unschedule fails partially.
|
||||
*
|
||||
* This operation MUST be executed with compactions and writer turned OFF.
|
||||
* @param compactionInstant Compaction Instant to be repaired
|
||||
* @param dryRun Dry Run Mode
|
||||
*/
|
||||
public List<RenameOpResult> repairCompaction(String compactionInstant,
|
||||
int parallelism, boolean dryRun) throws Exception {
|
||||
HoodieTableMetaClient metaClient = createMetaClient(false);
|
||||
List<ValidationOpResult> validationResults =
|
||||
validateCompactionPlan(metaClient, compactionInstant, parallelism);
|
||||
List<ValidationOpResult> failed = validationResults.stream()
|
||||
.filter(v -> !v.isSuccess()).collect(Collectors.toList());
|
||||
if (failed.isEmpty()) {
|
||||
return new ArrayList<>();
|
||||
}
|
||||
|
||||
final HoodieTableFileSystemView fsView = new HoodieTableFileSystemView(metaClient,
|
||||
metaClient.getCommitsAndCompactionTimeline());
|
||||
List<Pair<HoodieLogFile, HoodieLogFile>> renameActions = failed.stream().flatMap(v ->
|
||||
getRenamingActionsToAlignWithCompactionOperation(metaClient, compactionInstant,
|
||||
v.getOperation(), Option.of(fsView)).stream()).collect(Collectors.toList());
|
||||
return runRenamingOps(metaClient, renameActions, parallelism, dryRun);
|
||||
}
|
||||
|
||||
/**
|
||||
* Construction Compaction Plan from compaction instant
|
||||
*/
|
||||
private static HoodieCompactionPlan getCompactionPlan(HoodieTableMetaClient metaClient,
|
||||
String compactionInstant) throws IOException {
|
||||
HoodieCompactionPlan compactionPlan = AvroUtils.deserializeCompactionPlan(
|
||||
metaClient.getActiveTimeline().getInstantAuxiliaryDetails(
|
||||
HoodieTimeline.getCompactionRequestedInstant(compactionInstant)).get());
|
||||
return compactionPlan;
|
||||
}
|
||||
|
||||
/**
|
||||
* Get Renaming actions to ensure the log-files of merged file-slices is aligned with compaction operation. This
|
||||
* method is used to recover from failures during unschedule compaction operations.
|
||||
*
|
||||
* @param metaClient Hoodie Table Meta Client
|
||||
* @param compactionInstant Compaction Instant
|
||||
* @param op Compaction Operation
|
||||
* @param fsViewOpt File System View
|
||||
*/
|
||||
protected static List<Pair<HoodieLogFile, HoodieLogFile>> getRenamingActionsToAlignWithCompactionOperation(
|
||||
HoodieTableMetaClient metaClient, String compactionInstant, CompactionOperation op,
|
||||
Option<HoodieTableFileSystemView> fsViewOpt) {
|
||||
HoodieTableFileSystemView fileSystemView = fsViewOpt.isPresent() ? fsViewOpt.get() :
|
||||
new HoodieTableFileSystemView(metaClient, metaClient.getCommitsAndCompactionTimeline());
|
||||
HoodieInstant lastInstant = metaClient.getCommitsAndCompactionTimeline().lastInstant().get();
|
||||
FileSlice merged =
|
||||
fileSystemView.getLatestMergedFileSlicesBeforeOrOn(op.getPartitionPath(), lastInstant.getTimestamp())
|
||||
.filter(fs -> fs.getFileId().equals(op.getFileId())).findFirst().get();
|
||||
final int maxVersion =
|
||||
op.getDeltaFilePaths().stream().map(lf -> FSUtils.getFileVersionFromLog(new Path(lf)))
|
||||
.reduce((x, y) -> x > y ? x : y).orElse(0);
|
||||
List<HoodieLogFile> logFilesToBeMoved =
|
||||
merged.getLogFiles().filter(lf -> lf.getLogVersion() > maxVersion).collect(Collectors.toList());
|
||||
return logFilesToBeMoved.stream().map(lf -> {
|
||||
Preconditions.checkArgument(lf.getLogVersion() - maxVersion > 0,
|
||||
"Expect new log version to be sane");
|
||||
HoodieLogFile newLogFile = new HoodieLogFile(new Path(lf.getPath().getParent(),
|
||||
FSUtils.makeLogFileName(lf.getFileId(), "." + FSUtils.getFileExtensionFromLog(lf.getPath()),
|
||||
compactionInstant, lf.getLogVersion() - maxVersion, HoodieLogFormat.UNKNOWN_WRITE_TOKEN)));
|
||||
return Pair.of(lf, newLogFile);
|
||||
}).collect(Collectors.toList());
|
||||
}
|
||||
|
||||
/**
|
||||
* Rename log files. This is done for un-scheduling a pending compaction operation NOTE: Can only be used safely when
|
||||
* no writer (ingestion/compaction) is running.
|
||||
*
|
||||
* @param metaClient Hoodie Table Meta-Client
|
||||
* @param oldLogFile Old Log File
|
||||
* @param newLogFile New Log File
|
||||
*/
|
||||
protected static void renameLogFile(HoodieTableMetaClient metaClient, HoodieLogFile oldLogFile,
|
||||
HoodieLogFile newLogFile) throws IOException {
|
||||
FileStatus[] statuses = metaClient.getFs().listStatus(oldLogFile.getPath());
|
||||
Preconditions.checkArgument(statuses.length == 1, "Only one status must be present");
|
||||
Preconditions.checkArgument(statuses[0].isFile(), "Source File must exist");
|
||||
Preconditions.checkArgument(oldLogFile.getPath().getParent().equals(newLogFile.getPath().getParent()),
|
||||
"Log file must only be moved within the parent directory");
|
||||
metaClient.getFs().rename(oldLogFile.getPath(), newLogFile.getPath());
|
||||
}
|
||||
|
||||
/**
|
||||
* Check if a compaction operation is valid
|
||||
*
|
||||
* @param metaClient Hoodie Table Meta client
|
||||
* @param compactionInstant Compaction Instant
|
||||
* @param operation Compaction Operation
|
||||
* @param fsViewOpt File System View
|
||||
*/
|
||||
private ValidationOpResult validateCompactionOperation(HoodieTableMetaClient metaClient,
|
||||
String compactionInstant, CompactionOperation operation, Option<HoodieTableFileSystemView> fsViewOpt)
|
||||
throws IOException {
|
||||
HoodieTableFileSystemView fileSystemView = fsViewOpt.isPresent() ? fsViewOpt.get() :
|
||||
new HoodieTableFileSystemView(metaClient, metaClient.getCommitsAndCompactionTimeline());
|
||||
Option<HoodieInstant> lastInstant = metaClient.getCommitsAndCompactionTimeline().lastInstant();
|
||||
try {
|
||||
if (lastInstant.isPresent()) {
|
||||
Option<FileSlice> fileSliceOptional =
|
||||
Option.fromJavaOptional(fileSystemView.getLatestUnCompactedFileSlices(operation.getPartitionPath())
|
||||
.filter(fs -> fs.getFileId().equals(operation.getFileId())).findFirst());
|
||||
if (fileSliceOptional.isPresent()) {
|
||||
FileSlice fs = fileSliceOptional.get();
|
||||
Option<HoodieDataFile> df = fs.getDataFile();
|
||||
if (operation.getDataFilePath().isPresent()) {
|
||||
String expPath = metaClient.getFs().getFileStatus(new Path(operation.getDataFilePath().get())).getPath()
|
||||
.toString();
|
||||
Preconditions.checkArgument(df.isPresent(), "Data File must be present. File Slice was : "
|
||||
+ fs + ", operation :" + operation);
|
||||
Preconditions.checkArgument(df.get().getPath().equals(expPath),
|
||||
"Base Path in operation is specified as " + expPath + " but got path " + df.get().getPath());
|
||||
}
|
||||
Set<HoodieLogFile> logFilesInFileSlice = fs.getLogFiles().collect(Collectors.toSet());
|
||||
Set<HoodieLogFile> logFilesInCompactionOp = operation.getDeltaFilePaths().stream()
|
||||
.map(dp -> {
|
||||
try {
|
||||
FileStatus[] fileStatuses = metaClient.getFs().listStatus(new Path(dp));
|
||||
Preconditions.checkArgument(fileStatuses.length == 1, "Expect only 1 file-status");
|
||||
return new HoodieLogFile(fileStatuses[0]);
|
||||
} catch (FileNotFoundException fe) {
|
||||
throw new CompactionValidationException(fe.getMessage());
|
||||
} catch (IOException ioe) {
|
||||
throw new HoodieIOException(ioe.getMessage(), ioe);
|
||||
}
|
||||
}).collect(Collectors.toSet());
|
||||
Set<HoodieLogFile> missing =
|
||||
logFilesInCompactionOp.stream().filter(lf -> !logFilesInFileSlice.contains(lf))
|
||||
.collect(Collectors.toSet());
|
||||
Preconditions.checkArgument(missing.isEmpty(),
|
||||
"All log files specified in compaction operation is not present. Missing :" + missing
|
||||
+ ", Exp :" + logFilesInCompactionOp + ", Got :" + logFilesInFileSlice);
|
||||
Set<HoodieLogFile> diff =
|
||||
logFilesInFileSlice.stream().filter(lf -> !logFilesInCompactionOp.contains(lf))
|
||||
.collect(Collectors.toSet());
|
||||
Preconditions.checkArgument(diff.stream().allMatch(lf -> lf.getBaseCommitTime().equals(compactionInstant)),
|
||||
"There are some log-files which are neither specified in compaction plan "
|
||||
+ "nor present after compaction request instant. Some of these :" + diff);
|
||||
} else {
|
||||
throw new CompactionValidationException("Unable to find file-slice for file-id (" + operation.getFileId()
|
||||
+ " Compaction operation is invalid.");
|
||||
}
|
||||
} else {
|
||||
throw new CompactionValidationException("Unable to find any committed instant. Compaction Operation may "
|
||||
+ "be pointing to stale file-slices");
|
||||
}
|
||||
} catch (CompactionValidationException | IllegalArgumentException e) {
|
||||
return new ValidationOpResult(operation, false, Option.of(e));
|
||||
}
|
||||
return new ValidationOpResult(operation, true, Option.empty());
|
||||
}
|
||||
|
||||
/**
|
||||
* Execute Renaming operation
|
||||
*
|
||||
* @param metaClient HoodieTable MetaClient
|
||||
* @param renameActions List of rename operations
|
||||
*/
|
||||
private List<RenameOpResult> runRenamingOps(HoodieTableMetaClient metaClient,
|
||||
List<Pair<HoodieLogFile, HoodieLogFile>> renameActions, int parallelism, boolean dryRun) {
|
||||
if (renameActions.isEmpty()) {
|
||||
log.info("No renaming of log-files needed. Proceeding to removing file-id from compaction-plan");
|
||||
return new ArrayList<>();
|
||||
} else {
|
||||
log.info("The following compaction renaming operations needs to be performed to un-schedule");
|
||||
if (!dryRun) {
|
||||
return jsc.parallelize(renameActions, parallelism).map(lfPair -> {
|
||||
try {
|
||||
log.info("RENAME " + lfPair.getLeft().getPath() + " => " + lfPair.getRight().getPath());
|
||||
renameLogFile(metaClient, lfPair.getLeft(), lfPair.getRight());
|
||||
return new RenameOpResult(lfPair, true, Option.empty());
|
||||
} catch (IOException e) {
|
||||
log.error("Error renaming log file", e);
|
||||
log.error("\n\n\n***NOTE Compaction is in inconsistent state. Try running \"compaction repair "
|
||||
+ lfPair.getLeft().getBaseCommitTime() + "\" to recover from failure ***\n\n\n");
|
||||
return new RenameOpResult(lfPair, false, Option.of(e));
|
||||
}
|
||||
}).collect();
|
||||
} else {
|
||||
log.info("Dry-Run Mode activated for rename operations");
|
||||
return renameActions.parallelStream()
|
||||
.map(lfPair -> new RenameOpResult(lfPair, false, false, Option.empty()))
|
||||
.collect(Collectors.toList());
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Generate renaming actions for unscheduling a pending compaction plan. NOTE: Can only be used safely when no writer
|
||||
* (ingestion/compaction) is running.
|
||||
*
|
||||
* @param metaClient Hoodie Table MetaClient
|
||||
* @param compactionInstant Compaction Instant to be unscheduled
|
||||
* @param fsViewOpt Cached File System View
|
||||
* @param skipValidation Skip Validation
|
||||
* @return list of pairs of log-files (old, new) and for each pair, rename must be done to successfully unschedule
|
||||
* compaction.
|
||||
*/
|
||||
protected List<Pair<HoodieLogFile, HoodieLogFile>> getRenamingActionsForUnschedulingCompactionPlan(
|
||||
HoodieTableMetaClient metaClient, String compactionInstant, int parallelism,
|
||||
Option<HoodieTableFileSystemView> fsViewOpt, boolean skipValidation) throws IOException {
|
||||
HoodieTableFileSystemView fsView = fsViewOpt.isPresent() ? fsViewOpt.get() :
|
||||
new HoodieTableFileSystemView(metaClient, metaClient.getCommitsAndCompactionTimeline());
|
||||
HoodieCompactionPlan plan = getCompactionPlan(metaClient, compactionInstant);
|
||||
if (plan.getOperations() != null) {
|
||||
log.info("Number of Compaction Operations :" + plan.getOperations().size()
|
||||
+ " for instant :" + compactionInstant);
|
||||
List<CompactionOperation> ops = plan.getOperations().stream()
|
||||
.map(CompactionOperation::convertFromAvroRecordInstance).collect(Collectors.toList());
|
||||
return jsc.parallelize(ops, parallelism).flatMap(op -> {
|
||||
try {
|
||||
return getRenamingActionsForUnschedulingCompactionOperation(metaClient, compactionInstant,
|
||||
op, Option.of(fsView), skipValidation).iterator();
|
||||
} catch (IOException ioe) {
|
||||
throw new HoodieIOException(ioe.getMessage(), ioe);
|
||||
} catch (CompactionValidationException ve) {
|
||||
throw new HoodieException(ve);
|
||||
}
|
||||
}).collect();
|
||||
}
|
||||
log.warn("No operations for compaction instant : " + compactionInstant);
|
||||
return new ArrayList<>();
|
||||
}
|
||||
|
||||
/**
|
||||
* Generate renaming actions for unscheduling a compaction operation NOTE: Can only be used safely when no writer
|
||||
* (ingestion/compaction) is running.
|
||||
*
|
||||
* @param metaClient Hoodie Table MetaClient
|
||||
* @param compactionInstant Compaction Instant
|
||||
* @param operation Compaction Operation
|
||||
* @param fsViewOpt Cached File System View
|
||||
* @param skipValidation Skip Validation
|
||||
* @return list of pairs of log-files (old, new) and for each pair, rename must be done to successfully unschedule
|
||||
* compaction.
|
||||
*/
|
||||
public List<Pair<HoodieLogFile, HoodieLogFile>> getRenamingActionsForUnschedulingCompactionOperation(
|
||||
HoodieTableMetaClient metaClient, String compactionInstant, CompactionOperation operation,
|
||||
Option<HoodieTableFileSystemView> fsViewOpt, boolean skipValidation) throws IOException {
|
||||
List<Pair<HoodieLogFile, HoodieLogFile>> result = new ArrayList<>();
|
||||
HoodieTableFileSystemView fileSystemView = fsViewOpt.isPresent() ? fsViewOpt.get() :
|
||||
new HoodieTableFileSystemView(metaClient, metaClient.getCommitsAndCompactionTimeline());
|
||||
if (!skipValidation) {
|
||||
validateCompactionOperation(metaClient, compactionInstant, operation, Option.of(fileSystemView));
|
||||
}
|
||||
HoodieInstant lastInstant = metaClient.getCommitsAndCompactionTimeline().lastInstant().get();
|
||||
FileSlice merged =
|
||||
fileSystemView.getLatestMergedFileSlicesBeforeOrOn(operation.getPartitionPath(), lastInstant.getTimestamp())
|
||||
.filter(fs -> fs.getFileId().equals(operation.getFileId())).findFirst().get();
|
||||
List<HoodieLogFile> logFilesToRepair =
|
||||
merged.getLogFiles().filter(lf -> lf.getBaseCommitTime().equals(compactionInstant))
|
||||
.sorted(HoodieLogFile.getLogFileComparator())
|
||||
.collect(Collectors.toList());
|
||||
FileSlice fileSliceForCompaction =
|
||||
fileSystemView.getLatestFileSlicesBeforeOrOn(operation.getPartitionPath(), operation.getBaseInstantTime(), true)
|
||||
.filter(fs -> fs.getFileId().equals(operation.getFileId())).findFirst().get();
|
||||
int maxUsedVersion =
|
||||
fileSliceForCompaction.getLogFiles().findFirst().map(HoodieLogFile::getLogVersion)
|
||||
.orElse(HoodieLogFile.LOGFILE_BASE_VERSION - 1);
|
||||
String logExtn = fileSliceForCompaction.getLogFiles().findFirst().map(lf -> "." + lf.getFileExtension())
|
||||
.orElse(HoodieLogFile.DELTA_EXTENSION);
|
||||
String parentPath = fileSliceForCompaction.getDataFile().map(df -> new Path(df.getPath()).getParent().toString())
|
||||
.orElse(fileSliceForCompaction.getLogFiles().findFirst().map(lf -> lf.getPath().getParent().toString()).get());
|
||||
for (HoodieLogFile toRepair : logFilesToRepair) {
|
||||
int version = maxUsedVersion + 1;
|
||||
HoodieLogFile newLf = new HoodieLogFile(new Path(parentPath, FSUtils.makeLogFileName(operation.getFileId(),
|
||||
logExtn, operation.getBaseInstantTime(), version, HoodieLogFormat.UNKNOWN_WRITE_TOKEN)));
|
||||
result.add(Pair.of(toRepair, newLf));
|
||||
maxUsedVersion = version;
|
||||
}
|
||||
return result;
|
||||
}
|
||||
|
||||
/**
|
||||
* Generate renaming actions for unscheduling a fileId from pending compaction. NOTE: Can only be used safely when no
|
||||
* writer (ingestion/compaction) is running.
|
||||
*
|
||||
* @param metaClient Hoodie Table MetaClient
|
||||
* @param fgId FileGroupId to remove compaction
|
||||
* @param fsViewOpt Cached File System View
|
||||
* @param skipValidation Skip Validation
|
||||
* @return list of pairs of log-files (old, new) and for each pair, rename must be done to successfully unschedule
|
||||
* compaction.
|
||||
*/
|
||||
public List<Pair<HoodieLogFile, HoodieLogFile>> getRenamingActionsForUnschedulingCompactionForFileId(
|
||||
HoodieTableMetaClient metaClient, HoodieFileGroupId fgId,
|
||||
Option<HoodieTableFileSystemView> fsViewOpt, boolean skipValidation) throws IOException {
|
||||
Map<HoodieFileGroupId, Pair<String, HoodieCompactionOperation>> allPendingCompactions =
|
||||
CompactionUtils.getAllPendingCompactionOperations(metaClient);
|
||||
if (allPendingCompactions.containsKey(fgId)) {
|
||||
Pair<String, HoodieCompactionOperation> opWithInstant = allPendingCompactions.get(fgId);
|
||||
return getRenamingActionsForUnschedulingCompactionOperation(metaClient, opWithInstant.getKey(),
|
||||
CompactionOperation.convertFromAvroRecordInstance(opWithInstant.getValue()), fsViewOpt, skipValidation);
|
||||
}
|
||||
throw new HoodieException("FileGroupId " + fgId + " not in pending compaction");
|
||||
}
|
||||
|
||||
/**
|
||||
* Holds Operation result for Renaming
|
||||
*/
|
||||
public static class RenameOpResult extends OperationResult<RenameInfo> {
|
||||
|
||||
public RenameOpResult() {
|
||||
}
|
||||
|
||||
public RenameOpResult(Pair<HoodieLogFile, HoodieLogFile> op, boolean success,
|
||||
Option<Exception> exception) {
|
||||
super(new RenameInfo(op.getKey().getFileId(), op.getKey().getPath().toString(),
|
||||
op.getRight().getPath().toString()), success, exception);
|
||||
}
|
||||
|
||||
public RenameOpResult(
|
||||
Pair<HoodieLogFile, HoodieLogFile> op, boolean executed, boolean success,
|
||||
Option<Exception> exception) {
|
||||
super(new RenameInfo(op.getKey().getFileId(), op.getKey().getPath().toString(),
|
||||
op.getRight().getPath().toString()), executed, success, exception);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Holds Operation result for Renaming
|
||||
*/
|
||||
public static class ValidationOpResult extends OperationResult<CompactionOperation> {
|
||||
|
||||
public ValidationOpResult() {
|
||||
}
|
||||
|
||||
public ValidationOpResult(
|
||||
CompactionOperation operation, boolean success, Option<Exception> exception) {
|
||||
super(operation, success, exception);
|
||||
}
|
||||
}
|
||||
|
||||
public static class RenameInfo implements Serializable {
|
||||
|
||||
public String fileId;
|
||||
public String srcPath;
|
||||
public String destPath;
|
||||
|
||||
public RenameInfo() {
|
||||
}
|
||||
|
||||
public RenameInfo(String fileId, String srcPath, String destPath) {
|
||||
this.fileId = fileId;
|
||||
this.srcPath = srcPath;
|
||||
this.destPath = destPath;
|
||||
}
|
||||
}
|
||||
|
||||
public static class CompactionValidationException extends RuntimeException {
|
||||
|
||||
public CompactionValidationException(String msg) {
|
||||
super(msg);
|
||||
}
|
||||
}
|
||||
}
|
||||
224
hudi-client/src/main/java/org/apache/hudi/HoodieReadClient.java
Normal file
224
hudi-client/src/main/java/org/apache/hudi/HoodieReadClient.java
Normal file
@@ -0,0 +1,224 @@
|
||||
/*
|
||||
* 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;
|
||||
|
||||
import java.util.HashSet;
|
||||
import java.util.List;
|
||||
import java.util.Set;
|
||||
import java.util.stream.Collectors;
|
||||
import org.apache.hudi.avro.model.HoodieCompactionPlan;
|
||||
import org.apache.hudi.client.embedded.EmbeddedTimelineService;
|
||||
import org.apache.hudi.common.model.HoodieDataFile;
|
||||
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.HoodieTableMetaClient;
|
||||
import org.apache.hudi.common.table.HoodieTimeline;
|
||||
import org.apache.hudi.common.util.CompactionUtils;
|
||||
import org.apache.hudi.common.util.Option;
|
||||
import org.apache.hudi.common.util.collection.Pair;
|
||||
import org.apache.hudi.config.HoodieIndexConfig;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
import org.apache.hudi.exception.HoodieIndexException;
|
||||
import org.apache.hudi.index.HoodieIndex;
|
||||
import org.apache.hudi.table.HoodieTable;
|
||||
import org.apache.log4j.LogManager;
|
||||
import org.apache.log4j.Logger;
|
||||
import org.apache.spark.SparkConf;
|
||||
import org.apache.spark.api.java.JavaPairRDD;
|
||||
import org.apache.spark.api.java.JavaRDD;
|
||||
import org.apache.spark.api.java.JavaSparkContext;
|
||||
import org.apache.spark.sql.Dataset;
|
||||
import org.apache.spark.sql.Row;
|
||||
import org.apache.spark.sql.SQLContext;
|
||||
import org.apache.spark.sql.types.StructType;
|
||||
import scala.Tuple2;
|
||||
|
||||
/**
|
||||
* Provides an RDD based API for accessing/filtering Hoodie tables, based on keys.
|
||||
*/
|
||||
public class HoodieReadClient<T extends HoodieRecordPayload> extends AbstractHoodieClient {
|
||||
|
||||
private static final Logger logger = LogManager.getLogger(HoodieReadClient.class);
|
||||
|
||||
/**
|
||||
* TODO: We need to persist the index type into hoodie.properties and be able to access the index
|
||||
* just with a simple basepath pointing to the dataset. Until, then just always assume a
|
||||
* BloomIndex
|
||||
*/
|
||||
private final transient HoodieIndex<T> index;
|
||||
private final HoodieTimeline commitTimeline;
|
||||
private HoodieTable hoodieTable;
|
||||
private transient Option<SQLContext> sqlContextOpt;
|
||||
|
||||
/**
|
||||
* @param basePath path to Hoodie dataset
|
||||
*/
|
||||
public HoodieReadClient(JavaSparkContext jsc, String basePath,
|
||||
Option<EmbeddedTimelineService> timelineService) {
|
||||
this(jsc, HoodieWriteConfig.newBuilder().withPath(basePath)
|
||||
// by default we use HoodieBloomIndex
|
||||
.withIndexConfig(
|
||||
HoodieIndexConfig.newBuilder().withIndexType(HoodieIndex.IndexType.BLOOM).build())
|
||||
.build(), timelineService);
|
||||
}
|
||||
|
||||
/**
|
||||
* @param basePath path to Hoodie dataset
|
||||
*/
|
||||
public HoodieReadClient(JavaSparkContext jsc, String basePath) {
|
||||
this(jsc, basePath, Option.empty());
|
||||
}
|
||||
|
||||
/**
|
||||
* @param jsc
|
||||
* @param basePath
|
||||
* @param sqlContext
|
||||
*/
|
||||
public HoodieReadClient(JavaSparkContext jsc, String basePath, SQLContext sqlContext) {
|
||||
this(jsc, basePath);
|
||||
this.sqlContextOpt = Option.of(sqlContext);
|
||||
}
|
||||
|
||||
/**
|
||||
* @param clientConfig instance of HoodieWriteConfig
|
||||
*/
|
||||
public HoodieReadClient(JavaSparkContext jsc, HoodieWriteConfig clientConfig) {
|
||||
this(jsc, clientConfig, Option.empty());
|
||||
}
|
||||
|
||||
/**
|
||||
* @param clientConfig instance of HoodieWriteConfig
|
||||
*/
|
||||
public HoodieReadClient(JavaSparkContext jsc, HoodieWriteConfig clientConfig,
|
||||
Option<EmbeddedTimelineService> timelineService) {
|
||||
super(jsc, clientConfig, timelineService);
|
||||
final String basePath = clientConfig.getBasePath();
|
||||
// Create a Hoodie table which encapsulated the commits and files visible
|
||||
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath, true);
|
||||
this.hoodieTable = HoodieTable.getHoodieTable(metaClient, clientConfig, jsc);
|
||||
this.commitTimeline = metaClient.getCommitTimeline().filterCompletedInstants();
|
||||
this.index = HoodieIndex.createIndex(clientConfig, jsc);
|
||||
this.sqlContextOpt = Option.empty();
|
||||
}
|
||||
|
||||
/**
|
||||
* Adds support for accessing Hoodie built tables from SparkSQL, as you normally would.
|
||||
*
|
||||
* @return SparkConf object to be used to construct the SparkContext by caller
|
||||
*/
|
||||
public static SparkConf addHoodieSupport(SparkConf conf) {
|
||||
conf.set("spark.sql.hive.convertMetastoreParquet", "false");
|
||||
return conf;
|
||||
}
|
||||
|
||||
private void assertSqlContext() {
|
||||
if (!sqlContextOpt.isPresent()) {
|
||||
throw new IllegalStateException(
|
||||
"SQLContext must be set, when performing dataframe operations");
|
||||
}
|
||||
}
|
||||
|
||||
private Option<String> convertToDataFilePath(Option<Pair<String, String>> partitionPathFileIDPair) {
|
||||
if (partitionPathFileIDPair.isPresent()) {
|
||||
HoodieDataFile dataFile = hoodieTable.getROFileSystemView()
|
||||
.getLatestDataFile(partitionPathFileIDPair.get().getLeft(), partitionPathFileIDPair.get().getRight()).get();
|
||||
return Option.of(dataFile.getPath());
|
||||
} else {
|
||||
return Option.empty();
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Given a bunch of hoodie keys, fetches all the individual records out as a data frame
|
||||
*
|
||||
* @return a dataframe
|
||||
*/
|
||||
public Dataset<Row> readROView(JavaRDD<HoodieKey> hoodieKeys, int parallelism) {
|
||||
assertSqlContext();
|
||||
JavaPairRDD<HoodieKey, Option<Pair<String, String>>> lookupResultRDD = index
|
||||
.fetchRecordLocation(hoodieKeys, jsc, hoodieTable);
|
||||
JavaPairRDD<HoodieKey, Option<String>> keyToFileRDD = lookupResultRDD
|
||||
.mapToPair(r -> new Tuple2<>(r._1, convertToDataFilePath(r._2)));
|
||||
List<String> paths = keyToFileRDD.filter(keyFileTuple -> keyFileTuple._2().isPresent())
|
||||
.map(keyFileTuple -> keyFileTuple._2().get()).collect();
|
||||
|
||||
// record locations might be same for multiple keys, so need a unique list
|
||||
Set<String> uniquePaths = new HashSet<>(paths);
|
||||
Dataset<Row> originalDF = sqlContextOpt.get().read()
|
||||
.parquet(uniquePaths.toArray(new String[uniquePaths.size()]));
|
||||
StructType schema = originalDF.schema();
|
||||
JavaPairRDD<HoodieKey, Row> keyRowRDD = originalDF.javaRDD().mapToPair(row -> {
|
||||
HoodieKey key = new HoodieKey(row.getAs(HoodieRecord.RECORD_KEY_METADATA_FIELD),
|
||||
row.getAs(HoodieRecord.PARTITION_PATH_METADATA_FIELD));
|
||||
return new Tuple2<>(key, row);
|
||||
});
|
||||
|
||||
// Now, we need to further filter out, for only rows that match the supplied hoodie keys
|
||||
JavaRDD<Row> rowRDD = keyRowRDD.join(keyToFileRDD, parallelism).map(tuple -> tuple._2()._1());
|
||||
return sqlContextOpt.get().createDataFrame(rowRDD, schema);
|
||||
}
|
||||
|
||||
/**
|
||||
* Checks if the given [Keys] exists in the hoodie table and returns [Key, Option[FullFilePath]]
|
||||
* If the optional FullFilePath value is not present, then the key is not found. If the
|
||||
* FullFilePath value is present, it is the path component (without scheme) of the URI underlying
|
||||
* file
|
||||
*/
|
||||
public JavaPairRDD<HoodieKey, Option<String>> checkExists(JavaRDD<HoodieKey> hoodieKeys) {
|
||||
return index.fetchRecordLocation(hoodieKeys, jsc, hoodieTable);
|
||||
}
|
||||
|
||||
/**
|
||||
* Filter out HoodieRecords that already exists in the output folder. This is useful in
|
||||
* deduplication.
|
||||
*
|
||||
* @param hoodieRecords Input RDD of Hoodie records.
|
||||
* @return A subset of hoodieRecords RDD, with existing records filtered out.
|
||||
*/
|
||||
public JavaRDD<HoodieRecord<T>> filterExists(JavaRDD<HoodieRecord<T>> hoodieRecords) {
|
||||
JavaRDD<HoodieRecord<T>> recordsWithLocation = tagLocation(hoodieRecords);
|
||||
return recordsWithLocation.filter(v1 -> !v1.isCurrentLocationKnown());
|
||||
}
|
||||
|
||||
/**
|
||||
* Looks up the index and tags each incoming record with a location of a file that contains the
|
||||
* row (if it is actually present). Input RDD should contain no duplicates if needed.
|
||||
*
|
||||
* @param hoodieRecords Input RDD of Hoodie records
|
||||
* @return Tagged RDD of Hoodie records
|
||||
*/
|
||||
public JavaRDD<HoodieRecord<T>> tagLocation(JavaRDD<HoodieRecord<T>> hoodieRecords)
|
||||
throws HoodieIndexException {
|
||||
return index.tagLocation(hoodieRecords, jsc, hoodieTable);
|
||||
}
|
||||
|
||||
/**
|
||||
* Return all pending compactions with instant time for clients to decide what to compact next.
|
||||
* @return
|
||||
*/
|
||||
public List<Pair<String, HoodieCompactionPlan>> getPendingCompactions() {
|
||||
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(),
|
||||
hoodieTable.getMetaClient().getBasePath(), true);
|
||||
return CompactionUtils.getAllPendingCompactionPlans(metaClient).stream()
|
||||
.map(instantWorkloadPair ->
|
||||
Pair.of(instantWorkloadPair.getKey().getTimestamp(), instantWorkloadPair.getValue()))
|
||||
.collect(Collectors.toList());
|
||||
}
|
||||
}
|
||||
1417
hudi-client/src/main/java/org/apache/hudi/HoodieWriteClient.java
Normal file
1417
hudi-client/src/main/java/org/apache/hudi/HoodieWriteClient.java
Normal file
File diff suppressed because it is too large
Load Diff
186
hudi-client/src/main/java/org/apache/hudi/WriteStatus.java
Normal file
186
hudi-client/src/main/java/org/apache/hudi/WriteStatus.java
Normal file
@@ -0,0 +1,186 @@
|
||||
/*
|
||||
* 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;
|
||||
|
||||
import java.io.Serializable;
|
||||
import java.util.ArrayList;
|
||||
import java.util.HashMap;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Random;
|
||||
import org.apache.hudi.common.model.HoodieKey;
|
||||
import org.apache.hudi.common.model.HoodieRecord;
|
||||
import org.apache.hudi.common.model.HoodieWriteStat;
|
||||
import org.apache.hudi.common.util.Option;
|
||||
|
||||
/**
|
||||
* Status of a write operation.
|
||||
*/
|
||||
public class WriteStatus implements Serializable {
|
||||
|
||||
private static final long RANDOM_SEED = 9038412832L;
|
||||
|
||||
private final HashMap<HoodieKey, Throwable> errors = new HashMap<>();
|
||||
|
||||
private final List<HoodieRecord> writtenRecords = new ArrayList<>();
|
||||
|
||||
private final List<HoodieRecord> failedRecords = new ArrayList<>();
|
||||
|
||||
private Throwable globalError = null;
|
||||
|
||||
private String fileId = null;
|
||||
|
||||
private String partitionPath = null;
|
||||
|
||||
private HoodieWriteStat stat = null;
|
||||
|
||||
private long totalRecords = 0;
|
||||
private long totalErrorRecords = 0;
|
||||
|
||||
private final double failureFraction;
|
||||
private final boolean trackSuccessRecords;
|
||||
private final transient Random random;
|
||||
|
||||
public WriteStatus(Boolean trackSuccessRecords, Double failureFraction) {
|
||||
this.trackSuccessRecords = trackSuccessRecords;
|
||||
this.failureFraction = failureFraction;
|
||||
this.random = new Random(RANDOM_SEED);
|
||||
}
|
||||
|
||||
/**
|
||||
* Mark write as success, optionally using given parameters for the purpose of calculating some
|
||||
* aggregate metrics. This method is not meant to cache passed arguments, since WriteStatus
|
||||
* objects are collected in Spark Driver.
|
||||
*
|
||||
* @param record deflated {@code HoodieRecord} containing information that uniquely identifies
|
||||
* it.
|
||||
* @param optionalRecordMetadata optional metadata related to data contained in {@link
|
||||
* HoodieRecord} before deflation.
|
||||
*/
|
||||
public void markSuccess(HoodieRecord record, Option<Map<String, String>> optionalRecordMetadata) {
|
||||
if (trackSuccessRecords) {
|
||||
writtenRecords.add(record);
|
||||
}
|
||||
totalRecords++;
|
||||
}
|
||||
|
||||
/**
|
||||
* Mark write as failed, optionally using given parameters for the purpose of calculating some
|
||||
* aggregate metrics. This method is not meant to cache passed arguments, since WriteStatus
|
||||
* objects are collected in Spark Driver.
|
||||
*
|
||||
* @param record deflated {@code HoodieRecord} containing information that uniquely identifies
|
||||
* it.
|
||||
* @param optionalRecordMetadata optional metadata related to data contained in {@link
|
||||
* HoodieRecord} before deflation.
|
||||
*/
|
||||
public void markFailure(HoodieRecord record, Throwable t, Option<Map<String, String>> optionalRecordMetadata) {
|
||||
if (failedRecords.isEmpty() || (random.nextDouble() <= failureFraction)) {
|
||||
// Guaranteed to have at-least one error
|
||||
failedRecords.add(record);
|
||||
errors.put(record.getKey(), t);
|
||||
}
|
||||
totalRecords++;
|
||||
totalErrorRecords++;
|
||||
}
|
||||
|
||||
public String getFileId() {
|
||||
return fileId;
|
||||
}
|
||||
|
||||
public void setFileId(String fileId) {
|
||||
this.fileId = fileId;
|
||||
}
|
||||
|
||||
public boolean hasErrors() {
|
||||
return totalErrorRecords > 0;
|
||||
}
|
||||
|
||||
public boolean isErrored(HoodieKey key) {
|
||||
return errors.containsKey(key);
|
||||
}
|
||||
|
||||
public HashMap<HoodieKey, Throwable> getErrors() {
|
||||
return errors;
|
||||
}
|
||||
|
||||
public boolean hasGlobalError() {
|
||||
return globalError != null;
|
||||
}
|
||||
|
||||
public Throwable getGlobalError() {
|
||||
return this.globalError;
|
||||
}
|
||||
|
||||
public void setGlobalError(Throwable t) {
|
||||
this.globalError = t;
|
||||
}
|
||||
|
||||
public List<HoodieRecord> getWrittenRecords() {
|
||||
return writtenRecords;
|
||||
}
|
||||
|
||||
public List<HoodieRecord> getFailedRecords() {
|
||||
return failedRecords;
|
||||
}
|
||||
|
||||
public HoodieWriteStat getStat() {
|
||||
return stat;
|
||||
}
|
||||
|
||||
public void setStat(HoodieWriteStat stat) {
|
||||
this.stat = stat;
|
||||
}
|
||||
|
||||
public String getPartitionPath() {
|
||||
return partitionPath;
|
||||
}
|
||||
|
||||
public void setPartitionPath(String partitionPath) {
|
||||
this.partitionPath = partitionPath;
|
||||
}
|
||||
|
||||
public long getTotalRecords() {
|
||||
return totalRecords;
|
||||
}
|
||||
|
||||
public void setTotalRecords(long totalRecords) {
|
||||
this.totalRecords = totalRecords;
|
||||
}
|
||||
|
||||
public long getTotalErrorRecords() {
|
||||
return totalErrorRecords;
|
||||
}
|
||||
|
||||
public void setTotalErrorRecords(long totalErrorRecords) {
|
||||
this.totalErrorRecords = totalErrorRecords;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
final StringBuilder sb = new StringBuilder("WriteStatus {");
|
||||
sb.append("fileId=").append(fileId);
|
||||
sb.append(", globalError='").append(globalError).append('\'');
|
||||
sb.append(", hasErrors='").append(hasErrors()).append('\'');
|
||||
sb.append(", errorCount='").append(totalErrorRecords).append('\'');
|
||||
sb.append(", errorPct='").append((100.0 * totalErrorRecords) / totalRecords).append('\'');
|
||||
sb.append('}');
|
||||
return sb.toString();
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,105 @@
|
||||
/*
|
||||
* 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.embedded;
|
||||
|
||||
import java.io.IOException;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hudi.common.SerializableConfiguration;
|
||||
import org.apache.hudi.common.table.view.FileSystemViewManager;
|
||||
import org.apache.hudi.common.table.view.FileSystemViewStorageConfig;
|
||||
import org.apache.hudi.common.table.view.FileSystemViewStorageType;
|
||||
import org.apache.hudi.common.util.NetworkUtils;
|
||||
import org.apache.hudi.timeline.service.TimelineService;
|
||||
import org.apache.log4j.LogManager;
|
||||
import org.apache.log4j.Logger;
|
||||
import org.apache.spark.SparkConf;
|
||||
|
||||
/**
|
||||
* Timeline Service that runs as part of write client
|
||||
*/
|
||||
public class EmbeddedTimelineService {
|
||||
|
||||
private static Logger logger = LogManager.getLogger(EmbeddedTimelineService.class);
|
||||
|
||||
private int serverPort;
|
||||
private String hostAddr;
|
||||
private final SerializableConfiguration hadoopConf;
|
||||
private final FileSystemViewStorageConfig config;
|
||||
private transient FileSystemViewManager viewManager;
|
||||
private transient TimelineService server;
|
||||
|
||||
public EmbeddedTimelineService(Configuration hadoopConf, SparkConf sparkConf, FileSystemViewStorageConfig config) {
|
||||
setHostAddrFromSparkConf(sparkConf);
|
||||
if (hostAddr == null) {
|
||||
this.hostAddr = NetworkUtils.getHostname();
|
||||
}
|
||||
this.config = config;
|
||||
this.hadoopConf = new SerializableConfiguration(hadoopConf);
|
||||
this.viewManager = createViewManager();
|
||||
}
|
||||
|
||||
private FileSystemViewManager createViewManager() {
|
||||
// Using passed-in configs to build view storage configs
|
||||
FileSystemViewStorageConfig.Builder builder =
|
||||
FileSystemViewStorageConfig.newBuilder().fromProperties(config.getProps());
|
||||
FileSystemViewStorageType storageType = builder.build().getStorageType();
|
||||
if (storageType.equals(FileSystemViewStorageType.REMOTE_ONLY)
|
||||
|| storageType.equals(FileSystemViewStorageType.REMOTE_FIRST)) {
|
||||
// Reset to default if set to Remote
|
||||
builder.withStorageType(FileSystemViewStorageType.MEMORY);
|
||||
}
|
||||
return FileSystemViewManager.createViewManager(hadoopConf, builder.build());
|
||||
}
|
||||
|
||||
public void startServer() throws IOException {
|
||||
server = new TimelineService(0, viewManager, hadoopConf.get());
|
||||
serverPort = server.startService();
|
||||
logger.info("Started embedded timeline server at " + hostAddr + ":" + serverPort);
|
||||
}
|
||||
|
||||
private void setHostAddrFromSparkConf(SparkConf sparkConf) {
|
||||
String hostAddr = sparkConf.get("spark.driver.host", null);
|
||||
if (hostAddr != null) {
|
||||
logger.info("Overriding hostIp to (" + hostAddr + ") found in spark-conf. It was " + this.hostAddr);
|
||||
this.hostAddr = hostAddr;
|
||||
} else {
|
||||
logger.warn("Unable to find driver bind address from spark config");
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Retrieves proper view storage configs for remote clients to access this service
|
||||
*/
|
||||
public FileSystemViewStorageConfig getRemoteFileSystemViewConfig() {
|
||||
return FileSystemViewStorageConfig.newBuilder().withStorageType(FileSystemViewStorageType.REMOTE_FIRST)
|
||||
.withRemoteServerHost(hostAddr).withRemoteServerPort(serverPort).build();
|
||||
}
|
||||
|
||||
public FileSystemViewManager getViewManager() {
|
||||
return viewManager;
|
||||
}
|
||||
|
||||
public void stop() {
|
||||
if (null != server) {
|
||||
this.server.close();
|
||||
this.server = null;
|
||||
this.viewManager = null;
|
||||
}
|
||||
}
|
||||
}
|
||||
@@ -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.client.utils;
|
||||
|
||||
import org.apache.hudi.common.table.HoodieTableMetaClient;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
import org.apache.spark.api.java.JavaSparkContext;
|
||||
|
||||
public class ClientUtils {
|
||||
|
||||
/**
|
||||
* Create Consistency Aware MetaClient
|
||||
*
|
||||
* @param jsc JavaSparkContext
|
||||
* @param config HoodieWriteConfig
|
||||
* @param loadActiveTimelineOnLoad early loading of timeline
|
||||
*/
|
||||
public static HoodieTableMetaClient createMetaClient(JavaSparkContext jsc, HoodieWriteConfig config,
|
||||
boolean loadActiveTimelineOnLoad) {
|
||||
return new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), loadActiveTimelineOnLoad,
|
||||
config.getConsistencyGuardConfig());
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,312 @@
|
||||
/*
|
||||
* 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.config;
|
||||
|
||||
import com.google.common.base.Preconditions;
|
||||
import java.io.File;
|
||||
import java.io.FileReader;
|
||||
import java.io.IOException;
|
||||
import java.util.Properties;
|
||||
import javax.annotation.concurrent.Immutable;
|
||||
import org.apache.hudi.common.model.HoodieAvroPayload;
|
||||
import org.apache.hudi.common.model.HoodieCleaningPolicy;
|
||||
import org.apache.hudi.io.compact.strategy.CompactionStrategy;
|
||||
import org.apache.hudi.io.compact.strategy.LogFileSizeBasedCompactionStrategy;
|
||||
|
||||
/**
|
||||
* Compaction related config
|
||||
*/
|
||||
@Immutable
|
||||
public class HoodieCompactionConfig extends DefaultHoodieConfig {
|
||||
|
||||
public static final String CLEANER_POLICY_PROP = "hoodie.cleaner.policy";
|
||||
public static final String AUTO_CLEAN_PROP = "hoodie.clean.automatic";
|
||||
// Turn on inline compaction - after fw delta commits a inline compaction will be run
|
||||
public static final String INLINE_COMPACT_PROP = "hoodie.compact.inline";
|
||||
// Run a compaction every N delta commits
|
||||
public static final String INLINE_COMPACT_NUM_DELTA_COMMITS_PROP =
|
||||
"hoodie.compact.inline.max" + ".delta.commits";
|
||||
public static final String CLEANER_FILE_VERSIONS_RETAINED_PROP =
|
||||
"hoodie.cleaner.fileversions" + ".retained";
|
||||
public static final String CLEANER_COMMITS_RETAINED_PROP = "hoodie.cleaner.commits.retained";
|
||||
public static final String MAX_COMMITS_TO_KEEP_PROP = "hoodie.keep.max.commits";
|
||||
public static final String MIN_COMMITS_TO_KEEP_PROP = "hoodie.keep.min.commits";
|
||||
public static final String COMMITS_ARCHIVAL_BATCH_SIZE_PROP = "hoodie.commits.archival.batch";
|
||||
// Upsert uses this file size to compact new data onto existing files..
|
||||
public static final String PARQUET_SMALL_FILE_LIMIT_BYTES = "hoodie.parquet.small.file.limit";
|
||||
// By default, treat any file <= 100MB as a small file.
|
||||
public static final String DEFAULT_PARQUET_SMALL_FILE_LIMIT_BYTES = String.valueOf(104857600);
|
||||
/**
|
||||
* Configs related to specific table types
|
||||
**/
|
||||
// Number of inserts, that will be put each partition/bucket for writing
|
||||
public static final String COPY_ON_WRITE_TABLE_INSERT_SPLIT_SIZE =
|
||||
"hoodie.copyonwrite.insert" + ".split.size";
|
||||
// The rationale to pick the insert parallelism is the following. Writing out 100MB files,
|
||||
// with atleast 1kb records, means 100K records per file. we just overprovision to 500K
|
||||
public static final String DEFAULT_COPY_ON_WRITE_TABLE_INSERT_SPLIT_SIZE = String.valueOf(500000);
|
||||
// Config to control whether we control insert split sizes automatically based on average
|
||||
// record sizes
|
||||
public static final String COPY_ON_WRITE_TABLE_AUTO_SPLIT_INSERTS =
|
||||
"hoodie.copyonwrite.insert" + ".auto.split";
|
||||
// its off by default
|
||||
public static final String DEFAULT_COPY_ON_WRITE_TABLE_AUTO_SPLIT_INSERTS = String.valueOf(true);
|
||||
// This value is used as a guessimate for the record size, if we can't determine this from
|
||||
// previous commits
|
||||
public static final String COPY_ON_WRITE_TABLE_RECORD_SIZE_ESTIMATE =
|
||||
"hoodie.copyonwrite" + ".record.size.estimate";
|
||||
// Used to determine how much more can be packed into a small file, before it exceeds the size
|
||||
// limit.
|
||||
public static final String DEFAULT_COPY_ON_WRITE_TABLE_RECORD_SIZE_ESTIMATE = String
|
||||
.valueOf(1024);
|
||||
public static final String CLEANER_PARALLELISM = "hoodie.cleaner.parallelism";
|
||||
public static final String DEFAULT_CLEANER_PARALLELISM = String.valueOf(200);
|
||||
public static final String TARGET_IO_PER_COMPACTION_IN_MB_PROP = "hoodie.compaction.target.io";
|
||||
// 500GB of target IO per compaction (both read and write)
|
||||
public static final String DEFAULT_TARGET_IO_PER_COMPACTION_IN_MB = String.valueOf(500 * 1024);
|
||||
public static final String COMPACTION_STRATEGY_PROP = "hoodie.compaction.strategy";
|
||||
// 200GB of target IO per compaction
|
||||
public static final String DEFAULT_COMPACTION_STRATEGY = LogFileSizeBasedCompactionStrategy.class
|
||||
.getName();
|
||||
// used to merge records written to log file
|
||||
public static final String DEFAULT_PAYLOAD_CLASS = HoodieAvroPayload.class.getName();
|
||||
public static final String PAYLOAD_CLASS_PROP = "hoodie.compaction.payload.class";
|
||||
|
||||
// used to choose a trade off between IO vs Memory when performing compaction process
|
||||
// Depending on outputfile_size and memory provided, choose true to avoid OOM for large file
|
||||
// size + small memory
|
||||
public static final String COMPACTION_LAZY_BLOCK_READ_ENABLED_PROP =
|
||||
"hoodie.compaction.lazy" + ".block.read";
|
||||
public static final String DEFAULT_COMPACTION_LAZY_BLOCK_READ_ENABLED = "false";
|
||||
// used to choose whether to enable reverse log reading (reverse log traversal)
|
||||
public static final String COMPACTION_REVERSE_LOG_READ_ENABLED_PROP =
|
||||
"hoodie.compaction" + ".reverse.log.read";
|
||||
public static final String DEFAULT_COMPACTION_REVERSE_LOG_READ_ENABLED = "false";
|
||||
private static final String DEFAULT_CLEANER_POLICY = HoodieCleaningPolicy.KEEP_LATEST_COMMITS
|
||||
.name();
|
||||
private static final String DEFAULT_AUTO_CLEAN = "true";
|
||||
private static final String DEFAULT_INLINE_COMPACT = "false";
|
||||
private static final String DEFAULT_INLINE_COMPACT_NUM_DELTA_COMMITS = "1";
|
||||
private static final String DEFAULT_CLEANER_FILE_VERSIONS_RETAINED = "3";
|
||||
private static final String DEFAULT_CLEANER_COMMITS_RETAINED = "10";
|
||||
private static final String DEFAULT_MAX_COMMITS_TO_KEEP = "30";
|
||||
private static final String DEFAULT_MIN_COMMITS_TO_KEEP = "20";
|
||||
private static final String DEFAULT_COMMITS_ARCHIVAL_BATCH_SIZE = String.valueOf(10);
|
||||
public static final String TARGET_PARTITIONS_PER_DAYBASED_COMPACTION_PROP = "hoodie.compaction.daybased.target"
|
||||
+ ".partitions";
|
||||
// 500GB of target IO per compaction (both read and write)
|
||||
public static final String DEFAULT_TARGET_PARTITIONS_PER_DAYBASED_COMPACTION = String.valueOf(10);
|
||||
|
||||
private HoodieCompactionConfig(Properties props) {
|
||||
super(props);
|
||||
}
|
||||
|
||||
public static HoodieCompactionConfig.Builder newBuilder() {
|
||||
return new Builder();
|
||||
}
|
||||
|
||||
public static class Builder {
|
||||
|
||||
private final Properties props = new Properties();
|
||||
|
||||
public Builder fromFile(File propertiesFile) throws IOException {
|
||||
FileReader reader = new FileReader(propertiesFile);
|
||||
try {
|
||||
this.props.load(reader);
|
||||
return this;
|
||||
} finally {
|
||||
reader.close();
|
||||
}
|
||||
}
|
||||
|
||||
public Builder fromProperties(Properties props) {
|
||||
this.props.putAll(props);
|
||||
return this;
|
||||
}
|
||||
|
||||
|
||||
public Builder withAutoClean(Boolean autoClean) {
|
||||
props.setProperty(AUTO_CLEAN_PROP, String.valueOf(autoClean));
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder withInlineCompaction(Boolean inlineCompaction) {
|
||||
props.setProperty(INLINE_COMPACT_PROP, String.valueOf(inlineCompaction));
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder inlineCompactionEvery(int deltaCommits) {
|
||||
props.setProperty(INLINE_COMPACT_PROP, String.valueOf(deltaCommits));
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder withCleanerPolicy(HoodieCleaningPolicy policy) {
|
||||
props.setProperty(CLEANER_POLICY_PROP, policy.name());
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder retainFileVersions(int fileVersionsRetained) {
|
||||
props.setProperty(CLEANER_FILE_VERSIONS_RETAINED_PROP, String.valueOf(fileVersionsRetained));
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder retainCommits(int commitsRetained) {
|
||||
props.setProperty(CLEANER_COMMITS_RETAINED_PROP, String.valueOf(commitsRetained));
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder archiveCommitsWith(int minToKeep, int maxToKeep) {
|
||||
props.setProperty(MIN_COMMITS_TO_KEEP_PROP, String.valueOf(minToKeep));
|
||||
props.setProperty(MAX_COMMITS_TO_KEEP_PROP, String.valueOf(maxToKeep));
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder compactionSmallFileSize(long smallFileLimitBytes) {
|
||||
props.setProperty(PARQUET_SMALL_FILE_LIMIT_BYTES, String.valueOf(smallFileLimitBytes));
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder insertSplitSize(int insertSplitSize) {
|
||||
props.setProperty(COPY_ON_WRITE_TABLE_INSERT_SPLIT_SIZE, String.valueOf(insertSplitSize));
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder autoTuneInsertSplits(boolean autoTuneInsertSplits) {
|
||||
props.setProperty(COPY_ON_WRITE_TABLE_AUTO_SPLIT_INSERTS,
|
||||
String.valueOf(autoTuneInsertSplits));
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder approxRecordSize(int recordSizeEstimate) {
|
||||
props.setProperty(COPY_ON_WRITE_TABLE_RECORD_SIZE_ESTIMATE,
|
||||
String.valueOf(recordSizeEstimate));
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder withCleanerParallelism(int cleanerParallelism) {
|
||||
props.setProperty(CLEANER_PARALLELISM, String.valueOf(cleanerParallelism));
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder withCompactionStrategy(CompactionStrategy compactionStrategy) {
|
||||
props.setProperty(COMPACTION_STRATEGY_PROP, compactionStrategy.getClass().getName());
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder withPayloadClass(String payloadClassName) {
|
||||
props.setProperty(PAYLOAD_CLASS_PROP, payloadClassName);
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder withTargetIOPerCompactionInMB(long targetIOPerCompactionInMB) {
|
||||
props.setProperty(TARGET_IO_PER_COMPACTION_IN_MB_PROP,
|
||||
String.valueOf(targetIOPerCompactionInMB));
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder withMaxNumDeltaCommitsBeforeCompaction(int maxNumDeltaCommitsBeforeCompaction) {
|
||||
props.setProperty(INLINE_COMPACT_NUM_DELTA_COMMITS_PROP,
|
||||
String.valueOf(maxNumDeltaCommitsBeforeCompaction));
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder withCompactionLazyBlockReadEnabled(Boolean compactionLazyBlockReadEnabled) {
|
||||
props.setProperty(COMPACTION_LAZY_BLOCK_READ_ENABLED_PROP,
|
||||
String.valueOf(compactionLazyBlockReadEnabled));
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder withCompactionReverseLogReadEnabled(Boolean compactionReverseLogReadEnabled) {
|
||||
props.setProperty(COMPACTION_REVERSE_LOG_READ_ENABLED_PROP,
|
||||
String.valueOf(compactionReverseLogReadEnabled));
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder withTargetPartitionsPerDayBasedCompaction(int targetPartitionsPerCompaction) {
|
||||
props.setProperty(TARGET_PARTITIONS_PER_DAYBASED_COMPACTION_PROP,
|
||||
String.valueOf(targetPartitionsPerCompaction));
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder withCommitsArchivalBatchSize(int batchSize) {
|
||||
props.setProperty(COMMITS_ARCHIVAL_BATCH_SIZE_PROP, String.valueOf(batchSize));
|
||||
return this;
|
||||
}
|
||||
|
||||
public HoodieCompactionConfig build() {
|
||||
HoodieCompactionConfig config = new HoodieCompactionConfig(props);
|
||||
setDefaultOnCondition(props, !props.containsKey(AUTO_CLEAN_PROP), AUTO_CLEAN_PROP,
|
||||
DEFAULT_AUTO_CLEAN);
|
||||
setDefaultOnCondition(props, !props.containsKey(INLINE_COMPACT_PROP), INLINE_COMPACT_PROP,
|
||||
DEFAULT_INLINE_COMPACT);
|
||||
setDefaultOnCondition(props, !props.containsKey(INLINE_COMPACT_NUM_DELTA_COMMITS_PROP),
|
||||
INLINE_COMPACT_NUM_DELTA_COMMITS_PROP, DEFAULT_INLINE_COMPACT_NUM_DELTA_COMMITS);
|
||||
setDefaultOnCondition(props, !props.containsKey(CLEANER_POLICY_PROP), CLEANER_POLICY_PROP,
|
||||
DEFAULT_CLEANER_POLICY);
|
||||
setDefaultOnCondition(props, !props.containsKey(CLEANER_FILE_VERSIONS_RETAINED_PROP),
|
||||
CLEANER_FILE_VERSIONS_RETAINED_PROP, DEFAULT_CLEANER_FILE_VERSIONS_RETAINED);
|
||||
setDefaultOnCondition(props, !props.containsKey(CLEANER_COMMITS_RETAINED_PROP),
|
||||
CLEANER_COMMITS_RETAINED_PROP, DEFAULT_CLEANER_COMMITS_RETAINED);
|
||||
setDefaultOnCondition(props, !props.containsKey(MAX_COMMITS_TO_KEEP_PROP), MAX_COMMITS_TO_KEEP_PROP,
|
||||
DEFAULT_MAX_COMMITS_TO_KEEP);
|
||||
setDefaultOnCondition(props, !props.containsKey(MIN_COMMITS_TO_KEEP_PROP), MIN_COMMITS_TO_KEEP_PROP,
|
||||
DEFAULT_MIN_COMMITS_TO_KEEP);
|
||||
setDefaultOnCondition(props, !props.containsKey(PARQUET_SMALL_FILE_LIMIT_BYTES),
|
||||
PARQUET_SMALL_FILE_LIMIT_BYTES, DEFAULT_PARQUET_SMALL_FILE_LIMIT_BYTES);
|
||||
setDefaultOnCondition(props, !props.containsKey(COPY_ON_WRITE_TABLE_INSERT_SPLIT_SIZE),
|
||||
COPY_ON_WRITE_TABLE_INSERT_SPLIT_SIZE, DEFAULT_COPY_ON_WRITE_TABLE_INSERT_SPLIT_SIZE);
|
||||
setDefaultOnCondition(props, !props.containsKey(COPY_ON_WRITE_TABLE_AUTO_SPLIT_INSERTS),
|
||||
COPY_ON_WRITE_TABLE_AUTO_SPLIT_INSERTS, DEFAULT_COPY_ON_WRITE_TABLE_AUTO_SPLIT_INSERTS);
|
||||
setDefaultOnCondition(props, !props.containsKey(COPY_ON_WRITE_TABLE_RECORD_SIZE_ESTIMATE),
|
||||
COPY_ON_WRITE_TABLE_RECORD_SIZE_ESTIMATE,
|
||||
DEFAULT_COPY_ON_WRITE_TABLE_RECORD_SIZE_ESTIMATE);
|
||||
setDefaultOnCondition(props, !props.containsKey(CLEANER_PARALLELISM), CLEANER_PARALLELISM,
|
||||
DEFAULT_CLEANER_PARALLELISM);
|
||||
setDefaultOnCondition(props, !props.containsKey(COMPACTION_STRATEGY_PROP),
|
||||
COMPACTION_STRATEGY_PROP, DEFAULT_COMPACTION_STRATEGY);
|
||||
setDefaultOnCondition(props, !props.containsKey(PAYLOAD_CLASS_PROP),
|
||||
PAYLOAD_CLASS_PROP, DEFAULT_PAYLOAD_CLASS);
|
||||
setDefaultOnCondition(props, !props.containsKey(TARGET_IO_PER_COMPACTION_IN_MB_PROP),
|
||||
TARGET_IO_PER_COMPACTION_IN_MB_PROP, DEFAULT_TARGET_IO_PER_COMPACTION_IN_MB);
|
||||
setDefaultOnCondition(props, !props.containsKey(COMPACTION_LAZY_BLOCK_READ_ENABLED_PROP),
|
||||
COMPACTION_LAZY_BLOCK_READ_ENABLED_PROP, DEFAULT_COMPACTION_LAZY_BLOCK_READ_ENABLED);
|
||||
setDefaultOnCondition(props, !props.containsKey(COMPACTION_REVERSE_LOG_READ_ENABLED_PROP),
|
||||
COMPACTION_REVERSE_LOG_READ_ENABLED_PROP, DEFAULT_COMPACTION_REVERSE_LOG_READ_ENABLED);
|
||||
setDefaultOnCondition(props, !props.containsKey(TARGET_PARTITIONS_PER_DAYBASED_COMPACTION_PROP),
|
||||
TARGET_PARTITIONS_PER_DAYBASED_COMPACTION_PROP, DEFAULT_TARGET_PARTITIONS_PER_DAYBASED_COMPACTION);
|
||||
setDefaultOnCondition(props, !props.containsKey(COMMITS_ARCHIVAL_BATCH_SIZE_PROP),
|
||||
COMMITS_ARCHIVAL_BATCH_SIZE_PROP, DEFAULT_COMMITS_ARCHIVAL_BATCH_SIZE);
|
||||
|
||||
HoodieCleaningPolicy.valueOf(props.getProperty(CLEANER_POLICY_PROP));
|
||||
|
||||
// Ensure minInstantsToKeep > cleanerCommitsRetained, otherwise we will archive some
|
||||
// commit instant on timeline, that still has not been cleaned. Could miss some data via incr pull
|
||||
int minInstantsToKeep = Integer.parseInt(props.getProperty(HoodieCompactionConfig.MIN_COMMITS_TO_KEEP_PROP));
|
||||
int maxInstantsToKeep = Integer.parseInt(props.getProperty(HoodieCompactionConfig.MAX_COMMITS_TO_KEEP_PROP));
|
||||
int cleanerCommitsRetained = Integer
|
||||
.parseInt(props.getProperty(HoodieCompactionConfig.CLEANER_COMMITS_RETAINED_PROP));
|
||||
Preconditions.checkArgument(maxInstantsToKeep > minInstantsToKeep);
|
||||
Preconditions.checkArgument(minInstantsToKeep > cleanerCommitsRetained,
|
||||
String.format("Increase %s=%d to be greater than %s=%d. Otherwise, there is risk of incremental pull "
|
||||
+ "missing data from few instants.", HoodieCompactionConfig.MIN_COMMITS_TO_KEEP_PROP,
|
||||
minInstantsToKeep, HoodieCompactionConfig.CLEANER_COMMITS_RETAINED_PROP, cleanerCommitsRetained));
|
||||
return config;
|
||||
}
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,269 @@
|
||||
/*
|
||||
* 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.config;
|
||||
|
||||
import java.io.File;
|
||||
import java.io.FileReader;
|
||||
import java.io.IOException;
|
||||
import java.util.Properties;
|
||||
import org.apache.hudi.index.hbase.DefaultHBaseQPSResourceAllocator;
|
||||
|
||||
public class HoodieHBaseIndexConfig extends DefaultHoodieConfig {
|
||||
|
||||
public static final String HBASE_ZKQUORUM_PROP = "hoodie.index.hbase.zkquorum";
|
||||
public static final String HBASE_ZKPORT_PROP = "hoodie.index.hbase.zkport";
|
||||
public static final String HBASE_TABLENAME_PROP = "hoodie.index.hbase.table";
|
||||
public static final String HBASE_GET_BATCH_SIZE_PROP = "hoodie.index.hbase.get.batch.size";
|
||||
public static final String HBASE_ZK_ZNODEPARENT = "hoodie.index.hbase.zknode.path";
|
||||
/**
|
||||
* Note that if HBASE_PUT_BATCH_SIZE_AUTO_COMPUTE_PROP is set to true, this batch size will not
|
||||
* be honored for HBase Puts
|
||||
*/
|
||||
public static final String HBASE_PUT_BATCH_SIZE_PROP = "hoodie.index.hbase.put.batch.size";
|
||||
|
||||
/**
|
||||
* Property to set which implementation of HBase QPS resource allocator to be used
|
||||
*/
|
||||
public static final String HBASE_INDEX_QPS_ALLOCATOR_CLASS = "hoodie.index.hbase.qps.allocator.class";
|
||||
public static final String DEFAULT_HBASE_INDEX_QPS_ALLOCATOR_CLASS = DefaultHBaseQPSResourceAllocator.class.getName();
|
||||
/**
|
||||
* Property to set to enable auto computation of put batch size
|
||||
*/
|
||||
public static final String HBASE_PUT_BATCH_SIZE_AUTO_COMPUTE_PROP = "hoodie.index.hbase.put.batch.size.autocompute";
|
||||
public static final String DEFAULT_HBASE_PUT_BATCH_SIZE_AUTO_COMPUTE = "false";
|
||||
/**
|
||||
* Property to set the fraction of the global share of QPS that should be allocated to this job.
|
||||
* Let's say there are 3 jobs which have input size in terms of number of rows required for
|
||||
* HbaseIndexing as x, 2x, 3x respectively. Then this fraction for the jobs would be (0.17) 1/6,
|
||||
* 0.33 (2/6) and 0.5 (3/6) respectively.
|
||||
*/
|
||||
public static final String HBASE_QPS_FRACTION_PROP = "hoodie.index.hbase.qps.fraction";
|
||||
/**
|
||||
* Property to set maximum QPS allowed per Region Server. This should be same across various
|
||||
* jobs. This is intended to limit the aggregate QPS generated across various jobs to an Hbase
|
||||
* Region Server. It is recommended to set this value based on global indexing throughput needs
|
||||
* and most importantly, how much the HBase installation in use is able to tolerate without
|
||||
* Region Servers going down.
|
||||
*/
|
||||
public static String HBASE_MAX_QPS_PER_REGION_SERVER_PROP = "hoodie.index.hbase.max.qps.per.region.server";
|
||||
/**
|
||||
* Default batch size, used only for Get, but computed for Put
|
||||
*/
|
||||
public static final int DEFAULT_HBASE_BATCH_SIZE = 100;
|
||||
/**
|
||||
* A low default value.
|
||||
*/
|
||||
public static final int DEFAULT_HBASE_MAX_QPS_PER_REGION_SERVER = 1000;
|
||||
/**
|
||||
* Default is 50%, which means a total of 2 jobs can run using HbaseIndex without overwhelming
|
||||
* Region Servers
|
||||
*/
|
||||
public static final float DEFAULT_HBASE_QPS_FRACTION = 0.5f;
|
||||
|
||||
/**
|
||||
* Property to decide if HBASE_QPS_FRACTION_PROP is dynamically calculated based on volume
|
||||
*/
|
||||
public static final String HOODIE_INDEX_COMPUTE_QPS_DYNAMICALLY = "hoodie.index.hbase.dynamic_qps";
|
||||
public static final boolean DEFAULT_HOODIE_INDEX_COMPUTE_QPS_DYNAMICALLY = false;
|
||||
/**
|
||||
* Min and Max for HBASE_QPS_FRACTION_PROP to stabilize skewed volume workloads
|
||||
*/
|
||||
public static final String HBASE_MIN_QPS_FRACTION_PROP = "hoodie.index.hbase.min.qps.fraction";
|
||||
public static final String DEFAULT_HBASE_MIN_QPS_FRACTION_PROP = "0.002";
|
||||
|
||||
public static final String HBASE_MAX_QPS_FRACTION_PROP = "hoodie.index.hbase.max.qps.fraction";
|
||||
public static final String DEFAULT_HBASE_MAX_QPS_FRACTION_PROP = "0.06";
|
||||
/**
|
||||
* Hoodie index desired puts operation time in seconds
|
||||
*/
|
||||
public static final String HOODIE_INDEX_DESIRED_PUTS_TIME_IN_SECS = "hoodie.index.hbase.desired_puts_time_in_secs";
|
||||
public static final int DEFAULT_HOODIE_INDEX_DESIRED_PUTS_TIME_IN_SECS = 600;
|
||||
public static final String HBASE_SLEEP_MS_PUT_BATCH_PROP = "hoodie.index.hbase.sleep.ms.for.put.batch";
|
||||
public static final String HBASE_SLEEP_MS_GET_BATCH_PROP = "hoodie.index.hbase.sleep.ms.for.get.batch";
|
||||
public static final String HOODIE_INDEX_HBASE_ZK_SESSION_TIMEOUT_MS = "hoodie.index.hbase.zk.session_timeout_ms";
|
||||
public static final int DEFAULT_ZK_SESSION_TIMEOUT_MS = 60 * 1000;
|
||||
public static final String HOODIE_INDEX_HBASE_ZK_CONNECTION_TIMEOUT_MS =
|
||||
"hoodie.index.hbase.zk.connection_timeout_ms";
|
||||
public static final int DEFAULT_ZK_CONNECTION_TIMEOUT_MS = 15 * 1000;
|
||||
public static final String HBASE_ZK_PATH_QPS_ROOT = "hoodie.index.hbase.zkpath.qps_root";
|
||||
public static final String DEFAULT_HBASE_ZK_PATH_QPS_ROOT = "/QPS_ROOT";
|
||||
|
||||
public HoodieHBaseIndexConfig(final Properties props) {
|
||||
super(props);
|
||||
}
|
||||
|
||||
public static HoodieHBaseIndexConfig.Builder newBuilder() {
|
||||
return new HoodieHBaseIndexConfig.Builder();
|
||||
}
|
||||
|
||||
public static class Builder {
|
||||
|
||||
private final Properties props = new Properties();
|
||||
|
||||
public HoodieHBaseIndexConfig.Builder fromFile(File propertiesFile) throws IOException {
|
||||
FileReader reader = new FileReader(propertiesFile);
|
||||
try {
|
||||
this.props.load(reader);
|
||||
return this;
|
||||
} finally {
|
||||
reader.close();
|
||||
}
|
||||
}
|
||||
|
||||
public HoodieHBaseIndexConfig.Builder fromProperties(Properties props) {
|
||||
this.props.putAll(props);
|
||||
return this;
|
||||
}
|
||||
|
||||
public HoodieHBaseIndexConfig.Builder hbaseZkQuorum(String zkString) {
|
||||
props.setProperty(HBASE_ZKQUORUM_PROP, zkString);
|
||||
return this;
|
||||
}
|
||||
|
||||
public HoodieHBaseIndexConfig.Builder hbaseZkPort(int port) {
|
||||
props.setProperty(HBASE_ZKPORT_PROP, String.valueOf(port));
|
||||
return this;
|
||||
}
|
||||
|
||||
public HoodieHBaseIndexConfig.Builder hbaseTableName(String tableName) {
|
||||
props.setProperty(HBASE_TABLENAME_PROP, tableName);
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder hbaseZkZnodeQPSPath(String zkZnodeQPSPath) {
|
||||
props.setProperty(HBASE_ZK_PATH_QPS_ROOT, zkZnodeQPSPath);
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder hbaseIndexGetBatchSize(int getBatchSize) {
|
||||
props.setProperty(HBASE_GET_BATCH_SIZE_PROP, String.valueOf(getBatchSize));
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder hbaseIndexPutBatchSize(int putBatchSize) {
|
||||
props.setProperty(HBASE_PUT_BATCH_SIZE_PROP, String.valueOf(putBatchSize));
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder hbaseIndexPutBatchSizeAutoCompute(boolean putBatchSizeAutoCompute) {
|
||||
props.setProperty(HBASE_PUT_BATCH_SIZE_AUTO_COMPUTE_PROP, String.valueOf(putBatchSizeAutoCompute));
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder hbaseIndexDesiredPutsTime(int desiredPutsTime) {
|
||||
props.setProperty(HOODIE_INDEX_DESIRED_PUTS_TIME_IN_SECS, String.valueOf(desiredPutsTime));
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder hbaseIndexShouldComputeQPSDynamically(boolean shouldComputeQPsDynamically) {
|
||||
props.setProperty(HOODIE_INDEX_COMPUTE_QPS_DYNAMICALLY, String.valueOf(shouldComputeQPsDynamically));
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder hbaseIndexQPSFraction(float qpsFraction) {
|
||||
props.setProperty(HBASE_QPS_FRACTION_PROP, String.valueOf(qpsFraction));
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder hbaseIndexMinQPSFraction(float minQPSFraction) {
|
||||
props.setProperty(HBASE_MIN_QPS_FRACTION_PROP, String.valueOf(minQPSFraction));
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder hbaseIndexMaxQPSFraction(float maxQPSFraction) {
|
||||
props.setProperty(HBASE_MAX_QPS_FRACTION_PROP, String.valueOf(maxQPSFraction));
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder hbaseIndexSleepMsBetweenPutBatch(int sleepMsBetweenPutBatch) {
|
||||
props.setProperty(HBASE_SLEEP_MS_PUT_BATCH_PROP, String.valueOf(sleepMsBetweenPutBatch));
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder withQPSResourceAllocatorType(String qpsResourceAllocatorClass) {
|
||||
props.setProperty(HBASE_INDEX_QPS_ALLOCATOR_CLASS, qpsResourceAllocatorClass);
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder hbaseIndexZkSessionTimeout(int zkSessionTimeout) {
|
||||
props.setProperty(HOODIE_INDEX_HBASE_ZK_SESSION_TIMEOUT_MS, String.valueOf(zkSessionTimeout));
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder hbaseIndexZkConnectionTimeout(int zkConnectionTimeout) {
|
||||
props.setProperty(HOODIE_INDEX_HBASE_ZK_CONNECTION_TIMEOUT_MS, String.valueOf(zkConnectionTimeout));
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder hbaseZkZnodeParent(String zkZnodeParent) {
|
||||
props.setProperty(HBASE_ZK_ZNODEPARENT, zkZnodeParent);
|
||||
return this;
|
||||
}
|
||||
|
||||
/**
|
||||
* <p>
|
||||
* Method to set maximum QPS allowed per Region Server. This should be same across various
|
||||
* jobs. This is intended to limit the aggregate QPS generated across various jobs to an
|
||||
* Hbase Region Server.
|
||||
* </p>
|
||||
* <p>
|
||||
* It is recommended to set this value based on your global indexing throughput needs and
|
||||
* most importantly, how much your HBase installation is able to tolerate without Region
|
||||
* Servers going down.
|
||||
* </p>
|
||||
*/
|
||||
public HoodieHBaseIndexConfig.Builder hbaseIndexMaxQPSPerRegionServer(
|
||||
int maxQPSPerRegionServer) {
|
||||
// This should be same across various jobs
|
||||
props.setProperty(HoodieHBaseIndexConfig.HBASE_MAX_QPS_PER_REGION_SERVER_PROP,
|
||||
String.valueOf(maxQPSPerRegionServer));
|
||||
return this;
|
||||
}
|
||||
|
||||
public HoodieHBaseIndexConfig build() {
|
||||
HoodieHBaseIndexConfig config = new HoodieHBaseIndexConfig(props);
|
||||
setDefaultOnCondition(props, !props.containsKey(HBASE_GET_BATCH_SIZE_PROP),
|
||||
HBASE_GET_BATCH_SIZE_PROP, String.valueOf(DEFAULT_HBASE_BATCH_SIZE));
|
||||
setDefaultOnCondition(props, !props.containsKey(HBASE_PUT_BATCH_SIZE_PROP),
|
||||
HBASE_PUT_BATCH_SIZE_PROP, String.valueOf(DEFAULT_HBASE_BATCH_SIZE));
|
||||
setDefaultOnCondition(props, !props.containsKey(HBASE_PUT_BATCH_SIZE_AUTO_COMPUTE_PROP),
|
||||
HBASE_PUT_BATCH_SIZE_AUTO_COMPUTE_PROP, String.valueOf(DEFAULT_HBASE_PUT_BATCH_SIZE_AUTO_COMPUTE));
|
||||
setDefaultOnCondition(props, !props.containsKey(HBASE_QPS_FRACTION_PROP),
|
||||
HBASE_QPS_FRACTION_PROP, String.valueOf(DEFAULT_HBASE_QPS_FRACTION));
|
||||
setDefaultOnCondition(props, !props.containsKey(HBASE_MAX_QPS_PER_REGION_SERVER_PROP),
|
||||
HBASE_MAX_QPS_PER_REGION_SERVER_PROP, String.valueOf(DEFAULT_HBASE_MAX_QPS_PER_REGION_SERVER));
|
||||
setDefaultOnCondition(props, !props.containsKey(HOODIE_INDEX_COMPUTE_QPS_DYNAMICALLY),
|
||||
HOODIE_INDEX_COMPUTE_QPS_DYNAMICALLY, String.valueOf(DEFAULT_HOODIE_INDEX_COMPUTE_QPS_DYNAMICALLY));
|
||||
setDefaultOnCondition(props, !props.containsKey(HBASE_INDEX_QPS_ALLOCATOR_CLASS),
|
||||
HBASE_INDEX_QPS_ALLOCATOR_CLASS, String.valueOf(DEFAULT_HBASE_INDEX_QPS_ALLOCATOR_CLASS));
|
||||
setDefaultOnCondition(props, !props.containsKey(HOODIE_INDEX_DESIRED_PUTS_TIME_IN_SECS),
|
||||
HOODIE_INDEX_DESIRED_PUTS_TIME_IN_SECS, String.valueOf(DEFAULT_HOODIE_INDEX_DESIRED_PUTS_TIME_IN_SECS));
|
||||
setDefaultOnCondition(props, !props.containsKey(HBASE_ZK_PATH_QPS_ROOT),
|
||||
HBASE_ZK_PATH_QPS_ROOT, String.valueOf(DEFAULT_HBASE_ZK_PATH_QPS_ROOT));
|
||||
setDefaultOnCondition(props, !props.containsKey(HOODIE_INDEX_HBASE_ZK_SESSION_TIMEOUT_MS),
|
||||
HOODIE_INDEX_HBASE_ZK_SESSION_TIMEOUT_MS, String.valueOf(DEFAULT_ZK_SESSION_TIMEOUT_MS));
|
||||
setDefaultOnCondition(props, !props.containsKey(HOODIE_INDEX_HBASE_ZK_CONNECTION_TIMEOUT_MS),
|
||||
HOODIE_INDEX_HBASE_ZK_CONNECTION_TIMEOUT_MS, String.valueOf(DEFAULT_ZK_CONNECTION_TIMEOUT_MS));
|
||||
setDefaultOnCondition(props, !props.containsKey(HBASE_INDEX_QPS_ALLOCATOR_CLASS),
|
||||
HBASE_INDEX_QPS_ALLOCATOR_CLASS, String.valueOf(DEFAULT_HBASE_INDEX_QPS_ALLOCATOR_CLASS));
|
||||
return config;
|
||||
}
|
||||
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,203 @@
|
||||
/*
|
||||
* 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.config;
|
||||
|
||||
import java.io.File;
|
||||
import java.io.FileReader;
|
||||
import java.io.IOException;
|
||||
import java.util.Properties;
|
||||
import javax.annotation.concurrent.Immutable;
|
||||
import org.apache.hudi.index.HoodieIndex;
|
||||
|
||||
/**
|
||||
* Indexing related config
|
||||
*/
|
||||
@Immutable
|
||||
public class HoodieIndexConfig extends DefaultHoodieConfig {
|
||||
|
||||
public static final String INDEX_TYPE_PROP = "hoodie.index.type";
|
||||
public static final String DEFAULT_INDEX_TYPE = HoodieIndex.IndexType.BLOOM.name();
|
||||
|
||||
// ***** Bloom Index configs *****
|
||||
public static final String BLOOM_FILTER_NUM_ENTRIES = "hoodie.index.bloom.num_entries";
|
||||
public static final String DEFAULT_BLOOM_FILTER_NUM_ENTRIES = "60000";
|
||||
public static final String BLOOM_FILTER_FPP = "hoodie.index.bloom.fpp";
|
||||
public static final String DEFAULT_BLOOM_FILTER_FPP = "0.000000001";
|
||||
public static final String BLOOM_INDEX_PARALLELISM_PROP = "hoodie.bloom.index.parallelism";
|
||||
// Disable explicit bloom index parallelism setting by default - hoodie auto computes
|
||||
public static final String DEFAULT_BLOOM_INDEX_PARALLELISM = "0";
|
||||
public static final String BLOOM_INDEX_PRUNE_BY_RANGES_PROP =
|
||||
"hoodie.bloom.index.prune.by" + ".ranges";
|
||||
public static final String DEFAULT_BLOOM_INDEX_PRUNE_BY_RANGES = "true";
|
||||
public static final String BLOOM_INDEX_USE_CACHING_PROP = "hoodie.bloom.index.use.caching";
|
||||
public static final String DEFAULT_BLOOM_INDEX_USE_CACHING = "true";
|
||||
public static final String BLOOM_INDEX_TREE_BASED_FILTER_PROP = "hoodie.bloom.index.use.treebased.filter";
|
||||
public static final String DEFAULT_BLOOM_INDEX_TREE_BASED_FILTER = "true";
|
||||
// TODO: On by default. Once stable, we will remove the other mode.
|
||||
public static final String BLOOM_INDEX_BUCKETIZED_CHECKING_PROP = "hoodie.bloom.index.bucketized.checking";
|
||||
public static final String DEFAULT_BLOOM_INDEX_BUCKETIZED_CHECKING = "true";
|
||||
// 1B bloom filter checks happen in 250 seconds. 500ms to read a bloom filter.
|
||||
// 10M checks in 2500ms, thus amortizing the cost of reading bloom filter across partitions.
|
||||
public static final String BLOOM_INDEX_KEYS_PER_BUCKET_PROP = "hoodie.bloom.index.keys.per.bucket";
|
||||
public static final String DEFAULT_BLOOM_INDEX_KEYS_PER_BUCKET = "10000000";
|
||||
|
||||
// ***** HBase Index Configs *****
|
||||
public static final String HBASE_ZKQUORUM_PROP = "hoodie.index.hbase.zkquorum";
|
||||
public static final String HBASE_ZKPORT_PROP = "hoodie.index.hbase.zkport";
|
||||
public static final String HBASE_ZK_ZNODEPARENT = "hoodie.index.hbase.zknode.path";
|
||||
public static final String HBASE_TABLENAME_PROP = "hoodie.index.hbase.table";
|
||||
public static final String HBASE_GET_BATCH_SIZE_PROP = "hoodie.index.hbase.get.batch.size";
|
||||
public static final String HBASE_PUT_BATCH_SIZE_PROP = "hoodie.index.hbase.put.batch.size";
|
||||
public static final String DEFAULT_HBASE_BATCH_SIZE = "100";
|
||||
|
||||
|
||||
public static final String BLOOM_INDEX_INPUT_STORAGE_LEVEL =
|
||||
"hoodie.bloom.index.input.storage" + ".level";
|
||||
public static final String DEFAULT_BLOOM_INDEX_INPUT_STORAGE_LEVEL = "MEMORY_AND_DISK_SER";
|
||||
|
||||
private HoodieIndexConfig(Properties props) {
|
||||
super(props);
|
||||
}
|
||||
|
||||
public static HoodieIndexConfig.Builder newBuilder() {
|
||||
return new Builder();
|
||||
}
|
||||
|
||||
public static class Builder {
|
||||
|
||||
private final Properties props = new Properties();
|
||||
|
||||
public Builder fromFile(File propertiesFile) throws IOException {
|
||||
FileReader reader = new FileReader(propertiesFile);
|
||||
try {
|
||||
this.props.load(reader);
|
||||
return this;
|
||||
} finally {
|
||||
reader.close();
|
||||
}
|
||||
}
|
||||
|
||||
public Builder fromProperties(Properties props) {
|
||||
this.props.putAll(props);
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder withIndexType(HoodieIndex.IndexType indexType) {
|
||||
props.setProperty(INDEX_TYPE_PROP, indexType.name());
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder withHBaseIndexConfig(HoodieHBaseIndexConfig hBaseIndexConfig) {
|
||||
props.putAll(hBaseIndexConfig.getProps());
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder bloomFilterNumEntries(int numEntries) {
|
||||
props.setProperty(BLOOM_FILTER_NUM_ENTRIES, String.valueOf(numEntries));
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder bloomFilterFPP(double fpp) {
|
||||
props.setProperty(BLOOM_FILTER_FPP, String.valueOf(fpp));
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder hbaseZkQuorum(String zkString) {
|
||||
props.setProperty(HBASE_ZKQUORUM_PROP, zkString);
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder hbaseZkPort(int port) {
|
||||
props.setProperty(HBASE_ZKPORT_PROP, String.valueOf(port));
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder hbaseZkZnodeParent(String zkZnodeParent) {
|
||||
props.setProperty(HBASE_ZK_ZNODEPARENT, zkZnodeParent);
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder hbaseTableName(String tableName) {
|
||||
props.setProperty(HBASE_TABLENAME_PROP, tableName);
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder bloomIndexParallelism(int parallelism) {
|
||||
props.setProperty(BLOOM_INDEX_PARALLELISM_PROP, String.valueOf(parallelism));
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder bloomIndexPruneByRanges(boolean pruneRanges) {
|
||||
props.setProperty(BLOOM_INDEX_PRUNE_BY_RANGES_PROP, String.valueOf(pruneRanges));
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder bloomIndexUseCaching(boolean useCaching) {
|
||||
props.setProperty(BLOOM_INDEX_USE_CACHING_PROP, String.valueOf(useCaching));
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder bloomIndexTreebasedFilter(boolean useTreeFilter) {
|
||||
props.setProperty(BLOOM_INDEX_TREE_BASED_FILTER_PROP, String.valueOf(useTreeFilter));
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder bloomIndexBucketizedChecking(boolean bucketizedChecking) {
|
||||
props.setProperty(BLOOM_INDEX_BUCKETIZED_CHECKING_PROP, String.valueOf(bucketizedChecking));
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder bloomIndexKeysPerBucket(int keysPerBucket) {
|
||||
props.setProperty(BLOOM_INDEX_KEYS_PER_BUCKET_PROP, String.valueOf(keysPerBucket));
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder withBloomIndexInputStorageLevel(String level) {
|
||||
props.setProperty(BLOOM_INDEX_INPUT_STORAGE_LEVEL, level);
|
||||
return this;
|
||||
}
|
||||
|
||||
public HoodieIndexConfig build() {
|
||||
HoodieIndexConfig config = new HoodieIndexConfig(props);
|
||||
setDefaultOnCondition(props, !props.containsKey(INDEX_TYPE_PROP), INDEX_TYPE_PROP,
|
||||
DEFAULT_INDEX_TYPE);
|
||||
setDefaultOnCondition(props, !props.containsKey(BLOOM_FILTER_NUM_ENTRIES),
|
||||
BLOOM_FILTER_NUM_ENTRIES, DEFAULT_BLOOM_FILTER_NUM_ENTRIES);
|
||||
setDefaultOnCondition(props, !props.containsKey(BLOOM_FILTER_FPP), BLOOM_FILTER_FPP,
|
||||
DEFAULT_BLOOM_FILTER_FPP);
|
||||
setDefaultOnCondition(props, !props.containsKey(BLOOM_INDEX_PARALLELISM_PROP),
|
||||
BLOOM_INDEX_PARALLELISM_PROP, DEFAULT_BLOOM_INDEX_PARALLELISM);
|
||||
setDefaultOnCondition(props, !props.containsKey(BLOOM_INDEX_PRUNE_BY_RANGES_PROP),
|
||||
BLOOM_INDEX_PRUNE_BY_RANGES_PROP, DEFAULT_BLOOM_INDEX_PRUNE_BY_RANGES);
|
||||
setDefaultOnCondition(props, !props.containsKey(BLOOM_INDEX_USE_CACHING_PROP),
|
||||
BLOOM_INDEX_USE_CACHING_PROP, DEFAULT_BLOOM_INDEX_USE_CACHING);
|
||||
setDefaultOnCondition(props, !props.containsKey(BLOOM_INDEX_INPUT_STORAGE_LEVEL),
|
||||
BLOOM_INDEX_INPUT_STORAGE_LEVEL, DEFAULT_BLOOM_INDEX_INPUT_STORAGE_LEVEL);
|
||||
setDefaultOnCondition(props, !props.containsKey(BLOOM_INDEX_TREE_BASED_FILTER_PROP),
|
||||
BLOOM_INDEX_TREE_BASED_FILTER_PROP, DEFAULT_BLOOM_INDEX_TREE_BASED_FILTER);
|
||||
setDefaultOnCondition(props, !props.containsKey(BLOOM_INDEX_BUCKETIZED_CHECKING_PROP),
|
||||
BLOOM_INDEX_BUCKETIZED_CHECKING_PROP, DEFAULT_BLOOM_INDEX_BUCKETIZED_CHECKING);
|
||||
setDefaultOnCondition(props, !props.containsKey(BLOOM_INDEX_KEYS_PER_BUCKET_PROP),
|
||||
BLOOM_INDEX_KEYS_PER_BUCKET_PROP, DEFAULT_BLOOM_INDEX_KEYS_PER_BUCKET);
|
||||
// Throws IllegalArgumentException if the value set is not a known Hoodie Index Type
|
||||
HoodieIndex.IndexType.valueOf(props.getProperty(INDEX_TYPE_PROP));
|
||||
return config;
|
||||
}
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,181 @@
|
||||
/*
|
||||
* 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.config;
|
||||
|
||||
import java.io.File;
|
||||
import java.io.FileReader;
|
||||
import java.io.IOException;
|
||||
import java.util.Properties;
|
||||
import javax.annotation.concurrent.Immutable;
|
||||
import org.apache.spark.SparkEnv;
|
||||
import org.apache.spark.util.Utils;
|
||||
|
||||
/**
|
||||
* Memory related config
|
||||
*/
|
||||
@Immutable
|
||||
public class HoodieMemoryConfig extends DefaultHoodieConfig {
|
||||
|
||||
// This fraction is multiplied with the spark.memory.fraction to get a final fraction of heap space to use
|
||||
// during merge. This makes it easier to scale this value as one increases the spark.executor.memory
|
||||
public static final String MAX_MEMORY_FRACTION_FOR_MERGE_PROP = "hoodie.memory.merge.fraction";
|
||||
// Default max memory fraction during hash-merge, excess spills to disk
|
||||
public static final String DEFAULT_MAX_MEMORY_FRACTION_FOR_MERGE = String.valueOf(0.6);
|
||||
public static final String MAX_MEMORY_FRACTION_FOR_COMPACTION_PROP = "hoodie.memory.compaction.fraction";
|
||||
// Default max memory fraction during compaction, excess spills to disk
|
||||
public static final String DEFAULT_MAX_MEMORY_FRACTION_FOR_COMPACTION = String.valueOf(0.6);
|
||||
// Default memory size per compaction (used if SparkEnv is absent), excess spills to disk
|
||||
public static final long DEFAULT_MAX_MEMORY_FOR_SPILLABLE_MAP_IN_BYTES =
|
||||
1024 * 1024 * 1024L; // 1GB
|
||||
// Property to set the max memory for merge
|
||||
public static final String MAX_MEMORY_FOR_MERGE_PROP = "hoodie.memory.merge.max.size";
|
||||
// Property to set the max memory for compaction
|
||||
public static final String MAX_MEMORY_FOR_COMPACTION_PROP = "hoodie.memory.compaction.max.size";
|
||||
// Property to set the max memory for dfs inputstream buffer size
|
||||
public static final String MAX_DFS_STREAM_BUFFER_SIZE_PROP = "hoodie.memory.dfs.buffer.max.size";
|
||||
public static final int DEFAULT_MAX_DFS_STREAM_BUFFER_SIZE = 16 * 1024 * 1024; // 16MB
|
||||
public static final String SPILLABLE_MAP_BASE_PATH_PROP = "hoodie.memory.spillable.map.path";
|
||||
// Default file path prefix for spillable file
|
||||
public static final String DEFAULT_SPILLABLE_MAP_BASE_PATH = "/tmp/";
|
||||
|
||||
// Property to control how what fraction of the failed record, exceptions we report back to driver.
|
||||
public static final String WRITESTATUS_FAILURE_FRACTION_PROP = "hoodie.memory.writestatus.failure.fraction";
|
||||
// Default is 10%. If set to 100%, with lot of failures, this can cause memory pressure, cause OOMs and
|
||||
// mask actual data errors.
|
||||
public static final double DEFAULT_WRITESTATUS_FAILURE_FRACTION = 0.1;
|
||||
|
||||
private HoodieMemoryConfig(Properties props) {
|
||||
super(props);
|
||||
}
|
||||
|
||||
public static HoodieMemoryConfig.Builder newBuilder() {
|
||||
return new Builder();
|
||||
}
|
||||
|
||||
public static class Builder {
|
||||
|
||||
private final Properties props = new Properties();
|
||||
|
||||
public Builder fromFile(File propertiesFile) throws IOException {
|
||||
FileReader reader = new FileReader(propertiesFile);
|
||||
try {
|
||||
this.props.load(reader);
|
||||
return this;
|
||||
} finally {
|
||||
reader.close();
|
||||
}
|
||||
}
|
||||
|
||||
public Builder fromProperties(Properties props) {
|
||||
this.props.putAll(props);
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder withMaxMemoryFractionPerPartitionMerge(double maxMemoryFractionPerPartitionMerge) {
|
||||
props.setProperty(MAX_MEMORY_FRACTION_FOR_MERGE_PROP,
|
||||
String.valueOf(maxMemoryFractionPerPartitionMerge));
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder withMaxMemoryFractionPerCompaction(double maxMemoryFractionPerCompaction) {
|
||||
props.setProperty(MAX_MEMORY_FRACTION_FOR_COMPACTION_PROP,
|
||||
String.valueOf(maxMemoryFractionPerCompaction));
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder withMaxDFSStreamBufferSize(int maxStreamBufferSize) {
|
||||
props.setProperty(MAX_DFS_STREAM_BUFFER_SIZE_PROP,
|
||||
String.valueOf(maxStreamBufferSize));
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder withWriteStatusFailureFraction(double failureFraction) {
|
||||
props.setProperty(WRITESTATUS_FAILURE_FRACTION_PROP, String.valueOf(failureFraction));
|
||||
return this;
|
||||
}
|
||||
|
||||
/**
|
||||
* Dynamic calculation of max memory to use for for spillable map. user.available.memory = spark.executor.memory *
|
||||
* (1 - spark.memory.fraction) spillable.available.memory = user.available.memory * hoodie.memory.fraction. Anytime
|
||||
* the spark.executor.memory or the spark.memory.fraction is changed, the memory used for spillable map changes
|
||||
* accordingly
|
||||
*/
|
||||
private long getMaxMemoryAllowedForMerge(String maxMemoryFraction) {
|
||||
final String SPARK_EXECUTOR_MEMORY_PROP = "spark.executor.memory";
|
||||
final String SPARK_EXECUTOR_MEMORY_FRACTION_PROP = "spark.memory.fraction";
|
||||
// This is hard-coded in spark code {@link
|
||||
// https://github.com/apache/spark/blob/576c43fb4226e4efa12189b41c3bc862019862c6/core/src/main/scala/org/apache/
|
||||
// spark/memory/UnifiedMemoryManager.scala#L231} so have to re-define this here
|
||||
final String DEFAULT_SPARK_EXECUTOR_MEMORY_FRACTION = "0.6";
|
||||
// This is hard-coded in spark code {@link
|
||||
// https://github.com/apache/spark/blob/576c43fb4226e4efa12189b41c3bc862019862c6/core/src/main/scala/org/apache/
|
||||
// spark/SparkContext.scala#L471} so have to re-define this here
|
||||
final String DEFAULT_SPARK_EXECUTOR_MEMORY_MB = "1024"; // in MB
|
||||
|
||||
if (SparkEnv.get() != null) {
|
||||
// 1 GB is the default conf used by Spark, look at SparkContext.scala
|
||||
long executorMemoryInBytes = Utils.memoryStringToMb(SparkEnv.get().conf().get(SPARK_EXECUTOR_MEMORY_PROP,
|
||||
DEFAULT_SPARK_EXECUTOR_MEMORY_MB)) * 1024
|
||||
* 1024L;
|
||||
// 0.6 is the default value used by Spark,
|
||||
// look at {@link
|
||||
// https://github.com/apache/spark/blob/master/core/src/main/scala/org/apache/spark/SparkConf.scala#L507}
|
||||
double memoryFraction = Double
|
||||
.valueOf(SparkEnv.get().conf().get(SPARK_EXECUTOR_MEMORY_FRACTION_PROP,
|
||||
DEFAULT_SPARK_EXECUTOR_MEMORY_FRACTION));
|
||||
double maxMemoryFractionForMerge = Double.valueOf(maxMemoryFraction);
|
||||
double userAvailableMemory = executorMemoryInBytes * (1 - memoryFraction);
|
||||
long maxMemoryForMerge = (long) Math
|
||||
.floor(userAvailableMemory * maxMemoryFractionForMerge);
|
||||
return maxMemoryForMerge;
|
||||
} else {
|
||||
return DEFAULT_MAX_MEMORY_FOR_SPILLABLE_MAP_IN_BYTES;
|
||||
}
|
||||
}
|
||||
|
||||
public HoodieMemoryConfig build() {
|
||||
HoodieMemoryConfig config = new HoodieMemoryConfig(props);
|
||||
setDefaultOnCondition(props,
|
||||
!props.containsKey(MAX_MEMORY_FRACTION_FOR_COMPACTION_PROP),
|
||||
MAX_MEMORY_FRACTION_FOR_COMPACTION_PROP,
|
||||
DEFAULT_MAX_MEMORY_FRACTION_FOR_COMPACTION);
|
||||
setDefaultOnCondition(props,
|
||||
!props.containsKey(MAX_MEMORY_FRACTION_FOR_MERGE_PROP),
|
||||
MAX_MEMORY_FRACTION_FOR_MERGE_PROP, DEFAULT_MAX_MEMORY_FRACTION_FOR_MERGE);
|
||||
setDefaultOnCondition(props,
|
||||
!props.containsKey(MAX_MEMORY_FOR_MERGE_PROP),
|
||||
MAX_MEMORY_FOR_MERGE_PROP, String.valueOf(
|
||||
getMaxMemoryAllowedForMerge(props.getProperty(MAX_MEMORY_FRACTION_FOR_MERGE_PROP))));
|
||||
setDefaultOnCondition(props,
|
||||
!props.containsKey(MAX_MEMORY_FOR_COMPACTION_PROP),
|
||||
MAX_MEMORY_FOR_COMPACTION_PROP, String.valueOf(
|
||||
getMaxMemoryAllowedForMerge(props.getProperty(MAX_MEMORY_FRACTION_FOR_COMPACTION_PROP))));
|
||||
setDefaultOnCondition(props,
|
||||
!props.containsKey(MAX_DFS_STREAM_BUFFER_SIZE_PROP),
|
||||
MAX_DFS_STREAM_BUFFER_SIZE_PROP, String.valueOf(DEFAULT_MAX_DFS_STREAM_BUFFER_SIZE));
|
||||
setDefaultOnCondition(props,
|
||||
!props.containsKey(SPILLABLE_MAP_BASE_PATH_PROP),
|
||||
SPILLABLE_MAP_BASE_PATH_PROP, DEFAULT_SPILLABLE_MAP_BASE_PATH);
|
||||
setDefaultOnCondition(props,
|
||||
!props.containsKey(WRITESTATUS_FAILURE_FRACTION_PROP),
|
||||
WRITESTATUS_FAILURE_FRACTION_PROP, String.valueOf(DEFAULT_WRITESTATUS_FAILURE_FRACTION));
|
||||
return config;
|
||||
}
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,120 @@
|
||||
/*
|
||||
* 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.config;
|
||||
|
||||
import java.io.File;
|
||||
import java.io.FileReader;
|
||||
import java.io.IOException;
|
||||
import java.util.Properties;
|
||||
import javax.annotation.concurrent.Immutable;
|
||||
import org.apache.hudi.metrics.MetricsReporterType;
|
||||
|
||||
/**
|
||||
* Fetch the configurations used by the Metrics system.
|
||||
*/
|
||||
@Immutable
|
||||
public class HoodieMetricsConfig extends DefaultHoodieConfig {
|
||||
|
||||
public static final String METRIC_PREFIX = "hoodie.metrics";
|
||||
public static final String METRICS_ON = METRIC_PREFIX + ".on";
|
||||
public static final boolean DEFAULT_METRICS_ON = false;
|
||||
public static final String METRICS_REPORTER_TYPE = METRIC_PREFIX + ".reporter.type";
|
||||
public static final MetricsReporterType DEFAULT_METRICS_REPORTER_TYPE = MetricsReporterType
|
||||
.GRAPHITE;
|
||||
|
||||
// Graphite
|
||||
public static final String GRAPHITE_PREFIX = METRIC_PREFIX + ".graphite";
|
||||
public static final String GRAPHITE_SERVER_HOST = GRAPHITE_PREFIX + ".host";
|
||||
public static final String DEFAULT_GRAPHITE_SERVER_HOST = "localhost";
|
||||
|
||||
public static final String GRAPHITE_SERVER_PORT = GRAPHITE_PREFIX + ".port";
|
||||
public static final int DEFAULT_GRAPHITE_SERVER_PORT = 4756;
|
||||
|
||||
public static final String GRAPHITE_METRIC_PREFIX = GRAPHITE_PREFIX + ".metric.prefix";
|
||||
|
||||
private HoodieMetricsConfig(Properties props) {
|
||||
super(props);
|
||||
}
|
||||
|
||||
public static HoodieMetricsConfig.Builder newBuilder() {
|
||||
return new Builder();
|
||||
}
|
||||
|
||||
public static class Builder {
|
||||
|
||||
private final Properties props = new Properties();
|
||||
|
||||
public Builder fromFile(File propertiesFile) throws IOException {
|
||||
FileReader reader = new FileReader(propertiesFile);
|
||||
try {
|
||||
this.props.load(reader);
|
||||
return this;
|
||||
} finally {
|
||||
reader.close();
|
||||
}
|
||||
}
|
||||
|
||||
public Builder fromProperties(Properties props) {
|
||||
this.props.putAll(props);
|
||||
return this;
|
||||
}
|
||||
|
||||
|
||||
public Builder on(boolean metricsOn) {
|
||||
props.setProperty(METRICS_ON, String.valueOf(metricsOn));
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder withReporterType(String reporterType) {
|
||||
props.setProperty(METRICS_REPORTER_TYPE, reporterType);
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder toGraphiteHost(String host) {
|
||||
props.setProperty(GRAPHITE_SERVER_HOST, host);
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder onGraphitePort(int port) {
|
||||
props.setProperty(GRAPHITE_SERVER_PORT, String.valueOf(port));
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder usePrefix(String prefix) {
|
||||
props.setProperty(GRAPHITE_METRIC_PREFIX, prefix);
|
||||
return this;
|
||||
}
|
||||
|
||||
public HoodieMetricsConfig build() {
|
||||
HoodieMetricsConfig config = new HoodieMetricsConfig(props);
|
||||
setDefaultOnCondition(props, !props.containsKey(METRICS_ON), METRICS_ON,
|
||||
String.valueOf(DEFAULT_METRICS_ON));
|
||||
setDefaultOnCondition(props, !props.containsKey(METRICS_REPORTER_TYPE), METRICS_REPORTER_TYPE,
|
||||
DEFAULT_METRICS_REPORTER_TYPE.name());
|
||||
setDefaultOnCondition(props, !props.containsKey(GRAPHITE_SERVER_HOST), GRAPHITE_SERVER_HOST,
|
||||
DEFAULT_GRAPHITE_SERVER_HOST);
|
||||
setDefaultOnCondition(props, !props.containsKey(GRAPHITE_SERVER_PORT), GRAPHITE_SERVER_PORT,
|
||||
String.valueOf(DEFAULT_GRAPHITE_SERVER_PORT));
|
||||
setDefaultOnCondition(props, !props.containsKey(GRAPHITE_SERVER_PORT), GRAPHITE_SERVER_PORT,
|
||||
String.valueOf(DEFAULT_GRAPHITE_SERVER_PORT));
|
||||
return config;
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
@@ -0,0 +1,145 @@
|
||||
/*
|
||||
* 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.config;
|
||||
|
||||
import java.io.File;
|
||||
import java.io.FileReader;
|
||||
import java.io.IOException;
|
||||
import java.util.Properties;
|
||||
import javax.annotation.concurrent.Immutable;
|
||||
|
||||
/**
|
||||
* Storage related config
|
||||
*/
|
||||
@Immutable
|
||||
public class HoodieStorageConfig extends DefaultHoodieConfig {
|
||||
|
||||
public static final String PARQUET_FILE_MAX_BYTES = "hoodie.parquet.max.file.size";
|
||||
public static final String DEFAULT_PARQUET_FILE_MAX_BYTES = String.valueOf(120 * 1024 * 1024);
|
||||
public static final String PARQUET_BLOCK_SIZE_BYTES = "hoodie.parquet.block.size";
|
||||
public static final String DEFAULT_PARQUET_BLOCK_SIZE_BYTES = DEFAULT_PARQUET_FILE_MAX_BYTES;
|
||||
public static final String PARQUET_PAGE_SIZE_BYTES = "hoodie.parquet.page.size";
|
||||
public static final String DEFAULT_PARQUET_PAGE_SIZE_BYTES = String.valueOf(1 * 1024 * 1024);
|
||||
// used to size log files
|
||||
public static final String LOGFILE_SIZE_MAX_BYTES = "hoodie.logfile.max.size";
|
||||
public static final String DEFAULT_LOGFILE_SIZE_MAX_BYTES = String
|
||||
.valueOf(1024 * 1024 * 1024); // 1 GB
|
||||
// used to size data blocks in log file
|
||||
public static final String LOGFILE_DATA_BLOCK_SIZE_MAX_BYTES = "hoodie.logfile.data.block.max.size";
|
||||
public static final String DEFAULT_LOGFILE_DATA_BLOCK_SIZE_MAX_BYTES = String.valueOf(256 * 1024 * 1024); // 256 MB
|
||||
public static final String PARQUET_COMPRESSION_RATIO = "hoodie.parquet.compression.ratio";
|
||||
// Default compression ratio for parquet
|
||||
public static final String DEFAULT_STREAM_COMPRESSION_RATIO = String.valueOf(0.1);
|
||||
public static final String PARQUET_COMPRESSION_CODEC = "hoodie.parquet.compression.codec";
|
||||
// Default compression codec for parquet
|
||||
public static final String DEFAULT_PARQUET_COMPRESSION_CODEC = "gzip";
|
||||
public static final String LOGFILE_TO_PARQUET_COMPRESSION_RATIO = "hoodie.logfile.to.parquet.compression.ratio";
|
||||
// Default compression ratio for log file to parquet, general 3x
|
||||
public static final String DEFAULT_LOGFILE_TO_PARQUET_COMPRESSION_RATIO = String.valueOf(0.35);
|
||||
|
||||
private HoodieStorageConfig(Properties props) {
|
||||
super(props);
|
||||
}
|
||||
|
||||
public static HoodieStorageConfig.Builder newBuilder() {
|
||||
return new Builder();
|
||||
}
|
||||
|
||||
public static class Builder {
|
||||
|
||||
private final Properties props = new Properties();
|
||||
|
||||
public Builder fromFile(File propertiesFile) throws IOException {
|
||||
FileReader reader = new FileReader(propertiesFile);
|
||||
try {
|
||||
this.props.load(reader);
|
||||
return this;
|
||||
} finally {
|
||||
reader.close();
|
||||
}
|
||||
}
|
||||
|
||||
public Builder fromProperties(Properties props) {
|
||||
this.props.putAll(props);
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder limitFileSize(long maxFileSize) {
|
||||
props.setProperty(PARQUET_FILE_MAX_BYTES, String.valueOf(maxFileSize));
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder parquetBlockSize(int blockSize) {
|
||||
props.setProperty(PARQUET_BLOCK_SIZE_BYTES, String.valueOf(blockSize));
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder parquetPageSize(int pageSize) {
|
||||
props.setProperty(PARQUET_PAGE_SIZE_BYTES, String.valueOf(pageSize));
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder logFileDataBlockMaxSize(int dataBlockSize) {
|
||||
props.setProperty(LOGFILE_DATA_BLOCK_SIZE_MAX_BYTES, String.valueOf(dataBlockSize));
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder logFileMaxSize(int logFileSize) {
|
||||
props.setProperty(LOGFILE_SIZE_MAX_BYTES, String.valueOf(logFileSize));
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder parquetCompressionRatio(double parquetCompressionRatio) {
|
||||
props.setProperty(PARQUET_COMPRESSION_RATIO, String.valueOf(parquetCompressionRatio));
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder parquetCompressionCodec(String parquetCompressionCodec) {
|
||||
props.setProperty(PARQUET_COMPRESSION_CODEC, parquetCompressionCodec);
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder logFileToParquetCompressionRatio(double logFileToParquetCompressionRatio) {
|
||||
props.setProperty(LOGFILE_TO_PARQUET_COMPRESSION_RATIO, String.valueOf(logFileToParquetCompressionRatio));
|
||||
return this;
|
||||
}
|
||||
|
||||
public HoodieStorageConfig build() {
|
||||
HoodieStorageConfig config = new HoodieStorageConfig(props);
|
||||
setDefaultOnCondition(props, !props.containsKey(PARQUET_FILE_MAX_BYTES),
|
||||
PARQUET_FILE_MAX_BYTES, DEFAULT_PARQUET_FILE_MAX_BYTES);
|
||||
setDefaultOnCondition(props, !props.containsKey(PARQUET_BLOCK_SIZE_BYTES),
|
||||
PARQUET_BLOCK_SIZE_BYTES, DEFAULT_PARQUET_BLOCK_SIZE_BYTES);
|
||||
setDefaultOnCondition(props, !props.containsKey(PARQUET_PAGE_SIZE_BYTES),
|
||||
PARQUET_PAGE_SIZE_BYTES, DEFAULT_PARQUET_PAGE_SIZE_BYTES);
|
||||
setDefaultOnCondition(props, !props.containsKey(LOGFILE_DATA_BLOCK_SIZE_MAX_BYTES),
|
||||
LOGFILE_DATA_BLOCK_SIZE_MAX_BYTES, DEFAULT_LOGFILE_DATA_BLOCK_SIZE_MAX_BYTES);
|
||||
setDefaultOnCondition(props, !props.containsKey(LOGFILE_SIZE_MAX_BYTES),
|
||||
LOGFILE_SIZE_MAX_BYTES, DEFAULT_LOGFILE_SIZE_MAX_BYTES);
|
||||
setDefaultOnCondition(props, !props.containsKey(PARQUET_COMPRESSION_RATIO),
|
||||
PARQUET_COMPRESSION_RATIO, DEFAULT_STREAM_COMPRESSION_RATIO);
|
||||
setDefaultOnCondition(props, !props.containsKey(PARQUET_COMPRESSION_CODEC),
|
||||
PARQUET_COMPRESSION_CODEC, DEFAULT_PARQUET_COMPRESSION_CODEC);
|
||||
setDefaultOnCondition(props, !props.containsKey(LOGFILE_TO_PARQUET_COMPRESSION_RATIO),
|
||||
LOGFILE_TO_PARQUET_COMPRESSION_RATIO, DEFAULT_LOGFILE_TO_PARQUET_COMPRESSION_RATIO);
|
||||
return config;
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
@@ -0,0 +1,727 @@
|
||||
/*
|
||||
* 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.config;
|
||||
|
||||
import com.google.common.base.Preconditions;
|
||||
import java.io.File;
|
||||
import java.io.FileReader;
|
||||
import java.io.IOException;
|
||||
import java.io.InputStream;
|
||||
import java.util.Map;
|
||||
import java.util.Properties;
|
||||
import javax.annotation.concurrent.Immutable;
|
||||
import org.apache.hudi.HoodieWriteClient;
|
||||
import org.apache.hudi.WriteStatus;
|
||||
import org.apache.hudi.common.model.HoodieCleaningPolicy;
|
||||
import org.apache.hudi.common.table.view.FileSystemViewStorageConfig;
|
||||
import org.apache.hudi.common.util.ConsistencyGuardConfig;
|
||||
import org.apache.hudi.common.util.ReflectionUtils;
|
||||
import org.apache.hudi.index.HoodieIndex;
|
||||
import org.apache.hudi.io.compact.strategy.CompactionStrategy;
|
||||
import org.apache.hudi.metrics.MetricsReporterType;
|
||||
import org.apache.parquet.hadoop.metadata.CompressionCodecName;
|
||||
import org.apache.spark.storage.StorageLevel;
|
||||
|
||||
/**
|
||||
* Class storing configs for the {@link HoodieWriteClient}
|
||||
*/
|
||||
@Immutable
|
||||
public class HoodieWriteConfig extends DefaultHoodieConfig {
|
||||
|
||||
public static final String TABLE_NAME = "hoodie.table.name";
|
||||
private static final String BASE_PATH_PROP = "hoodie.base.path";
|
||||
private static final String AVRO_SCHEMA = "hoodie.avro.schema";
|
||||
private static final String DEFAULT_PARALLELISM = "1500";
|
||||
private static final String INSERT_PARALLELISM = "hoodie.insert.shuffle.parallelism";
|
||||
private static final String BULKINSERT_PARALLELISM = "hoodie.bulkinsert.shuffle.parallelism";
|
||||
private static final String UPSERT_PARALLELISM = "hoodie.upsert.shuffle.parallelism";
|
||||
private static final String WRITE_BUFFER_LIMIT_BYTES = "hoodie.write.buffer.limit.bytes";
|
||||
private static final String DEFAULT_WRITE_BUFFER_LIMIT_BYTES = String.valueOf(4 * 1024 * 1024);
|
||||
private static final String COMBINE_BEFORE_INSERT_PROP = "hoodie.combine.before.insert";
|
||||
private static final String DEFAULT_COMBINE_BEFORE_INSERT = "false";
|
||||
private static final String COMBINE_BEFORE_UPSERT_PROP = "hoodie.combine.before.upsert";
|
||||
private static final String DEFAULT_COMBINE_BEFORE_UPSERT = "true";
|
||||
private static final String WRITE_STATUS_STORAGE_LEVEL = "hoodie.write.status.storage.level";
|
||||
private static final String DEFAULT_WRITE_STATUS_STORAGE_LEVEL = "MEMORY_AND_DISK_SER";
|
||||
private static final String HOODIE_AUTO_COMMIT_PROP = "hoodie.auto.commit";
|
||||
private static final String DEFAULT_HOODIE_AUTO_COMMIT = "true";
|
||||
private static final String HOODIE_ASSUME_DATE_PARTITIONING_PROP =
|
||||
"hoodie.assume.date" + ".partitioning";
|
||||
private static final String DEFAULT_ASSUME_DATE_PARTITIONING = "false";
|
||||
private static final String HOODIE_WRITE_STATUS_CLASS_PROP = "hoodie.writestatus.class";
|
||||
private static final String DEFAULT_HOODIE_WRITE_STATUS_CLASS = WriteStatus.class.getName();
|
||||
private static final String FINALIZE_WRITE_PARALLELISM = "hoodie.finalize.write.parallelism";
|
||||
private static final String DEFAULT_FINALIZE_WRITE_PARALLELISM = DEFAULT_PARALLELISM;
|
||||
|
||||
private static final String EMBEDDED_TIMELINE_SERVER_ENABLED = "hoodie.embed.timeline.server";
|
||||
private static final String DEFAULT_EMBEDDED_TIMELINE_SERVER_ENABLED = "false";
|
||||
|
||||
private static final String FAIL_ON_TIMELINE_ARCHIVING_ENABLED_PROP = "hoodie.fail.on.timeline.archiving";
|
||||
private static final String DEFAULT_FAIL_ON_TIMELINE_ARCHIVING_ENABLED = "true";
|
||||
// time between successive attempts to ensure written data's metadata is consistent on storage
|
||||
private static final String INITIAL_CONSISTENCY_CHECK_INTERVAL_MS_PROP =
|
||||
"hoodie.consistency.check.initial_interval_ms";
|
||||
private static long DEFAULT_INITIAL_CONSISTENCY_CHECK_INTERVAL_MS = 2000L;
|
||||
|
||||
// max interval time
|
||||
private static final String MAX_CONSISTENCY_CHECK_INTERVAL_MS_PROP = "hoodie.consistency.check.max_interval_ms";
|
||||
private static long DEFAULT_MAX_CONSISTENCY_CHECK_INTERVAL_MS = 300000L;
|
||||
|
||||
// maximum number of checks, for consistency of written data. Will wait upto 256 Secs
|
||||
private static final String MAX_CONSISTENCY_CHECKS_PROP = "hoodie.consistency.check.max_checks";
|
||||
private static int DEFAULT_MAX_CONSISTENCY_CHECKS = 7;
|
||||
|
||||
private ConsistencyGuardConfig consistencyGuardConfig;
|
||||
|
||||
// Hoodie Write Client transparently rewrites File System View config when embedded mode is enabled
|
||||
// We keep track of original config and rewritten config
|
||||
private final FileSystemViewStorageConfig clientSpecifiedViewStorageConfig;
|
||||
private FileSystemViewStorageConfig viewStorageConfig;
|
||||
|
||||
private HoodieWriteConfig(Properties props) {
|
||||
super(props);
|
||||
Properties newProps = new Properties();
|
||||
newProps.putAll(props);
|
||||
this.consistencyGuardConfig = ConsistencyGuardConfig.newBuilder().fromProperties(newProps).build();
|
||||
this.clientSpecifiedViewStorageConfig = FileSystemViewStorageConfig.newBuilder().fromProperties(newProps).build();
|
||||
this.viewStorageConfig = clientSpecifiedViewStorageConfig;
|
||||
}
|
||||
|
||||
public static HoodieWriteConfig.Builder newBuilder() {
|
||||
return new Builder();
|
||||
}
|
||||
|
||||
/**
|
||||
* base properties
|
||||
**/
|
||||
public String getBasePath() {
|
||||
return props.getProperty(BASE_PATH_PROP);
|
||||
}
|
||||
|
||||
public String getSchema() {
|
||||
return props.getProperty(AVRO_SCHEMA);
|
||||
}
|
||||
|
||||
public String getTableName() {
|
||||
return props.getProperty(TABLE_NAME);
|
||||
}
|
||||
|
||||
public Boolean shouldAutoCommit() {
|
||||
return Boolean.parseBoolean(props.getProperty(HOODIE_AUTO_COMMIT_PROP));
|
||||
}
|
||||
|
||||
public Boolean shouldAssumeDatePartitioning() {
|
||||
return Boolean.parseBoolean(props.getProperty(HOODIE_ASSUME_DATE_PARTITIONING_PROP));
|
||||
}
|
||||
|
||||
public int getBulkInsertShuffleParallelism() {
|
||||
return Integer.parseInt(props.getProperty(BULKINSERT_PARALLELISM));
|
||||
}
|
||||
|
||||
public int getInsertShuffleParallelism() {
|
||||
return Integer.parseInt(props.getProperty(INSERT_PARALLELISM));
|
||||
}
|
||||
|
||||
public int getUpsertShuffleParallelism() {
|
||||
return Integer.parseInt(props.getProperty(UPSERT_PARALLELISM));
|
||||
}
|
||||
|
||||
public int getWriteBufferLimitBytes() {
|
||||
return Integer
|
||||
.parseInt(props.getProperty(WRITE_BUFFER_LIMIT_BYTES, DEFAULT_WRITE_BUFFER_LIMIT_BYTES));
|
||||
}
|
||||
|
||||
public boolean shouldCombineBeforeInsert() {
|
||||
return Boolean.parseBoolean(props.getProperty(COMBINE_BEFORE_INSERT_PROP));
|
||||
}
|
||||
|
||||
public boolean shouldCombineBeforeUpsert() {
|
||||
return Boolean.parseBoolean(props.getProperty(COMBINE_BEFORE_UPSERT_PROP));
|
||||
}
|
||||
|
||||
public StorageLevel getWriteStatusStorageLevel() {
|
||||
return StorageLevel.fromString(props.getProperty(WRITE_STATUS_STORAGE_LEVEL));
|
||||
}
|
||||
|
||||
public String getWriteStatusClassName() {
|
||||
return props.getProperty(HOODIE_WRITE_STATUS_CLASS_PROP);
|
||||
}
|
||||
|
||||
public int getFinalizeWriteParallelism() {
|
||||
return Integer.parseInt(props.getProperty(FINALIZE_WRITE_PARALLELISM));
|
||||
}
|
||||
|
||||
public boolean isEmbeddedTimelineServerEnabled() {
|
||||
return Boolean.parseBoolean(props.getProperty(EMBEDDED_TIMELINE_SERVER_ENABLED));
|
||||
}
|
||||
|
||||
public boolean isFailOnTimelineArchivingEnabled() {
|
||||
return Boolean.parseBoolean(props.getProperty(FAIL_ON_TIMELINE_ARCHIVING_ENABLED_PROP));
|
||||
}
|
||||
|
||||
public int getMaxConsistencyChecks() {
|
||||
return Integer.parseInt(props.getProperty(MAX_CONSISTENCY_CHECKS_PROP));
|
||||
}
|
||||
|
||||
public int getInitialConsistencyCheckIntervalMs() {
|
||||
return Integer.parseInt(props.getProperty(INITIAL_CONSISTENCY_CHECK_INTERVAL_MS_PROP));
|
||||
}
|
||||
|
||||
public int getMaxConsistencyCheckIntervalMs() {
|
||||
return Integer.parseInt(props.getProperty(MAX_CONSISTENCY_CHECK_INTERVAL_MS_PROP));
|
||||
}
|
||||
|
||||
/**
|
||||
* compaction properties
|
||||
**/
|
||||
public HoodieCleaningPolicy getCleanerPolicy() {
|
||||
return HoodieCleaningPolicy
|
||||
.valueOf(props.getProperty(HoodieCompactionConfig.CLEANER_POLICY_PROP));
|
||||
}
|
||||
|
||||
public int getCleanerFileVersionsRetained() {
|
||||
return Integer
|
||||
.parseInt(props.getProperty(HoodieCompactionConfig.CLEANER_FILE_VERSIONS_RETAINED_PROP));
|
||||
}
|
||||
|
||||
public int getCleanerCommitsRetained() {
|
||||
return Integer
|
||||
.parseInt(props.getProperty(HoodieCompactionConfig.CLEANER_COMMITS_RETAINED_PROP));
|
||||
}
|
||||
|
||||
public int getMaxCommitsToKeep() {
|
||||
return Integer.parseInt(props.getProperty(HoodieCompactionConfig.MAX_COMMITS_TO_KEEP_PROP));
|
||||
}
|
||||
|
||||
public int getMinCommitsToKeep() {
|
||||
return Integer.parseInt(props.getProperty(HoodieCompactionConfig.MIN_COMMITS_TO_KEEP_PROP));
|
||||
}
|
||||
|
||||
public int getParquetSmallFileLimit() {
|
||||
return Integer
|
||||
.parseInt(props.getProperty(HoodieCompactionConfig.PARQUET_SMALL_FILE_LIMIT_BYTES));
|
||||
}
|
||||
|
||||
public int getCopyOnWriteInsertSplitSize() {
|
||||
return Integer
|
||||
.parseInt(props.getProperty(HoodieCompactionConfig.COPY_ON_WRITE_TABLE_INSERT_SPLIT_SIZE));
|
||||
}
|
||||
|
||||
public int getCopyOnWriteRecordSizeEstimate() {
|
||||
return Integer.parseInt(
|
||||
props.getProperty(HoodieCompactionConfig.COPY_ON_WRITE_TABLE_RECORD_SIZE_ESTIMATE));
|
||||
}
|
||||
|
||||
public boolean shouldAutoTuneInsertSplits() {
|
||||
return Boolean.parseBoolean(
|
||||
props.getProperty(HoodieCompactionConfig.COPY_ON_WRITE_TABLE_AUTO_SPLIT_INSERTS));
|
||||
}
|
||||
|
||||
public int getCleanerParallelism() {
|
||||
return Integer.parseInt(props.getProperty(HoodieCompactionConfig.CLEANER_PARALLELISM));
|
||||
}
|
||||
|
||||
public boolean isAutoClean() {
|
||||
return Boolean.parseBoolean(props.getProperty(HoodieCompactionConfig.AUTO_CLEAN_PROP));
|
||||
}
|
||||
|
||||
public boolean isInlineCompaction() {
|
||||
return Boolean.parseBoolean(props.getProperty(HoodieCompactionConfig.INLINE_COMPACT_PROP));
|
||||
}
|
||||
|
||||
public int getInlineCompactDeltaCommitMax() {
|
||||
return Integer
|
||||
.parseInt(props.getProperty(HoodieCompactionConfig.INLINE_COMPACT_NUM_DELTA_COMMITS_PROP));
|
||||
}
|
||||
|
||||
public CompactionStrategy getCompactionStrategy() {
|
||||
return ReflectionUtils
|
||||
.loadClass(props.getProperty(HoodieCompactionConfig.COMPACTION_STRATEGY_PROP));
|
||||
}
|
||||
|
||||
public Long getTargetIOPerCompactionInMB() {
|
||||
return Long
|
||||
.parseLong(props.getProperty(HoodieCompactionConfig.TARGET_IO_PER_COMPACTION_IN_MB_PROP));
|
||||
}
|
||||
|
||||
public Boolean getCompactionLazyBlockReadEnabled() {
|
||||
return Boolean
|
||||
.valueOf(props.getProperty(HoodieCompactionConfig.COMPACTION_LAZY_BLOCK_READ_ENABLED_PROP));
|
||||
}
|
||||
|
||||
public Boolean getCompactionReverseLogReadEnabled() {
|
||||
return Boolean.valueOf(
|
||||
props.getProperty(HoodieCompactionConfig.COMPACTION_REVERSE_LOG_READ_ENABLED_PROP));
|
||||
}
|
||||
|
||||
public String getPayloadClass() {
|
||||
return props.getProperty(HoodieCompactionConfig.PAYLOAD_CLASS_PROP);
|
||||
}
|
||||
|
||||
public int getTargetPartitionsPerDayBasedCompaction() {
|
||||
return Integer
|
||||
.parseInt(props.getProperty(HoodieCompactionConfig.TARGET_PARTITIONS_PER_DAYBASED_COMPACTION_PROP));
|
||||
}
|
||||
|
||||
public int getCommitArchivalBatchSize() {
|
||||
return Integer
|
||||
.parseInt(props.getProperty(HoodieCompactionConfig.COMMITS_ARCHIVAL_BATCH_SIZE_PROP));
|
||||
}
|
||||
|
||||
/**
|
||||
* index properties
|
||||
**/
|
||||
public HoodieIndex.IndexType getIndexType() {
|
||||
return HoodieIndex.IndexType.valueOf(props.getProperty(HoodieIndexConfig.INDEX_TYPE_PROP));
|
||||
}
|
||||
|
||||
public int getBloomFilterNumEntries() {
|
||||
return Integer.parseInt(props.getProperty(HoodieIndexConfig.BLOOM_FILTER_NUM_ENTRIES));
|
||||
}
|
||||
|
||||
public double getBloomFilterFPP() {
|
||||
return Double.parseDouble(props.getProperty(HoodieIndexConfig.BLOOM_FILTER_FPP));
|
||||
}
|
||||
|
||||
public String getHbaseZkQuorum() {
|
||||
return props.getProperty(HoodieHBaseIndexConfig.HBASE_ZKQUORUM_PROP);
|
||||
}
|
||||
|
||||
public int getHbaseZkPort() {
|
||||
return Integer.parseInt(props.getProperty(HoodieHBaseIndexConfig.HBASE_ZKPORT_PROP));
|
||||
}
|
||||
|
||||
public String getHBaseZkZnodeParent() {
|
||||
return props.getProperty(HoodieIndexConfig.HBASE_ZK_ZNODEPARENT);
|
||||
}
|
||||
|
||||
public String getHbaseTableName() {
|
||||
return props.getProperty(HoodieHBaseIndexConfig.HBASE_TABLENAME_PROP);
|
||||
}
|
||||
|
||||
public int getHbaseIndexGetBatchSize() {
|
||||
return Integer.valueOf(props.getProperty(HoodieHBaseIndexConfig.HBASE_GET_BATCH_SIZE_PROP));
|
||||
}
|
||||
|
||||
public int getHbaseIndexPutBatchSize() {
|
||||
return Integer.valueOf(props.getProperty(HoodieHBaseIndexConfig.HBASE_PUT_BATCH_SIZE_PROP));
|
||||
}
|
||||
|
||||
public Boolean getHbaseIndexPutBatchSizeAutoCompute() {
|
||||
return Boolean.valueOf(props.getProperty(HoodieHBaseIndexConfig.HBASE_PUT_BATCH_SIZE_AUTO_COMPUTE_PROP));
|
||||
}
|
||||
|
||||
public String getHBaseQPSResourceAllocatorClass() {
|
||||
return props.getProperty(HoodieHBaseIndexConfig.HBASE_INDEX_QPS_ALLOCATOR_CLASS);
|
||||
}
|
||||
|
||||
public String getHBaseQPSZKnodePath() {
|
||||
return props.getProperty(HoodieHBaseIndexConfig.HBASE_ZK_PATH_QPS_ROOT);
|
||||
}
|
||||
|
||||
public String getHBaseZkZnodeSessionTimeout() {
|
||||
return props.getProperty(HoodieHBaseIndexConfig.HOODIE_INDEX_HBASE_ZK_SESSION_TIMEOUT_MS);
|
||||
}
|
||||
|
||||
public String getHBaseZkZnodeConnectionTimeout() {
|
||||
return props.getProperty(HoodieHBaseIndexConfig.HOODIE_INDEX_HBASE_ZK_CONNECTION_TIMEOUT_MS);
|
||||
}
|
||||
|
||||
public boolean getHBaseIndexShouldComputeQPSDynamically() {
|
||||
return Boolean.valueOf(props.getProperty(HoodieHBaseIndexConfig.HOODIE_INDEX_COMPUTE_QPS_DYNAMICALLY));
|
||||
}
|
||||
|
||||
public int getHBaseIndexDesiredPutsTime() {
|
||||
return Integer.valueOf(props.getProperty(HoodieHBaseIndexConfig.HOODIE_INDEX_DESIRED_PUTS_TIME_IN_SECS));
|
||||
}
|
||||
|
||||
/**
|
||||
* Fraction of the global share of QPS that should be allocated to this job.
|
||||
* Let's say there are 3 jobs which have input size in terms of number of rows
|
||||
* required for HbaseIndexing as x, 2x, 3x respectively. Then this fraction for
|
||||
* the jobs would be (0.17) 1/6, 0.33 (2/6) and 0.5 (3/6) respectively.
|
||||
*/
|
||||
public float getHbaseIndexQPSFraction() {
|
||||
return Float.parseFloat(props.getProperty(HoodieHBaseIndexConfig.HBASE_QPS_FRACTION_PROP));
|
||||
}
|
||||
|
||||
public float getHBaseIndexMinQPSFraction() {
|
||||
return Float.parseFloat(props.getProperty(HoodieHBaseIndexConfig.HBASE_MIN_QPS_FRACTION_PROP));
|
||||
}
|
||||
|
||||
public float getHBaseIndexMaxQPSFraction() {
|
||||
return Float.parseFloat(props.getProperty(HoodieHBaseIndexConfig.HBASE_MAX_QPS_FRACTION_PROP));
|
||||
}
|
||||
|
||||
/**
|
||||
* This should be same across various jobs. This is intended to limit the aggregate
|
||||
* QPS generated across various Hoodie jobs to an Hbase Region Server
|
||||
*/
|
||||
public int getHbaseIndexMaxQPSPerRegionServer() {
|
||||
return Integer.parseInt(props.getProperty(HoodieHBaseIndexConfig.HBASE_MAX_QPS_PER_REGION_SERVER_PROP));
|
||||
}
|
||||
|
||||
public int getBloomIndexParallelism() {
|
||||
return Integer.parseInt(props.getProperty(HoodieIndexConfig.BLOOM_INDEX_PARALLELISM_PROP));
|
||||
}
|
||||
|
||||
public boolean getBloomIndexPruneByRanges() {
|
||||
return Boolean
|
||||
.parseBoolean(props.getProperty(HoodieIndexConfig.BLOOM_INDEX_PRUNE_BY_RANGES_PROP));
|
||||
}
|
||||
|
||||
public boolean getBloomIndexUseCaching() {
|
||||
return Boolean.parseBoolean(props.getProperty(HoodieIndexConfig.BLOOM_INDEX_USE_CACHING_PROP));
|
||||
}
|
||||
|
||||
public boolean useBloomIndexTreebasedFilter() {
|
||||
return Boolean.parseBoolean(props.getProperty(HoodieIndexConfig.BLOOM_INDEX_TREE_BASED_FILTER_PROP));
|
||||
}
|
||||
|
||||
public boolean useBloomIndexBucketizedChecking() {
|
||||
return Boolean.parseBoolean(props.getProperty(HoodieIndexConfig.BLOOM_INDEX_BUCKETIZED_CHECKING_PROP));
|
||||
}
|
||||
|
||||
public int getBloomIndexKeysPerBucket() {
|
||||
return Integer.parseInt(props.getProperty(HoodieIndexConfig.BLOOM_INDEX_KEYS_PER_BUCKET_PROP));
|
||||
}
|
||||
|
||||
public StorageLevel getBloomIndexInputStorageLevel() {
|
||||
return StorageLevel
|
||||
.fromString(props.getProperty(HoodieIndexConfig.BLOOM_INDEX_INPUT_STORAGE_LEVEL));
|
||||
}
|
||||
|
||||
/**
|
||||
* storage properties
|
||||
**/
|
||||
public long getParquetMaxFileSize() {
|
||||
return Long.parseLong(props.getProperty(HoodieStorageConfig.PARQUET_FILE_MAX_BYTES));
|
||||
}
|
||||
|
||||
public int getParquetBlockSize() {
|
||||
return Integer.parseInt(props.getProperty(HoodieStorageConfig.PARQUET_BLOCK_SIZE_BYTES));
|
||||
}
|
||||
|
||||
public int getParquetPageSize() {
|
||||
return Integer.parseInt(props.getProperty(HoodieStorageConfig.PARQUET_PAGE_SIZE_BYTES));
|
||||
}
|
||||
|
||||
public int getLogFileDataBlockMaxSize() {
|
||||
return Integer
|
||||
.parseInt(props.getProperty(HoodieStorageConfig.LOGFILE_DATA_BLOCK_SIZE_MAX_BYTES));
|
||||
}
|
||||
|
||||
public int getLogFileMaxSize() {
|
||||
return Integer.parseInt(props.getProperty(HoodieStorageConfig.LOGFILE_SIZE_MAX_BYTES));
|
||||
}
|
||||
|
||||
public double getParquetCompressionRatio() {
|
||||
return Double.valueOf(props.getProperty(HoodieStorageConfig.PARQUET_COMPRESSION_RATIO));
|
||||
}
|
||||
|
||||
public CompressionCodecName getParquetCompressionCodec() {
|
||||
return CompressionCodecName.fromConf(props.getProperty(HoodieStorageConfig.PARQUET_COMPRESSION_CODEC));
|
||||
}
|
||||
|
||||
public double getLogFileToParquetCompressionRatio() {
|
||||
return Double.valueOf(props.getProperty(HoodieStorageConfig.LOGFILE_TO_PARQUET_COMPRESSION_RATIO));
|
||||
}
|
||||
|
||||
/**
|
||||
* metrics properties
|
||||
**/
|
||||
public boolean isMetricsOn() {
|
||||
return Boolean.parseBoolean(props.getProperty(HoodieMetricsConfig.METRICS_ON));
|
||||
}
|
||||
|
||||
public MetricsReporterType getMetricsReporterType() {
|
||||
return MetricsReporterType
|
||||
.valueOf(props.getProperty(HoodieMetricsConfig.METRICS_REPORTER_TYPE));
|
||||
}
|
||||
|
||||
public String getGraphiteServerHost() {
|
||||
return props.getProperty(HoodieMetricsConfig.GRAPHITE_SERVER_HOST);
|
||||
}
|
||||
|
||||
public int getGraphiteServerPort() {
|
||||
return Integer.parseInt(props.getProperty(HoodieMetricsConfig.GRAPHITE_SERVER_PORT));
|
||||
}
|
||||
|
||||
public String getGraphiteMetricPrefix() {
|
||||
return props.getProperty(HoodieMetricsConfig.GRAPHITE_METRIC_PREFIX);
|
||||
}
|
||||
|
||||
/**
|
||||
* memory configs
|
||||
*/
|
||||
public Double getMaxMemoryFractionPerPartitionMerge() {
|
||||
return Double.valueOf(props.getProperty(HoodieMemoryConfig.MAX_MEMORY_FRACTION_FOR_MERGE_PROP));
|
||||
}
|
||||
|
||||
public Double getMaxMemoryFractionPerCompaction() {
|
||||
return Double
|
||||
.valueOf(
|
||||
props.getProperty(HoodieMemoryConfig.MAX_MEMORY_FRACTION_FOR_COMPACTION_PROP));
|
||||
}
|
||||
|
||||
public Long getMaxMemoryPerPartitionMerge() {
|
||||
return Long.valueOf(props.getProperty(HoodieMemoryConfig.MAX_MEMORY_FOR_MERGE_PROP));
|
||||
}
|
||||
|
||||
public Long getMaxMemoryPerCompaction() {
|
||||
return Long.valueOf(props.getProperty(HoodieMemoryConfig.MAX_MEMORY_FOR_COMPACTION_PROP));
|
||||
}
|
||||
|
||||
public int getMaxDFSStreamBufferSize() {
|
||||
return Integer.valueOf(props.getProperty(HoodieMemoryConfig.MAX_DFS_STREAM_BUFFER_SIZE_PROP));
|
||||
}
|
||||
|
||||
public String getSpillableMapBasePath() {
|
||||
return props.getProperty(HoodieMemoryConfig.SPILLABLE_MAP_BASE_PATH_PROP);
|
||||
}
|
||||
|
||||
public double getWriteStatusFailureFraction() {
|
||||
return Double.valueOf(props.getProperty(HoodieMemoryConfig.WRITESTATUS_FAILURE_FRACTION_PROP));
|
||||
}
|
||||
|
||||
public ConsistencyGuardConfig getConsistencyGuardConfig() {
|
||||
return consistencyGuardConfig;
|
||||
}
|
||||
|
||||
public void setConsistencyGuardConfig(ConsistencyGuardConfig consistencyGuardConfig) {
|
||||
this.consistencyGuardConfig = consistencyGuardConfig;
|
||||
}
|
||||
|
||||
public FileSystemViewStorageConfig getViewStorageConfig() {
|
||||
return viewStorageConfig;
|
||||
}
|
||||
|
||||
public void setViewStorageConfig(FileSystemViewStorageConfig viewStorageConfig) {
|
||||
this.viewStorageConfig = viewStorageConfig;
|
||||
}
|
||||
|
||||
public void resetViewStorageConfig() {
|
||||
this.setViewStorageConfig(getClientSpecifiedViewStorageConfig());
|
||||
}
|
||||
|
||||
public FileSystemViewStorageConfig getClientSpecifiedViewStorageConfig() {
|
||||
return clientSpecifiedViewStorageConfig;
|
||||
}
|
||||
|
||||
public static class Builder {
|
||||
|
||||
private final Properties props = new Properties();
|
||||
private boolean isIndexConfigSet = false;
|
||||
private boolean isStorageConfigSet = false;
|
||||
private boolean isCompactionConfigSet = false;
|
||||
private boolean isMetricsConfigSet = false;
|
||||
private boolean isMemoryConfigSet = false;
|
||||
private boolean isViewConfigSet = false;
|
||||
private boolean isConsistencyGuardSet = false;
|
||||
|
||||
public Builder fromFile(File propertiesFile) throws IOException {
|
||||
FileReader reader = new FileReader(propertiesFile);
|
||||
try {
|
||||
this.props.load(reader);
|
||||
return this;
|
||||
} finally {
|
||||
reader.close();
|
||||
}
|
||||
}
|
||||
|
||||
public Builder fromInputStream(InputStream inputStream) throws IOException {
|
||||
try {
|
||||
this.props.load(inputStream);
|
||||
return this;
|
||||
} finally {
|
||||
inputStream.close();
|
||||
}
|
||||
}
|
||||
|
||||
public Builder withProps(Map kvprops) {
|
||||
props.putAll(kvprops);
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder withPath(String basePath) {
|
||||
props.setProperty(BASE_PATH_PROP, basePath);
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder withSchema(String schemaStr) {
|
||||
props.setProperty(AVRO_SCHEMA, schemaStr);
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder forTable(String tableName) {
|
||||
props.setProperty(TABLE_NAME, tableName);
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder withBulkInsertParallelism(int bulkInsertParallelism) {
|
||||
props.setProperty(BULKINSERT_PARALLELISM, String.valueOf(bulkInsertParallelism));
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder withParallelism(int insertShuffleParallelism, int upsertShuffleParallelism) {
|
||||
props.setProperty(INSERT_PARALLELISM, String.valueOf(insertShuffleParallelism));
|
||||
props.setProperty(UPSERT_PARALLELISM, String.valueOf(upsertShuffleParallelism));
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder withWriteBufferLimitBytes(int writeBufferLimit) {
|
||||
props.setProperty(WRITE_BUFFER_LIMIT_BYTES, String.valueOf(writeBufferLimit));
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder combineInput(boolean onInsert, boolean onUpsert) {
|
||||
props.setProperty(COMBINE_BEFORE_INSERT_PROP, String.valueOf(onInsert));
|
||||
props.setProperty(COMBINE_BEFORE_UPSERT_PROP, String.valueOf(onUpsert));
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder withWriteStatusStorageLevel(String level) {
|
||||
props.setProperty(WRITE_STATUS_STORAGE_LEVEL, level);
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder withIndexConfig(HoodieIndexConfig indexConfig) {
|
||||
props.putAll(indexConfig.getProps());
|
||||
isIndexConfigSet = true;
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder withStorageConfig(HoodieStorageConfig storageConfig) {
|
||||
props.putAll(storageConfig.getProps());
|
||||
isStorageConfigSet = true;
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder withCompactionConfig(HoodieCompactionConfig compactionConfig) {
|
||||
props.putAll(compactionConfig.getProps());
|
||||
isCompactionConfigSet = true;
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder withMetricsConfig(HoodieMetricsConfig metricsConfig) {
|
||||
props.putAll(metricsConfig.getProps());
|
||||
isMetricsConfigSet = true;
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder withMemoryConfig(HoodieMemoryConfig memoryConfig) {
|
||||
props.putAll(memoryConfig.getProps());
|
||||
isMemoryConfigSet = true;
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder withAutoCommit(boolean autoCommit) {
|
||||
props.setProperty(HOODIE_AUTO_COMMIT_PROP, String.valueOf(autoCommit));
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder withAssumeDatePartitioning(boolean assumeDatePartitioning) {
|
||||
props.setProperty(HOODIE_ASSUME_DATE_PARTITIONING_PROP,
|
||||
String.valueOf(assumeDatePartitioning));
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder withWriteStatusClass(Class<? extends WriteStatus> writeStatusClass) {
|
||||
props.setProperty(HOODIE_WRITE_STATUS_CLASS_PROP, writeStatusClass.getName());
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder withFileSystemViewConfig(FileSystemViewStorageConfig viewStorageConfig) {
|
||||
props.putAll(viewStorageConfig.getProps());
|
||||
isViewConfigSet = true;
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder withConsistencyGuardConfig(ConsistencyGuardConfig consistencyGuardConfig) {
|
||||
props.putAll(consistencyGuardConfig.getProps());
|
||||
isConsistencyGuardSet = true;
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder withFinalizeWriteParallelism(int parallelism) {
|
||||
props.setProperty(FINALIZE_WRITE_PARALLELISM, String.valueOf(parallelism));
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder withEmbeddedTimelineServerEnabled(boolean enabled) {
|
||||
props.setProperty(EMBEDDED_TIMELINE_SERVER_ENABLED, String.valueOf(enabled));
|
||||
return this;
|
||||
}
|
||||
|
||||
public HoodieWriteConfig build() {
|
||||
// Check for mandatory properties
|
||||
setDefaultOnCondition(props, !props.containsKey(INSERT_PARALLELISM), INSERT_PARALLELISM,
|
||||
DEFAULT_PARALLELISM);
|
||||
setDefaultOnCondition(props, !props.containsKey(BULKINSERT_PARALLELISM),
|
||||
BULKINSERT_PARALLELISM, DEFAULT_PARALLELISM);
|
||||
setDefaultOnCondition(props, !props.containsKey(UPSERT_PARALLELISM), UPSERT_PARALLELISM,
|
||||
DEFAULT_PARALLELISM);
|
||||
setDefaultOnCondition(props, !props.containsKey(COMBINE_BEFORE_INSERT_PROP),
|
||||
COMBINE_BEFORE_INSERT_PROP, DEFAULT_COMBINE_BEFORE_INSERT);
|
||||
setDefaultOnCondition(props, !props.containsKey(COMBINE_BEFORE_UPSERT_PROP),
|
||||
COMBINE_BEFORE_UPSERT_PROP, DEFAULT_COMBINE_BEFORE_UPSERT);
|
||||
setDefaultOnCondition(props, !props.containsKey(WRITE_STATUS_STORAGE_LEVEL),
|
||||
WRITE_STATUS_STORAGE_LEVEL, DEFAULT_WRITE_STATUS_STORAGE_LEVEL);
|
||||
setDefaultOnCondition(props, !props.containsKey(HOODIE_AUTO_COMMIT_PROP),
|
||||
HOODIE_AUTO_COMMIT_PROP, DEFAULT_HOODIE_AUTO_COMMIT);
|
||||
setDefaultOnCondition(props, !props.containsKey(HOODIE_ASSUME_DATE_PARTITIONING_PROP),
|
||||
HOODIE_ASSUME_DATE_PARTITIONING_PROP, DEFAULT_ASSUME_DATE_PARTITIONING);
|
||||
setDefaultOnCondition(props, !props.containsKey(HOODIE_WRITE_STATUS_CLASS_PROP),
|
||||
HOODIE_WRITE_STATUS_CLASS_PROP, DEFAULT_HOODIE_WRITE_STATUS_CLASS);
|
||||
setDefaultOnCondition(props, !props.containsKey(FINALIZE_WRITE_PARALLELISM),
|
||||
FINALIZE_WRITE_PARALLELISM, DEFAULT_FINALIZE_WRITE_PARALLELISM);
|
||||
setDefaultOnCondition(props, !props.containsKey(EMBEDDED_TIMELINE_SERVER_ENABLED),
|
||||
EMBEDDED_TIMELINE_SERVER_ENABLED, DEFAULT_EMBEDDED_TIMELINE_SERVER_ENABLED);
|
||||
setDefaultOnCondition(props, !props.containsKey(INITIAL_CONSISTENCY_CHECK_INTERVAL_MS_PROP),
|
||||
INITIAL_CONSISTENCY_CHECK_INTERVAL_MS_PROP, String.valueOf(DEFAULT_INITIAL_CONSISTENCY_CHECK_INTERVAL_MS));
|
||||
setDefaultOnCondition(props, !props.containsKey(MAX_CONSISTENCY_CHECK_INTERVAL_MS_PROP),
|
||||
MAX_CONSISTENCY_CHECK_INTERVAL_MS_PROP, String.valueOf(DEFAULT_MAX_CONSISTENCY_CHECK_INTERVAL_MS));
|
||||
setDefaultOnCondition(props, !props.containsKey(MAX_CONSISTENCY_CHECKS_PROP),
|
||||
MAX_CONSISTENCY_CHECKS_PROP, String.valueOf(DEFAULT_MAX_CONSISTENCY_CHECKS));
|
||||
setDefaultOnCondition(props, !props.containsKey(FAIL_ON_TIMELINE_ARCHIVING_ENABLED_PROP),
|
||||
FAIL_ON_TIMELINE_ARCHIVING_ENABLED_PROP, DEFAULT_FAIL_ON_TIMELINE_ARCHIVING_ENABLED);
|
||||
|
||||
// Make sure the props is propagated
|
||||
setDefaultOnCondition(props, !isIndexConfigSet,
|
||||
HoodieIndexConfig.newBuilder().fromProperties(props).build());
|
||||
setDefaultOnCondition(props, !isStorageConfigSet,
|
||||
HoodieStorageConfig.newBuilder().fromProperties(props).build());
|
||||
setDefaultOnCondition(props, !isCompactionConfigSet,
|
||||
HoodieCompactionConfig.newBuilder().fromProperties(props).build());
|
||||
setDefaultOnCondition(props, !isMetricsConfigSet,
|
||||
HoodieMetricsConfig.newBuilder().fromProperties(props).build());
|
||||
setDefaultOnCondition(props, !isMemoryConfigSet,
|
||||
HoodieMemoryConfig.newBuilder().fromProperties(props).build());
|
||||
setDefaultOnCondition(props, !isViewConfigSet,
|
||||
FileSystemViewStorageConfig.newBuilder().fromProperties(props).build());
|
||||
setDefaultOnCondition(props, !isConsistencyGuardSet,
|
||||
ConsistencyGuardConfig.newBuilder().fromProperties(props).build());
|
||||
|
||||
// Build WriteConfig at the end
|
||||
HoodieWriteConfig config = new HoodieWriteConfig(props);
|
||||
Preconditions.checkArgument(config.getBasePath() != null);
|
||||
return config;
|
||||
}
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,34 @@
|
||||
/*
|
||||
* 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.exception;
|
||||
|
||||
/**
|
||||
* <p> Exception thrown for any higher level errors when <code>HoodieClient</code> is doing a delta
|
||||
* commit </p>
|
||||
*/
|
||||
public class HoodieAppendException extends HoodieException {
|
||||
|
||||
public HoodieAppendException(String msg, Throwable e) {
|
||||
super(msg, e);
|
||||
}
|
||||
|
||||
public HoodieAppendException(String msg) {
|
||||
super(msg);
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,34 @@
|
||||
/*
|
||||
* 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.exception;
|
||||
|
||||
/**
|
||||
* <p> Exception thrown for any higher level errors when <code>HoodieClient</code> is doing a Commit
|
||||
* </p>
|
||||
*/
|
||||
public class HoodieCommitException extends HoodieException {
|
||||
|
||||
public HoodieCommitException(String msg) {
|
||||
super(msg);
|
||||
}
|
||||
|
||||
public HoodieCommitException(String msg, Throwable e) {
|
||||
super(msg, e);
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,31 @@
|
||||
/*
|
||||
* 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.exception;
|
||||
|
||||
public class HoodieCompactionException extends HoodieException {
|
||||
|
||||
public HoodieCompactionException(String msg) {
|
||||
super(msg);
|
||||
}
|
||||
|
||||
public HoodieCompactionException(String msg, Throwable e) {
|
||||
super(msg, e);
|
||||
}
|
||||
}
|
||||
|
||||
@@ -0,0 +1,36 @@
|
||||
/*
|
||||
* 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.exception;
|
||||
|
||||
|
||||
/**
|
||||
* <p> Exception thrown when dependent system is not available </p>
|
||||
*/
|
||||
public class HoodieDependentSystemUnavailableException extends HoodieException {
|
||||
|
||||
public static final String HBASE = "HBASE";
|
||||
|
||||
public HoodieDependentSystemUnavailableException(String system, String connectURL) {
|
||||
super(getLogMessage(system, connectURL));
|
||||
}
|
||||
|
||||
private static String getLogMessage(String system, String connectURL) {
|
||||
return "System " + system + " unavailable. Tried to connect to " + connectURL;
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,30 @@
|
||||
/*
|
||||
* 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.exception;
|
||||
|
||||
/**
|
||||
* <p> Exception thrown for any higher level errors when <code>HoodieClient</code> is doing a bulk
|
||||
* insert </p>
|
||||
*/
|
||||
public class HoodieInsertException extends HoodieException {
|
||||
|
||||
public HoodieInsertException(String msg, Throwable e) {
|
||||
super(msg, e);
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,30 @@
|
||||
/*
|
||||
* 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.exception;
|
||||
|
||||
public class HoodieRollbackException extends HoodieException {
|
||||
|
||||
public HoodieRollbackException(String msg, Throwable e) {
|
||||
super(msg, e);
|
||||
}
|
||||
|
||||
public HoodieRollbackException(String msg) {
|
||||
super(msg);
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,30 @@
|
||||
/*
|
||||
* 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.exception;
|
||||
|
||||
public class HoodieSavepointException extends HoodieException {
|
||||
|
||||
public HoodieSavepointException(String msg, Throwable e) {
|
||||
super(msg, e);
|
||||
}
|
||||
|
||||
public HoodieSavepointException(String msg) {
|
||||
super(msg);
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,34 @@
|
||||
/*
|
||||
* 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.exception;
|
||||
|
||||
/**
|
||||
* <p> Exception thrown for any higher level errors when <code>HoodieClient</code> is doing a
|
||||
* incremental upsert </p>
|
||||
*/
|
||||
public class HoodieUpsertException extends HoodieException {
|
||||
|
||||
public HoodieUpsertException(String msg, Throwable e) {
|
||||
super(msg, e);
|
||||
}
|
||||
|
||||
public HoodieUpsertException(String msg) {
|
||||
super(msg);
|
||||
}
|
||||
}
|
||||
@@ -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.func;
|
||||
|
||||
import java.util.Iterator;
|
||||
import java.util.List;
|
||||
import org.apache.hudi.WriteStatus;
|
||||
import org.apache.hudi.common.model.HoodieRecord;
|
||||
import org.apache.hudi.common.model.HoodieRecordPayload;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
import org.apache.hudi.table.HoodieTable;
|
||||
import org.apache.spark.api.java.function.Function2;
|
||||
|
||||
|
||||
/**
|
||||
* Map function that handles a sorted stream of HoodieRecords
|
||||
*/
|
||||
public class BulkInsertMapFunction<T extends HoodieRecordPayload> implements
|
||||
Function2<Integer, Iterator<HoodieRecord<T>>, Iterator<List<WriteStatus>>> {
|
||||
|
||||
private String commitTime;
|
||||
private HoodieWriteConfig config;
|
||||
private HoodieTable<T> hoodieTable;
|
||||
private List<String> fileIDPrefixes;
|
||||
|
||||
public BulkInsertMapFunction(String commitTime, HoodieWriteConfig config,
|
||||
HoodieTable<T> hoodieTable, List<String> fileIDPrefixes) {
|
||||
this.commitTime = commitTime;
|
||||
this.config = config;
|
||||
this.hoodieTable = hoodieTable;
|
||||
this.fileIDPrefixes = fileIDPrefixes;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Iterator<List<WriteStatus>> call(Integer partition, Iterator<HoodieRecord<T>> sortedRecordItr) {
|
||||
return new CopyOnWriteLazyInsertIterable<>(sortedRecordItr, config, commitTime, hoodieTable,
|
||||
fileIDPrefixes.get(partition));
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,172 @@
|
||||
/*
|
||||
* 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.func;
|
||||
|
||||
import java.util.ArrayList;
|
||||
import java.util.Iterator;
|
||||
import java.util.List;
|
||||
import java.util.function.Function;
|
||||
import org.apache.avro.Schema;
|
||||
import org.apache.avro.generic.IndexedRecord;
|
||||
import org.apache.hudi.WriteStatus;
|
||||
import org.apache.hudi.common.model.HoodieRecord;
|
||||
import org.apache.hudi.common.model.HoodieRecordPayload;
|
||||
import org.apache.hudi.common.util.Option;
|
||||
import org.apache.hudi.common.util.queue.BoundedInMemoryExecutor;
|
||||
import org.apache.hudi.common.util.queue.BoundedInMemoryQueueConsumer;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
import org.apache.hudi.exception.HoodieException;
|
||||
import org.apache.hudi.io.HoodieCreateHandle;
|
||||
import org.apache.hudi.io.HoodieWriteHandle;
|
||||
import org.apache.hudi.table.HoodieTable;
|
||||
|
||||
/**
|
||||
* Lazy Iterable, that writes a stream of HoodieRecords sorted by the partitionPath, into new
|
||||
* files.
|
||||
*/
|
||||
public class CopyOnWriteLazyInsertIterable<T extends HoodieRecordPayload> extends
|
||||
LazyIterableIterator<HoodieRecord<T>, List<WriteStatus>> {
|
||||
|
||||
protected final HoodieWriteConfig hoodieConfig;
|
||||
protected final String commitTime;
|
||||
protected final HoodieTable<T> hoodieTable;
|
||||
protected final String idPrefix;
|
||||
protected int numFilesWritten;
|
||||
|
||||
public CopyOnWriteLazyInsertIterable(Iterator<HoodieRecord<T>> sortedRecordItr, HoodieWriteConfig config,
|
||||
String commitTime, HoodieTable<T> hoodieTable, String idPrefix) {
|
||||
super(sortedRecordItr);
|
||||
this.hoodieConfig = config;
|
||||
this.commitTime = commitTime;
|
||||
this.hoodieTable = hoodieTable;
|
||||
this.idPrefix = idPrefix;
|
||||
this.numFilesWritten = 0;
|
||||
}
|
||||
|
||||
// Used for caching HoodieRecord along with insertValue. We need this to offload computation work to buffering thread.
|
||||
static class HoodieInsertValueGenResult<T extends HoodieRecord> {
|
||||
public T record;
|
||||
public Option<IndexedRecord> insertValue;
|
||||
// It caches the exception seen while fetching insert value.
|
||||
public Option<Exception> exception = Option.empty();
|
||||
|
||||
public HoodieInsertValueGenResult(T record, Schema schema) {
|
||||
this.record = record;
|
||||
try {
|
||||
this.insertValue = record.getData().getInsertValue(schema);
|
||||
} catch (Exception e) {
|
||||
this.exception = Option.of(e);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Transformer function to help transform a HoodieRecord. This transformer is used by BufferedIterator to offload some
|
||||
* expensive operations of transformation to the reader thread.
|
||||
*/
|
||||
static <T extends HoodieRecordPayload> Function<HoodieRecord<T>,
|
||||
HoodieInsertValueGenResult<HoodieRecord>> getTransformFunction(Schema schema) {
|
||||
return hoodieRecord -> new HoodieInsertValueGenResult(hoodieRecord, schema);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected void start() {
|
||||
}
|
||||
|
||||
@Override
|
||||
protected List<WriteStatus> computeNext() {
|
||||
// Executor service used for launching writer thread.
|
||||
BoundedInMemoryExecutor<HoodieRecord<T>,
|
||||
HoodieInsertValueGenResult<HoodieRecord>, List<WriteStatus>> bufferedIteratorExecutor = null;
|
||||
try {
|
||||
final Schema schema = new Schema.Parser().parse(hoodieConfig.getSchema());
|
||||
bufferedIteratorExecutor =
|
||||
new SparkBoundedInMemoryExecutor<>(hoodieConfig, inputItr,
|
||||
getInsertHandler(), getTransformFunction(schema));
|
||||
final List<WriteStatus> result = bufferedIteratorExecutor.execute();
|
||||
assert result != null && !result.isEmpty() && !bufferedIteratorExecutor.isRemaining();
|
||||
return result;
|
||||
} catch (Exception e) {
|
||||
throw new HoodieException(e);
|
||||
} finally {
|
||||
if (null != bufferedIteratorExecutor) {
|
||||
bufferedIteratorExecutor.shutdownNow();
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
protected void end() {
|
||||
}
|
||||
|
||||
protected String getNextFileId(String idPfx) {
|
||||
return String.format("%s-%d", idPfx, numFilesWritten++);
|
||||
}
|
||||
|
||||
protected CopyOnWriteInsertHandler getInsertHandler() {
|
||||
return new CopyOnWriteInsertHandler();
|
||||
}
|
||||
|
||||
/**
|
||||
* Consumes stream of hoodie records from in-memory queue and
|
||||
* writes to one or more create-handles
|
||||
*/
|
||||
protected class CopyOnWriteInsertHandler extends
|
||||
BoundedInMemoryQueueConsumer<HoodieInsertValueGenResult<HoodieRecord>, List<WriteStatus>> {
|
||||
|
||||
protected final List<WriteStatus> statuses = new ArrayList<>();
|
||||
protected HoodieWriteHandle handle;
|
||||
|
||||
@Override
|
||||
protected void consumeOneRecord(HoodieInsertValueGenResult<HoodieRecord> payload) {
|
||||
final HoodieRecord insertPayload = payload.record;
|
||||
// lazily initialize the handle, for the first time
|
||||
if (handle == null) {
|
||||
handle = new HoodieCreateHandle(hoodieConfig, commitTime, hoodieTable, insertPayload.getPartitionPath(),
|
||||
getNextFileId(idPrefix));
|
||||
}
|
||||
|
||||
if (handle.canWrite(payload.record)) {
|
||||
// write the payload, if the handle has capacity
|
||||
handle.write(insertPayload, payload.insertValue, payload.exception);
|
||||
} else {
|
||||
// handle is full.
|
||||
statuses.add(handle.close());
|
||||
// Need to handle the rejected payload & open new handle
|
||||
handle = new HoodieCreateHandle(hoodieConfig, commitTime, hoodieTable, insertPayload.getPartitionPath(),
|
||||
getNextFileId(idPrefix));
|
||||
handle.write(insertPayload, payload.insertValue, payload.exception); // we should be able to write 1 payload.
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
protected void finish() {
|
||||
if (handle != null) {
|
||||
statuses.add(handle.close());
|
||||
}
|
||||
handle = null;
|
||||
assert statuses.size() > 0;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected List<WriteStatus> getResult() {
|
||||
return statuses;
|
||||
}
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,131 @@
|
||||
/*
|
||||
* 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.func;
|
||||
|
||||
import java.util.Iterator;
|
||||
|
||||
/**
|
||||
* (NOTE: Adapted from Apache SystemML) This class is a generic base class for lazy, single pass
|
||||
* inputItr classes in order to simplify the implementation of lazy iterators for mapPartitions use
|
||||
* cases. Note [SPARK-3369], which gives the reasons for backwards compatibility with regard to the
|
||||
* iterable API despite Spark's single pass nature.
|
||||
* <p>
|
||||
* Provide a way to obtain a inputItr of type O (output), out of an inputItr of type I (input)
|
||||
* <p>
|
||||
* Things to remember: - Assumes Spark calls hasNext() to check for elements, before calling next()
|
||||
* to obtain them - Assumes hasNext() gets called atleast once. - Concrete Implementation is
|
||||
* responsible for calling inputIterator.next() and doing the processing in computeNext()
|
||||
*/
|
||||
public abstract class LazyIterableIterator<I, O> implements Iterable<O>, Iterator<O> {
|
||||
|
||||
protected Iterator<I> inputItr = null;
|
||||
private boolean consumed = false;
|
||||
private boolean startCalled = false;
|
||||
private boolean endCalled = false;
|
||||
|
||||
public LazyIterableIterator(Iterator<I> in) {
|
||||
inputItr = in;
|
||||
}
|
||||
|
||||
/**
|
||||
* Called once, before any elements are processed
|
||||
*/
|
||||
protected abstract void start();
|
||||
|
||||
/**
|
||||
* Block computation to be overwritten by sub classes.
|
||||
*/
|
||||
protected abstract O computeNext();
|
||||
|
||||
|
||||
/**
|
||||
* Called once, after all elements are processed.
|
||||
*/
|
||||
protected abstract void end();
|
||||
|
||||
//////////////////
|
||||
// iterable implementation
|
||||
|
||||
private void invokeStartIfNeeded() {
|
||||
if (!startCalled) {
|
||||
startCalled = true;
|
||||
try {
|
||||
start();
|
||||
} catch (Exception e) {
|
||||
throw new RuntimeException("Error in start()");
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
private void invokeEndIfNeeded() {
|
||||
// make the calls out to begin() & end()
|
||||
if (!endCalled) {
|
||||
endCalled = true;
|
||||
// if we are out of elements, and end has not been called yet
|
||||
try {
|
||||
end();
|
||||
} catch (Exception e) {
|
||||
throw new RuntimeException("Error in end()");
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public Iterator<O> iterator() {
|
||||
//check for consumed inputItr
|
||||
if (consumed) {
|
||||
throw new RuntimeException("Invalid repeated inputItr consumption.");
|
||||
}
|
||||
|
||||
//hand out self as inputItr exactly once (note: do not hand out the input
|
||||
//inputItr since it is consumed by the self inputItr implementation)
|
||||
consumed = true;
|
||||
return this;
|
||||
}
|
||||
|
||||
//////////////////
|
||||
// inputItr implementation
|
||||
|
||||
@Override
|
||||
public boolean hasNext() {
|
||||
boolean ret = inputItr.hasNext();
|
||||
// make sure, there is exactly one call to start()
|
||||
invokeStartIfNeeded();
|
||||
if (!ret) {
|
||||
// if we are out of elements, and end has not been called yet
|
||||
invokeEndIfNeeded();
|
||||
}
|
||||
|
||||
return ret;
|
||||
}
|
||||
|
||||
@Override
|
||||
public O next() {
|
||||
try {
|
||||
return computeNext();
|
||||
} catch (Exception ex) {
|
||||
throw new RuntimeException(ex);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void remove() {
|
||||
throw new RuntimeException("Unsupported remove operation.");
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,72 @@
|
||||
/*
|
||||
* 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.func;
|
||||
|
||||
import java.util.ArrayList;
|
||||
import java.util.Iterator;
|
||||
import java.util.List;
|
||||
import org.apache.hudi.WriteStatus;
|
||||
import org.apache.hudi.common.model.HoodieRecord;
|
||||
import org.apache.hudi.common.model.HoodieRecordPayload;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
import org.apache.hudi.io.HoodieAppendHandle;
|
||||
import org.apache.hudi.table.HoodieTable;
|
||||
|
||||
/**
|
||||
* Lazy Iterable, that writes a stream of HoodieRecords sorted by the partitionPath, into new
|
||||
* log files.
|
||||
*/
|
||||
public class MergeOnReadLazyInsertIterable<T extends HoodieRecordPayload> extends
|
||||
CopyOnWriteLazyInsertIterable<T> {
|
||||
|
||||
public MergeOnReadLazyInsertIterable(Iterator<HoodieRecord<T>> sortedRecordItr, HoodieWriteConfig config,
|
||||
String commitTime, HoodieTable<T> hoodieTable, String idPfx) {
|
||||
super(sortedRecordItr, config, commitTime, hoodieTable, idPfx);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected CopyOnWriteInsertHandler getInsertHandler() {
|
||||
return new MergeOnReadInsertHandler();
|
||||
}
|
||||
|
||||
protected class MergeOnReadInsertHandler extends CopyOnWriteInsertHandler {
|
||||
|
||||
@Override
|
||||
protected void consumeOneRecord(HoodieInsertValueGenResult<HoodieRecord> payload) {
|
||||
final HoodieRecord insertPayload = payload.record;
|
||||
List<WriteStatus> statuses = new ArrayList<>();
|
||||
// lazily initialize the handle, for the first time
|
||||
if (handle == null) {
|
||||
handle = new HoodieAppendHandle(hoodieConfig, commitTime, hoodieTable, getNextFileId(idPrefix));
|
||||
}
|
||||
if (handle.canWrite(insertPayload)) {
|
||||
// write the payload, if the handle has capacity
|
||||
handle.write(insertPayload, payload.insertValue, payload.exception);
|
||||
} else {
|
||||
// handle is full.
|
||||
handle.close();
|
||||
statuses.add(handle.getWriteStatus());
|
||||
// Need to handle the rejected payload & open new handle
|
||||
handle = new HoodieAppendHandle(hoodieConfig, commitTime, hoodieTable, getNextFileId(idPrefix));
|
||||
handle.write(insertPayload, payload.insertValue, payload.exception); // we should be able to write 1 payload.
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
@@ -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.func;
|
||||
|
||||
import java.io.Serializable;
|
||||
import org.apache.hudi.common.util.Option;
|
||||
|
||||
/**
|
||||
* Holds Operation result. Used as a result container for Compaction Admin Client (running as part of Spark-launcher
|
||||
* process) to communicate results back to Hoodie CLI process.
|
||||
*/
|
||||
public class OperationResult<T> implements Serializable {
|
||||
|
||||
private T operation;
|
||||
private boolean executed;
|
||||
private boolean success;
|
||||
private Option<Exception> exception;
|
||||
|
||||
public OperationResult() {
|
||||
}
|
||||
|
||||
public OperationResult(T operation, boolean success, Option<Exception> exception) {
|
||||
this.operation = operation;
|
||||
this.success = success;
|
||||
this.exception = exception;
|
||||
this.executed = true;
|
||||
}
|
||||
|
||||
public OperationResult(T operation, boolean executed, boolean success, Option<Exception> exception) {
|
||||
this.operation = operation;
|
||||
this.success = success;
|
||||
this.exception = exception;
|
||||
this.executed = executed;
|
||||
}
|
||||
|
||||
public T getOperation() {
|
||||
return operation;
|
||||
}
|
||||
|
||||
public boolean isSuccess() {
|
||||
return success;
|
||||
}
|
||||
|
||||
public boolean isExecuted() {
|
||||
return executed;
|
||||
}
|
||||
|
||||
public Option<Exception> getException() {
|
||||
return exception;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
return "OperationResult{"
|
||||
+ "operation=" + operation
|
||||
+ ", executed=" + executed
|
||||
+ ", success=" + success
|
||||
+ ", exception=" + exception
|
||||
+ '}';
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,75 @@
|
||||
/*
|
||||
* 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.func;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.Iterator;
|
||||
import org.apache.hudi.common.util.queue.BoundedInMemoryQueue;
|
||||
import org.apache.hudi.exception.HoodieIOException;
|
||||
import org.apache.parquet.hadoop.ParquetReader;
|
||||
|
||||
/**
|
||||
* This class wraps a parquet reader and provides an iterator based api to
|
||||
* read from a parquet file. This is used in {@link BoundedInMemoryQueue}
|
||||
*/
|
||||
public class ParquetReaderIterator<T> implements Iterator<T> {
|
||||
|
||||
// Parquet reader for an existing parquet file
|
||||
private final ParquetReader<T> parquetReader;
|
||||
// Holds the next entry returned by the parquet reader
|
||||
private T next;
|
||||
|
||||
public ParquetReaderIterator(ParquetReader<T> parquetReader) {
|
||||
this.parquetReader = parquetReader;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean hasNext() {
|
||||
try {
|
||||
// To handle when hasNext() is called multiple times for idempotency and/or the first time
|
||||
if (this.next == null) {
|
||||
this.next = parquetReader.read();
|
||||
}
|
||||
return this.next != null;
|
||||
} catch (IOException io) {
|
||||
throw new HoodieIOException("unable to read next record from parquet file ", io);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public T next() {
|
||||
try {
|
||||
// To handle case when next() is called before hasNext()
|
||||
if (this.next == null) {
|
||||
if (!hasNext()) {
|
||||
throw new HoodieIOException("No more records left to read from parquet file");
|
||||
}
|
||||
}
|
||||
T retVal = this.next;
|
||||
this.next = parquetReader.read();
|
||||
return retVal;
|
||||
} catch (IOException io) {
|
||||
throw new HoodieIOException("unable to read next record from parquet file ", io);
|
||||
}
|
||||
}
|
||||
|
||||
public void close() throws IOException {
|
||||
parquetReader.close();
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,57 @@
|
||||
/*
|
||||
* 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.func;
|
||||
|
||||
import java.util.Iterator;
|
||||
import java.util.function.Function;
|
||||
import org.apache.hudi.common.util.Option;
|
||||
import org.apache.hudi.common.util.queue.BoundedInMemoryExecutor;
|
||||
import org.apache.hudi.common.util.queue.BoundedInMemoryQueueConsumer;
|
||||
import org.apache.hudi.common.util.queue.BoundedInMemoryQueueProducer;
|
||||
import org.apache.hudi.common.util.queue.IteratorBasedQueueProducer;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
import org.apache.spark.TaskContext;
|
||||
import org.apache.spark.TaskContext$;
|
||||
|
||||
public class SparkBoundedInMemoryExecutor<I, O, E> extends BoundedInMemoryExecutor<I, O, E> {
|
||||
|
||||
// Need to set current spark thread's TaskContext into newly launched thread so that new thread can access
|
||||
// TaskContext properties.
|
||||
final TaskContext sparkThreadTaskContext;
|
||||
|
||||
public SparkBoundedInMemoryExecutor(final HoodieWriteConfig hoodieConfig, final Iterator<I> inputItr,
|
||||
BoundedInMemoryQueueConsumer<O, E> consumer,
|
||||
Function<I, O> bufferedIteratorTransform) {
|
||||
this(hoodieConfig, new IteratorBasedQueueProducer<>(inputItr), consumer, bufferedIteratorTransform);
|
||||
}
|
||||
|
||||
public SparkBoundedInMemoryExecutor(final HoodieWriteConfig hoodieConfig,
|
||||
BoundedInMemoryQueueProducer<I> producer,
|
||||
BoundedInMemoryQueueConsumer<O, E> consumer,
|
||||
Function<I, O> bufferedIteratorTransform) {
|
||||
super(hoodieConfig.getWriteBufferLimitBytes(), producer,
|
||||
Option.of(consumer), bufferedIteratorTransform);
|
||||
this.sparkThreadTaskContext = TaskContext.get();
|
||||
}
|
||||
|
||||
public void preExecute() {
|
||||
// Passing parent thread's TaskContext to newly launched thread for it to access original TaskContext properties.
|
||||
TaskContext$.MODULE$.setTaskContext(sparkThreadTaskContext);
|
||||
}
|
||||
}
|
||||
127
hudi-client/src/main/java/org/apache/hudi/index/HoodieIndex.java
Normal file
127
hudi-client/src/main/java/org/apache/hudi/index/HoodieIndex.java
Normal file
@@ -0,0 +1,127 @@
|
||||
/*
|
||||
* 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.index;
|
||||
|
||||
import java.io.Serializable;
|
||||
import org.apache.hudi.WriteStatus;
|
||||
import org.apache.hudi.common.model.FileSlice;
|
||||
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.util.Option;
|
||||
import org.apache.hudi.common.util.collection.Pair;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
import org.apache.hudi.exception.HoodieIndexException;
|
||||
import org.apache.hudi.index.bloom.HoodieBloomIndex;
|
||||
import org.apache.hudi.index.bloom.HoodieGlobalBloomIndex;
|
||||
import org.apache.hudi.index.hbase.HBaseIndex;
|
||||
import org.apache.hudi.table.HoodieTable;
|
||||
import org.apache.spark.api.java.JavaPairRDD;
|
||||
import org.apache.spark.api.java.JavaRDD;
|
||||
import org.apache.spark.api.java.JavaSparkContext;
|
||||
|
||||
/**
|
||||
* Base class for different types of indexes to determine the mapping from uuid
|
||||
*/
|
||||
public abstract class HoodieIndex<T extends HoodieRecordPayload> implements Serializable {
|
||||
|
||||
protected final HoodieWriteConfig config;
|
||||
|
||||
protected HoodieIndex(HoodieWriteConfig config) {
|
||||
this.config = config;
|
||||
}
|
||||
|
||||
|
||||
public static <T extends HoodieRecordPayload> HoodieIndex<T> createIndex(HoodieWriteConfig config,
|
||||
JavaSparkContext jsc) throws HoodieIndexException {
|
||||
switch (config.getIndexType()) {
|
||||
case HBASE:
|
||||
return new HBaseIndex<>(config);
|
||||
case INMEMORY:
|
||||
return new InMemoryHashIndex<>(config);
|
||||
case BLOOM:
|
||||
return new HoodieBloomIndex<>(config);
|
||||
case GLOBAL_BLOOM:
|
||||
return new HoodieGlobalBloomIndex<>(config);
|
||||
default:
|
||||
throw new HoodieIndexException("Index type unspecified, set " + config.getIndexType());
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Checks if the given [Keys] exists in the hoodie table and returns [Key, Option[partitionPath, fileID]]
|
||||
* If the optional is empty, then the key is not found.
|
||||
*/
|
||||
public abstract JavaPairRDD<HoodieKey, Option<Pair<String, String>>> fetchRecordLocation(
|
||||
JavaRDD<HoodieKey> hoodieKeys, final JavaSparkContext jsc, HoodieTable<T> hoodieTable);
|
||||
|
||||
/**
|
||||
* Looks up the index and tags each incoming record with a location of a file that contains the
|
||||
* row (if it is actually present)
|
||||
*/
|
||||
public abstract JavaRDD<HoodieRecord<T>> tagLocation(JavaRDD<HoodieRecord<T>> recordRDD,
|
||||
JavaSparkContext jsc, HoodieTable<T> hoodieTable) throws HoodieIndexException;
|
||||
|
||||
/**
|
||||
* Extracts the location of written records, and updates the index.
|
||||
* <p>
|
||||
* TODO(vc): We may need to propagate the record as well in a WriteStatus class
|
||||
*/
|
||||
public abstract JavaRDD<WriteStatus> updateLocation(JavaRDD<WriteStatus> writeStatusRDD, JavaSparkContext jsc,
|
||||
HoodieTable<T> hoodieTable)
|
||||
throws HoodieIndexException;
|
||||
|
||||
/**
|
||||
* Rollback the efffects of the commit made at commitTime.
|
||||
*/
|
||||
public abstract boolean rollbackCommit(String commitTime);
|
||||
|
||||
/**
|
||||
* An index is `global` if {@link HoodieKey} to fileID mapping, does not depend on the
|
||||
* `partitionPath`. Such an implementation is able to obtain the same mapping, for two hoodie keys
|
||||
* with same `recordKey` but different `partitionPath`
|
||||
*
|
||||
* @return whether or not, the index implementation is global in nature
|
||||
*/
|
||||
public abstract boolean isGlobal();
|
||||
|
||||
/**
|
||||
* This is used by storage to determine, if its safe to send inserts, straight to the log, i.e
|
||||
* having a {@link FileSlice}, with no data file.
|
||||
*
|
||||
* @return Returns true/false depending on whether the impl has this capability
|
||||
*/
|
||||
public abstract boolean canIndexLogFiles();
|
||||
|
||||
|
||||
/**
|
||||
* An index is "implicit" with respect to storage, if just writing new data to a file slice,
|
||||
* updates the index as well. This is used by storage, to save memory footprint in certain cases.
|
||||
*/
|
||||
public abstract boolean isImplicitWithStorage();
|
||||
|
||||
/**
|
||||
* Each index type should implement it's own logic to release any resources acquired during the process.
|
||||
*/
|
||||
public void close() {}
|
||||
|
||||
public enum IndexType {
|
||||
HBASE, INMEMORY, BLOOM, GLOBAL_BLOOM
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,142 @@
|
||||
/*
|
||||
* 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.index;
|
||||
|
||||
import java.util.ArrayList;
|
||||
import java.util.Iterator;
|
||||
import java.util.List;
|
||||
import java.util.concurrent.ConcurrentHashMap;
|
||||
import java.util.concurrent.ConcurrentMap;
|
||||
import org.apache.hudi.WriteStatus;
|
||||
import org.apache.hudi.common.model.HoodieKey;
|
||||
import org.apache.hudi.common.model.HoodieRecord;
|
||||
import org.apache.hudi.common.model.HoodieRecordLocation;
|
||||
import org.apache.hudi.common.model.HoodieRecordPayload;
|
||||
import org.apache.hudi.common.util.Option;
|
||||
import org.apache.hudi.common.util.collection.Pair;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
import org.apache.hudi.table.HoodieTable;
|
||||
import org.apache.spark.api.java.JavaPairRDD;
|
||||
import org.apache.spark.api.java.JavaRDD;
|
||||
import org.apache.spark.api.java.JavaSparkContext;
|
||||
import org.apache.spark.api.java.function.Function;
|
||||
import org.apache.spark.api.java.function.Function2;
|
||||
|
||||
|
||||
/**
|
||||
* Hoodie Index implementation backed by an in-memory Hash map. <p> ONLY USE FOR LOCAL TESTING
|
||||
*/
|
||||
public class InMemoryHashIndex<T extends HoodieRecordPayload> extends HoodieIndex<T> {
|
||||
|
||||
private static ConcurrentMap<HoodieKey, HoodieRecordLocation> recordLocationMap;
|
||||
|
||||
public InMemoryHashIndex(HoodieWriteConfig config) {
|
||||
super(config);
|
||||
synchronized (InMemoryHashIndex.class) {
|
||||
if (recordLocationMap == null) {
|
||||
recordLocationMap = new ConcurrentHashMap<>();
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public JavaPairRDD<HoodieKey, Option<Pair<String, String>>> fetchRecordLocation(JavaRDD<HoodieKey> hoodieKeys,
|
||||
JavaSparkContext jsc, HoodieTable<T> hoodieTable) {
|
||||
throw new UnsupportedOperationException("InMemory index does not implement check exist yet");
|
||||
}
|
||||
|
||||
@Override
|
||||
public JavaRDD<HoodieRecord<T>> tagLocation(JavaRDD<HoodieRecord<T>> recordRDD, JavaSparkContext jsc,
|
||||
HoodieTable<T> hoodieTable) {
|
||||
return recordRDD.mapPartitionsWithIndex(this.new LocationTagFunction(), true);
|
||||
}
|
||||
|
||||
@Override
|
||||
public JavaRDD<WriteStatus> updateLocation(JavaRDD<WriteStatus> writeStatusRDD, JavaSparkContext jsc,
|
||||
HoodieTable<T> hoodieTable) {
|
||||
return writeStatusRDD.map(new Function<WriteStatus, WriteStatus>() {
|
||||
@Override
|
||||
public WriteStatus call(WriteStatus writeStatus) {
|
||||
for (HoodieRecord record : writeStatus.getWrittenRecords()) {
|
||||
if (!writeStatus.isErrored(record.getKey())) {
|
||||
HoodieKey key = record.getKey();
|
||||
Option<HoodieRecordLocation> newLocation = record.getNewLocation();
|
||||
if (newLocation.isPresent()) {
|
||||
recordLocationMap.put(key, newLocation.get());
|
||||
} else {
|
||||
//Delete existing index for a deleted record
|
||||
recordLocationMap.remove(key);
|
||||
}
|
||||
}
|
||||
}
|
||||
return writeStatus;
|
||||
}
|
||||
});
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean rollbackCommit(String commitTime) {
|
||||
return true;
|
||||
}
|
||||
|
||||
/**
|
||||
* Only looks up by recordKey
|
||||
*/
|
||||
@Override
|
||||
public boolean isGlobal() {
|
||||
return true;
|
||||
}
|
||||
|
||||
/**
|
||||
* Mapping is available in HBase already.
|
||||
*/
|
||||
@Override
|
||||
public boolean canIndexLogFiles() {
|
||||
return true;
|
||||
}
|
||||
|
||||
/**
|
||||
* Index needs to be explicitly updated after storage write.
|
||||
*/
|
||||
@Override
|
||||
public boolean isImplicitWithStorage() {
|
||||
return false;
|
||||
}
|
||||
|
||||
/**
|
||||
* Function that tags each HoodieRecord with an existing location, if known.
|
||||
*/
|
||||
class LocationTagFunction implements
|
||||
Function2<Integer, Iterator<HoodieRecord<T>>, Iterator<HoodieRecord<T>>> {
|
||||
|
||||
@Override
|
||||
public Iterator<HoodieRecord<T>> call(Integer partitionNum,
|
||||
Iterator<HoodieRecord<T>> hoodieRecordIterator) {
|
||||
List<HoodieRecord<T>> taggedRecords = new ArrayList<>();
|
||||
while (hoodieRecordIterator.hasNext()) {
|
||||
HoodieRecord<T> rec = hoodieRecordIterator.next();
|
||||
if (recordLocationMap.containsKey(rec.getKey())) {
|
||||
rec.setCurrentLocation(recordLocationMap.get(rec.getKey()));
|
||||
}
|
||||
taggedRecords.add(rec);
|
||||
}
|
||||
return taggedRecords.iterator();
|
||||
}
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,98 @@
|
||||
/*
|
||||
* 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.index.bloom;
|
||||
|
||||
import com.google.common.base.Objects;
|
||||
import java.io.Serializable;
|
||||
|
||||
/**
|
||||
* Metadata about a given file group, useful for index lookup
|
||||
*/
|
||||
public class BloomIndexFileInfo implements Serializable {
|
||||
|
||||
private final String fileId;
|
||||
|
||||
private final String minRecordKey;
|
||||
|
||||
private final String maxRecordKey;
|
||||
|
||||
public BloomIndexFileInfo(String fileId, String minRecordKey, String maxRecordKey) {
|
||||
this.fileId = fileId;
|
||||
this.minRecordKey = minRecordKey;
|
||||
this.maxRecordKey = maxRecordKey;
|
||||
}
|
||||
|
||||
public BloomIndexFileInfo(String fileId) {
|
||||
this.fileId = fileId;
|
||||
this.minRecordKey = null;
|
||||
this.maxRecordKey = null;
|
||||
}
|
||||
|
||||
public String getFileId() {
|
||||
return fileId;
|
||||
}
|
||||
|
||||
public String getMinRecordKey() {
|
||||
return minRecordKey;
|
||||
}
|
||||
|
||||
public String getMaxRecordKey() {
|
||||
return maxRecordKey;
|
||||
}
|
||||
|
||||
public boolean hasKeyRanges() {
|
||||
return minRecordKey != null && maxRecordKey != null;
|
||||
}
|
||||
|
||||
/**
|
||||
* Does the given key fall within the range (inclusive)
|
||||
*/
|
||||
public boolean isKeyInRange(String recordKey) {
|
||||
return minRecordKey.compareTo(recordKey) <= 0 && maxRecordKey.compareTo(recordKey) >= 0;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean equals(Object o) {
|
||||
if (this == o) {
|
||||
return true;
|
||||
}
|
||||
if (o == null || getClass() != o.getClass()) {
|
||||
return false;
|
||||
}
|
||||
|
||||
BloomIndexFileInfo that = (BloomIndexFileInfo) o;
|
||||
return Objects.equal(that.fileId, fileId) && Objects.equal(that.minRecordKey, minRecordKey)
|
||||
&& Objects.equal(that.maxRecordKey, maxRecordKey);
|
||||
|
||||
}
|
||||
|
||||
@Override
|
||||
public int hashCode() {
|
||||
return Objects.hashCode(fileId, minRecordKey, maxRecordKey);
|
||||
}
|
||||
|
||||
public String toString() {
|
||||
final StringBuilder sb = new StringBuilder("BloomIndexFileInfo {");
|
||||
sb.append(" fileId=").append(fileId);
|
||||
sb.append(" minRecordKey=").append(minRecordKey);
|
||||
sb.append(" maxRecordKey=").append(maxRecordKey);
|
||||
sb.append('}');
|
||||
return sb.toString();
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,156 @@
|
||||
/*
|
||||
* 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.index.bloom;
|
||||
|
||||
import com.google.common.annotations.VisibleForTesting;
|
||||
import com.google.common.hash.Hashing;
|
||||
import java.nio.charset.StandardCharsets;
|
||||
import java.util.ArrayList;
|
||||
import java.util.HashMap;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.concurrent.atomic.AtomicInteger;
|
||||
import org.apache.hudi.common.util.collection.Pair;
|
||||
import org.apache.log4j.LogManager;
|
||||
import org.apache.log4j.Logger;
|
||||
import org.apache.spark.Partitioner;
|
||||
|
||||
/**
|
||||
* Partitions bloom filter checks by spreading out comparisons across buckets of work.
|
||||
*
|
||||
* Each bucket incurs the following cost
|
||||
* <pre>
|
||||
* 1) Read bloom filter from file footer
|
||||
* 2) Check keys against bloom filter
|
||||
* 3) [Conditional] If any key had a hit, open file and check
|
||||
* </pre>
|
||||
*
|
||||
* The partitioner performs a two phase bin packing algorithm, to pack enough work into each bucket such that cost of
|
||||
* (1) & (3) is amortized. Also, avoids any skews in the sort based approach, by directly partitioning by the file to be
|
||||
* checked against and ensuring each partition has similar number of buckets. Performance tests show that this approach
|
||||
* could bound the amount of skew to std_dev(numberOfBucketsPerPartition) * cost of (3), lower than sort partitioning.
|
||||
*
|
||||
* Approach has two goals :
|
||||
* <pre>
|
||||
* 1) Pack as many buckets from same file group into same partition, to amortize cost of (1) and (2) further
|
||||
* 2) Spread buckets across partitions evenly to achieve skew reduction
|
||||
* </pre>
|
||||
*/
|
||||
public class BucketizedBloomCheckPartitioner extends Partitioner {
|
||||
|
||||
private static Logger logger = LogManager.getLogger(BucketizedBloomCheckPartitioner.class);
|
||||
|
||||
private int partitions;
|
||||
|
||||
/**
|
||||
* Stores the final mapping of a file group to a list of partitions for its keys.
|
||||
*/
|
||||
private Map<String, List<Integer>> fileGroupToPartitions;
|
||||
|
||||
/**
|
||||
* Create a partitioner that computes a plan based on provided workload characteristics.
|
||||
*
|
||||
* @param targetPartitions maximum number of partitions to target
|
||||
* @param fileGroupToComparisons number of expected comparisons per file group
|
||||
* @param keysPerBucket maximum number of keys to pack in a single bucket
|
||||
*/
|
||||
public BucketizedBloomCheckPartitioner(int targetPartitions, Map<String, Long> fileGroupToComparisons,
|
||||
int keysPerBucket) {
|
||||
this.fileGroupToPartitions = new HashMap<>();
|
||||
|
||||
Map<String, Integer> bucketsPerFileGroup = new HashMap<>();
|
||||
// Compute the buckets needed per file group, using simple uniform distribution
|
||||
fileGroupToComparisons.forEach((f, c) ->
|
||||
bucketsPerFileGroup.put(f, (int) Math.ceil((c * 1.0) / keysPerBucket)));
|
||||
int totalBuckets = bucketsPerFileGroup.values().stream().mapToInt(i -> i).sum();
|
||||
// If totalBuckets > targetPartitions, no need to have extra partitions
|
||||
this.partitions = Math.min(targetPartitions, totalBuckets);
|
||||
|
||||
// PHASE 1 : start filling upto minimum number of buckets into partitions, taking all but one bucket from each file
|
||||
// This tries to first optimize for goal 1 above, with knowledge that each partition needs a certain minimum number
|
||||
// of buckets and assigns buckets in the same order as file groups. If we were to simply round robin, then buckets
|
||||
// for a file group is more or less guaranteed to be placed on different partitions all the time.
|
||||
int minBucketsPerPartition = Math.max((int) Math.floor((1.0 * totalBuckets) / partitions), 1);
|
||||
logger.info(String.format("TotalBuckets %d, min_buckets/partition %d", totalBuckets, minBucketsPerPartition));
|
||||
int[] bucketsFilled = new int[partitions];
|
||||
Map<String, AtomicInteger> bucketsFilledPerFileGroup = new HashMap<>();
|
||||
int partitionIndex = 0;
|
||||
for (Map.Entry<String, Integer> e : bucketsPerFileGroup.entrySet()) {
|
||||
for (int b = 0; b < Math.max(1, e.getValue() - 1); b++) {
|
||||
// keep filled counts upto date
|
||||
bucketsFilled[partitionIndex]++;
|
||||
AtomicInteger cnt = bucketsFilledPerFileGroup.getOrDefault(e.getKey(), new AtomicInteger(0));
|
||||
cnt.incrementAndGet();
|
||||
bucketsFilledPerFileGroup.put(e.getKey(), cnt);
|
||||
|
||||
// mark this partition against the file group
|
||||
List<Integer> partitionList = this.fileGroupToPartitions.getOrDefault(e.getKey(), new ArrayList<>());
|
||||
partitionList.add(partitionIndex);
|
||||
this.fileGroupToPartitions.put(e.getKey(), partitionList);
|
||||
|
||||
// switch to new partition if needed
|
||||
if (bucketsFilled[partitionIndex] >= minBucketsPerPartition) {
|
||||
partitionIndex = (partitionIndex + 1) % partitions;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
// PHASE 2 : for remaining unassigned buckets, round robin over partitions once. Since we withheld 1 bucket from
|
||||
// each file group uniformly, this remaining is also an uniform mix across file groups. We just round robin to
|
||||
// optimize for goal 2.
|
||||
for (Map.Entry<String, Integer> e : bucketsPerFileGroup.entrySet()) {
|
||||
int remaining = e.getValue() - bucketsFilledPerFileGroup.get(e.getKey()).intValue();
|
||||
for (int r = 0; r < remaining; r++) {
|
||||
// mark this partition against the file group
|
||||
this.fileGroupToPartitions.get(e.getKey()).add(partitionIndex);
|
||||
bucketsFilled[partitionIndex]++;
|
||||
partitionIndex = (partitionIndex + 1) % partitions;
|
||||
}
|
||||
}
|
||||
|
||||
if (logger.isDebugEnabled()) {
|
||||
logger.debug("Partitions assigned per file groups :" + fileGroupToPartitions);
|
||||
StringBuilder str = new StringBuilder();
|
||||
for (int i = 0; i < bucketsFilled.length; i++) {
|
||||
str.append("p" + i + " : " + bucketsFilled[i] + ",");
|
||||
}
|
||||
logger.debug("Num buckets assigned per file group :" + str);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public int numPartitions() {
|
||||
return partitions;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int getPartition(Object key) {
|
||||
final Pair<String, String> parts = (Pair<String, String>) key;
|
||||
final long hashOfKey = Hashing.md5().hashString(parts.getRight(), StandardCharsets.UTF_8).asLong();
|
||||
final List<Integer> candidatePartitions = fileGroupToPartitions.get(parts.getLeft());
|
||||
final int idx = (int) Math.floorMod(hashOfKey, candidatePartitions.size());
|
||||
assert idx >= 0;
|
||||
return candidatePartitions.get(idx);
|
||||
}
|
||||
|
||||
@VisibleForTesting
|
||||
Map<String, List<Integer>> getFileGroupToPartitions() {
|
||||
return fileGroupToPartitions;
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,397 @@
|
||||
/*
|
||||
* 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.index.bloom;
|
||||
|
||||
import static java.util.stream.Collectors.groupingBy;
|
||||
import static java.util.stream.Collectors.mapping;
|
||||
import static java.util.stream.Collectors.toList;
|
||||
|
||||
import com.google.common.annotations.VisibleForTesting;
|
||||
import java.util.ArrayList;
|
||||
import java.util.HashMap;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.stream.Collectors;
|
||||
import org.apache.hudi.WriteStatus;
|
||||
import org.apache.hudi.common.model.HoodieKey;
|
||||
import org.apache.hudi.common.model.HoodieRecord;
|
||||
import org.apache.hudi.common.model.HoodieRecordLocation;
|
||||
import org.apache.hudi.common.model.HoodieRecordPayload;
|
||||
import org.apache.hudi.common.table.timeline.HoodieInstant;
|
||||
import org.apache.hudi.common.util.Option;
|
||||
import org.apache.hudi.common.util.collection.Pair;
|
||||
import org.apache.hudi.config.HoodieIndexConfig;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
import org.apache.hudi.exception.MetadataNotFoundException;
|
||||
import org.apache.hudi.index.HoodieIndex;
|
||||
import org.apache.hudi.io.HoodieRangeInfoHandle;
|
||||
import org.apache.hudi.table.HoodieTable;
|
||||
import org.apache.log4j.LogManager;
|
||||
import org.apache.log4j.Logger;
|
||||
import org.apache.spark.Partitioner;
|
||||
import org.apache.spark.api.java.JavaPairRDD;
|
||||
import org.apache.spark.api.java.JavaRDD;
|
||||
import org.apache.spark.api.java.JavaSparkContext;
|
||||
import org.apache.spark.storage.StorageLevel;
|
||||
import scala.Tuple2;
|
||||
|
||||
/**
|
||||
* Indexing mechanism based on bloom filter. Each parquet file includes its row_key bloom filter in its metadata.
|
||||
*/
|
||||
public class HoodieBloomIndex<T extends HoodieRecordPayload> extends HoodieIndex<T> {
|
||||
|
||||
// we need to limit the join such that it stays within 1.5GB per Spark partition. (SPARK-1476)
|
||||
private static final int SPARK_MAXIMUM_BYTES_PER_PARTITION = 1500 * 1024 * 1024;
|
||||
// this is how much a triplet of (partitionPath, fileId, recordKey) costs.
|
||||
private static final int BYTES_PER_PARTITION_FILE_KEY_TRIPLET = 300;
|
||||
private static Logger logger = LogManager.getLogger(HoodieBloomIndex.class);
|
||||
private static int MAX_ITEMS_PER_SHUFFLE_PARTITION =
|
||||
SPARK_MAXIMUM_BYTES_PER_PARTITION / BYTES_PER_PARTITION_FILE_KEY_TRIPLET;
|
||||
|
||||
public HoodieBloomIndex(HoodieWriteConfig config) {
|
||||
super(config);
|
||||
}
|
||||
|
||||
@Override
|
||||
public JavaRDD<HoodieRecord<T>> tagLocation(JavaRDD<HoodieRecord<T>> recordRDD, JavaSparkContext jsc,
|
||||
HoodieTable<T> hoodieTable) {
|
||||
|
||||
// Step 0: cache the input record RDD
|
||||
if (config.getBloomIndexUseCaching()) {
|
||||
recordRDD.persist(config.getBloomIndexInputStorageLevel());
|
||||
}
|
||||
|
||||
// Step 1: Extract out thinner JavaPairRDD of (partitionPath, recordKey)
|
||||
JavaPairRDD<String, String> partitionRecordKeyPairRDD = recordRDD
|
||||
.mapToPair(record -> new Tuple2<>(record.getPartitionPath(), record.getRecordKey()));
|
||||
|
||||
// Lookup indexes for all the partition/recordkey pair
|
||||
JavaPairRDD<HoodieKey, HoodieRecordLocation> keyFilenamePairRDD = lookupIndex(partitionRecordKeyPairRDD, jsc,
|
||||
hoodieTable);
|
||||
|
||||
// Cache the result, for subsequent stages.
|
||||
if (config.getBloomIndexUseCaching()) {
|
||||
keyFilenamePairRDD.persist(StorageLevel.MEMORY_AND_DISK_SER());
|
||||
}
|
||||
if (logger.isDebugEnabled()) {
|
||||
long totalTaggedRecords = keyFilenamePairRDD.count();
|
||||
logger.debug("Number of update records (ones tagged with a fileID): " + totalTaggedRecords);
|
||||
}
|
||||
|
||||
// Step 4: Tag the incoming records, as inserts or updates, by joining with existing record keys
|
||||
// Cost: 4 sec.
|
||||
JavaRDD<HoodieRecord<T>> taggedRecordRDD = tagLocationBacktoRecords(keyFilenamePairRDD,
|
||||
recordRDD);
|
||||
|
||||
if (config.getBloomIndexUseCaching()) {
|
||||
recordRDD.unpersist(); // unpersist the input Record RDD
|
||||
keyFilenamePairRDD.unpersist();
|
||||
}
|
||||
|
||||
return taggedRecordRDD;
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns an RDD mapping each HoodieKey with a partitionPath/fileID which contains it. Option.Empty if the key is
|
||||
* not found.
|
||||
*
|
||||
* @param hoodieKeys keys to lookup
|
||||
* @param jsc spark context
|
||||
* @param hoodieTable hoodie table object
|
||||
*/
|
||||
@Override
|
||||
public JavaPairRDD<HoodieKey, Option<Pair<String, String>>> fetchRecordLocation(JavaRDD<HoodieKey> hoodieKeys,
|
||||
JavaSparkContext jsc, HoodieTable<T> hoodieTable) {
|
||||
JavaPairRDD<String, String> partitionRecordKeyPairRDD = hoodieKeys
|
||||
.mapToPair(key -> new Tuple2<>(key.getPartitionPath(), key.getRecordKey()));
|
||||
|
||||
// Lookup indexes for all the partition/recordkey pair
|
||||
JavaPairRDD<HoodieKey, HoodieRecordLocation> recordKeyLocationRDD = lookupIndex(partitionRecordKeyPairRDD, jsc,
|
||||
hoodieTable);
|
||||
JavaPairRDD<HoodieKey, String> keyHoodieKeyPairRDD = hoodieKeys.mapToPair(key -> new Tuple2<>(key, null));
|
||||
|
||||
return keyHoodieKeyPairRDD.leftOuterJoin(recordKeyLocationRDD).mapToPair(keyLoc -> {
|
||||
Option<Pair<String, String>> partitionPathFileidPair;
|
||||
if (keyLoc._2._2.isPresent()) {
|
||||
partitionPathFileidPair = Option.of(Pair.of(keyLoc._1().getPartitionPath(), keyLoc._2._2.get().getFileId()));
|
||||
} else {
|
||||
partitionPathFileidPair = Option.empty();
|
||||
}
|
||||
return new Tuple2<>(keyLoc._1, partitionPathFileidPair);
|
||||
});
|
||||
}
|
||||
|
||||
/**
|
||||
* Lookup the location for each record key and return the pair<record_key,location> for all record keys already
|
||||
* present and drop the record keys if not present
|
||||
*/
|
||||
private JavaPairRDD<HoodieKey, HoodieRecordLocation> lookupIndex(
|
||||
JavaPairRDD<String, String> partitionRecordKeyPairRDD, final JavaSparkContext jsc,
|
||||
final HoodieTable hoodieTable) {
|
||||
// Obtain records per partition, in the incoming records
|
||||
Map<String, Long> recordsPerPartition = partitionRecordKeyPairRDD.countByKey();
|
||||
List<String> affectedPartitionPathList = new ArrayList<>(recordsPerPartition.keySet());
|
||||
|
||||
// Step 2: Load all involved files as <Partition, filename> pairs
|
||||
List<Tuple2<String, BloomIndexFileInfo>> fileInfoList = loadInvolvedFiles(affectedPartitionPathList, jsc,
|
||||
hoodieTable);
|
||||
final Map<String, List<BloomIndexFileInfo>> partitionToFileInfo = fileInfoList.stream()
|
||||
.collect(groupingBy(Tuple2::_1, mapping(Tuple2::_2, toList())));
|
||||
|
||||
// Step 3: Obtain a RDD, for each incoming record, that already exists, with the file id,
|
||||
// that contains it.
|
||||
Map<String, Long> comparisonsPerFileGroup = computeComparisonsPerFileGroup(recordsPerPartition, partitionToFileInfo,
|
||||
partitionRecordKeyPairRDD);
|
||||
int safeParallelism = computeSafeParallelism(recordsPerPartition, comparisonsPerFileGroup);
|
||||
int joinParallelism = determineParallelism(partitionRecordKeyPairRDD.partitions().size(), safeParallelism);
|
||||
return findMatchingFilesForRecordKeys(partitionToFileInfo, partitionRecordKeyPairRDD, joinParallelism,
|
||||
hoodieTable, comparisonsPerFileGroup);
|
||||
}
|
||||
|
||||
/**
|
||||
* Compute the estimated number of bloom filter comparisons to be performed on each file group
|
||||
*/
|
||||
private Map<String, Long> computeComparisonsPerFileGroup(final Map<String, Long> recordsPerPartition,
|
||||
final Map<String, List<BloomIndexFileInfo>> partitionToFileInfo,
|
||||
JavaPairRDD<String, String> partitionRecordKeyPairRDD) {
|
||||
|
||||
Map<String, Long> fileToComparisons;
|
||||
if (config.getBloomIndexPruneByRanges()) {
|
||||
// we will just try exploding the input and then count to determine comparisons
|
||||
// FIX(vc): Only do sampling here and extrapolate?
|
||||
fileToComparisons = explodeRecordRDDWithFileComparisons(partitionToFileInfo,
|
||||
partitionRecordKeyPairRDD).mapToPair(t -> t).countByKey();
|
||||
} else {
|
||||
fileToComparisons = new HashMap<>();
|
||||
partitionToFileInfo.entrySet().stream().forEach(e -> {
|
||||
for (BloomIndexFileInfo fileInfo : e.getValue()) {
|
||||
//each file needs to be compared against all the records coming into the partition
|
||||
fileToComparisons.put(fileInfo.getFileId(), recordsPerPartition.get(e.getKey()));
|
||||
}
|
||||
});
|
||||
}
|
||||
return fileToComparisons;
|
||||
}
|
||||
|
||||
/**
|
||||
* Compute the minimum parallelism needed to play well with the spark 2GB limitation.. The index lookup can be skewed
|
||||
* in three dimensions : #files, #partitions, #records <p> To be able to smoothly handle skews, we need to compute how
|
||||
* to split each partitions into subpartitions. We do it here, in a way that keeps the amount of each Spark join
|
||||
* partition to < 2GB. <p> If {@link HoodieIndexConfig#BLOOM_INDEX_PARALLELISM_PROP} is
|
||||
* specified as a NON-zero number, then that is used explicitly.
|
||||
*/
|
||||
int computeSafeParallelism(Map<String, Long> recordsPerPartition, Map<String, Long> comparisonsPerFileGroup) {
|
||||
long totalComparisons = comparisonsPerFileGroup.values().stream().mapToLong(Long::longValue).sum();
|
||||
long totalFiles = comparisonsPerFileGroup.size();
|
||||
long totalRecords = recordsPerPartition.values().stream().mapToLong(Long::longValue).sum();
|
||||
int parallelism = (int) (totalComparisons / MAX_ITEMS_PER_SHUFFLE_PARTITION + 1);
|
||||
logger.info(String.format("TotalRecords %d, TotalFiles %d, TotalAffectedPartitions %d, TotalComparisons %d, "
|
||||
+ "SafeParallelism %d", totalRecords, totalFiles, recordsPerPartition.size(), totalComparisons, parallelism));
|
||||
return parallelism;
|
||||
}
|
||||
|
||||
/**
|
||||
* Its crucial to pick the right parallelism. <p> totalSubPartitions : this is deemed safe limit, to be nice with
|
||||
* Spark. inputParallelism : typically number of input file splits <p> We pick the max such that, we are always safe,
|
||||
* but go higher if say a there are a lot of input files. (otherwise, we will fallback to number of partitions in
|
||||
* input and end up with slow performance)
|
||||
*/
|
||||
private int determineParallelism(int inputParallelism, int totalSubPartitions) {
|
||||
// If bloom index parallelism is set, use it to to check against the input parallelism and
|
||||
// take the max
|
||||
int indexParallelism = Math.max(inputParallelism, config.getBloomIndexParallelism());
|
||||
int joinParallelism = Math.max(totalSubPartitions, indexParallelism);
|
||||
logger.info("InputParallelism: ${" + inputParallelism + "}, " + "IndexParallelism: ${" + config
|
||||
.getBloomIndexParallelism() + "}, " + "TotalSubParts: ${" + totalSubPartitions + "}, "
|
||||
+ "Join Parallelism set to : " + joinParallelism);
|
||||
return joinParallelism;
|
||||
}
|
||||
|
||||
/**
|
||||
* Load all involved files as <Partition, filename> pair RDD.
|
||||
*/
|
||||
@VisibleForTesting
|
||||
List<Tuple2<String, BloomIndexFileInfo>> loadInvolvedFiles(List<String> partitions, final JavaSparkContext jsc,
|
||||
final HoodieTable hoodieTable) {
|
||||
|
||||
// Obtain the latest data files from all the partitions.
|
||||
List<Pair<String, String>> partitionPathFileIDList = jsc
|
||||
.parallelize(partitions, Math.max(partitions.size(), 1))
|
||||
.flatMap(partitionPath -> {
|
||||
Option<HoodieInstant> latestCommitTime = hoodieTable.getMetaClient().getCommitsTimeline()
|
||||
.filterCompletedInstants().lastInstant();
|
||||
List<Pair<String, String>> filteredFiles = new ArrayList<>();
|
||||
if (latestCommitTime.isPresent()) {
|
||||
filteredFiles = hoodieTable.getROFileSystemView()
|
||||
.getLatestDataFilesBeforeOrOn(partitionPath, latestCommitTime.get().getTimestamp())
|
||||
.map(f -> Pair.of(partitionPath, f.getFileId())).collect(toList());
|
||||
}
|
||||
return filteredFiles.iterator();
|
||||
}).collect();
|
||||
|
||||
if (config.getBloomIndexPruneByRanges()) {
|
||||
// also obtain file ranges, if range pruning is enabled
|
||||
return jsc.parallelize(partitionPathFileIDList, Math.max(partitionPathFileIDList.size(), 1)).mapToPair(pf -> {
|
||||
try {
|
||||
HoodieRangeInfoHandle<T> rangeInfoHandle = new HoodieRangeInfoHandle<T>(config, hoodieTable, pf);
|
||||
String[] minMaxKeys = rangeInfoHandle.getMinMaxKeys();
|
||||
return new Tuple2<>(pf.getKey(), new BloomIndexFileInfo(pf.getValue(), minMaxKeys[0], minMaxKeys[1]));
|
||||
} catch (MetadataNotFoundException me) {
|
||||
logger.warn("Unable to find range metadata in file :" + pf);
|
||||
return new Tuple2<>(pf.getKey(), new BloomIndexFileInfo(pf.getValue()));
|
||||
}
|
||||
}).collect();
|
||||
} else {
|
||||
return partitionPathFileIDList.stream()
|
||||
.map(pf -> new Tuple2<>(pf.getKey(), new BloomIndexFileInfo(pf.getValue())))
|
||||
.collect(toList());
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
@Override
|
||||
public boolean rollbackCommit(String commitTime) {
|
||||
// Nope, don't need to do anything.
|
||||
return true;
|
||||
}
|
||||
|
||||
/**
|
||||
* This is not global, since we depend on the partitionPath to do the lookup
|
||||
*/
|
||||
@Override
|
||||
public boolean isGlobal() {
|
||||
return false;
|
||||
}
|
||||
|
||||
/**
|
||||
* No indexes into log files yet.
|
||||
*/
|
||||
@Override
|
||||
public boolean canIndexLogFiles() {
|
||||
return false;
|
||||
}
|
||||
|
||||
/**
|
||||
* Bloom filters are stored, into the same data files.
|
||||
*/
|
||||
@Override
|
||||
public boolean isImplicitWithStorage() {
|
||||
return true;
|
||||
}
|
||||
|
||||
/**
|
||||
* For each incoming record, produce N output records, 1 each for each file against which the record's key needs to be
|
||||
* checked. For datasets, where the keys have a definite insert order (e.g: timestamp as prefix), the number of files
|
||||
* to be compared gets cut down a lot from range pruning.
|
||||
*
|
||||
* Sub-partition to ensure the records can be looked up against files & also prune file<=>record comparisons based on
|
||||
* recordKey ranges in the index info.
|
||||
*/
|
||||
@VisibleForTesting
|
||||
JavaRDD<Tuple2<String, HoodieKey>> explodeRecordRDDWithFileComparisons(
|
||||
final Map<String, List<BloomIndexFileInfo>> partitionToFileIndexInfo,
|
||||
JavaPairRDD<String, String> partitionRecordKeyPairRDD) {
|
||||
IndexFileFilter indexFileFilter = config.useBloomIndexTreebasedFilter()
|
||||
? new IntervalTreeBasedIndexFileFilter(partitionToFileIndexInfo)
|
||||
: new ListBasedIndexFileFilter(partitionToFileIndexInfo);
|
||||
|
||||
return partitionRecordKeyPairRDD.map(partitionRecordKeyPair -> {
|
||||
String recordKey = partitionRecordKeyPair._2();
|
||||
String partitionPath = partitionRecordKeyPair._1();
|
||||
|
||||
return indexFileFilter.getMatchingFiles(partitionPath, recordKey).stream()
|
||||
.map(matchingFile -> new Tuple2<>(matchingFile, new HoodieKey(recordKey, partitionPath)))
|
||||
.collect(Collectors.toList());
|
||||
}).flatMap(List::iterator);
|
||||
}
|
||||
|
||||
/**
|
||||
* Find out <RowKey, filename> pair. All workload grouped by file-level. <p> Join PairRDD(PartitionPath, RecordKey)
|
||||
* and PairRDD(PartitionPath, File) & then repartition such that each RDD partition is a file, then for each file, we
|
||||
* do (1) load bloom filter, (2) load rowKeys, (3) Tag rowKey <p> Make sure the parallelism is atleast the groupby
|
||||
* parallelism for tagging location
|
||||
*/
|
||||
@VisibleForTesting
|
||||
JavaPairRDD<HoodieKey, HoodieRecordLocation> findMatchingFilesForRecordKeys(
|
||||
final Map<String, List<BloomIndexFileInfo>> partitionToFileIndexInfo,
|
||||
JavaPairRDD<String, String> partitionRecordKeyPairRDD, int shuffleParallelism, HoodieTable hoodieTable,
|
||||
Map<String, Long> fileGroupToComparisons) {
|
||||
JavaRDD<Tuple2<String, HoodieKey>> fileComparisonsRDD =
|
||||
explodeRecordRDDWithFileComparisons(partitionToFileIndexInfo, partitionRecordKeyPairRDD);
|
||||
|
||||
if (config.useBloomIndexBucketizedChecking()) {
|
||||
Partitioner partitioner = new BucketizedBloomCheckPartitioner(
|
||||
shuffleParallelism,
|
||||
fileGroupToComparisons,
|
||||
config.getBloomIndexKeysPerBucket()
|
||||
);
|
||||
|
||||
fileComparisonsRDD = fileComparisonsRDD
|
||||
.mapToPair(t -> new Tuple2<>(Pair.of(t._1, t._2.getRecordKey()), t))
|
||||
.repartitionAndSortWithinPartitions(partitioner)
|
||||
.map(Tuple2::_2);
|
||||
} else {
|
||||
fileComparisonsRDD = fileComparisonsRDD.sortBy(Tuple2::_1, true, shuffleParallelism);
|
||||
}
|
||||
|
||||
return fileComparisonsRDD
|
||||
.mapPartitionsWithIndex(new HoodieBloomIndexCheckFunction(hoodieTable, config), true)
|
||||
.flatMap(List::iterator)
|
||||
.filter(lr -> lr.getMatchingRecordKeys().size() > 0)
|
||||
.flatMapToPair(lookupResult -> lookupResult.getMatchingRecordKeys().stream()
|
||||
.map(recordKey -> new Tuple2<>(new HoodieKey(recordKey, lookupResult.getPartitionPath()),
|
||||
new HoodieRecordLocation(lookupResult.getBaseInstantTime(), lookupResult.getFileId())))
|
||||
.collect(Collectors.toList())
|
||||
.iterator());
|
||||
}
|
||||
|
||||
HoodieRecord<T> getTaggedRecord(HoodieRecord<T> inputRecord,
|
||||
Option<HoodieRecordLocation> location) {
|
||||
HoodieRecord<T> record = inputRecord;
|
||||
if (location.isPresent()) {
|
||||
// When you have a record in multiple files in the same partition, then rowKeyRecordPairRDD
|
||||
// will have 2 entries with the same exact in memory copy of the HoodieRecord and the 2
|
||||
// separate filenames that the record is found in. This will result in setting
|
||||
// currentLocation 2 times and it will fail the second time. So creating a new in memory
|
||||
// copy of the hoodie record.
|
||||
record = new HoodieRecord<>(inputRecord);
|
||||
record.setCurrentLocation(location.get());
|
||||
}
|
||||
return record;
|
||||
}
|
||||
|
||||
/**
|
||||
* Tag the <rowKey, filename> back to the original HoodieRecord RDD.
|
||||
*/
|
||||
protected JavaRDD<HoodieRecord<T>> tagLocationBacktoRecords(
|
||||
JavaPairRDD<HoodieKey, HoodieRecordLocation> keyFilenamePairRDD, JavaRDD<HoodieRecord<T>> recordRDD) {
|
||||
JavaPairRDD<HoodieKey, HoodieRecord<T>> keyRecordPairRDD = recordRDD
|
||||
.mapToPair(record -> new Tuple2<>(record.getKey(), record));
|
||||
// Here as the recordRDD might have more data than rowKeyRDD (some rowKeys' fileId is null),
|
||||
// so we do left outer join.
|
||||
return keyRecordPairRDD.leftOuterJoin(keyFilenamePairRDD).values().map(
|
||||
v1 -> getTaggedRecord(v1._1, Option.ofNullable(v1._2.orNull())));
|
||||
}
|
||||
|
||||
@Override
|
||||
public JavaRDD<WriteStatus> updateLocation(JavaRDD<WriteStatus> writeStatusRDD, JavaSparkContext jsc,
|
||||
HoodieTable<T> hoodieTable) {
|
||||
return writeStatusRDD;
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,119 @@
|
||||
/*
|
||||
* 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.index.bloom;
|
||||
|
||||
import java.util.ArrayList;
|
||||
import java.util.Iterator;
|
||||
import java.util.List;
|
||||
import org.apache.hudi.common.model.HoodieKey;
|
||||
import org.apache.hudi.common.util.collection.Pair;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
import org.apache.hudi.exception.HoodieException;
|
||||
import org.apache.hudi.exception.HoodieIndexException;
|
||||
import org.apache.hudi.func.LazyIterableIterator;
|
||||
import org.apache.hudi.io.HoodieKeyLookupHandle;
|
||||
import org.apache.hudi.io.HoodieKeyLookupHandle.KeyLookupResult;
|
||||
import org.apache.hudi.table.HoodieTable;
|
||||
import org.apache.spark.api.java.function.Function2;
|
||||
import scala.Tuple2;
|
||||
|
||||
/**
|
||||
* Function performing actual checking of RDD partition containing (fileId, hoodieKeys) against the
|
||||
* actual files
|
||||
*/
|
||||
public class HoodieBloomIndexCheckFunction implements
|
||||
Function2<Integer, Iterator<Tuple2<String, HoodieKey>>, Iterator<List<KeyLookupResult>>> {
|
||||
|
||||
private final HoodieTable hoodieTable;
|
||||
|
||||
private final HoodieWriteConfig config;
|
||||
|
||||
public HoodieBloomIndexCheckFunction(HoodieTable hoodieTable, HoodieWriteConfig config) {
|
||||
this.hoodieTable = hoodieTable;
|
||||
this.config = config;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Iterator<List<KeyLookupResult>> call(Integer partition,
|
||||
Iterator<Tuple2<String, HoodieKey>> fileParitionRecordKeyTripletItr) {
|
||||
return new LazyKeyCheckIterator(fileParitionRecordKeyTripletItr);
|
||||
}
|
||||
|
||||
class LazyKeyCheckIterator extends LazyIterableIterator<Tuple2<String, HoodieKey>, List<KeyLookupResult>> {
|
||||
|
||||
private HoodieKeyLookupHandle keyLookupHandle;
|
||||
|
||||
LazyKeyCheckIterator(
|
||||
Iterator<Tuple2<String, HoodieKey>> filePartitionRecordKeyTripletItr) {
|
||||
super(filePartitionRecordKeyTripletItr);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected void start() {
|
||||
}
|
||||
|
||||
@Override
|
||||
protected List<HoodieKeyLookupHandle.KeyLookupResult> computeNext() {
|
||||
|
||||
List<HoodieKeyLookupHandle.KeyLookupResult> ret = new ArrayList<>();
|
||||
try {
|
||||
// process one file in each go.
|
||||
while (inputItr.hasNext()) {
|
||||
Tuple2<String, HoodieKey> currentTuple = inputItr.next();
|
||||
String fileId = currentTuple._1;
|
||||
String partitionPath = currentTuple._2.getPartitionPath();
|
||||
String recordKey = currentTuple._2.getRecordKey();
|
||||
Pair<String, String> partitionPathFilePair = Pair.of(partitionPath, fileId);
|
||||
|
||||
// lazily init state
|
||||
if (keyLookupHandle == null) {
|
||||
keyLookupHandle = new HoodieKeyLookupHandle(config, hoodieTable, partitionPathFilePair);
|
||||
}
|
||||
|
||||
// if continue on current file
|
||||
if (keyLookupHandle.getPartitionPathFilePair().equals(partitionPathFilePair)) {
|
||||
keyLookupHandle.addKey(recordKey);
|
||||
} else {
|
||||
// do the actual checking of file & break out
|
||||
ret.add(keyLookupHandle.getLookupResult());
|
||||
keyLookupHandle = new HoodieKeyLookupHandle(config, hoodieTable, partitionPathFilePair);
|
||||
keyLookupHandle.addKey(recordKey);
|
||||
break;
|
||||
}
|
||||
}
|
||||
|
||||
// handle case, where we ran out of input, close pending work, update return val
|
||||
if (!inputItr.hasNext()) {
|
||||
ret.add(keyLookupHandle.getLookupResult());
|
||||
}
|
||||
} catch (Throwable e) {
|
||||
if (e instanceof HoodieException) {
|
||||
throw e;
|
||||
}
|
||||
throw new HoodieIndexException("Error checking bloom filter index. ", e);
|
||||
}
|
||||
|
||||
return ret;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected void end() {
|
||||
}
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,125 @@
|
||||
/*
|
||||
* 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.index.bloom;
|
||||
|
||||
import com.google.common.annotations.VisibleForTesting;
|
||||
import java.io.IOException;
|
||||
import java.util.HashMap;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Map.Entry;
|
||||
import java.util.stream.Collectors;
|
||||
import org.apache.hudi.common.model.HoodieKey;
|
||||
import org.apache.hudi.common.model.HoodieRecord;
|
||||
import org.apache.hudi.common.model.HoodieRecordLocation;
|
||||
import org.apache.hudi.common.model.HoodieRecordPayload;
|
||||
import org.apache.hudi.common.table.HoodieTableMetaClient;
|
||||
import org.apache.hudi.common.util.FSUtils;
|
||||
import org.apache.hudi.common.util.Option;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
import org.apache.hudi.exception.HoodieIOException;
|
||||
import org.apache.hudi.table.HoodieTable;
|
||||
import org.apache.spark.api.java.JavaPairRDD;
|
||||
import org.apache.spark.api.java.JavaRDD;
|
||||
import org.apache.spark.api.java.JavaSparkContext;
|
||||
import scala.Tuple2;
|
||||
|
||||
/**
|
||||
* This filter will only work with hoodie dataset since it will only load partitions with .hoodie_partition_metadata
|
||||
* file in it.
|
||||
*/
|
||||
public class HoodieGlobalBloomIndex<T extends HoodieRecordPayload> extends HoodieBloomIndex<T> {
|
||||
|
||||
public HoodieGlobalBloomIndex(HoodieWriteConfig config) {
|
||||
super(config);
|
||||
}
|
||||
|
||||
/**
|
||||
* Load all involved files as <Partition, filename> pair RDD from all partitions in the table.
|
||||
*/
|
||||
@Override
|
||||
@VisibleForTesting
|
||||
List<Tuple2<String, BloomIndexFileInfo>> loadInvolvedFiles(List<String> partitions, final JavaSparkContext jsc,
|
||||
final HoodieTable hoodieTable) {
|
||||
HoodieTableMetaClient metaClient = hoodieTable.getMetaClient();
|
||||
try {
|
||||
List<String> allPartitionPaths = FSUtils
|
||||
.getAllPartitionPaths(metaClient.getFs(), metaClient.getBasePath(),
|
||||
config.shouldAssumeDatePartitioning());
|
||||
return super.loadInvolvedFiles(allPartitionPaths, jsc, hoodieTable);
|
||||
} catch (IOException e) {
|
||||
throw new HoodieIOException("Failed to load all partitions", e);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* For each incoming record, produce N output records, 1 each for each file against which the record's key needs to be
|
||||
* checked. For datasets, where the keys have a definite insert order (e.g: timestamp as prefix), the number of files
|
||||
* to be compared gets cut down a lot from range pruning.
|
||||
*
|
||||
* Sub-partition to ensure the records can be looked up against files & also prune file<=>record comparisons based on
|
||||
* recordKey ranges in the index info. the partition path of the incoming record (partitionRecordKeyPairRDD._2()) will
|
||||
* be ignored since the search scope should be bigger than that
|
||||
*/
|
||||
|
||||
@Override
|
||||
@VisibleForTesting
|
||||
JavaRDD<Tuple2<String, HoodieKey>> explodeRecordRDDWithFileComparisons(
|
||||
final Map<String, List<BloomIndexFileInfo>> partitionToFileIndexInfo,
|
||||
JavaPairRDD<String, String> partitionRecordKeyPairRDD) {
|
||||
Map<String, String> indexToPartitionMap = new HashMap<>();
|
||||
for (Entry<String, List<BloomIndexFileInfo>> entry : partitionToFileIndexInfo.entrySet()) {
|
||||
entry.getValue().forEach(indexFile -> indexToPartitionMap.put(indexFile.getFileId(), entry.getKey()));
|
||||
}
|
||||
|
||||
IndexFileFilter indexFileFilter = config.getBloomIndexPruneByRanges()
|
||||
? new IntervalTreeBasedGlobalIndexFileFilter(partitionToFileIndexInfo)
|
||||
: new ListBasedGlobalIndexFileFilter(partitionToFileIndexInfo);
|
||||
|
||||
return partitionRecordKeyPairRDD.map(partitionRecordKeyPair -> {
|
||||
String recordKey = partitionRecordKeyPair._2();
|
||||
String partitionPath = partitionRecordKeyPair._1();
|
||||
|
||||
return indexFileFilter.getMatchingFiles(partitionPath, recordKey).stream()
|
||||
.map(file -> new Tuple2<>(file, new HoodieKey(recordKey, indexToPartitionMap.get(file))))
|
||||
.collect(Collectors.toList());
|
||||
}).flatMap(List::iterator);
|
||||
}
|
||||
|
||||
|
||||
/**
|
||||
* Tagging for global index should only consider the record key
|
||||
*/
|
||||
@Override
|
||||
protected JavaRDD<HoodieRecord<T>> tagLocationBacktoRecords(
|
||||
JavaPairRDD<HoodieKey, HoodieRecordLocation> keyFilenamePairRDD, JavaRDD<HoodieRecord<T>> recordRDD) {
|
||||
JavaPairRDD<String, HoodieRecord<T>> rowKeyRecordPairRDD = recordRDD
|
||||
.mapToPair(record -> new Tuple2<>(record.getRecordKey(), record));
|
||||
|
||||
// Here as the recordRDD might have more data than rowKeyRDD (some rowKeys' fileId is null),
|
||||
// so we do left outer join.
|
||||
return rowKeyRecordPairRDD.leftOuterJoin(keyFilenamePairRDD.mapToPair(p -> new Tuple2<>(p._1.getRecordKey(), p._2)))
|
||||
.values().map(value -> getTaggedRecord(value._1, Option.ofNullable(value._2.orNull())));
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean isGlobal() {
|
||||
return true;
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,38 @@
|
||||
/*
|
||||
* 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.index.bloom;
|
||||
|
||||
import java.io.Serializable;
|
||||
import java.util.Set;
|
||||
|
||||
/**
|
||||
* IndexFile filter to assist in look up of a record key.
|
||||
*/
|
||||
public interface IndexFileFilter extends Serializable {
|
||||
|
||||
/**
|
||||
* Fetches all matching files for a given record key and partition.
|
||||
*
|
||||
* @param partitionPath the partition path of interest
|
||||
* @param recordKey the record key to be looked up
|
||||
* @return the {@link Set} of matching file names where the record could potentially be present.
|
||||
*/
|
||||
Set<String> getMatchingFiles(String partitionPath, String recordKey);
|
||||
|
||||
}
|
||||
@@ -0,0 +1,67 @@
|
||||
/*
|
||||
* 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.index.bloom;
|
||||
|
||||
import java.util.Collection;
|
||||
import java.util.Collections;
|
||||
import java.util.HashSet;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
import java.util.stream.Collectors;
|
||||
|
||||
/**
|
||||
* Interval Tree based index look up for Global Index. Builds an {@link KeyRangeLookupTree} for all index files (across
|
||||
* all partitions) and uses it to search for matching index files for any given recordKey that needs to be looked up.
|
||||
*/
|
||||
class IntervalTreeBasedGlobalIndexFileFilter implements IndexFileFilter {
|
||||
|
||||
private final KeyRangeLookupTree indexLookUpTree = new KeyRangeLookupTree();
|
||||
private final Set<String> filesWithNoRanges = new HashSet<>();
|
||||
|
||||
/**
|
||||
* Instantiates {@link IntervalTreeBasedGlobalIndexFileFilter}
|
||||
*
|
||||
* @param partitionToFileIndexInfo Map of partition to List of {@link BloomIndexFileInfo}s
|
||||
*/
|
||||
IntervalTreeBasedGlobalIndexFileFilter(final Map<String, List<BloomIndexFileInfo>> partitionToFileIndexInfo) {
|
||||
List<BloomIndexFileInfo> allIndexFiles = partitionToFileIndexInfo.values().stream().flatMap(Collection::stream)
|
||||
.collect(Collectors.toList());
|
||||
// Note that the interval tree implementation doesn't have auto-balancing to ensure logN search time.
|
||||
// So, we are shuffling the input here hoping the tree will not have any skewness. If not, the tree could be skewed
|
||||
// which could result in N search time instead of NlogN.
|
||||
Collections.shuffle(allIndexFiles);
|
||||
allIndexFiles.forEach(indexFile -> {
|
||||
if (indexFile.hasKeyRanges()) {
|
||||
indexLookUpTree.insert(new KeyRangeNode(indexFile.getMinRecordKey(),
|
||||
indexFile.getMaxRecordKey(), indexFile.getFileId()));
|
||||
} else {
|
||||
filesWithNoRanges.add(indexFile.getFileId());
|
||||
}
|
||||
});
|
||||
}
|
||||
|
||||
@Override
|
||||
public Set<String> getMatchingFiles(String partitionPath, String recordKey) {
|
||||
Set<String> toReturn = new HashSet<>();
|
||||
toReturn.addAll(indexLookUpTree.getMatchingIndexFiles(recordKey));
|
||||
toReturn.addAll(filesWithNoRanges);
|
||||
return toReturn;
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,76 @@
|
||||
/*
|
||||
* 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.index.bloom;
|
||||
|
||||
import java.util.Collections;
|
||||
import java.util.HashMap;
|
||||
import java.util.HashSet;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
|
||||
/**
|
||||
* Interval Tree based index look up. Builds an {@link KeyRangeLookupTree} for every partition and uses it to search for
|
||||
* matching index files for any given recordKey that needs to be looked up.
|
||||
*/
|
||||
class IntervalTreeBasedIndexFileFilter implements IndexFileFilter {
|
||||
|
||||
private final Map<String, KeyRangeLookupTree> partitionToFileIndexLookUpTree = new HashMap<>();
|
||||
private final Map<String, Set<String>> partitionToFilesWithNoRanges = new HashMap<>();
|
||||
|
||||
/**
|
||||
* Instantiates {@link IntervalTreeBasedIndexFileFilter}
|
||||
*
|
||||
* @param partitionToFileIndexInfo Map of partition to List of {@link BloomIndexFileInfo}s
|
||||
*/
|
||||
IntervalTreeBasedIndexFileFilter(final Map<String, List<BloomIndexFileInfo>> partitionToFileIndexInfo) {
|
||||
partitionToFileIndexInfo.forEach((partition, bloomIndexFiles) -> {
|
||||
// Note that the interval tree implementation doesn't have auto-balancing to ensure logN search time.
|
||||
// So, we are shuffling the input here hoping the tree will not have any skewness. If not, the tree could be
|
||||
// skewed which could result in N search time instead of logN.
|
||||
Collections.shuffle(bloomIndexFiles);
|
||||
KeyRangeLookupTree lookUpTree = new KeyRangeLookupTree();
|
||||
bloomIndexFiles.forEach(indexFileInfo -> {
|
||||
if (indexFileInfo.hasKeyRanges()) {
|
||||
lookUpTree.insert(new KeyRangeNode(indexFileInfo.getMinRecordKey(),
|
||||
indexFileInfo.getMaxRecordKey(), indexFileInfo.getFileId()));
|
||||
} else {
|
||||
if (!partitionToFilesWithNoRanges.containsKey(partition)) {
|
||||
partitionToFilesWithNoRanges.put(partition, new HashSet<>());
|
||||
}
|
||||
partitionToFilesWithNoRanges.get(partition).add(indexFileInfo.getFileId());
|
||||
}
|
||||
});
|
||||
partitionToFileIndexLookUpTree.put(partition, lookUpTree);
|
||||
});
|
||||
}
|
||||
|
||||
@Override
|
||||
public Set<String> getMatchingFiles(String partitionPath, String recordKey) {
|
||||
Set<String> toReturn = new HashSet<>();
|
||||
// could be null, if there are no files in a given partition yet or if all index files have no ranges
|
||||
if (partitionToFileIndexLookUpTree.containsKey(partitionPath)) {
|
||||
toReturn.addAll(partitionToFileIndexLookUpTree.get(partitionPath).getMatchingIndexFiles(recordKey));
|
||||
}
|
||||
if (partitionToFilesWithNoRanges.containsKey(partitionPath)) {
|
||||
toReturn.addAll(partitionToFilesWithNoRanges.get(partitionPath));
|
||||
}
|
||||
return toReturn;
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,49 @@
|
||||
/*
|
||||
* 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.index.bloom;
|
||||
|
||||
import java.util.List;
|
||||
|
||||
/**
|
||||
* Encapsulates the result from a key lookup
|
||||
*/
|
||||
public class KeyLookupResult {
|
||||
|
||||
private final String fileName;
|
||||
private final List<String> matchingRecordKeys;
|
||||
private final String partitionPath;
|
||||
|
||||
public KeyLookupResult(String fileName, String partitionPath, List<String> matchingRecordKeys) {
|
||||
this.fileName = fileName;
|
||||
this.partitionPath = partitionPath;
|
||||
this.matchingRecordKeys = matchingRecordKeys;
|
||||
}
|
||||
|
||||
public String getFileName() {
|
||||
return fileName;
|
||||
}
|
||||
|
||||
public List<String> getMatchingRecordKeys() {
|
||||
return matchingRecordKeys;
|
||||
}
|
||||
|
||||
public String getPartitionPath() {
|
||||
return partitionPath;
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,156 @@
|
||||
/*
|
||||
* 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.index.bloom;
|
||||
|
||||
import java.io.Serializable;
|
||||
import java.util.HashSet;
|
||||
import java.util.Set;
|
||||
|
||||
/**
|
||||
* Look up tree implemented as interval trees to search for any given key in (N logN) time complexity.
|
||||
*/
|
||||
class KeyRangeLookupTree implements Serializable {
|
||||
|
||||
private KeyRangeNode root;
|
||||
|
||||
/**
|
||||
* @return the root of the tree. Could be {@code null}
|
||||
*/
|
||||
public KeyRangeNode getRoot() {
|
||||
return root;
|
||||
}
|
||||
|
||||
/**
|
||||
* Inserts a new {@link KeyRangeNode} to this look up tree.
|
||||
*
|
||||
* @param newNode the new {@link KeyRangeNode} to be inserted
|
||||
*/
|
||||
void insert(KeyRangeNode newNode) {
|
||||
root = insert(getRoot(), newNode);
|
||||
}
|
||||
|
||||
/**
|
||||
* Inserts a new {@link KeyRangeNode} to this look up tree.
|
||||
*
|
||||
* If no root exists, make {@code newNode} as the root and return the new root.
|
||||
*
|
||||
* If current root and newNode matches with min record key and max record key,
|
||||
* merge two nodes. In other words, add files from {@code newNode} to current root.
|
||||
* Return current root.
|
||||
*
|
||||
* If current root is < newNode
|
||||
* if current root has no right sub tree
|
||||
* update current root's right sub tree max and min
|
||||
* set newNode as right sub tree
|
||||
* else
|
||||
* update root's right sub tree min and max with newNode's min and max record key as applicable
|
||||
* recursively call insert() with root's right subtree as new root
|
||||
*
|
||||
* else // current root is >= newNode
|
||||
* if current root has no left sub tree
|
||||
* update current root's left sub tree max and min
|
||||
* set newNode as left sub tree
|
||||
* else
|
||||
* update root's left sub tree min and max with newNode's min and max record key as applicable
|
||||
* recursively call insert() with root's left subtree as new root
|
||||
*
|
||||
* @param root refers to the current root of the look up tree
|
||||
* @param newNode newNode the new {@link KeyRangeNode} to be inserted
|
||||
*/
|
||||
private KeyRangeNode insert(KeyRangeNode root, KeyRangeNode newNode) {
|
||||
if (root == null) {
|
||||
root = newNode;
|
||||
return root;
|
||||
}
|
||||
|
||||
if (root.compareTo(newNode) == 0) {
|
||||
root.addFiles(newNode.getFileNameList());
|
||||
return root;
|
||||
}
|
||||
|
||||
if (root.compareTo(newNode) < 0) {
|
||||
if (root.getRight() == null) {
|
||||
root.setRightSubTreeMax(newNode.getMaxRecordKey());
|
||||
root.setRightSubTreeMin(newNode.getMinRecordKey());
|
||||
root.setRight(newNode);
|
||||
} else {
|
||||
if (root.getRightSubTreeMax().compareTo(newNode.getMaxRecordKey()) < 0) {
|
||||
root.setRightSubTreeMax(newNode.getMaxRecordKey());
|
||||
}
|
||||
if (root.getRightSubTreeMin().compareTo(newNode.getMinRecordKey()) > 0) {
|
||||
root.setRightSubTreeMin(newNode.getMinRecordKey());
|
||||
}
|
||||
insert(root.getRight(), newNode);
|
||||
}
|
||||
} else {
|
||||
if (root.getLeft() == null) {
|
||||
root.setLeftSubTreeMax(newNode.getMaxRecordKey());
|
||||
root.setLeftSubTreeMin(newNode.getMinRecordKey());
|
||||
root.setLeft(newNode);
|
||||
} else {
|
||||
if (root.getLeftSubTreeMax().compareTo(newNode.getMaxRecordKey()) < 0) {
|
||||
root.setLeftSubTreeMax(newNode.getMaxRecordKey());
|
||||
}
|
||||
if (root.getLeftSubTreeMin().compareTo(newNode.getMinRecordKey()) > 0) {
|
||||
root.setLeftSubTreeMin(newNode.getMinRecordKey());
|
||||
}
|
||||
insert(root.getLeft(), newNode);
|
||||
}
|
||||
}
|
||||
return root;
|
||||
}
|
||||
|
||||
/**
|
||||
* Fetches all the matching index files where the key could possibly be present.
|
||||
*
|
||||
* @param lookupKey the key to be searched for
|
||||
* @return the {@link Set} of matching index file names
|
||||
*/
|
||||
Set<String> getMatchingIndexFiles(String lookupKey) {
|
||||
Set<String> matchingFileNameSet = new HashSet<>();
|
||||
getMatchingIndexFiles(getRoot(), lookupKey, matchingFileNameSet);
|
||||
return matchingFileNameSet;
|
||||
}
|
||||
|
||||
/**
|
||||
* Fetches all the matching index files where the key could possibly be present.
|
||||
*
|
||||
* @param root refers to the current root of the look up tree
|
||||
* @param lookupKey the key to be searched for
|
||||
*/
|
||||
private void getMatchingIndexFiles(KeyRangeNode root, String lookupKey, Set<String> matchingFileNameSet) {
|
||||
if (root == null) {
|
||||
return;
|
||||
}
|
||||
|
||||
if (root.getMinRecordKey().compareTo(lookupKey) <= 0 && lookupKey.compareTo(root.getMaxRecordKey()) <= 0) {
|
||||
matchingFileNameSet.addAll(root.getFileNameList());
|
||||
}
|
||||
|
||||
if (root.getLeftSubTreeMax() != null && root.getLeftSubTreeMin().compareTo(lookupKey) <= 0
|
||||
&& lookupKey.compareTo(root.getLeftSubTreeMax()) <= 0) {
|
||||
getMatchingIndexFiles(root.getLeft(), lookupKey, matchingFileNameSet);
|
||||
}
|
||||
|
||||
if (root.getRightSubTreeMax() != null && root.getRightSubTreeMin().compareTo(lookupKey) <= 0
|
||||
&& lookupKey.compareTo(root.getRightSubTreeMax()) <= 0) {
|
||||
getMatchingIndexFiles(root.getRight(), lookupKey, matchingFileNameSet);
|
||||
}
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,153 @@
|
||||
/*
|
||||
* 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.index.bloom;
|
||||
|
||||
import java.io.Serializable;
|
||||
import java.util.ArrayList;
|
||||
import java.util.List;
|
||||
|
||||
/**
|
||||
* Represents a node in the {@link KeyRangeLookupTree}. Holds information pertaining to a single index file, viz file
|
||||
* name, min record key and max record key.
|
||||
*/
|
||||
class KeyRangeNode implements Comparable<KeyRangeNode>, Serializable {
|
||||
|
||||
private final List<String> fileNameList = new ArrayList<>();
|
||||
private final String minRecordKey;
|
||||
private final String maxRecordKey;
|
||||
private String rightSubTreeMax = null;
|
||||
private String leftSubTreeMax = null;
|
||||
private String rightSubTreeMin = null;
|
||||
private String leftSubTreeMin = null;
|
||||
private KeyRangeNode left = null;
|
||||
private KeyRangeNode right = null;
|
||||
|
||||
/**
|
||||
* Instantiates a new {@link KeyRangeNode}
|
||||
*
|
||||
* @param minRecordKey min record key of the index file
|
||||
* @param maxRecordKey max record key of the index file
|
||||
* @param fileName file name of the index file
|
||||
*/
|
||||
KeyRangeNode(String minRecordKey, String maxRecordKey, String fileName) {
|
||||
this.fileNameList.add(fileName);
|
||||
this.minRecordKey = minRecordKey;
|
||||
this.maxRecordKey = maxRecordKey;
|
||||
}
|
||||
|
||||
/**
|
||||
* Adds a new file name list to existing list of file names.
|
||||
*
|
||||
* @param newFiles {@link List} of file names to be added
|
||||
*/
|
||||
void addFiles(List<String> newFiles) {
|
||||
this.fileNameList.addAll(newFiles);
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
return "KeyRangeNode{"
|
||||
+ "minRecordKey='" + minRecordKey + '\''
|
||||
+ ", maxRecordKey='" + maxRecordKey + '\''
|
||||
+ ", fileNameList=" + fileNameList
|
||||
+ ", rightSubTreeMax='" + rightSubTreeMax + '\''
|
||||
+ ", leftSubTreeMax='" + leftSubTreeMax + '\''
|
||||
+ ", rightSubTreeMin='" + rightSubTreeMin + '\''
|
||||
+ ", leftSubTreeMin='" + leftSubTreeMin + '\''
|
||||
+ '}';
|
||||
}
|
||||
|
||||
/**
|
||||
* Compares the min record key of two nodes, followed by max record key.
|
||||
*
|
||||
* @param that the {@link KeyRangeNode} to be compared with
|
||||
* @return the result of comparison. 0 if both min and max are equal in both. 1 if this {@link KeyRangeNode} is
|
||||
* greater than the {@code that} keyRangeNode. -1 if {@code that} keyRangeNode is greater than this {@link
|
||||
* KeyRangeNode}
|
||||
*/
|
||||
@Override
|
||||
public int compareTo(KeyRangeNode that) {
|
||||
int compareValue = minRecordKey.compareTo(that.minRecordKey);
|
||||
if (compareValue == 0) {
|
||||
return maxRecordKey.compareTo(that.maxRecordKey);
|
||||
} else {
|
||||
return compareValue;
|
||||
}
|
||||
}
|
||||
|
||||
public List<String> getFileNameList() {
|
||||
return fileNameList;
|
||||
}
|
||||
|
||||
public String getMinRecordKey() {
|
||||
return minRecordKey;
|
||||
}
|
||||
|
||||
public String getMaxRecordKey() {
|
||||
return maxRecordKey;
|
||||
}
|
||||
|
||||
public String getRightSubTreeMin() {
|
||||
return rightSubTreeMin;
|
||||
}
|
||||
|
||||
public void setRightSubTreeMin(String rightSubTreeMin) {
|
||||
this.rightSubTreeMin = rightSubTreeMin;
|
||||
}
|
||||
|
||||
public String getLeftSubTreeMin() {
|
||||
return leftSubTreeMin;
|
||||
}
|
||||
|
||||
public void setLeftSubTreeMin(String leftSubTreeMin) {
|
||||
this.leftSubTreeMin = leftSubTreeMin;
|
||||
}
|
||||
|
||||
public String getRightSubTreeMax() {
|
||||
return rightSubTreeMax;
|
||||
}
|
||||
|
||||
public void setRightSubTreeMax(String rightSubTreeMax) {
|
||||
this.rightSubTreeMax = rightSubTreeMax;
|
||||
}
|
||||
|
||||
public String getLeftSubTreeMax() {
|
||||
return leftSubTreeMax;
|
||||
}
|
||||
|
||||
public void setLeftSubTreeMax(String leftSubTreeMax) {
|
||||
this.leftSubTreeMax = leftSubTreeMax;
|
||||
}
|
||||
|
||||
public KeyRangeNode getLeft() {
|
||||
return left;
|
||||
}
|
||||
|
||||
public void setLeft(KeyRangeNode left) {
|
||||
this.left = left;
|
||||
}
|
||||
|
||||
public KeyRangeNode getRight() {
|
||||
return right;
|
||||
}
|
||||
|
||||
public void setRight(KeyRangeNode right) {
|
||||
this.right = right;
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,53 @@
|
||||
/*
|
||||
* 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.index.bloom;
|
||||
|
||||
import java.util.HashSet;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
|
||||
class ListBasedGlobalIndexFileFilter extends ListBasedIndexFileFilter {
|
||||
|
||||
/**
|
||||
* Instantiates {@link ListBasedGlobalIndexFileFilter}
|
||||
*
|
||||
* @param partitionToFileIndexInfo Map of partition to List of {@link BloomIndexFileInfo}
|
||||
*/
|
||||
ListBasedGlobalIndexFileFilter(
|
||||
Map<String, List<BloomIndexFileInfo>> partitionToFileIndexInfo) {
|
||||
super(partitionToFileIndexInfo);
|
||||
}
|
||||
|
||||
@Override
|
||||
public Set<String> getMatchingFiles(String partitionPath, String recordKey) {
|
||||
Set<String> toReturn = new HashSet<>();
|
||||
partitionToFileIndexInfo.values().forEach(indexInfos -> {
|
||||
if (indexInfos != null) { // could be null, if there are no files in a given partition yet.
|
||||
// for each candidate file in partition, that needs to be compared.
|
||||
for (BloomIndexFileInfo indexInfo : indexInfos) {
|
||||
if (shouldCompareWithFile(indexInfo, recordKey)) {
|
||||
toReturn.add(indexInfo.getFileId());
|
||||
}
|
||||
}
|
||||
}
|
||||
});
|
||||
return toReturn;
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,65 @@
|
||||
/*
|
||||
* 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.index.bloom;
|
||||
|
||||
import java.util.HashSet;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
|
||||
/**
|
||||
* Simple implementation of {@link IndexFileFilter}. Sequentially goes through every index file in a given partition to
|
||||
* search for potential index files to be searched for a given record key.
|
||||
*/
|
||||
class ListBasedIndexFileFilter implements IndexFileFilter {
|
||||
|
||||
final Map<String, List<BloomIndexFileInfo>> partitionToFileIndexInfo;
|
||||
|
||||
/**
|
||||
* Instantiates {@link ListBasedIndexFileFilter}
|
||||
*
|
||||
* @param partitionToFileIndexInfo Map of partition to List of {@link BloomIndexFileInfo}
|
||||
*/
|
||||
ListBasedIndexFileFilter(final Map<String, List<BloomIndexFileInfo>> partitionToFileIndexInfo) {
|
||||
this.partitionToFileIndexInfo = partitionToFileIndexInfo;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Set<String> getMatchingFiles(String partitionPath, String recordKey) {
|
||||
List<BloomIndexFileInfo> indexInfos = partitionToFileIndexInfo.get(partitionPath);
|
||||
Set<String> toReturn = new HashSet<>();
|
||||
if (indexInfos != null) { // could be null, if there are no files in a given partition yet.
|
||||
// for each candidate file in partition, that needs to be compared.
|
||||
for (BloomIndexFileInfo indexInfo : indexInfos) {
|
||||
if (shouldCompareWithFile(indexInfo, recordKey)) {
|
||||
toReturn.add(indexInfo.getFileId());
|
||||
}
|
||||
}
|
||||
}
|
||||
return toReturn;
|
||||
}
|
||||
|
||||
/**
|
||||
* if we dont have key ranges, then also we need to compare against the file. no other choice if we do, then only
|
||||
* compare the file if the record key falls in range.
|
||||
*/
|
||||
protected boolean shouldCompareWithFile(BloomIndexFileInfo indexInfo, String recordKey) {
|
||||
return !indexInfo.hasKeyRanges() || indexInfo.isKeyInRange(recordKey);
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,51 @@
|
||||
/*
|
||||
* 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.index.hbase;
|
||||
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
import org.apache.log4j.LogManager;
|
||||
import org.apache.log4j.Logger;
|
||||
|
||||
public class DefaultHBaseQPSResourceAllocator implements HBaseIndexQPSResourceAllocator {
|
||||
private HoodieWriteConfig hoodieWriteConfig;
|
||||
private static Logger logger = LogManager.getLogger(DefaultHBaseQPSResourceAllocator.class);
|
||||
|
||||
public DefaultHBaseQPSResourceAllocator(HoodieWriteConfig hoodieWriteConfig) {
|
||||
this.hoodieWriteConfig = hoodieWriteConfig;
|
||||
}
|
||||
|
||||
@Override
|
||||
public float calculateQPSFractionForPutsTime(final long numPuts, final int numRegionServers) {
|
||||
// Just return the configured qps_fraction without calculating it runtime
|
||||
return hoodieWriteConfig.getHbaseIndexQPSFraction();
|
||||
}
|
||||
|
||||
@Override
|
||||
public float acquireQPSResources(final float desiredQPSFraction, final long numPuts) {
|
||||
// Return the requested QPSFraction in this default implementation
|
||||
return desiredQPSFraction;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void releaseQPSResources() {
|
||||
// Do nothing, as there are no resources locked in default implementation
|
||||
logger.info(String.format("Release QPS resources called for %s with default implementation, do nothing",
|
||||
this.hoodieWriteConfig.getHbaseTableName()));
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,548 @@
|
||||
/*
|
||||
* 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.index.hbase;
|
||||
|
||||
import com.google.common.annotations.VisibleForTesting;
|
||||
import java.io.IOException;
|
||||
import java.io.Serializable;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Iterator;
|
||||
import java.util.LinkedList;
|
||||
import java.util.List;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.hbase.HBaseConfiguration;
|
||||
import org.apache.hadoop.hbase.TableName;
|
||||
import org.apache.hadoop.hbase.client.Connection;
|
||||
import org.apache.hadoop.hbase.client.ConnectionFactory;
|
||||
import org.apache.hadoop.hbase.client.Delete;
|
||||
import org.apache.hadoop.hbase.client.Get;
|
||||
import org.apache.hadoop.hbase.client.HTable;
|
||||
import org.apache.hadoop.hbase.client.Put;
|
||||
import org.apache.hadoop.hbase.client.RegionLocator;
|
||||
import org.apache.hadoop.hbase.client.Result;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
import org.apache.hudi.WriteStatus;
|
||||
import org.apache.hudi.common.model.HoodieKey;
|
||||
import org.apache.hudi.common.model.HoodieRecord;
|
||||
import org.apache.hudi.common.model.HoodieRecordLocation;
|
||||
import org.apache.hudi.common.model.HoodieRecordPayload;
|
||||
import org.apache.hudi.common.table.HoodieTableMetaClient;
|
||||
import org.apache.hudi.common.table.HoodieTimeline;
|
||||
import org.apache.hudi.common.table.timeline.HoodieInstant;
|
||||
import org.apache.hudi.common.util.Option;
|
||||
import org.apache.hudi.common.util.ReflectionUtils;
|
||||
import org.apache.hudi.common.util.collection.Pair;
|
||||
import org.apache.hudi.config.HoodieIndexConfig;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
import org.apache.hudi.exception.HoodieDependentSystemUnavailableException;
|
||||
import org.apache.hudi.exception.HoodieIndexException;
|
||||
import org.apache.hudi.index.HoodieIndex;
|
||||
import org.apache.hudi.table.HoodieTable;
|
||||
import org.apache.log4j.LogManager;
|
||||
import org.apache.log4j.Logger;
|
||||
import org.apache.spark.SparkConf;
|
||||
import org.apache.spark.api.java.JavaPairRDD;
|
||||
import org.apache.spark.api.java.JavaRDD;
|
||||
import org.apache.spark.api.java.JavaSparkContext;
|
||||
import org.apache.spark.api.java.function.Function2;
|
||||
import scala.Tuple2;
|
||||
|
||||
/**
|
||||
* Hoodie Index implementation backed by HBase
|
||||
*/
|
||||
public class HBaseIndex<T extends HoodieRecordPayload> extends HoodieIndex<T> {
|
||||
|
||||
public static final String DEFAULT_SPARK_EXECUTOR_INSTANCES_CONFIG_NAME =
|
||||
"spark.executor.instances";
|
||||
public static final String DEFAULT_SPARK_DYNAMIC_ALLOCATION_ENABLED_CONFIG_NAME =
|
||||
"spark.dynamicAllocation.enabled";
|
||||
public static final String DEFAULT_SPARK_DYNAMIC_ALLOCATION_MAX_EXECUTORS_CONFIG_NAME =
|
||||
"spark.dynamicAllocation.maxExecutors";
|
||||
|
||||
private static final byte[] SYSTEM_COLUMN_FAMILY = Bytes.toBytes("_s");
|
||||
private static final byte[] COMMIT_TS_COLUMN = Bytes.toBytes("commit_ts");
|
||||
private static final byte[] FILE_NAME_COLUMN = Bytes.toBytes("file_name");
|
||||
private static final byte[] PARTITION_PATH_COLUMN = Bytes.toBytes("partition_path");
|
||||
private static final int SLEEP_TIME_MILLISECONDS = 100;
|
||||
|
||||
private static Logger logger = LogManager.getLogger(HBaseIndex.class);
|
||||
private static Connection hbaseConnection = null;
|
||||
private HBaseIndexQPSResourceAllocator hBaseIndexQPSResourceAllocator = null;
|
||||
private float qpsFraction;
|
||||
private int maxQpsPerRegionServer;
|
||||
/**
|
||||
* multiPutBatchSize will be computed and re-set in updateLocation if
|
||||
* {@link HoodieIndexConfig.HBASE_PUT_BATCH_SIZE_AUTO_COMPUTE_PROP} is set to true
|
||||
*/
|
||||
private Integer multiPutBatchSize;
|
||||
private Integer numRegionServersForTable;
|
||||
private final String tableName;
|
||||
private HbasePutBatchSizeCalculator putBatchSizeCalculator;
|
||||
|
||||
public HBaseIndex(HoodieWriteConfig config) {
|
||||
super(config);
|
||||
this.tableName = config.getHbaseTableName();
|
||||
addShutDownHook();
|
||||
init(config);
|
||||
}
|
||||
|
||||
private void init(HoodieWriteConfig config) {
|
||||
this.multiPutBatchSize = config.getHbaseIndexGetBatchSize();
|
||||
this.qpsFraction = config.getHbaseIndexQPSFraction();
|
||||
this.maxQpsPerRegionServer = config.getHbaseIndexMaxQPSPerRegionServer();
|
||||
this.putBatchSizeCalculator = new HbasePutBatchSizeCalculator();
|
||||
this.hBaseIndexQPSResourceAllocator = createQPSResourceAllocator(this.config);
|
||||
}
|
||||
|
||||
@VisibleForTesting
|
||||
public HBaseIndexQPSResourceAllocator createQPSResourceAllocator(HoodieWriteConfig config) {
|
||||
try {
|
||||
logger.info("createQPSResourceAllocator :" + config.getHBaseQPSResourceAllocatorClass());
|
||||
final HBaseIndexQPSResourceAllocator resourceAllocator =
|
||||
(HBaseIndexQPSResourceAllocator) ReflectionUtils.loadClass(
|
||||
config.getHBaseQPSResourceAllocatorClass(), config);
|
||||
return resourceAllocator;
|
||||
} catch (Exception e) {
|
||||
logger.warn("error while instantiating HBaseIndexQPSResourceAllocator", e);
|
||||
}
|
||||
return new DefaultHBaseQPSResourceAllocator(config);
|
||||
}
|
||||
|
||||
@Override
|
||||
public JavaPairRDD<HoodieKey, Option<Pair<String, String>>> fetchRecordLocation(JavaRDD<HoodieKey> hoodieKeys,
|
||||
JavaSparkContext jsc, HoodieTable<T> hoodieTable) {
|
||||
throw new UnsupportedOperationException("HBase index does not implement check exist");
|
||||
}
|
||||
|
||||
private Connection getHBaseConnection() {
|
||||
Configuration hbaseConfig = HBaseConfiguration.create();
|
||||
String quorum = config.getHbaseZkQuorum();
|
||||
hbaseConfig.set("hbase.zookeeper.quorum", quorum);
|
||||
String zkZnodeParent = config.getHBaseZkZnodeParent();
|
||||
if (zkZnodeParent != null) {
|
||||
hbaseConfig.set("zookeeper.znode.parent", zkZnodeParent);
|
||||
}
|
||||
String port = String.valueOf(config.getHbaseZkPort());
|
||||
hbaseConfig.set("hbase.zookeeper.property.clientPort", port);
|
||||
try {
|
||||
return ConnectionFactory.createConnection(hbaseConfig);
|
||||
} catch (IOException e) {
|
||||
throw new HoodieDependentSystemUnavailableException(
|
||||
HoodieDependentSystemUnavailableException.HBASE, quorum + ":" + port);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Since we are sharing the HbaseConnection across tasks in a JVM, make sure the HbaseConnectio is
|
||||
* closed when JVM exits
|
||||
*/
|
||||
private void addShutDownHook() {
|
||||
Runtime.getRuntime().addShutdownHook(new Thread() {
|
||||
public void run() {
|
||||
try {
|
||||
hbaseConnection.close();
|
||||
} catch (Exception e) {
|
||||
// fail silently for any sort of exception
|
||||
}
|
||||
}
|
||||
});
|
||||
}
|
||||
|
||||
/**
|
||||
* Ensure that any resources used for indexing are released here.
|
||||
*/
|
||||
public void close() {
|
||||
this.hBaseIndexQPSResourceAllocator.releaseQPSResources();
|
||||
}
|
||||
|
||||
private Get generateStatement(String key) throws IOException {
|
||||
return new Get(Bytes.toBytes(key)).setMaxVersions(1)
|
||||
.addColumn(SYSTEM_COLUMN_FAMILY, COMMIT_TS_COLUMN)
|
||||
.addColumn(SYSTEM_COLUMN_FAMILY, FILE_NAME_COLUMN)
|
||||
.addColumn(SYSTEM_COLUMN_FAMILY, PARTITION_PATH_COLUMN);
|
||||
}
|
||||
|
||||
private boolean checkIfValidCommit(HoodieTableMetaClient metaClient, String commitTs) {
|
||||
HoodieTimeline commitTimeline = metaClient.getActiveTimeline().filterCompletedInstants();
|
||||
// Check if the last commit ts for this row is 1) present in the timeline or
|
||||
// 2) is less than the first commit ts in the timeline
|
||||
return !commitTimeline.empty() && (commitTimeline
|
||||
.containsInstant(new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, commitTs))
|
||||
|| HoodieTimeline
|
||||
.compareTimestamps(commitTimeline.firstInstant().get().getTimestamp(), commitTs,
|
||||
HoodieTimeline.GREATER));
|
||||
}
|
||||
|
||||
/**
|
||||
* Function that tags each HoodieRecord with an existing location, if known.
|
||||
*/
|
||||
private Function2<Integer, Iterator<HoodieRecord<T>>,
|
||||
Iterator<HoodieRecord<T>>> locationTagFunction(HoodieTableMetaClient metaClient) {
|
||||
|
||||
return (Function2<Integer, Iterator<HoodieRecord<T>>, Iterator<HoodieRecord<T>>>)
|
||||
(partitionNum, hoodieRecordIterator) -> {
|
||||
|
||||
Integer multiGetBatchSize = config.getHbaseIndexGetBatchSize();
|
||||
|
||||
// Grab the global HBase connection
|
||||
synchronized (HBaseIndex.class) {
|
||||
if (hbaseConnection == null || hbaseConnection.isClosed()) {
|
||||
hbaseConnection = getHBaseConnection();
|
||||
}
|
||||
}
|
||||
List<HoodieRecord<T>> taggedRecords = new ArrayList<>();
|
||||
HTable hTable = null;
|
||||
try {
|
||||
hTable = (HTable) hbaseConnection.getTable(TableName.valueOf(tableName));
|
||||
List<Get> statements = new ArrayList<>();
|
||||
List<HoodieRecord> currentBatchOfRecords = new LinkedList<>();
|
||||
// Do the tagging.
|
||||
while (hoodieRecordIterator.hasNext()) {
|
||||
HoodieRecord rec = hoodieRecordIterator.next();
|
||||
statements.add(generateStatement(rec.getRecordKey()));
|
||||
currentBatchOfRecords.add(rec);
|
||||
// iterator till we reach batch size
|
||||
if (statements.size() >= multiGetBatchSize || !hoodieRecordIterator.hasNext()) {
|
||||
// get results for batch from Hbase
|
||||
Result[] results = doGet(hTable, statements);
|
||||
// clear statements to be GC'd
|
||||
statements.clear();
|
||||
for (Result result : results) {
|
||||
// first, attempt to grab location from HBase
|
||||
HoodieRecord currentRecord = currentBatchOfRecords.remove(0);
|
||||
if (result.getRow() != null) {
|
||||
String keyFromResult = Bytes.toString(result.getRow());
|
||||
String commitTs = Bytes
|
||||
.toString(result.getValue(SYSTEM_COLUMN_FAMILY, COMMIT_TS_COLUMN));
|
||||
String fileId = Bytes
|
||||
.toString(result.getValue(SYSTEM_COLUMN_FAMILY, FILE_NAME_COLUMN));
|
||||
String partitionPath = Bytes
|
||||
.toString(result.getValue(SYSTEM_COLUMN_FAMILY, PARTITION_PATH_COLUMN));
|
||||
|
||||
if (checkIfValidCommit(metaClient, commitTs)) {
|
||||
currentRecord = new HoodieRecord(
|
||||
new HoodieKey(currentRecord.getRecordKey(), partitionPath),
|
||||
currentRecord.getData());
|
||||
currentRecord.setCurrentLocation(new HoodieRecordLocation(commitTs, fileId));
|
||||
taggedRecords.add(currentRecord);
|
||||
// the key from Result and the key being processed should be same
|
||||
assert (currentRecord.getRecordKey().contentEquals(keyFromResult));
|
||||
} else { //if commit is invalid, treat this as a new taggedRecord
|
||||
taggedRecords.add(currentRecord);
|
||||
}
|
||||
} else {
|
||||
taggedRecords.add(currentRecord);
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
} catch (IOException e) {
|
||||
throw new HoodieIndexException(
|
||||
"Failed to Tag indexed locations because of exception with HBase Client", e);
|
||||
} finally {
|
||||
if (hTable != null) {
|
||||
try {
|
||||
hTable.close();
|
||||
} catch (IOException e) {
|
||||
// Ignore
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
return taggedRecords.iterator();
|
||||
};
|
||||
}
|
||||
|
||||
private Result[] doGet(HTable hTable, List<Get> keys) throws IOException {
|
||||
sleepForTime(SLEEP_TIME_MILLISECONDS);
|
||||
return hTable.get(keys);
|
||||
}
|
||||
|
||||
@Override
|
||||
public JavaRDD<HoodieRecord<T>> tagLocation(JavaRDD<HoodieRecord<T>> recordRDD, JavaSparkContext jsc,
|
||||
HoodieTable<T> hoodieTable) {
|
||||
return recordRDD.mapPartitionsWithIndex(locationTagFunction(hoodieTable.getMetaClient()), true);
|
||||
}
|
||||
|
||||
private Function2<Integer, Iterator<WriteStatus>, Iterator<WriteStatus>> updateLocationFunction() {
|
||||
|
||||
return (Function2<Integer, Iterator<WriteStatus>, Iterator<WriteStatus>>) (partition, statusIterator) -> {
|
||||
|
||||
List<WriteStatus> writeStatusList = new ArrayList<>();
|
||||
// Grab the global HBase connection
|
||||
synchronized (HBaseIndex.class) {
|
||||
if (hbaseConnection == null || hbaseConnection.isClosed()) {
|
||||
hbaseConnection = getHBaseConnection();
|
||||
}
|
||||
}
|
||||
HTable hTable = null;
|
||||
try {
|
||||
hTable = (HTable) hbaseConnection.getTable(TableName.valueOf(tableName));
|
||||
while (statusIterator.hasNext()) {
|
||||
WriteStatus writeStatus = statusIterator.next();
|
||||
List<Put> puts = new ArrayList<>();
|
||||
List<Delete> deletes = new ArrayList<>();
|
||||
try {
|
||||
for (HoodieRecord rec : writeStatus.getWrittenRecords()) {
|
||||
if (!writeStatus.isErrored(rec.getKey())) {
|
||||
Option<HoodieRecordLocation> loc = rec.getNewLocation();
|
||||
if (loc.isPresent()) {
|
||||
if (rec.getCurrentLocation() != null) {
|
||||
// This is an update, no need to update index
|
||||
continue;
|
||||
}
|
||||
Put put = new Put(Bytes.toBytes(rec.getRecordKey()));
|
||||
put.addColumn(SYSTEM_COLUMN_FAMILY, COMMIT_TS_COLUMN,
|
||||
Bytes.toBytes(loc.get().getInstantTime()));
|
||||
put.addColumn(SYSTEM_COLUMN_FAMILY, FILE_NAME_COLUMN,
|
||||
Bytes.toBytes(loc.get().getFileId()));
|
||||
put.addColumn(SYSTEM_COLUMN_FAMILY, PARTITION_PATH_COLUMN,
|
||||
Bytes.toBytes(rec.getPartitionPath()));
|
||||
puts.add(put);
|
||||
} else {
|
||||
//Delete existing index for a deleted record
|
||||
Delete delete = new Delete(Bytes.toBytes(rec.getRecordKey()));
|
||||
deletes.add(delete);
|
||||
}
|
||||
}
|
||||
if (puts.size() + deletes.size() < multiPutBatchSize) {
|
||||
continue;
|
||||
}
|
||||
doPutsAndDeletes(hTable, puts, deletes);
|
||||
}
|
||||
//process remaining puts and deletes, if any
|
||||
doPutsAndDeletes(hTable, puts, deletes);
|
||||
} catch (Exception e) {
|
||||
Exception we = new Exception("Error updating index for " + writeStatus, e);
|
||||
logger.error(we);
|
||||
writeStatus.setGlobalError(we);
|
||||
}
|
||||
writeStatusList.add(writeStatus);
|
||||
}
|
||||
} catch (IOException e) {
|
||||
throw new HoodieIndexException(
|
||||
"Failed to Update Index locations because of exception with HBase Client", e);
|
||||
} finally {
|
||||
if (hTable != null) {
|
||||
try {
|
||||
hTable.close();
|
||||
} catch (IOException e) {
|
||||
// Ignore
|
||||
}
|
||||
}
|
||||
}
|
||||
return writeStatusList.iterator();
|
||||
};
|
||||
}
|
||||
|
||||
/**
|
||||
* Helper method to facilitate performing puts and deletes in Hbase
|
||||
*/
|
||||
private void doPutsAndDeletes(HTable hTable, List<Put> puts, List<Delete> deletes)
|
||||
throws IOException {
|
||||
if (puts.size() > 0) {
|
||||
hTable.put(puts);
|
||||
}
|
||||
if (deletes.size() > 0) {
|
||||
hTable.delete(deletes);
|
||||
}
|
||||
hTable.flushCommits();
|
||||
puts.clear();
|
||||
deletes.clear();
|
||||
sleepForTime(SLEEP_TIME_MILLISECONDS);
|
||||
}
|
||||
|
||||
private static void sleepForTime(int sleepTimeMs) {
|
||||
try {
|
||||
Thread.sleep(sleepTimeMs);
|
||||
} catch (InterruptedException e) {
|
||||
logger.error("Sleep interrupted during throttling", e);
|
||||
throw new RuntimeException(e);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public JavaRDD<WriteStatus> updateLocation(JavaRDD<WriteStatus> writeStatusRDD, JavaSparkContext jsc,
|
||||
HoodieTable<T> hoodieTable) {
|
||||
final HBaseIndexQPSResourceAllocator hBaseIndexQPSResourceAllocator = createQPSResourceAllocator(this.config);
|
||||
setPutBatchSize(writeStatusRDD, hBaseIndexQPSResourceAllocator, jsc);
|
||||
logger.info("multiPutBatchSize: before hbase puts" + multiPutBatchSize);
|
||||
JavaRDD<WriteStatus> writeStatusJavaRDD = writeStatusRDD.mapPartitionsWithIndex(
|
||||
updateLocationFunction(), true);
|
||||
// caching the index updated status RDD
|
||||
writeStatusJavaRDD = writeStatusJavaRDD.persist(config.getWriteStatusStorageLevel());
|
||||
return writeStatusJavaRDD;
|
||||
}
|
||||
|
||||
private void setPutBatchSize(JavaRDD<WriteStatus> writeStatusRDD,
|
||||
HBaseIndexQPSResourceAllocator hBaseIndexQPSResourceAllocator,
|
||||
final JavaSparkContext jsc) {
|
||||
if (config.getHbaseIndexPutBatchSizeAutoCompute()) {
|
||||
SparkConf conf = jsc.getConf();
|
||||
int maxExecutors = conf.getInt(DEFAULT_SPARK_EXECUTOR_INSTANCES_CONFIG_NAME, 1);
|
||||
if (conf.getBoolean(DEFAULT_SPARK_DYNAMIC_ALLOCATION_ENABLED_CONFIG_NAME, false)) {
|
||||
maxExecutors = Math.max(maxExecutors, conf.getInt(
|
||||
DEFAULT_SPARK_DYNAMIC_ALLOCATION_MAX_EXECUTORS_CONFIG_NAME, 1));
|
||||
}
|
||||
|
||||
/*
|
||||
Each writeStatus represents status information from a write done in one of the IOHandles.
|
||||
If a writeStatus has any insert, it implies that the corresponding task contacts HBase for
|
||||
doing puts, since we only do puts for inserts from HBaseIndex.
|
||||
*/
|
||||
final Tuple2<Long, Integer> numPutsParallelismTuple = getHBasePutAccessParallelism(writeStatusRDD);
|
||||
final long numPuts = numPutsParallelismTuple._1;
|
||||
final int hbasePutsParallelism = numPutsParallelismTuple._2;
|
||||
this.numRegionServersForTable = getNumRegionServersAliveForTable();
|
||||
final float desiredQPSFraction = hBaseIndexQPSResourceAllocator
|
||||
.calculateQPSFractionForPutsTime(numPuts, this.numRegionServersForTable);
|
||||
logger.info("Desired QPSFraction :" + desiredQPSFraction);
|
||||
logger.info("Number HBase puts :" + numPuts);
|
||||
logger.info("Hbase Puts Parallelism :" + hbasePutsParallelism);
|
||||
final float availableQpsFraction = hBaseIndexQPSResourceAllocator
|
||||
.acquireQPSResources(desiredQPSFraction, numPuts);
|
||||
logger.info("Allocated QPS Fraction :" + availableQpsFraction);
|
||||
multiPutBatchSize = putBatchSizeCalculator
|
||||
.getBatchSize(
|
||||
numRegionServersForTable,
|
||||
maxQpsPerRegionServer,
|
||||
hbasePutsParallelism,
|
||||
maxExecutors,
|
||||
SLEEP_TIME_MILLISECONDS,
|
||||
availableQpsFraction);
|
||||
logger.info("multiPutBatchSize :" + multiPutBatchSize);
|
||||
}
|
||||
}
|
||||
|
||||
@VisibleForTesting
|
||||
public Tuple2<Long, Integer> getHBasePutAccessParallelism(final JavaRDD<WriteStatus> writeStatusRDD) {
|
||||
final JavaPairRDD<Long, Integer> insertOnlyWriteStatusRDD =
|
||||
writeStatusRDD.filter(w -> w.getStat().getNumInserts() > 0)
|
||||
.mapToPair(w -> new Tuple2<>(w.getStat().getNumInserts(), 1));
|
||||
return insertOnlyWriteStatusRDD.fold(new Tuple2<>(0L, 0), (w, c) -> new Tuple2<>(w._1 + c._1, w._2 + c._2));
|
||||
}
|
||||
|
||||
public static class HbasePutBatchSizeCalculator implements Serializable {
|
||||
|
||||
private static final int MILLI_SECONDS_IN_A_SECOND = 1000;
|
||||
private static Logger logger = LogManager.getLogger(HbasePutBatchSizeCalculator.class);
|
||||
|
||||
/**
|
||||
* Calculate putBatch size so that sum of requests across multiple jobs in a second does not exceed
|
||||
* maxQpsPerRegionServer for each Region Server. Multiplying qpsFraction to reduce the aggregate load on common RS
|
||||
* across topics. Assumption here is that all tables have regions across all RS, which is not necessarily true for
|
||||
* smaller tables. So, they end up getting a smaller share of QPS than they deserve, but it might be ok.
|
||||
* <p>
|
||||
* Example: int putBatchSize = batchSizeCalculator.getBatchSize(10, 16667, 1200, 200, 100, 0.1f)
|
||||
* </p>
|
||||
* <p>
|
||||
* Expected batchSize is 8 because in that case, total request sent to a Region Server in one second is:
|
||||
*
|
||||
* 8 (batchSize) * 200 (parallelism) * 10 (maxReqsInOneSecond) * 10 (numRegionServers) * 0.1 (qpsFraction)) =>
|
||||
* 16000. We assume requests get distributed to Region Servers uniformly, so each RS gets 1600 requests which
|
||||
* happens to be 10% of 16667 (maxQPSPerRegionServer), as expected.
|
||||
* </p>
|
||||
* <p> Assumptions made here <li> In a batch, writes get evenly distributed to each RS for that
|
||||
* table. Since we do writes only in the case of inserts and not updates, for this assumption to fail, inserts would
|
||||
* have to be skewed towards few RS, likelihood of which is less if Hbase table is pre-split and rowKeys are UUIDs
|
||||
* (random strings). If this assumption fails, then it is possible for some RS to receive more than
|
||||
* maxQpsPerRegionServer QPS, but for simplicity, we are going ahead with this model, since this is meant to be a
|
||||
* lightweight distributed throttling mechanism without maintaining a global context. So if this assumption breaks,
|
||||
* we are hoping the HBase Master relocates hot-spot regions to new Region Servers.
|
||||
*
|
||||
* </li> <li> For Region Server stability, throttling at a second level granularity is fine.
|
||||
* Although, within a second, the sum of queries might be within maxQpsPerRegionServer, there could be peaks at some
|
||||
* sub second intervals. So, the assumption is that these peaks are tolerated by the Region Server (which at max can
|
||||
* be maxQpsPerRegionServer). </li> </p>
|
||||
*/
|
||||
public int getBatchSize(int numRegionServersForTable, int maxQpsPerRegionServer,
|
||||
int numTasksDuringPut, int maxExecutors, int sleepTimeMs, float qpsFraction) {
|
||||
int numRSAlive = numRegionServersForTable;
|
||||
int maxReqPerSec = (int) (qpsFraction * numRSAlive * maxQpsPerRegionServer);
|
||||
int numTasks = numTasksDuringPut;
|
||||
int maxParallelPuts = Math.max(1, Math.min(numTasks, maxExecutors));
|
||||
int maxReqsSentPerTaskPerSec = MILLI_SECONDS_IN_A_SECOND / sleepTimeMs;
|
||||
int multiPutBatchSize = Math.max(1, maxReqPerSec / (maxParallelPuts * maxReqsSentPerTaskPerSec));
|
||||
logger.info("HbaseIndexThrottling: qpsFraction :" + qpsFraction);
|
||||
logger.info("HbaseIndexThrottling: numRSAlive :" + numRSAlive);
|
||||
logger.info("HbaseIndexThrottling: maxReqPerSec :" + maxReqPerSec);
|
||||
logger.info("HbaseIndexThrottling: numTasks :" + numTasks);
|
||||
logger.info("HbaseIndexThrottling: maxExecutors :" + maxExecutors);
|
||||
logger.info("HbaseIndexThrottling: maxParallelPuts :" + maxParallelPuts);
|
||||
logger.info("HbaseIndexThrottling: maxReqsSentPerTaskPerSec :" + maxReqsSentPerTaskPerSec);
|
||||
logger.info("HbaseIndexThrottling: numRegionServersForTable :" + numRegionServersForTable);
|
||||
logger.info("HbaseIndexThrottling: multiPutBatchSize :" + multiPutBatchSize);
|
||||
return multiPutBatchSize;
|
||||
}
|
||||
}
|
||||
|
||||
private Integer getNumRegionServersAliveForTable() {
|
||||
// This is being called in the driver, so there is only one connection
|
||||
// from the driver, so ok to use a local connection variable.
|
||||
if (numRegionServersForTable == null) {
|
||||
try (Connection conn = getHBaseConnection()) {
|
||||
RegionLocator regionLocator = conn
|
||||
.getRegionLocator(TableName.valueOf(tableName));
|
||||
numRegionServersForTable = Math.toIntExact(
|
||||
regionLocator.getAllRegionLocations().stream().map(e -> e.getServerName()).distinct()
|
||||
.count());
|
||||
return numRegionServersForTable;
|
||||
} catch (IOException e) {
|
||||
logger.error(e);
|
||||
throw new RuntimeException(e);
|
||||
}
|
||||
}
|
||||
return numRegionServersForTable;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean rollbackCommit(String commitTime) {
|
||||
// Rollback in HbaseIndex is managed via method {@link #checkIfValidCommit()}
|
||||
return true;
|
||||
}
|
||||
|
||||
/**
|
||||
* Only looks up by recordKey
|
||||
*/
|
||||
@Override
|
||||
public boolean isGlobal() {
|
||||
return true;
|
||||
}
|
||||
|
||||
/**
|
||||
* Mapping is available in HBase already.
|
||||
*/
|
||||
@Override
|
||||
public boolean canIndexLogFiles() {
|
||||
return true;
|
||||
}
|
||||
|
||||
/**
|
||||
* Index needs to be explicitly updated after storage write.
|
||||
*/
|
||||
@Override
|
||||
public boolean isImplicitWithStorage() {
|
||||
return false;
|
||||
}
|
||||
|
||||
@VisibleForTesting
|
||||
public void setHbaseConnection(Connection hbaseConnection) {
|
||||
HBaseIndex.hbaseConnection = hbaseConnection;
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,51 @@
|
||||
/*
|
||||
* 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.index.hbase;
|
||||
|
||||
import java.io.Serializable;
|
||||
|
||||
/**
|
||||
* <code>HBaseIndexQPSResourceAllocator</code> defines methods to manage resource allocation for HBase index operations
|
||||
*/
|
||||
public interface HBaseIndexQPSResourceAllocator extends Serializable {
|
||||
|
||||
/**
|
||||
* This method returns the QPS Fraction value that needs to be acquired such that the respective
|
||||
* HBase index operation can be completed in desiredPutsTime.
|
||||
*
|
||||
* @param numPuts Number of inserts to be written to HBase index
|
||||
* @param desiredPutsTimeInSecs Total expected time for the HBase inserts operation
|
||||
* @return QPS fraction that needs to be acquired.
|
||||
*/
|
||||
float calculateQPSFractionForPutsTime(final long numPuts, final int desiredPutsTimeInSecs);
|
||||
|
||||
/**
|
||||
* This method acquires the requested QPS Fraction against HBase cluster for index operation.
|
||||
*
|
||||
* @param desiredQPSFraction QPS fraction that needs to be requested and acquired
|
||||
* @param numPuts Number of inserts to be written to HBase index
|
||||
* @return value of the acquired QPS Fraction.
|
||||
*/
|
||||
float acquireQPSResources(final float desiredQPSFraction, final long numPuts);
|
||||
|
||||
/**
|
||||
* This method releases the acquired QPS Fraction
|
||||
*/
|
||||
void releaseQPSResources();
|
||||
}
|
||||
@@ -0,0 +1,322 @@
|
||||
/*
|
||||
* 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.io;
|
||||
|
||||
import com.beust.jcommander.internal.Maps;
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Iterator;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.concurrent.atomic.AtomicLong;
|
||||
import org.apache.avro.generic.GenericRecord;
|
||||
import org.apache.avro.generic.IndexedRecord;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hudi.WriteStatus;
|
||||
import org.apache.hudi.common.model.FileSlice;
|
||||
import org.apache.hudi.common.model.HoodieDeltaWriteStat;
|
||||
import org.apache.hudi.common.model.HoodieKey;
|
||||
import org.apache.hudi.common.model.HoodieLogFile;
|
||||
import org.apache.hudi.common.model.HoodieRecord;
|
||||
import org.apache.hudi.common.model.HoodieRecordLocation;
|
||||
import org.apache.hudi.common.model.HoodieRecordPayload;
|
||||
import org.apache.hudi.common.model.HoodieWriteStat;
|
||||
import org.apache.hudi.common.model.HoodieWriteStat.RuntimeStats;
|
||||
import org.apache.hudi.common.table.TableFileSystemView.RealtimeView;
|
||||
import org.apache.hudi.common.table.log.HoodieLogFormat;
|
||||
import org.apache.hudi.common.table.log.HoodieLogFormat.Writer;
|
||||
import org.apache.hudi.common.table.log.block.HoodieAvroDataBlock;
|
||||
import org.apache.hudi.common.table.log.block.HoodieDeleteBlock;
|
||||
import org.apache.hudi.common.table.log.block.HoodieLogBlock;
|
||||
import org.apache.hudi.common.table.log.block.HoodieLogBlock.HeaderMetadataType;
|
||||
import org.apache.hudi.common.util.FSUtils;
|
||||
import org.apache.hudi.common.util.HoodieAvroUtils;
|
||||
import org.apache.hudi.common.util.Option;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
import org.apache.hudi.exception.HoodieAppendException;
|
||||
import org.apache.hudi.exception.HoodieUpsertException;
|
||||
import org.apache.hudi.table.HoodieTable;
|
||||
import org.apache.log4j.LogManager;
|
||||
import org.apache.log4j.Logger;
|
||||
import org.apache.spark.TaskContext;
|
||||
import org.apache.spark.util.SizeEstimator;
|
||||
|
||||
/**
|
||||
* IO Operation to append data onto an existing file.
|
||||
*/
|
||||
public class HoodieAppendHandle<T extends HoodieRecordPayload> extends HoodieWriteHandle<T> {
|
||||
|
||||
private static Logger logger = LogManager.getLogger(HoodieAppendHandle.class);
|
||||
// This acts as the sequenceID for records written
|
||||
private static AtomicLong recordIndex = new AtomicLong(1);
|
||||
private final String fileId;
|
||||
// Buffer for holding records in memory before they are flushed to disk
|
||||
private List<IndexedRecord> recordList = new ArrayList<>();
|
||||
// Buffer for holding records (to be deleted) in memory before they are flushed to disk
|
||||
private List<HoodieKey> keysToDelete = new ArrayList<>();
|
||||
|
||||
private String partitionPath;
|
||||
private Iterator<HoodieRecord<T>> recordItr;
|
||||
// Total number of records written during an append
|
||||
private long recordsWritten = 0;
|
||||
// Total number of records deleted during an append
|
||||
private long recordsDeleted = 0;
|
||||
// Total number of records updated during an append
|
||||
private long updatedRecordsWritten = 0;
|
||||
// Average record size for a HoodieRecord. This size is updated at the end of every log block flushed to disk
|
||||
private long averageRecordSize = 0;
|
||||
private HoodieLogFile currentLogFile;
|
||||
private Writer writer;
|
||||
// Flag used to initialize some metadata
|
||||
private boolean doInit = true;
|
||||
// Total number of bytes written during this append phase (an estimation)
|
||||
private long estimatedNumberOfBytesWritten;
|
||||
// Number of records that must be written to meet the max block size for a log block
|
||||
private int numberOfRecords = 0;
|
||||
// Max block size to limit to for a log block
|
||||
private int maxBlockSize = config.getLogFileDataBlockMaxSize();
|
||||
// Header metadata for a log block
|
||||
private Map<HeaderMetadataType, String> header = Maps.newHashMap();
|
||||
// Total number of new records inserted into the delta file
|
||||
private long insertRecordsWritten = 0;
|
||||
|
||||
public HoodieAppendHandle(HoodieWriteConfig config, String commitTime, HoodieTable<T> hoodieTable,
|
||||
String fileId, Iterator<HoodieRecord<T>> recordItr) {
|
||||
super(config, commitTime, fileId, hoodieTable);
|
||||
writeStatus.setStat(new HoodieDeltaWriteStat());
|
||||
this.fileId = fileId;
|
||||
this.recordItr = recordItr;
|
||||
}
|
||||
|
||||
public HoodieAppendHandle(HoodieWriteConfig config, String commitTime, HoodieTable<T> hoodieTable, String fileId) {
|
||||
this(config, commitTime, hoodieTable, fileId, null);
|
||||
}
|
||||
|
||||
private void init(HoodieRecord record) {
|
||||
if (doInit) {
|
||||
this.partitionPath = record.getPartitionPath();
|
||||
// extract some information from the first record
|
||||
RealtimeView rtView = hoodieTable.getRTFileSystemView();
|
||||
Option<FileSlice> fileSlice = rtView.getLatestFileSlice(partitionPath, fileId);
|
||||
// Set the base commit time as the current commitTime for new inserts into log files
|
||||
String baseInstantTime = instantTime;
|
||||
if (fileSlice.isPresent()) {
|
||||
baseInstantTime = fileSlice.get().getBaseInstantTime();
|
||||
} else {
|
||||
// This means there is no base data file, start appending to a new log file
|
||||
fileSlice = Option.of(new FileSlice(partitionPath, baseInstantTime, this.fileId));
|
||||
logger.info("New InsertHandle for partition :" + partitionPath);
|
||||
}
|
||||
writeStatus.getStat().setPrevCommit(baseInstantTime);
|
||||
writeStatus.setFileId(fileId);
|
||||
writeStatus.setPartitionPath(partitionPath);
|
||||
writeStatus.getStat().setPartitionPath(partitionPath);
|
||||
writeStatus.getStat().setFileId(fileId);
|
||||
averageRecordSize = SizeEstimator.estimate(record);
|
||||
try {
|
||||
this.writer = createLogWriter(fileSlice, baseInstantTime);
|
||||
this.currentLogFile = writer.getLogFile();
|
||||
((HoodieDeltaWriteStat) writeStatus.getStat()).setLogVersion(currentLogFile.getLogVersion());
|
||||
((HoodieDeltaWriteStat) writeStatus.getStat()).setLogOffset(writer.getCurrentSize());
|
||||
} catch (Exception e) {
|
||||
logger.error("Error in update task at commit " + instantTime, e);
|
||||
writeStatus.setGlobalError(e);
|
||||
throw new HoodieUpsertException(
|
||||
"Failed to initialize HoodieAppendHandle for FileId: " + fileId + " on commit "
|
||||
+ instantTime + " on HDFS path " + hoodieTable.getMetaClient().getBasePath()
|
||||
+ partitionPath, e);
|
||||
}
|
||||
Path path = new Path(partitionPath, writer.getLogFile().getFileName());
|
||||
writeStatus.getStat().setPath(path.toString());
|
||||
doInit = false;
|
||||
}
|
||||
}
|
||||
|
||||
private Option<IndexedRecord> getIndexedRecord(HoodieRecord<T> hoodieRecord) {
|
||||
Option recordMetadata = hoodieRecord.getData().getMetadata();
|
||||
try {
|
||||
Option<IndexedRecord> avroRecord = hoodieRecord.getData().getInsertValue(originalSchema);
|
||||
if (avroRecord.isPresent()) {
|
||||
// Convert GenericRecord to GenericRecord with hoodie commit metadata in schema
|
||||
avroRecord = Option.of(rewriteRecord((GenericRecord) avroRecord.get()));
|
||||
String seqId = HoodieRecord.generateSequenceId(instantTime, TaskContext.getPartitionId(),
|
||||
recordIndex.getAndIncrement());
|
||||
HoodieAvroUtils
|
||||
.addHoodieKeyToRecord((GenericRecord) avroRecord.get(), hoodieRecord.getRecordKey(),
|
||||
hoodieRecord.getPartitionPath(), fileId);
|
||||
HoodieAvroUtils
|
||||
.addCommitMetadataToRecord((GenericRecord) avroRecord.get(), instantTime, seqId);
|
||||
// If currentLocation is present, then this is an update
|
||||
if (hoodieRecord.getCurrentLocation() != null) {
|
||||
updatedRecordsWritten++;
|
||||
} else {
|
||||
insertRecordsWritten++;
|
||||
}
|
||||
recordsWritten++;
|
||||
} else {
|
||||
recordsDeleted++;
|
||||
}
|
||||
|
||||
writeStatus.markSuccess(hoodieRecord, recordMetadata);
|
||||
// deflate record payload after recording success. This will help users access payload as a
|
||||
// part of marking
|
||||
// record successful.
|
||||
hoodieRecord.deflate();
|
||||
return avroRecord;
|
||||
} catch (Exception e) {
|
||||
logger.error("Error writing record " + hoodieRecord, e);
|
||||
writeStatus.markFailure(hoodieRecord, e, recordMetadata);
|
||||
}
|
||||
return Option.empty();
|
||||
}
|
||||
|
||||
// TODO (NA) - Perform a writerSchema check of current input record with the last writerSchema on log file
|
||||
// to make sure we don't append records with older (shorter) writerSchema than already appended
|
||||
public void doAppend() {
|
||||
while (recordItr.hasNext()) {
|
||||
HoodieRecord record = recordItr.next();
|
||||
init(record);
|
||||
flushToDiskIfRequired(record);
|
||||
writeToBuffer(record);
|
||||
}
|
||||
doAppend(header);
|
||||
estimatedNumberOfBytesWritten += averageRecordSize * numberOfRecords;
|
||||
}
|
||||
|
||||
private void doAppend(Map<HeaderMetadataType, String> header) {
|
||||
try {
|
||||
header.put(HoodieLogBlock.HeaderMetadataType.INSTANT_TIME, instantTime);
|
||||
header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, writerSchema.toString());
|
||||
if (recordList.size() > 0) {
|
||||
writer = writer.appendBlock(new HoodieAvroDataBlock(recordList, header));
|
||||
recordList.clear();
|
||||
}
|
||||
if (keysToDelete.size() > 0) {
|
||||
writer = writer.appendBlock(
|
||||
new HoodieDeleteBlock(keysToDelete.stream().toArray(HoodieKey[]::new), header));
|
||||
keysToDelete.clear();
|
||||
}
|
||||
} catch (Exception e) {
|
||||
throw new HoodieAppendException(
|
||||
"Failed while appending records to " + currentLogFile.getPath(), e);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean canWrite(HoodieRecord record) {
|
||||
return config.getParquetMaxFileSize() >= estimatedNumberOfBytesWritten * config
|
||||
.getLogFileToParquetCompressionRatio();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void write(HoodieRecord record, Option<IndexedRecord> insertValue) {
|
||||
Option recordMetadata = record.getData().getMetadata();
|
||||
try {
|
||||
init(record);
|
||||
flushToDiskIfRequired(record);
|
||||
writeToBuffer(record);
|
||||
} catch (Throwable t) {
|
||||
// Not throwing exception from here, since we don't want to fail the entire job
|
||||
// for a single record
|
||||
writeStatus.markFailure(record, t, recordMetadata);
|
||||
logger.error("Error writing record " + record, t);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public WriteStatus close() {
|
||||
try {
|
||||
// flush any remaining records to disk
|
||||
doAppend(header);
|
||||
long sizeInBytes = writer.getCurrentSize();
|
||||
if (writer != null) {
|
||||
writer.close();
|
||||
}
|
||||
|
||||
HoodieWriteStat stat = writeStatus.getStat();
|
||||
stat.setFileId(this.fileId);
|
||||
stat.setNumWrites(recordsWritten);
|
||||
stat.setNumUpdateWrites(updatedRecordsWritten);
|
||||
stat.setNumInserts(insertRecordsWritten);
|
||||
stat.setNumDeletes(recordsDeleted);
|
||||
stat.setTotalWriteBytes(estimatedNumberOfBytesWritten);
|
||||
stat.setFileSizeInBytes(sizeInBytes);
|
||||
stat.setTotalWriteErrors(writeStatus.getTotalErrorRecords());
|
||||
RuntimeStats runtimeStats = new RuntimeStats();
|
||||
runtimeStats.setTotalUpsertTime(timer.endTimer());
|
||||
stat.setRuntimeStats(runtimeStats);
|
||||
|
||||
logger.info(String.format("AppendHandle for partitionPath %s fileID %s, took %d ms.",
|
||||
stat.getPartitionPath(), stat.getFileId(), runtimeStats.getTotalUpsertTime()));
|
||||
|
||||
return writeStatus;
|
||||
} catch (IOException e) {
|
||||
throw new HoodieUpsertException("Failed to close UpdateHandle", e);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public WriteStatus getWriteStatus() {
|
||||
return writeStatus;
|
||||
}
|
||||
|
||||
private Writer createLogWriter(Option<FileSlice> fileSlice, String baseCommitTime)
|
||||
throws IOException, InterruptedException {
|
||||
Option<HoodieLogFile> latestLogFile = fileSlice.get().getLatestLogFile();
|
||||
|
||||
return HoodieLogFormat.newWriterBuilder()
|
||||
.onParentPath(FSUtils.getPartitionPath(hoodieTable.getMetaClient().getBasePath(), partitionPath))
|
||||
.withFileId(fileId).overBaseCommit(baseCommitTime).withLogVersion(
|
||||
latestLogFile.map(HoodieLogFile::getLogVersion).orElse(HoodieLogFile.LOGFILE_BASE_VERSION))
|
||||
.withSizeThreshold(config.getLogFileMaxSize()).withFs(fs)
|
||||
.withLogWriteToken(
|
||||
latestLogFile.map(x -> FSUtils.getWriteTokenFromLogPath(x.getPath())).orElse(writeToken))
|
||||
.withRolloverLogWriteToken(writeToken)
|
||||
.withFileExtension(HoodieLogFile.DELTA_EXTENSION).build();
|
||||
}
|
||||
|
||||
private void writeToBuffer(HoodieRecord<T> record) {
|
||||
// update the new location of the record, so we know where to find it next
|
||||
record.setNewLocation(new HoodieRecordLocation(instantTime, fileId));
|
||||
Option<IndexedRecord> indexedRecord = getIndexedRecord(record);
|
||||
if (indexedRecord.isPresent()) {
|
||||
recordList.add(indexedRecord.get());
|
||||
} else {
|
||||
keysToDelete.add(record.getKey());
|
||||
}
|
||||
numberOfRecords++;
|
||||
}
|
||||
|
||||
/**
|
||||
* Checks if the number of records have reached the set threshold and then flushes the records to disk
|
||||
*/
|
||||
private void flushToDiskIfRequired(HoodieRecord record) {
|
||||
// Append if max number of records reached to achieve block size
|
||||
if (numberOfRecords >= (int) (maxBlockSize / averageRecordSize)) {
|
||||
// Recompute averageRecordSize before writing a new block and update existing value with
|
||||
// avg of new and old
|
||||
logger.info("AvgRecordSize => " + averageRecordSize);
|
||||
averageRecordSize = (averageRecordSize + SizeEstimator.estimate(record)) / 2;
|
||||
doAppend(header);
|
||||
estimatedNumberOfBytesWritten += averageRecordSize * numberOfRecords;
|
||||
numberOfRecords = 0;
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
@@ -0,0 +1,273 @@
|
||||
/*
|
||||
* 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.io;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Iterator;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.stream.Collectors;
|
||||
import org.apache.hudi.common.model.CompactionOperation;
|
||||
import org.apache.hudi.common.model.FileSlice;
|
||||
import org.apache.hudi.common.model.HoodieCleaningPolicy;
|
||||
import org.apache.hudi.common.model.HoodieCommitMetadata;
|
||||
import org.apache.hudi.common.model.HoodieDataFile;
|
||||
import org.apache.hudi.common.model.HoodieFileGroup;
|
||||
import org.apache.hudi.common.model.HoodieFileGroupId;
|
||||
import org.apache.hudi.common.model.HoodieRecordPayload;
|
||||
import org.apache.hudi.common.model.HoodieTableType;
|
||||
import org.apache.hudi.common.table.HoodieTimeline;
|
||||
import org.apache.hudi.common.table.SyncableFileSystemView;
|
||||
import org.apache.hudi.common.table.timeline.HoodieInstant;
|
||||
import org.apache.hudi.common.util.Option;
|
||||
import org.apache.hudi.common.util.collection.Pair;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
import org.apache.hudi.table.HoodieTable;
|
||||
import org.apache.log4j.LogManager;
|
||||
import org.apache.log4j.Logger;
|
||||
|
||||
/**
|
||||
* Cleaner is responsible for garbage collecting older files in a given partition path, such that
|
||||
* <p> 1) It provides sufficient time for existing queries running on older versions, to close <p>
|
||||
* 2) It bounds the growth of the files in the file system <p> TODO: Should all cleaning be done
|
||||
* based on {@link HoodieCommitMetadata}
|
||||
*/
|
||||
public class HoodieCleanHelper<T extends HoodieRecordPayload<T>> {
|
||||
|
||||
private static Logger logger = LogManager.getLogger(HoodieCleanHelper.class);
|
||||
|
||||
private final SyncableFileSystemView fileSystemView;
|
||||
private final HoodieTimeline commitTimeline;
|
||||
private final Map<HoodieFileGroupId, CompactionOperation> fgIdToPendingCompactionOperations;
|
||||
private HoodieTable<T> hoodieTable;
|
||||
private HoodieWriteConfig config;
|
||||
|
||||
public HoodieCleanHelper(HoodieTable<T> hoodieTable, HoodieWriteConfig config) {
|
||||
this.hoodieTable = hoodieTable;
|
||||
this.fileSystemView = hoodieTable.getHoodieView();
|
||||
this.commitTimeline = hoodieTable.getCompletedCommitTimeline();
|
||||
this.config = config;
|
||||
this.fgIdToPendingCompactionOperations =
|
||||
((SyncableFileSystemView)hoodieTable.getRTFileSystemView()).getPendingCompactionOperations()
|
||||
.map(entry -> Pair.of(new HoodieFileGroupId(entry.getValue().getPartitionPath(),
|
||||
entry.getValue().getFileId()), entry.getValue()))
|
||||
.collect(Collectors.toMap(Pair::getKey, Pair::getValue));
|
||||
}
|
||||
|
||||
/**
|
||||
* Selects the older versions of files for cleaning, such that it bounds the number of versions of
|
||||
* each file. This policy is useful, if you are simply interested in querying the table, and you
|
||||
* don't want too many versions for a single file (i.e run it with versionsRetained = 1)
|
||||
*/
|
||||
private List<String> getFilesToCleanKeepingLatestVersions(String partitionPath)
|
||||
throws IOException {
|
||||
logger.info("Cleaning " + partitionPath + ", retaining latest " + config
|
||||
.getCleanerFileVersionsRetained() + " file versions. ");
|
||||
List<HoodieFileGroup> fileGroups = fileSystemView.getAllFileGroups(partitionPath)
|
||||
.collect(Collectors.toList());
|
||||
List<String> deletePaths = new ArrayList<>();
|
||||
// Collect all the datafiles savepointed by all the savepoints
|
||||
List<String> savepointedFiles = hoodieTable.getSavepoints().stream()
|
||||
.flatMap(s -> hoodieTable.getSavepointedDataFiles(s)).collect(Collectors.toList());
|
||||
|
||||
for (HoodieFileGroup fileGroup : fileGroups) {
|
||||
int keepVersions = config.getCleanerFileVersionsRetained();
|
||||
// do not cleanup slice required for pending compaction
|
||||
Iterator<FileSlice> fileSliceIterator = fileGroup.getAllFileSlices()
|
||||
.filter(fs -> !isFileSliceNeededForPendingCompaction(fs)).iterator();
|
||||
if (isFileGroupInPendingCompaction(fileGroup)) {
|
||||
// We have already saved the last version of file-groups for pending compaction Id
|
||||
keepVersions--;
|
||||
}
|
||||
|
||||
while (fileSliceIterator.hasNext() && keepVersions > 0) {
|
||||
// Skip this most recent version
|
||||
FileSlice nextSlice = fileSliceIterator.next();
|
||||
Option<HoodieDataFile> dataFile = nextSlice.getDataFile();
|
||||
if (dataFile.isPresent() && savepointedFiles.contains(dataFile.get().getFileName())) {
|
||||
// do not clean up a savepoint data file
|
||||
continue;
|
||||
}
|
||||
keepVersions--;
|
||||
}
|
||||
// Delete the remaining files
|
||||
while (fileSliceIterator.hasNext()) {
|
||||
FileSlice nextSlice = fileSliceIterator.next();
|
||||
if (nextSlice.getDataFile().isPresent()) {
|
||||
HoodieDataFile dataFile = nextSlice.getDataFile().get();
|
||||
deletePaths.add(dataFile.getPath());
|
||||
}
|
||||
if (hoodieTable.getMetaClient().getTableType() == HoodieTableType.MERGE_ON_READ) {
|
||||
// If merge on read, then clean the log files for the commits as well
|
||||
deletePaths.addAll(nextSlice.getLogFiles().map(file -> file.getPath().toString())
|
||||
.collect(Collectors.toList()));
|
||||
}
|
||||
}
|
||||
}
|
||||
return deletePaths;
|
||||
}
|
||||
|
||||
|
||||
/**
|
||||
* Selects the versions for file for cleaning, such that it <p> - Leaves the latest version of the
|
||||
* file untouched - For older versions, - It leaves all the commits untouched which has occured in
|
||||
* last <code>config.getCleanerCommitsRetained()</code> commits - It leaves ONE commit before this
|
||||
* window. We assume that the max(query execution time) == commit_batch_time *
|
||||
* config.getCleanerCommitsRetained(). This is 12 hours by default. This is essential to leave the
|
||||
* file used by the query thats running for the max time. <p> This provides the effect of having
|
||||
* lookback into all changes that happened in the last X commits. (eg: if you retain 24 commits,
|
||||
* and commit batch time is 30 mins, then you have 12 hrs of lookback) <p> This policy is the
|
||||
* default.
|
||||
*/
|
||||
private List<String> getFilesToCleanKeepingLatestCommits(String partitionPath)
|
||||
throws IOException {
|
||||
int commitsRetained = config.getCleanerCommitsRetained();
|
||||
logger
|
||||
.info("Cleaning " + partitionPath + ", retaining latest " + commitsRetained + " commits. ");
|
||||
List<String> deletePaths = new ArrayList<>();
|
||||
|
||||
// Collect all the datafiles savepointed by all the savepoints
|
||||
List<String> savepointedFiles = hoodieTable.getSavepoints().stream()
|
||||
.flatMap(s -> hoodieTable.getSavepointedDataFiles(s)).collect(Collectors.toList());
|
||||
|
||||
// determine if we have enough commits, to start cleaning.
|
||||
if (commitTimeline.countInstants() > commitsRetained) {
|
||||
HoodieInstant earliestCommitToRetain = getEarliestCommitToRetain().get();
|
||||
List<HoodieFileGroup> fileGroups = fileSystemView.getAllFileGroups(partitionPath)
|
||||
.collect(Collectors.toList());
|
||||
for (HoodieFileGroup fileGroup : fileGroups) {
|
||||
List<FileSlice> fileSliceList = fileGroup.getAllFileSlices().collect(Collectors.toList());
|
||||
|
||||
if (fileSliceList.isEmpty()) {
|
||||
continue;
|
||||
}
|
||||
|
||||
String lastVersion = fileSliceList.get(0).getBaseInstantTime();
|
||||
String lastVersionBeforeEarliestCommitToRetain = getLatestVersionBeforeCommit(fileSliceList,
|
||||
earliestCommitToRetain);
|
||||
|
||||
// Ensure there are more than 1 version of the file (we only clean old files from updates)
|
||||
// i.e always spare the last commit.
|
||||
for (FileSlice aSlice : fileSliceList) {
|
||||
Option<HoodieDataFile> aFile = aSlice.getDataFile();
|
||||
String fileCommitTime = aSlice.getBaseInstantTime();
|
||||
if (aFile.isPresent() && savepointedFiles.contains(aFile.get().getFileName())) {
|
||||
// do not clean up a savepoint data file
|
||||
continue;
|
||||
}
|
||||
// Dont delete the latest commit and also the last commit before the earliest commit we
|
||||
// are retaining
|
||||
// The window of commit retain == max query run time. So a query could be running which
|
||||
// still
|
||||
// uses this file.
|
||||
if (fileCommitTime.equals(lastVersion) || (fileCommitTime.equals(lastVersionBeforeEarliestCommitToRetain))) {
|
||||
// move on to the next file
|
||||
continue;
|
||||
}
|
||||
|
||||
// Always keep the last commit
|
||||
if (!isFileSliceNeededForPendingCompaction(aSlice)
|
||||
&& HoodieTimeline
|
||||
.compareTimestamps(earliestCommitToRetain.getTimestamp(), fileCommitTime,
|
||||
HoodieTimeline.GREATER)) {
|
||||
// this is a commit, that should be cleaned.
|
||||
aFile.ifPresent(hoodieDataFile -> deletePaths.add(hoodieDataFile.getPath()));
|
||||
if (hoodieTable.getMetaClient().getTableType() == HoodieTableType.MERGE_ON_READ) {
|
||||
// If merge on read, then clean the log files for the commits as well
|
||||
deletePaths.addAll(aSlice.getLogFiles().map(file -> file.getPath().toString())
|
||||
.collect(Collectors.toList()));
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
return deletePaths;
|
||||
}
|
||||
|
||||
/**
|
||||
* Gets the latest version < commitTime. This version file could still be used by queries.
|
||||
*/
|
||||
private String getLatestVersionBeforeCommit(List<FileSlice> fileSliceList,
|
||||
HoodieInstant commitTime) {
|
||||
for (FileSlice file : fileSliceList) {
|
||||
String fileCommitTime = file.getBaseInstantTime();
|
||||
if (HoodieTimeline
|
||||
.compareTimestamps(commitTime.getTimestamp(), fileCommitTime, HoodieTimeline.GREATER)) {
|
||||
// fileList is sorted on the reverse, so the first commit we find <= commitTime is the
|
||||
// one we want
|
||||
return fileCommitTime;
|
||||
}
|
||||
}
|
||||
// There is no version of this file which is <= commitTime
|
||||
return null;
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns files to be cleaned for the given partitionPath based on cleaning policy.
|
||||
*/
|
||||
public List<String> getDeletePaths(String partitionPath) throws IOException {
|
||||
HoodieCleaningPolicy policy = config.getCleanerPolicy();
|
||||
List<String> deletePaths;
|
||||
if (policy == HoodieCleaningPolicy.KEEP_LATEST_COMMITS) {
|
||||
deletePaths = getFilesToCleanKeepingLatestCommits(partitionPath);
|
||||
} else if (policy == HoodieCleaningPolicy.KEEP_LATEST_FILE_VERSIONS) {
|
||||
deletePaths = getFilesToCleanKeepingLatestVersions(partitionPath);
|
||||
} else {
|
||||
throw new IllegalArgumentException("Unknown cleaning policy : " + policy.name());
|
||||
}
|
||||
logger.info(deletePaths.size() + " patterns used to delete in partition path:" + partitionPath);
|
||||
|
||||
return deletePaths;
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns earliest commit to retain based on cleaning policy.
|
||||
*/
|
||||
public Option<HoodieInstant> getEarliestCommitToRetain() {
|
||||
Option<HoodieInstant> earliestCommitToRetain = Option.empty();
|
||||
int commitsRetained = config.getCleanerCommitsRetained();
|
||||
if (config.getCleanerPolicy() == HoodieCleaningPolicy.KEEP_LATEST_COMMITS
|
||||
&& commitTimeline.countInstants() > commitsRetained) {
|
||||
earliestCommitToRetain = commitTimeline
|
||||
.nthInstant(commitTimeline.countInstants() - commitsRetained);
|
||||
}
|
||||
return earliestCommitToRetain;
|
||||
}
|
||||
|
||||
/**
|
||||
* Determine if file slice needed to be preserved for pending compaction
|
||||
* @param fileSlice File Slice
|
||||
* @return true if file slice needs to be preserved, false otherwise.
|
||||
*/
|
||||
private boolean isFileSliceNeededForPendingCompaction(FileSlice fileSlice) {
|
||||
CompactionOperation op = fgIdToPendingCompactionOperations.get(fileSlice.getFileGroupId());
|
||||
if (null != op) {
|
||||
// If file slice's instant time is newer or same as that of operation, do not clean
|
||||
return HoodieTimeline.compareTimestamps(fileSlice.getBaseInstantTime(), op.getBaseInstantTime(),
|
||||
HoodieTimeline.GREATER_OR_EQUAL);
|
||||
}
|
||||
return false;
|
||||
}
|
||||
|
||||
private boolean isFileGroupInPendingCompaction(HoodieFileGroup fg) {
|
||||
return fgIdToPendingCompactionOperations.containsKey(fg.getFileGroupId());
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,337 @@
|
||||
/*
|
||||
* 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.io;
|
||||
|
||||
import com.fasterxml.jackson.databind.DeserializationFeature;
|
||||
import com.fasterxml.jackson.databind.ObjectMapper;
|
||||
import com.google.common.collect.Maps;
|
||||
import com.google.common.collect.Sets;
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Comparator;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.stream.Collectors;
|
||||
import java.util.stream.Stream;
|
||||
import org.apache.avro.Schema;
|
||||
import org.apache.avro.generic.IndexedRecord;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hudi.avro.model.HoodieArchivedMetaEntry;
|
||||
import org.apache.hudi.avro.model.HoodieCleanMetadata;
|
||||
import org.apache.hudi.avro.model.HoodieRollbackMetadata;
|
||||
import org.apache.hudi.avro.model.HoodieSavepointMetadata;
|
||||
import org.apache.hudi.common.model.ActionType;
|
||||
import org.apache.hudi.common.model.HoodieArchivedLogFile;
|
||||
import org.apache.hudi.common.model.HoodieCommitMetadata;
|
||||
import org.apache.hudi.common.model.HoodieRollingStatMetadata;
|
||||
import org.apache.hudi.common.table.HoodieTableMetaClient;
|
||||
import org.apache.hudi.common.table.HoodieTimeline;
|
||||
import org.apache.hudi.common.table.log.HoodieLogFormat;
|
||||
import org.apache.hudi.common.table.log.HoodieLogFormat.Writer;
|
||||
import org.apache.hudi.common.table.log.block.HoodieAvroDataBlock;
|
||||
import org.apache.hudi.common.table.log.block.HoodieLogBlock;
|
||||
import org.apache.hudi.common.table.log.block.HoodieLogBlock.HeaderMetadataType;
|
||||
import org.apache.hudi.common.table.timeline.HoodieActiveTimeline;
|
||||
import org.apache.hudi.common.table.timeline.HoodieArchivedTimeline;
|
||||
import org.apache.hudi.common.table.timeline.HoodieInstant;
|
||||
import org.apache.hudi.common.util.AvroUtils;
|
||||
import org.apache.hudi.common.util.Option;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
import org.apache.hudi.exception.HoodieCommitException;
|
||||
import org.apache.hudi.exception.HoodieException;
|
||||
import org.apache.hudi.exception.HoodieIOException;
|
||||
import org.apache.hudi.table.HoodieTable;
|
||||
import org.apache.log4j.LogManager;
|
||||
import org.apache.log4j.Logger;
|
||||
import org.apache.spark.api.java.JavaSparkContext;
|
||||
|
||||
/**
|
||||
* Archiver to bound the growth of <action>.commit files
|
||||
*/
|
||||
public class HoodieCommitArchiveLog {
|
||||
|
||||
private static Logger log = LogManager.getLogger(HoodieCommitArchiveLog.class);
|
||||
|
||||
private final Path archiveFilePath;
|
||||
private final HoodieTableMetaClient metaClient;
|
||||
private final HoodieWriteConfig config;
|
||||
private Writer writer;
|
||||
|
||||
public HoodieCommitArchiveLog(HoodieWriteConfig config, HoodieTableMetaClient metaClient) {
|
||||
this.config = config;
|
||||
this.metaClient = metaClient;
|
||||
this.archiveFilePath = HoodieArchivedTimeline.getArchiveLogPath(metaClient.getArchivePath());
|
||||
}
|
||||
|
||||
private Writer openWriter() {
|
||||
try {
|
||||
if (this.writer == null) {
|
||||
return HoodieLogFormat.newWriterBuilder().onParentPath(archiveFilePath.getParent())
|
||||
.withFileId(archiveFilePath.getName())
|
||||
.withFileExtension(HoodieArchivedLogFile.ARCHIVE_EXTENSION).withFs(metaClient.getFs())
|
||||
.overBaseCommit("").build();
|
||||
} else {
|
||||
return this.writer;
|
||||
}
|
||||
} catch (InterruptedException | IOException e) {
|
||||
throw new HoodieException("Unable to initialize HoodieLogFormat writer", e);
|
||||
}
|
||||
}
|
||||
|
||||
private void close() {
|
||||
try {
|
||||
if (this.writer != null) {
|
||||
this.writer.close();
|
||||
}
|
||||
} catch (IOException e) {
|
||||
throw new HoodieException("Unable to close HoodieLogFormat writer", e);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Check if commits need to be archived. If yes, archive commits.
|
||||
*/
|
||||
public boolean archiveIfRequired(final JavaSparkContext jsc) throws IOException {
|
||||
try {
|
||||
List<HoodieInstant> instantsToArchive = getInstantsToArchive(jsc).collect(Collectors.toList());
|
||||
boolean success = true;
|
||||
if (instantsToArchive.iterator().hasNext()) {
|
||||
this.writer = openWriter();
|
||||
log.info("Archiving instants " + instantsToArchive);
|
||||
archive(instantsToArchive);
|
||||
success = deleteArchivedInstants(instantsToArchive);
|
||||
} else {
|
||||
log.info("No Instants to archive");
|
||||
}
|
||||
return success;
|
||||
} finally {
|
||||
close();
|
||||
}
|
||||
}
|
||||
|
||||
private Stream<HoodieInstant> getInstantsToArchive(JavaSparkContext jsc) {
|
||||
|
||||
// TODO : rename to max/minInstantsToKeep
|
||||
int maxCommitsToKeep = config.getMaxCommitsToKeep();
|
||||
int minCommitsToKeep = config.getMinCommitsToKeep();
|
||||
|
||||
HoodieTable table = HoodieTable.getHoodieTable(metaClient, config, jsc);
|
||||
|
||||
// GroupBy each action and limit each action timeline to maxCommitsToKeep
|
||||
// TODO: Handle ROLLBACK_ACTION in future
|
||||
// ROLLBACK_ACTION is currently not defined in HoodieActiveTimeline
|
||||
HoodieTimeline cleanAndRollbackTimeline = table.getActiveTimeline()
|
||||
.getTimelineOfActions(Sets.newHashSet(HoodieTimeline.CLEAN_ACTION))
|
||||
.filterCompletedInstants();
|
||||
Stream<HoodieInstant> instants = cleanAndRollbackTimeline.getInstants()
|
||||
.collect(Collectors.groupingBy(s -> s.getAction())).entrySet().stream().map(i -> {
|
||||
if (i.getValue().size() > maxCommitsToKeep) {
|
||||
return i.getValue().subList(0, i.getValue().size() - minCommitsToKeep);
|
||||
} else {
|
||||
return new ArrayList<HoodieInstant>();
|
||||
}
|
||||
}).flatMap(i -> i.stream());
|
||||
|
||||
//TODO (na) : Add a way to return actions associated with a timeline and then merge/unify
|
||||
// with logic above to avoid Stream.concats
|
||||
HoodieTimeline commitTimeline = table.getCompletedCommitsTimeline();
|
||||
Option<HoodieInstant> oldestPendingCompactionInstant =
|
||||
table.getActiveTimeline().filterPendingCompactionTimeline().firstInstant();
|
||||
|
||||
// We cannot have any holes in the commit timeline. We cannot archive any commits which are
|
||||
// made after the first savepoint present.
|
||||
Option<HoodieInstant> firstSavepoint = table.getCompletedSavepointTimeline().firstInstant();
|
||||
if (!commitTimeline.empty() && commitTimeline.countInstants() > maxCommitsToKeep) {
|
||||
// Actually do the commits
|
||||
instants = Stream.concat(instants, commitTimeline.getInstants()
|
||||
.filter(s -> {
|
||||
// if no savepoint present, then dont filter
|
||||
return !(firstSavepoint.isPresent() && HoodieTimeline
|
||||
.compareTimestamps(firstSavepoint.get().getTimestamp(), s.getTimestamp(),
|
||||
HoodieTimeline.LESSER_OR_EQUAL));
|
||||
})
|
||||
.filter(s -> {
|
||||
// Ensure commits >= oldest pending compaction commit is retained
|
||||
return oldestPendingCompactionInstant.map(instant -> {
|
||||
return HoodieTimeline.compareTimestamps(instant.getTimestamp(), s.getTimestamp(), HoodieTimeline.GREATER);
|
||||
}).orElse(true);
|
||||
})
|
||||
.limit(commitTimeline.countInstants() - minCommitsToKeep));
|
||||
}
|
||||
|
||||
return instants;
|
||||
}
|
||||
|
||||
private boolean deleteArchivedInstants(List<HoodieInstant> archivedInstants) throws IOException {
|
||||
log.info("Deleting instants " + archivedInstants);
|
||||
boolean success = true;
|
||||
for (HoodieInstant archivedInstant : archivedInstants) {
|
||||
Path commitFile = new Path(metaClient.getMetaPath(), archivedInstant.getFileName());
|
||||
try {
|
||||
if (metaClient.getFs().exists(commitFile)) {
|
||||
success &= metaClient.getFs().delete(commitFile, false);
|
||||
log.info("Archived and deleted instant file " + commitFile);
|
||||
}
|
||||
} catch (IOException e) {
|
||||
throw new HoodieIOException("Failed to delete archived instant " + archivedInstant, e);
|
||||
}
|
||||
}
|
||||
|
||||
// Remove older meta-data from auxiliary path too
|
||||
Option<HoodieInstant> latestCommitted =
|
||||
Option.fromJavaOptional(archivedInstants.stream()
|
||||
.filter(i -> {
|
||||
return i.isCompleted()
|
||||
&& (i.getAction().equals(HoodieTimeline.COMMIT_ACTION) || (i.getAction().equals(
|
||||
HoodieTimeline.DELTA_COMMIT_ACTION)));
|
||||
}).max(Comparator.comparing(HoodieInstant::getTimestamp)));
|
||||
if (latestCommitted.isPresent()) {
|
||||
success &= deleteAllInstantsOlderorEqualsInAuxMetaFolder(latestCommitted.get());
|
||||
}
|
||||
return success;
|
||||
}
|
||||
|
||||
/**
|
||||
* Remove older instants from auxiliary meta folder
|
||||
*
|
||||
* @param thresholdInstant Hoodie Instant
|
||||
* @return success if all eligible file deleted successfully
|
||||
* @throws IOException in case of error
|
||||
*/
|
||||
private boolean deleteAllInstantsOlderorEqualsInAuxMetaFolder(HoodieInstant thresholdInstant)
|
||||
throws IOException {
|
||||
List<HoodieInstant> instants =
|
||||
HoodieTableMetaClient.scanHoodieInstantsFromFileSystem(metaClient.getFs(),
|
||||
new Path(metaClient.getMetaAuxiliaryPath()),
|
||||
HoodieActiveTimeline.VALID_EXTENSIONS_IN_ACTIVE_TIMELINE);
|
||||
|
||||
List<HoodieInstant> instantsToBeDeleted =
|
||||
instants.stream().filter(instant1 -> HoodieTimeline.compareTimestamps(instant1.getTimestamp(),
|
||||
thresholdInstant.getTimestamp(), HoodieTimeline.LESSER_OR_EQUAL)).collect(Collectors.toList());
|
||||
|
||||
boolean success = true;
|
||||
for (HoodieInstant deleteInstant : instantsToBeDeleted) {
|
||||
log.info("Deleting instant " + deleteInstant + " in auxiliary meta path " + metaClient.getMetaAuxiliaryPath());
|
||||
Path metaFile = new Path(metaClient.getMetaAuxiliaryPath(), deleteInstant.getFileName());
|
||||
if (metaClient.getFs().exists(metaFile)) {
|
||||
success &= metaClient.getFs().delete(metaFile, false);
|
||||
log.info("Deleted instant file in auxiliary metapath : " + metaFile);
|
||||
}
|
||||
}
|
||||
return success;
|
||||
}
|
||||
|
||||
public void archive(List<HoodieInstant> instants) throws HoodieCommitException {
|
||||
try {
|
||||
HoodieTimeline commitTimeline = metaClient.getActiveTimeline().getAllCommitsTimeline()
|
||||
.filterCompletedInstants();
|
||||
Schema wrapperSchema = HoodieArchivedMetaEntry.getClassSchema();
|
||||
log.info("Wrapper schema " + wrapperSchema.toString());
|
||||
List<IndexedRecord> records = new ArrayList<>();
|
||||
for (HoodieInstant hoodieInstant : instants) {
|
||||
try {
|
||||
records.add(convertToAvroRecord(commitTimeline, hoodieInstant));
|
||||
if (records.size() >= this.config.getCommitArchivalBatchSize()) {
|
||||
writeToFile(wrapperSchema, records);
|
||||
}
|
||||
} catch (Exception e) {
|
||||
log.error("Failed to archive commits, .commit file: " + hoodieInstant.getFileName(), e);
|
||||
if (this.config.isFailOnTimelineArchivingEnabled()) {
|
||||
throw e;
|
||||
}
|
||||
}
|
||||
}
|
||||
writeToFile(wrapperSchema, records);
|
||||
} catch (Exception e) {
|
||||
throw new HoodieCommitException("Failed to archive commits", e);
|
||||
}
|
||||
}
|
||||
|
||||
public Path getArchiveFilePath() {
|
||||
return archiveFilePath;
|
||||
}
|
||||
|
||||
private void writeToFile(Schema wrapperSchema, List<IndexedRecord> records) throws Exception {
|
||||
if (records.size() > 0) {
|
||||
Map<HeaderMetadataType, String> header = Maps.newHashMap();
|
||||
header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, wrapperSchema.toString());
|
||||
HoodieAvroDataBlock block = new HoodieAvroDataBlock(records, header);
|
||||
this.writer = writer.appendBlock(block);
|
||||
records.clear();
|
||||
}
|
||||
}
|
||||
|
||||
private IndexedRecord convertToAvroRecord(HoodieTimeline commitTimeline,
|
||||
HoodieInstant hoodieInstant) throws IOException {
|
||||
HoodieArchivedMetaEntry archivedMetaWrapper = new HoodieArchivedMetaEntry();
|
||||
archivedMetaWrapper.setCommitTime(hoodieInstant.getTimestamp());
|
||||
switch (hoodieInstant.getAction()) {
|
||||
case HoodieTimeline.CLEAN_ACTION: {
|
||||
archivedMetaWrapper.setHoodieCleanMetadata(AvroUtils
|
||||
.deserializeAvroMetadata(commitTimeline.getInstantDetails(hoodieInstant).get(),
|
||||
HoodieCleanMetadata.class));
|
||||
archivedMetaWrapper.setActionType(ActionType.clean.name());
|
||||
break;
|
||||
}
|
||||
case HoodieTimeline.COMMIT_ACTION: {
|
||||
HoodieCommitMetadata commitMetadata = HoodieCommitMetadata
|
||||
.fromBytes(commitTimeline.getInstantDetails(hoodieInstant).get(), HoodieCommitMetadata.class);
|
||||
archivedMetaWrapper.setHoodieCommitMetadata(commitMetadataConverter(commitMetadata));
|
||||
archivedMetaWrapper.setActionType(ActionType.commit.name());
|
||||
break;
|
||||
}
|
||||
case HoodieTimeline.ROLLBACK_ACTION: {
|
||||
archivedMetaWrapper.setHoodieRollbackMetadata(AvroUtils
|
||||
.deserializeAvroMetadata(commitTimeline.getInstantDetails(hoodieInstant).get(),
|
||||
HoodieRollbackMetadata.class));
|
||||
archivedMetaWrapper.setActionType(ActionType.rollback.name());
|
||||
break;
|
||||
}
|
||||
case HoodieTimeline.SAVEPOINT_ACTION: {
|
||||
archivedMetaWrapper.setHoodieSavePointMetadata(AvroUtils
|
||||
.deserializeAvroMetadata(commitTimeline.getInstantDetails(hoodieInstant).get(),
|
||||
HoodieSavepointMetadata.class));
|
||||
archivedMetaWrapper.setActionType(ActionType.savepoint.name());
|
||||
break;
|
||||
}
|
||||
case HoodieTimeline.DELTA_COMMIT_ACTION: {
|
||||
HoodieCommitMetadata commitMetadata = HoodieCommitMetadata
|
||||
.fromBytes(commitTimeline.getInstantDetails(hoodieInstant).get(), HoodieCommitMetadata.class);
|
||||
archivedMetaWrapper.setHoodieCommitMetadata(commitMetadataConverter(commitMetadata));
|
||||
archivedMetaWrapper.setActionType(ActionType.commit.name());
|
||||
break;
|
||||
}
|
||||
default:
|
||||
throw new UnsupportedOperationException("Action not fully supported yet");
|
||||
}
|
||||
return archivedMetaWrapper;
|
||||
}
|
||||
|
||||
private org.apache.hudi.avro.model.HoodieCommitMetadata commitMetadataConverter(
|
||||
HoodieCommitMetadata hoodieCommitMetadata) {
|
||||
ObjectMapper mapper = new ObjectMapper();
|
||||
//Need this to ignore other public get() methods
|
||||
mapper.configure(DeserializationFeature.FAIL_ON_UNKNOWN_PROPERTIES, false);
|
||||
org.apache.hudi.avro.model.HoodieCommitMetadata avroMetaData = mapper
|
||||
.convertValue(hoodieCommitMetadata, org.apache.hudi.avro.model.HoodieCommitMetadata.class);
|
||||
// Do not archive Rolling Stats, cannot set to null since AVRO will throw null pointer
|
||||
avroMetaData.getExtraMetadata().put(HoodieRollingStatMetadata.ROLLING_STAT_METADATA_KEY, "");
|
||||
return avroMetaData;
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,183 @@
|
||||
/*
|
||||
* 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.io;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.Iterator;
|
||||
import org.apache.avro.generic.GenericRecord;
|
||||
import org.apache.avro.generic.IndexedRecord;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hudi.WriteStatus;
|
||||
import org.apache.hudi.common.model.HoodiePartitionMetadata;
|
||||
import org.apache.hudi.common.model.HoodieRecord;
|
||||
import org.apache.hudi.common.model.HoodieRecordLocation;
|
||||
import org.apache.hudi.common.model.HoodieRecordPayload;
|
||||
import org.apache.hudi.common.model.HoodieWriteStat;
|
||||
import org.apache.hudi.common.model.HoodieWriteStat.RuntimeStats;
|
||||
import org.apache.hudi.common.util.FSUtils;
|
||||
import org.apache.hudi.common.util.Option;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
import org.apache.hudi.exception.HoodieInsertException;
|
||||
import org.apache.hudi.io.storage.HoodieStorageWriter;
|
||||
import org.apache.hudi.io.storage.HoodieStorageWriterFactory;
|
||||
import org.apache.hudi.table.HoodieTable;
|
||||
import org.apache.log4j.LogManager;
|
||||
import org.apache.log4j.Logger;
|
||||
import org.apache.spark.TaskContext;
|
||||
|
||||
public class HoodieCreateHandle<T extends HoodieRecordPayload> extends HoodieWriteHandle<T> {
|
||||
|
||||
private static Logger logger = LogManager.getLogger(HoodieCreateHandle.class);
|
||||
|
||||
private final HoodieStorageWriter<IndexedRecord> storageWriter;
|
||||
private final Path path;
|
||||
private long recordsWritten = 0;
|
||||
private long insertRecordsWritten = 0;
|
||||
private long recordsDeleted = 0;
|
||||
private Iterator<HoodieRecord<T>> recordIterator;
|
||||
private boolean useWriterSchema = false;
|
||||
|
||||
public HoodieCreateHandle(HoodieWriteConfig config, String commitTime, HoodieTable<T> hoodieTable,
|
||||
String partitionPath, String fileId) {
|
||||
super(config, commitTime, fileId, hoodieTable);
|
||||
writeStatus.setFileId(fileId);
|
||||
writeStatus.setPartitionPath(partitionPath);
|
||||
|
||||
this.path = makeNewPath(partitionPath);
|
||||
|
||||
try {
|
||||
HoodiePartitionMetadata partitionMetadata = new HoodiePartitionMetadata(fs, commitTime,
|
||||
new Path(config.getBasePath()), FSUtils.getPartitionPath(config.getBasePath(), partitionPath));
|
||||
partitionMetadata.trySave(TaskContext.getPartitionId());
|
||||
createMarkerFile(partitionPath);
|
||||
this.storageWriter = HoodieStorageWriterFactory
|
||||
.getStorageWriter(commitTime, path, hoodieTable, config, writerSchema);
|
||||
} catch (IOException e) {
|
||||
throw new HoodieInsertException(
|
||||
"Failed to initialize HoodieStorageWriter for path " + path, e);
|
||||
}
|
||||
logger.info("New CreateHandle for partition :" + partitionPath + " with fileId " + fileId);
|
||||
}
|
||||
|
||||
/**
|
||||
* Called by the compactor code path
|
||||
*/
|
||||
public HoodieCreateHandle(HoodieWriteConfig config, String commitTime, HoodieTable<T> hoodieTable,
|
||||
String partitionPath, String fileId, Iterator<HoodieRecord<T>> recordIterator) {
|
||||
this(config, commitTime, hoodieTable, partitionPath, fileId);
|
||||
this.recordIterator = recordIterator;
|
||||
this.useWriterSchema = true;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean canWrite(HoodieRecord record) {
|
||||
return storageWriter.canWrite() && record.getPartitionPath().equals(writeStatus.getPartitionPath());
|
||||
}
|
||||
|
||||
/**
|
||||
* Perform the actual writing of the given record into the backing file.
|
||||
*/
|
||||
public void write(HoodieRecord record, Option<IndexedRecord> avroRecord) {
|
||||
Option recordMetadata = record.getData().getMetadata();
|
||||
try {
|
||||
if (avroRecord.isPresent()) {
|
||||
// Convert GenericRecord to GenericRecord with hoodie commit metadata in schema
|
||||
IndexedRecord recordWithMetadataInSchema = rewriteRecord((GenericRecord) avroRecord.get());
|
||||
storageWriter.writeAvroWithMetadata(recordWithMetadataInSchema, record);
|
||||
// update the new location of record, so we know where to find it next
|
||||
record.setNewLocation(new HoodieRecordLocation(instantTime, writeStatus.getFileId()));
|
||||
recordsWritten++;
|
||||
insertRecordsWritten++;
|
||||
} else {
|
||||
recordsDeleted++;
|
||||
}
|
||||
writeStatus.markSuccess(record, recordMetadata);
|
||||
// deflate record payload after recording success. This will help users access payload as a
|
||||
// part of marking
|
||||
// record successful.
|
||||
record.deflate();
|
||||
} catch (Throwable t) {
|
||||
// Not throwing exception from here, since we don't want to fail the entire job
|
||||
// for a single record
|
||||
writeStatus.markFailure(record, t, recordMetadata);
|
||||
logger.error("Error writing record " + record, t);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Writes all records passed
|
||||
*/
|
||||
public void write() {
|
||||
try {
|
||||
while (recordIterator.hasNext()) {
|
||||
HoodieRecord<T> record = recordIterator.next();
|
||||
if (useWriterSchema) {
|
||||
write(record, record.getData().getInsertValue(writerSchema));
|
||||
} else {
|
||||
write(record, record.getData().getInsertValue(originalSchema));
|
||||
}
|
||||
}
|
||||
} catch (IOException io) {
|
||||
throw new HoodieInsertException(
|
||||
"Failed to insert records for path " + path, io);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public WriteStatus getWriteStatus() {
|
||||
return writeStatus;
|
||||
}
|
||||
|
||||
/**
|
||||
* Performs actions to durably, persist the current changes and returns a WriteStatus object
|
||||
*/
|
||||
@Override
|
||||
public WriteStatus close() {
|
||||
logger.info("Closing the file " + writeStatus.getFileId() + " as we are done with all the records "
|
||||
+ recordsWritten);
|
||||
try {
|
||||
|
||||
storageWriter.close();
|
||||
|
||||
HoodieWriteStat stat = new HoodieWriteStat();
|
||||
stat.setPartitionPath(writeStatus.getPartitionPath());
|
||||
stat.setNumWrites(recordsWritten);
|
||||
stat.setNumDeletes(recordsDeleted);
|
||||
stat.setNumInserts(insertRecordsWritten);
|
||||
stat.setPrevCommit(HoodieWriteStat.NULL_COMMIT);
|
||||
stat.setFileId(writeStatus.getFileId());
|
||||
stat.setPath(new Path(config.getBasePath()), path);
|
||||
long fileSizeInBytes = FSUtils.getFileSize(fs, path);
|
||||
stat.setTotalWriteBytes(fileSizeInBytes);
|
||||
stat.setFileSizeInBytes(fileSizeInBytes);
|
||||
stat.setTotalWriteErrors(writeStatus.getTotalErrorRecords());
|
||||
RuntimeStats runtimeStats = new RuntimeStats();
|
||||
runtimeStats.setTotalCreateTime(timer.endTimer());
|
||||
stat.setRuntimeStats(runtimeStats);
|
||||
writeStatus.setStat(stat);
|
||||
|
||||
logger.info(String.format("CreateHandle for partitionPath %s fileID %s, took %d ms.",
|
||||
stat.getPartitionPath(), stat.getFileId(), runtimeStats.getTotalCreateTime()));
|
||||
|
||||
return writeStatus;
|
||||
} catch (IOException e) {
|
||||
throw new HoodieInsertException("Failed to close the Insert Handle for path " + path, e);
|
||||
}
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,42 @@
|
||||
/*
|
||||
* 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.io;
|
||||
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hudi.common.model.HoodieRecordPayload;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
import org.apache.hudi.table.HoodieTable;
|
||||
|
||||
|
||||
public abstract class HoodieIOHandle<T extends HoodieRecordPayload> {
|
||||
|
||||
protected final String instantTime;
|
||||
protected final HoodieWriteConfig config;
|
||||
protected final FileSystem fs;
|
||||
protected final HoodieTable<T> hoodieTable;
|
||||
|
||||
HoodieIOHandle(HoodieWriteConfig config, String instantTime, HoodieTable<T> hoodieTable) {
|
||||
this.instantTime = instantTime;
|
||||
this.config = config;
|
||||
this.hoodieTable = hoodieTable;
|
||||
this.fs = getFileSystem();
|
||||
}
|
||||
|
||||
protected abstract FileSystem getFileSystem();
|
||||
}
|
||||
@@ -0,0 +1,158 @@
|
||||
/*
|
||||
* 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.io;
|
||||
|
||||
import java.util.ArrayList;
|
||||
import java.util.HashSet;
|
||||
import java.util.List;
|
||||
import java.util.Set;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hudi.common.BloomFilter;
|
||||
import org.apache.hudi.common.model.HoodieDataFile;
|
||||
import org.apache.hudi.common.model.HoodieRecordPayload;
|
||||
import org.apache.hudi.common.model.HoodieTableType;
|
||||
import org.apache.hudi.common.util.HoodieTimer;
|
||||
import org.apache.hudi.common.util.ParquetUtils;
|
||||
import org.apache.hudi.common.util.collection.Pair;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
import org.apache.hudi.exception.HoodieIndexException;
|
||||
import org.apache.hudi.table.HoodieTable;
|
||||
import org.apache.log4j.LogManager;
|
||||
import org.apache.log4j.Logger;
|
||||
|
||||
/**
|
||||
* Takes a bunch of keys and returns ones that are present in the file group.
|
||||
*/
|
||||
public class HoodieKeyLookupHandle<T extends HoodieRecordPayload> extends HoodieReadHandle<T> {
|
||||
|
||||
private static Logger logger = LogManager.getLogger(HoodieKeyLookupHandle.class);
|
||||
|
||||
private final HoodieTableType tableType;
|
||||
|
||||
private final BloomFilter bloomFilter;
|
||||
|
||||
private final List<String> candidateRecordKeys;
|
||||
|
||||
private long totalKeysChecked;
|
||||
|
||||
public HoodieKeyLookupHandle(HoodieWriteConfig config, HoodieTable<T> hoodieTable,
|
||||
Pair<String, String> partitionPathFilePair) {
|
||||
super(config, null, hoodieTable, partitionPathFilePair);
|
||||
this.tableType = hoodieTable.getMetaClient().getTableType();
|
||||
this.candidateRecordKeys = new ArrayList<>();
|
||||
this.totalKeysChecked = 0;
|
||||
HoodieTimer timer = new HoodieTimer().startTimer();
|
||||
this.bloomFilter = ParquetUtils.readBloomFilterFromParquetMetadata(hoodieTable.getHadoopConf(),
|
||||
new Path(getLatestDataFile().getPath()));
|
||||
logger.info(String.format("Read bloom filter from %s in %d ms", partitionPathFilePair, timer.endTimer()));
|
||||
}
|
||||
|
||||
/**
|
||||
* Given a list of row keys and one file, return only row keys existing in that file.
|
||||
*/
|
||||
public static List<String> checkCandidatesAgainstFile(Configuration configuration,
|
||||
List<String> candidateRecordKeys, Path filePath) throws HoodieIndexException {
|
||||
List<String> foundRecordKeys = new ArrayList<>();
|
||||
try {
|
||||
// Load all rowKeys from the file, to double-confirm
|
||||
if (!candidateRecordKeys.isEmpty()) {
|
||||
HoodieTimer timer = new HoodieTimer().startTimer();
|
||||
Set<String> fileRowKeys = ParquetUtils.filterParquetRowKeys(configuration, filePath,
|
||||
new HashSet<>(candidateRecordKeys));
|
||||
foundRecordKeys.addAll(fileRowKeys);
|
||||
logger.info(String.format("Checked keys against file %s, in %d ms. #candidates (%d) #found (%d)", filePath,
|
||||
timer.endTimer(), candidateRecordKeys.size(), foundRecordKeys.size()));
|
||||
if (logger.isDebugEnabled()) {
|
||||
logger.debug("Keys matching for file " + filePath + " => " + foundRecordKeys);
|
||||
}
|
||||
}
|
||||
} catch (Exception e) {
|
||||
throw new HoodieIndexException("Error checking candidate keys against file.", e);
|
||||
}
|
||||
return foundRecordKeys;
|
||||
}
|
||||
|
||||
/**
|
||||
* Adds the key for look up.
|
||||
*/
|
||||
public void addKey(String recordKey) {
|
||||
// check record key against bloom filter of current file & add to possible keys if needed
|
||||
if (bloomFilter.mightContain(recordKey)) {
|
||||
if (logger.isDebugEnabled()) {
|
||||
logger.debug("Record key " + recordKey + " matches bloom filter in " + partitionPathFilePair);
|
||||
}
|
||||
candidateRecordKeys.add(recordKey);
|
||||
}
|
||||
totalKeysChecked++;
|
||||
}
|
||||
|
||||
/**
|
||||
* Of all the keys, that were added, return a list of keys that were actually found in the file group.
|
||||
*/
|
||||
public KeyLookupResult getLookupResult() {
|
||||
if (logger.isDebugEnabled()) {
|
||||
logger.debug("#The candidate row keys for " + partitionPathFilePair + " => " + candidateRecordKeys);
|
||||
}
|
||||
|
||||
HoodieDataFile dataFile = getLatestDataFile();
|
||||
List<String> matchingKeys = checkCandidatesAgainstFile(hoodieTable.getHadoopConf(), candidateRecordKeys,
|
||||
new Path(dataFile.getPath()));
|
||||
logger.info(String.format("Total records (%d), bloom filter candidates (%d)/fp(%d), actual matches (%d)",
|
||||
totalKeysChecked, candidateRecordKeys.size(), candidateRecordKeys.size() - matchingKeys.size(),
|
||||
matchingKeys.size()));
|
||||
return new KeyLookupResult(partitionPathFilePair.getRight(), partitionPathFilePair.getLeft(),
|
||||
dataFile.getCommitTime(), matchingKeys);
|
||||
}
|
||||
|
||||
/**
|
||||
* Encapsulates the result from a key lookup
|
||||
*/
|
||||
public static class KeyLookupResult {
|
||||
|
||||
private final String fileId;
|
||||
private final String baseInstantTime;
|
||||
private final List<String> matchingRecordKeys;
|
||||
private final String partitionPath;
|
||||
|
||||
public KeyLookupResult(String fileId, String partitionPath, String baseInstantTime,
|
||||
List<String> matchingRecordKeys) {
|
||||
this.fileId = fileId;
|
||||
this.partitionPath = partitionPath;
|
||||
this.baseInstantTime = baseInstantTime;
|
||||
this.matchingRecordKeys = matchingRecordKeys;
|
||||
}
|
||||
|
||||
public String getFileId() {
|
||||
return fileId;
|
||||
}
|
||||
|
||||
public String getBaseInstantTime() {
|
||||
return baseInstantTime;
|
||||
}
|
||||
|
||||
public String getPartitionPath() {
|
||||
return partitionPath;
|
||||
}
|
||||
|
||||
public List<String> getMatchingRecordKeys() {
|
||||
return matchingRecordKeys;
|
||||
}
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,361 @@
|
||||
/*
|
||||
* 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.io;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.HashSet;
|
||||
import java.util.Iterator;
|
||||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
import org.apache.avro.Schema;
|
||||
import org.apache.avro.generic.GenericRecord;
|
||||
import org.apache.avro.generic.IndexedRecord;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hudi.WriteStatus;
|
||||
import org.apache.hudi.common.model.HoodieDataFile;
|
||||
import org.apache.hudi.common.model.HoodiePartitionMetadata;
|
||||
import org.apache.hudi.common.model.HoodieRecord;
|
||||
import org.apache.hudi.common.model.HoodieRecordLocation;
|
||||
import org.apache.hudi.common.model.HoodieRecordPayload;
|
||||
import org.apache.hudi.common.model.HoodieWriteStat;
|
||||
import org.apache.hudi.common.model.HoodieWriteStat.RuntimeStats;
|
||||
import org.apache.hudi.common.util.DefaultSizeEstimator;
|
||||
import org.apache.hudi.common.util.FSUtils;
|
||||
import org.apache.hudi.common.util.HoodieAvroUtils;
|
||||
import org.apache.hudi.common.util.HoodieRecordSizeEstimator;
|
||||
import org.apache.hudi.common.util.Option;
|
||||
import org.apache.hudi.common.util.collection.ExternalSpillableMap;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
import org.apache.hudi.exception.HoodieIOException;
|
||||
import org.apache.hudi.exception.HoodieUpsertException;
|
||||
import org.apache.hudi.io.storage.HoodieStorageWriter;
|
||||
import org.apache.hudi.io.storage.HoodieStorageWriterFactory;
|
||||
import org.apache.hudi.table.HoodieTable;
|
||||
import org.apache.log4j.LogManager;
|
||||
import org.apache.log4j.Logger;
|
||||
import org.apache.spark.TaskContext;
|
||||
|
||||
@SuppressWarnings("Duplicates")
|
||||
public class HoodieMergeHandle<T extends HoodieRecordPayload> extends HoodieWriteHandle<T> {
|
||||
|
||||
private static Logger logger = LogManager.getLogger(HoodieMergeHandle.class);
|
||||
|
||||
private Map<String, HoodieRecord<T>> keyToNewRecords;
|
||||
private Set<String> writtenRecordKeys;
|
||||
private HoodieStorageWriter<IndexedRecord> storageWriter;
|
||||
private Path newFilePath;
|
||||
private Path oldFilePath;
|
||||
private long recordsWritten = 0;
|
||||
private long recordsDeleted = 0;
|
||||
private long updatedRecordsWritten = 0;
|
||||
private long insertRecordsWritten = 0;
|
||||
private boolean useWriterSchema;
|
||||
|
||||
public HoodieMergeHandle(HoodieWriteConfig config, String commitTime, HoodieTable<T> hoodieTable,
|
||||
Iterator<HoodieRecord<T>> recordItr, String fileId) {
|
||||
super(config, commitTime, fileId, hoodieTable);
|
||||
String partitionPath = init(fileId, recordItr);
|
||||
init(fileId, partitionPath,
|
||||
hoodieTable.getROFileSystemView().getLatestDataFile(partitionPath, fileId).get());
|
||||
}
|
||||
|
||||
/**
|
||||
* Called by compactor code path
|
||||
*/
|
||||
public HoodieMergeHandle(HoodieWriteConfig config, String commitTime, HoodieTable<T> hoodieTable,
|
||||
Map<String, HoodieRecord<T>> keyToNewRecords, String fileId, HoodieDataFile dataFileToBeMerged) {
|
||||
super(config, commitTime, fileId, hoodieTable);
|
||||
this.keyToNewRecords = keyToNewRecords;
|
||||
this.useWriterSchema = true;
|
||||
init(fileId, keyToNewRecords.get(keyToNewRecords.keySet().stream().findFirst().get())
|
||||
.getPartitionPath(), dataFileToBeMerged);
|
||||
}
|
||||
|
||||
|
||||
public static Schema createHoodieWriteSchema(Schema originalSchema) {
|
||||
return HoodieAvroUtils.addMetadataFields(originalSchema);
|
||||
}
|
||||
|
||||
public Path makeNewPath(String partitionPath) {
|
||||
Path path = FSUtils.getPartitionPath(config.getBasePath(), partitionPath);
|
||||
try {
|
||||
fs.mkdirs(path); // create a new partition as needed.
|
||||
} catch (IOException e) {
|
||||
throw new HoodieIOException("Failed to make dir " + path, e);
|
||||
}
|
||||
|
||||
return new Path(path.toString(), FSUtils.makeDataFileName(instantTime, writeToken, fileId));
|
||||
}
|
||||
|
||||
public Schema getWriterSchema() {
|
||||
return writerSchema;
|
||||
}
|
||||
|
||||
/**
|
||||
* Determines whether we can accept the incoming records, into the current file, depending on
|
||||
* <p>
|
||||
* - Whether it belongs to the same partitionPath as existing records - Whether the current file written bytes lt max
|
||||
* file size
|
||||
*/
|
||||
public boolean canWrite(HoodieRecord record) {
|
||||
return false;
|
||||
}
|
||||
|
||||
/**
|
||||
* Perform the actual writing of the given record into the backing file.
|
||||
*/
|
||||
public void write(HoodieRecord record, Option<IndexedRecord> insertValue) {
|
||||
// NO_OP
|
||||
}
|
||||
|
||||
/**
|
||||
* Perform the actual writing of the given record into the backing file.
|
||||
*/
|
||||
public void write(HoodieRecord record, Option<IndexedRecord> avroRecord, Option<Exception> exception) {
|
||||
Option recordMetadata = record.getData().getMetadata();
|
||||
if (exception.isPresent() && exception.get() instanceof Throwable) {
|
||||
// Not throwing exception from here, since we don't want to fail the entire job for a single record
|
||||
writeStatus.markFailure(record, exception.get(), recordMetadata);
|
||||
logger.error("Error writing record " + record, exception.get());
|
||||
} else {
|
||||
write(record, avroRecord);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Rewrite the GenericRecord with the Schema containing the Hoodie Metadata fields
|
||||
*/
|
||||
protected GenericRecord rewriteRecord(GenericRecord record) {
|
||||
return HoodieAvroUtils.rewriteRecord(record, writerSchema);
|
||||
}
|
||||
|
||||
/**
|
||||
* Extract old file path, initialize StorageWriter and WriteStatus
|
||||
*/
|
||||
private void init(String fileId, String partitionPath, HoodieDataFile dataFileToBeMerged) {
|
||||
logger.info("partitionPath:" + partitionPath + ", fileId to be merged:" + fileId);
|
||||
this.writtenRecordKeys = new HashSet<>();
|
||||
writeStatus.setStat(new HoodieWriteStat());
|
||||
try {
|
||||
String latestValidFilePath = dataFileToBeMerged.getFileName();
|
||||
writeStatus.getStat().setPrevCommit(FSUtils.getCommitTime(latestValidFilePath));
|
||||
|
||||
HoodiePartitionMetadata partitionMetadata = new HoodiePartitionMetadata(fs, instantTime,
|
||||
new Path(config.getBasePath()), FSUtils.getPartitionPath(config.getBasePath(), partitionPath));
|
||||
partitionMetadata.trySave(TaskContext.getPartitionId());
|
||||
|
||||
oldFilePath = new Path(
|
||||
config.getBasePath() + "/" + partitionPath + "/" + latestValidFilePath);
|
||||
String relativePath = new Path((partitionPath.isEmpty() ? "" : partitionPath + "/")
|
||||
+ FSUtils.makeDataFileName(instantTime, writeToken, fileId)).toString();
|
||||
newFilePath = new Path(config.getBasePath(), relativePath);
|
||||
|
||||
logger.info(String
|
||||
.format("Merging new data into oldPath %s, as newPath %s", oldFilePath.toString(),
|
||||
newFilePath.toString()));
|
||||
// file name is same for all records, in this bunch
|
||||
writeStatus.setFileId(fileId);
|
||||
writeStatus.setPartitionPath(partitionPath);
|
||||
writeStatus.getStat().setPartitionPath(partitionPath);
|
||||
writeStatus.getStat().setFileId(fileId);
|
||||
writeStatus.getStat().setPath(new Path(config.getBasePath()), newFilePath);
|
||||
|
||||
// Create Marker file
|
||||
createMarkerFile(partitionPath);
|
||||
|
||||
// Create the writer for writing the new version file
|
||||
storageWriter = HoodieStorageWriterFactory
|
||||
.getStorageWriter(instantTime, newFilePath, hoodieTable, config, writerSchema);
|
||||
} catch (IOException io) {
|
||||
logger.error("Error in update task at commit " + instantTime, io);
|
||||
writeStatus.setGlobalError(io);
|
||||
throw new HoodieUpsertException(
|
||||
"Failed to initialize HoodieUpdateHandle for FileId: " + fileId + " on commit "
|
||||
+ instantTime + " on path " + hoodieTable.getMetaClient().getBasePath(), io);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Load the new incoming records in a map and return partitionPath
|
||||
*/
|
||||
private String init(String fileId, Iterator<HoodieRecord<T>> newRecordsItr) {
|
||||
try {
|
||||
// Load the new records in a map
|
||||
logger.info("MaxMemoryPerPartitionMerge => " + config.getMaxMemoryPerPartitionMerge());
|
||||
this.keyToNewRecords = new ExternalSpillableMap<>(config.getMaxMemoryPerPartitionMerge(),
|
||||
config.getSpillableMapBasePath(), new DefaultSizeEstimator(), new HoodieRecordSizeEstimator(originalSchema));
|
||||
} catch (IOException io) {
|
||||
throw new HoodieIOException("Cannot instantiate an ExternalSpillableMap", io);
|
||||
}
|
||||
String partitionPath = null;
|
||||
while (newRecordsItr.hasNext()) {
|
||||
HoodieRecord<T> record = newRecordsItr.next();
|
||||
partitionPath = record.getPartitionPath();
|
||||
keyToNewRecords.put(record.getRecordKey(), record);
|
||||
// update the new location of the record, so we know where to find it next
|
||||
record.setNewLocation(new HoodieRecordLocation(instantTime, fileId));
|
||||
}
|
||||
logger.info("Number of entries in MemoryBasedMap => "
|
||||
+ ((ExternalSpillableMap) keyToNewRecords).getInMemoryMapNumEntries()
|
||||
+ "Total size in bytes of MemoryBasedMap => "
|
||||
+ ((ExternalSpillableMap) keyToNewRecords).getCurrentInMemoryMapSize()
|
||||
+ "Number of entries in DiskBasedMap => "
|
||||
+ ((ExternalSpillableMap) keyToNewRecords).getDiskBasedMapNumEntries()
|
||||
+ "Size of file spilled to disk => "
|
||||
+ ((ExternalSpillableMap) keyToNewRecords).getSizeOfFileOnDiskInBytes());
|
||||
return partitionPath;
|
||||
}
|
||||
|
||||
private boolean writeUpdateRecord(HoodieRecord<T> hoodieRecord, Option<IndexedRecord> indexedRecord) {
|
||||
if (indexedRecord.isPresent()) {
|
||||
updatedRecordsWritten++;
|
||||
}
|
||||
return writeRecord(hoodieRecord, indexedRecord);
|
||||
}
|
||||
|
||||
private boolean writeRecord(HoodieRecord<T> hoodieRecord, Option<IndexedRecord> indexedRecord) {
|
||||
Option recordMetadata = hoodieRecord.getData().getMetadata();
|
||||
try {
|
||||
if (indexedRecord.isPresent()) {
|
||||
// Convert GenericRecord to GenericRecord with hoodie commit metadata in schema
|
||||
IndexedRecord recordWithMetadataInSchema = rewriteRecord((GenericRecord) indexedRecord.get());
|
||||
storageWriter.writeAvroWithMetadata(recordWithMetadataInSchema, hoodieRecord);
|
||||
recordsWritten++;
|
||||
} else {
|
||||
recordsDeleted++;
|
||||
}
|
||||
|
||||
writeStatus.markSuccess(hoodieRecord, recordMetadata);
|
||||
// deflate record payload after recording success. This will help users access payload as a
|
||||
// part of marking
|
||||
// record successful.
|
||||
hoodieRecord.deflate();
|
||||
return true;
|
||||
} catch (Exception e) {
|
||||
logger.error("Error writing record " + hoodieRecord, e);
|
||||
writeStatus.markFailure(hoodieRecord, e, recordMetadata);
|
||||
}
|
||||
return false;
|
||||
}
|
||||
|
||||
/**
|
||||
* Go through an old record. Here if we detect a newer version shows up, we write the new one to
|
||||
* the file.
|
||||
*/
|
||||
public void write(GenericRecord oldRecord) {
|
||||
String key = oldRecord.get(HoodieRecord.RECORD_KEY_METADATA_FIELD).toString();
|
||||
boolean copyOldRecord = true;
|
||||
if (keyToNewRecords.containsKey(key)) {
|
||||
// If we have duplicate records that we are updating, then the hoodie record will be deflated after
|
||||
// writing the first record. So make a copy of the record to be merged
|
||||
HoodieRecord<T> hoodieRecord = new HoodieRecord<>(keyToNewRecords.get(key));
|
||||
try {
|
||||
Option<IndexedRecord> combinedAvroRecord = hoodieRecord.getData()
|
||||
.combineAndGetUpdateValue(oldRecord, useWriterSchema ? writerSchema : originalSchema);
|
||||
if (writeUpdateRecord(hoodieRecord, combinedAvroRecord)) {
|
||||
/* ONLY WHEN
|
||||
* 1) we have an update for this key AND
|
||||
* 2) We are able to successfully write the the combined new value
|
||||
*
|
||||
* We no longer need to copy the old record over.
|
||||
*/
|
||||
copyOldRecord = false;
|
||||
}
|
||||
writtenRecordKeys.add(key);
|
||||
} catch (Exception e) {
|
||||
throw new HoodieUpsertException(
|
||||
"Failed to combine/merge new record with old value in storage, for new record {"
|
||||
+ keyToNewRecords.get(key) + "}, old value {" + oldRecord + "}", e);
|
||||
}
|
||||
}
|
||||
|
||||
if (copyOldRecord) {
|
||||
// this should work as it is, since this is an existing record
|
||||
String errMsg = "Failed to merge old record into new file for key " + key + " from old file "
|
||||
+ getOldFilePath() + " to new file " + newFilePath;
|
||||
try {
|
||||
storageWriter.writeAvro(key, oldRecord);
|
||||
} catch (ClassCastException e) {
|
||||
logger.error("Schema mismatch when rewriting old record " + oldRecord + " from file "
|
||||
+ getOldFilePath() + " to file " + newFilePath + " with writerSchema " + writerSchema
|
||||
.toString(true));
|
||||
throw new HoodieUpsertException(errMsg, e);
|
||||
} catch (IOException e) {
|
||||
logger.error("Failed to merge old record into new file for key " + key + " from old file "
|
||||
+ getOldFilePath() + " to new file " + newFilePath, e);
|
||||
throw new HoodieUpsertException(errMsg, e);
|
||||
}
|
||||
recordsWritten++;
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public WriteStatus close() {
|
||||
try {
|
||||
// write out any pending records (this can happen when inserts are turned into updates)
|
||||
for (String key : keyToNewRecords.keySet()) {
|
||||
if (!writtenRecordKeys.contains(key)) {
|
||||
HoodieRecord<T> hoodieRecord = keyToNewRecords.get(key);
|
||||
if (useWriterSchema) {
|
||||
writeRecord(hoodieRecord, hoodieRecord.getData().getInsertValue(writerSchema));
|
||||
} else {
|
||||
writeRecord(hoodieRecord, hoodieRecord.getData().getInsertValue(originalSchema));
|
||||
}
|
||||
insertRecordsWritten++;
|
||||
}
|
||||
}
|
||||
keyToNewRecords.clear();
|
||||
writtenRecordKeys.clear();
|
||||
|
||||
if (storageWriter != null) {
|
||||
storageWriter.close();
|
||||
}
|
||||
|
||||
long fileSizeInBytes = FSUtils.getFileSize(fs, newFilePath);
|
||||
HoodieWriteStat stat = writeStatus.getStat();
|
||||
|
||||
stat.setTotalWriteBytes(fileSizeInBytes);
|
||||
stat.setFileSizeInBytes(fileSizeInBytes);
|
||||
stat.setNumWrites(recordsWritten);
|
||||
stat.setNumDeletes(recordsDeleted);
|
||||
stat.setNumUpdateWrites(updatedRecordsWritten);
|
||||
stat.setNumInserts(insertRecordsWritten);
|
||||
stat.setTotalWriteErrors(writeStatus.getTotalErrorRecords());
|
||||
RuntimeStats runtimeStats = new RuntimeStats();
|
||||
runtimeStats.setTotalUpsertTime(timer.endTimer());
|
||||
stat.setRuntimeStats(runtimeStats);
|
||||
|
||||
logger.info(String.format("MergeHandle for partitionPath %s fileID %s, took %d ms.",
|
||||
stat.getPartitionPath(), stat.getFileId(), runtimeStats.getTotalUpsertTime()));
|
||||
|
||||
return writeStatus;
|
||||
} catch (IOException e) {
|
||||
throw new HoodieUpsertException("Failed to close UpdateHandle", e);
|
||||
}
|
||||
}
|
||||
|
||||
public Path getOldFilePath() {
|
||||
return oldFilePath;
|
||||
}
|
||||
|
||||
@Override
|
||||
public WriteStatus getWriteStatus() {
|
||||
return writeStatus;
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,43 @@
|
||||
/*
|
||||
* 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.io;
|
||||
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hudi.common.model.HoodieDataFile;
|
||||
import org.apache.hudi.common.model.HoodieRecordPayload;
|
||||
import org.apache.hudi.common.util.ParquetUtils;
|
||||
import org.apache.hudi.common.util.collection.Pair;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
import org.apache.hudi.table.HoodieTable;
|
||||
|
||||
/**
|
||||
* Extract range information for a given file slice
|
||||
*/
|
||||
public class HoodieRangeInfoHandle<T extends HoodieRecordPayload> extends HoodieReadHandle<T> {
|
||||
|
||||
public HoodieRangeInfoHandle(HoodieWriteConfig config, HoodieTable<T> hoodieTable,
|
||||
Pair<String, String> partitionPathFilePair) {
|
||||
super(config, null, hoodieTable, partitionPathFilePair);
|
||||
}
|
||||
|
||||
public String[] getMinMaxKeys() {
|
||||
HoodieDataFile dataFile = getLatestDataFile();
|
||||
return ParquetUtils.readMinMaxRecordKeys(hoodieTable.getHadoopConf(), new Path(dataFile.getPath()));
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,58 @@
|
||||
/*
|
||||
* 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.io;
|
||||
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hudi.common.model.HoodieDataFile;
|
||||
import org.apache.hudi.common.model.HoodieRecordPayload;
|
||||
import org.apache.hudi.common.util.collection.Pair;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
import org.apache.hudi.table.HoodieTable;
|
||||
|
||||
/**
|
||||
* Base class for read operations done logically on the file group.
|
||||
*/
|
||||
public abstract class HoodieReadHandle<T extends HoodieRecordPayload> extends HoodieIOHandle {
|
||||
|
||||
protected final Pair<String, String> partitionPathFilePair;
|
||||
|
||||
public HoodieReadHandle(HoodieWriteConfig config, String instantTime, HoodieTable<T> hoodieTable,
|
||||
Pair<String, String> partitionPathFilePair) {
|
||||
super(config, instantTime, hoodieTable);
|
||||
this.partitionPathFilePair = partitionPathFilePair;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected FileSystem getFileSystem() {
|
||||
return hoodieTable.getMetaClient().getFs();
|
||||
}
|
||||
|
||||
public Pair<String, String> getPartitionPathFilePair() {
|
||||
return partitionPathFilePair;
|
||||
}
|
||||
|
||||
public String getFileId() {
|
||||
return partitionPathFilePair.getRight();
|
||||
}
|
||||
|
||||
protected HoodieDataFile getLatestDataFile() {
|
||||
return hoodieTable.getROFileSystemView()
|
||||
.getLatestDataFile(partitionPathFilePair.getLeft(), partitionPathFilePair.getRight()).get();
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,170 @@
|
||||
/*
|
||||
* 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.io;
|
||||
|
||||
import java.io.IOException;
|
||||
import org.apache.avro.Schema;
|
||||
import org.apache.avro.generic.GenericRecord;
|
||||
import org.apache.avro.generic.IndexedRecord;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hudi.WriteStatus;
|
||||
import org.apache.hudi.common.model.HoodieRecord;
|
||||
import org.apache.hudi.common.model.HoodieRecordPayload;
|
||||
import org.apache.hudi.common.util.FSUtils;
|
||||
import org.apache.hudi.common.util.HoodieAvroUtils;
|
||||
import org.apache.hudi.common.util.HoodieTimer;
|
||||
import org.apache.hudi.common.util.Option;
|
||||
import org.apache.hudi.common.util.ReflectionUtils;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
import org.apache.hudi.exception.HoodieException;
|
||||
import org.apache.hudi.exception.HoodieIOException;
|
||||
import org.apache.hudi.table.HoodieTable;
|
||||
import org.apache.log4j.LogManager;
|
||||
import org.apache.log4j.Logger;
|
||||
import org.apache.spark.TaskContext;
|
||||
|
||||
/**
|
||||
* Base class for all write operations logically performed at the file group level.
|
||||
*/
|
||||
public abstract class HoodieWriteHandle<T extends HoodieRecordPayload> extends HoodieIOHandle {
|
||||
|
||||
private static Logger logger = LogManager.getLogger(HoodieWriteHandle.class);
|
||||
protected final Schema originalSchema;
|
||||
protected final Schema writerSchema;
|
||||
protected HoodieTimer timer;
|
||||
protected final WriteStatus writeStatus;
|
||||
protected final String fileId;
|
||||
protected final String writeToken;
|
||||
|
||||
public HoodieWriteHandle(HoodieWriteConfig config, String instantTime, String fileId, HoodieTable<T> hoodieTable) {
|
||||
super(config, instantTime, hoodieTable);
|
||||
this.fileId = fileId;
|
||||
this.writeToken = makeSparkWriteToken();
|
||||
this.originalSchema = new Schema.Parser().parse(config.getSchema());
|
||||
this.writerSchema = createHoodieWriteSchema(originalSchema);
|
||||
this.timer = new HoodieTimer().startTimer();
|
||||
this.writeStatus = (WriteStatus) ReflectionUtils.loadClass(config.getWriteStatusClassName(),
|
||||
!hoodieTable.getIndex().isImplicitWithStorage(),
|
||||
config.getWriteStatusFailureFraction());
|
||||
}
|
||||
|
||||
/**
|
||||
* Generate a write token based on the currently running spark task and its place in the spark dag.
|
||||
*/
|
||||
private static String makeSparkWriteToken() {
|
||||
return FSUtils.makeWriteToken(TaskContext.getPartitionId(), TaskContext.get().stageId(),
|
||||
TaskContext.get().taskAttemptId());
|
||||
}
|
||||
|
||||
public static Schema createHoodieWriteSchema(Schema originalSchema) {
|
||||
return HoodieAvroUtils.addMetadataFields(originalSchema);
|
||||
}
|
||||
|
||||
public Path makeNewPath(String partitionPath) {
|
||||
Path path = FSUtils.getPartitionPath(config.getBasePath(), partitionPath);
|
||||
try {
|
||||
fs.mkdirs(path); // create a new partition as needed.
|
||||
} catch (IOException e) {
|
||||
throw new HoodieIOException("Failed to make dir " + path, e);
|
||||
}
|
||||
|
||||
return new Path(path.toString(), FSUtils.makeDataFileName(instantTime, writeToken, fileId));
|
||||
}
|
||||
|
||||
/**
|
||||
* Creates an empty marker file corresponding to storage writer path
|
||||
*
|
||||
* @param partitionPath Partition path
|
||||
*/
|
||||
protected void createMarkerFile(String partitionPath) {
|
||||
Path markerPath = makeNewMarkerPath(partitionPath);
|
||||
try {
|
||||
logger.info("Creating Marker Path=" + markerPath);
|
||||
fs.create(markerPath, false).close();
|
||||
} catch (IOException e) {
|
||||
throw new HoodieException("Failed to create marker file " + markerPath, e);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* THe marker path will be <base-path>/.hoodie/.temp/<instant_ts>/2019/04/25/filename
|
||||
*/
|
||||
private Path makeNewMarkerPath(String partitionPath) {
|
||||
Path markerRootPath = new Path(hoodieTable.getMetaClient().getMarkerFolderPath(instantTime));
|
||||
Path path = FSUtils.getPartitionPath(markerRootPath, partitionPath);
|
||||
try {
|
||||
fs.mkdirs(path); // create a new partition as needed.
|
||||
} catch (IOException e) {
|
||||
throw new HoodieIOException("Failed to make dir " + path, e);
|
||||
}
|
||||
return new Path(path.toString(), FSUtils.makeMarkerFile(instantTime, writeToken, fileId));
|
||||
}
|
||||
|
||||
public Schema getWriterSchema() {
|
||||
return writerSchema;
|
||||
}
|
||||
|
||||
/**
|
||||
* Determines whether we can accept the incoming records, into the current file, depending on
|
||||
* <p>
|
||||
* - Whether it belongs to the same partitionPath as existing records - Whether the current file written bytes lt max
|
||||
* file size
|
||||
*/
|
||||
public boolean canWrite(HoodieRecord record) {
|
||||
return false;
|
||||
}
|
||||
|
||||
/**
|
||||
* Perform the actual writing of the given record into the backing file.
|
||||
*/
|
||||
public void write(HoodieRecord record, Option<IndexedRecord> insertValue) {
|
||||
// NO_OP
|
||||
}
|
||||
|
||||
/**
|
||||
* Perform the actual writing of the given record into the backing file.
|
||||
*/
|
||||
public void write(HoodieRecord record, Option<IndexedRecord> avroRecord, Option<Exception> exception) {
|
||||
Option recordMetadata = record.getData().getMetadata();
|
||||
if (exception.isPresent() && exception.get() instanceof Throwable) {
|
||||
// Not throwing exception from here, since we don't want to fail the entire job for a single record
|
||||
writeStatus.markFailure(record, exception.get(), recordMetadata);
|
||||
logger.error("Error writing record " + record, exception.get());
|
||||
} else {
|
||||
write(record, avroRecord);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Rewrite the GenericRecord with the Schema containing the Hoodie Metadata fields
|
||||
*/
|
||||
protected GenericRecord rewriteRecord(GenericRecord record) {
|
||||
return HoodieAvroUtils.rewriteRecord(record, writerSchema);
|
||||
}
|
||||
|
||||
public abstract WriteStatus close();
|
||||
|
||||
public abstract WriteStatus getWriteStatus();
|
||||
|
||||
@Override
|
||||
protected FileSystem getFileSystem() {
|
||||
return hoodieTable.getMetaClient().getFs();
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,59 @@
|
||||
/*
|
||||
* 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.io.compact;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.io.Serializable;
|
||||
import java.util.Set;
|
||||
import org.apache.hudi.WriteStatus;
|
||||
import org.apache.hudi.avro.model.HoodieCompactionPlan;
|
||||
import org.apache.hudi.common.model.HoodieFileGroupId;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
import org.apache.hudi.table.HoodieTable;
|
||||
import org.apache.spark.api.java.JavaRDD;
|
||||
import org.apache.spark.api.java.JavaSparkContext;
|
||||
|
||||
/**
|
||||
* A HoodieCompactor runs compaction on a hoodie table
|
||||
*/
|
||||
public interface HoodieCompactor extends Serializable {
|
||||
|
||||
/**
|
||||
* Generate a new compaction plan for scheduling
|
||||
*
|
||||
* @param jsc Spark Context
|
||||
* @param hoodieTable Hoodie Table
|
||||
* @param config Hoodie Write Configuration
|
||||
* @param compactionCommitTime scheduled compaction commit time
|
||||
* @param fgIdsInPendingCompactions partition-fileId pairs for which compaction is pending
|
||||
* @return Compaction Plan
|
||||
* @throws IOException when encountering errors
|
||||
*/
|
||||
HoodieCompactionPlan generateCompactionPlan(JavaSparkContext jsc,
|
||||
HoodieTable hoodieTable, HoodieWriteConfig config, String compactionCommitTime,
|
||||
Set<HoodieFileGroupId> fgIdsInPendingCompactions)
|
||||
throws IOException;
|
||||
|
||||
/**
|
||||
* Execute compaction operations and report back status
|
||||
*/
|
||||
JavaRDD<WriteStatus> compact(JavaSparkContext jsc,
|
||||
HoodieCompactionPlan compactionPlan, HoodieTable hoodieTable, HoodieWriteConfig config,
|
||||
String compactionInstantTime) throws IOException;
|
||||
}
|
||||
@@ -0,0 +1,236 @@
|
||||
/*
|
||||
* 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.io.compact;
|
||||
|
||||
import static java.util.stream.Collectors.toList;
|
||||
|
||||
import com.google.common.base.Preconditions;
|
||||
import com.google.common.collect.Lists;
|
||||
import com.google.common.collect.Sets;
|
||||
import java.io.IOException;
|
||||
import java.util.Collection;
|
||||
import java.util.Iterator;
|
||||
import java.util.List;
|
||||
import java.util.Set;
|
||||
import java.util.stream.Collectors;
|
||||
import java.util.stream.StreamSupport;
|
||||
import org.apache.avro.Schema;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hudi.WriteStatus;
|
||||
import org.apache.hudi.avro.model.HoodieCompactionOperation;
|
||||
import org.apache.hudi.avro.model.HoodieCompactionPlan;
|
||||
import org.apache.hudi.common.model.CompactionOperation;
|
||||
import org.apache.hudi.common.model.HoodieDataFile;
|
||||
import org.apache.hudi.common.model.HoodieFileGroupId;
|
||||
import org.apache.hudi.common.model.HoodieLogFile;
|
||||
import org.apache.hudi.common.model.HoodieTableType;
|
||||
import org.apache.hudi.common.model.HoodieWriteStat.RuntimeStats;
|
||||
import org.apache.hudi.common.table.HoodieTableMetaClient;
|
||||
import org.apache.hudi.common.table.HoodieTimeline;
|
||||
import org.apache.hudi.common.table.TableFileSystemView.RealtimeView;
|
||||
import org.apache.hudi.common.table.log.HoodieMergedLogRecordScanner;
|
||||
import org.apache.hudi.common.util.CompactionUtils;
|
||||
import org.apache.hudi.common.util.FSUtils;
|
||||
import org.apache.hudi.common.util.HoodieAvroUtils;
|
||||
import org.apache.hudi.common.util.Option;
|
||||
import org.apache.hudi.common.util.collection.Pair;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
import org.apache.hudi.io.compact.strategy.CompactionStrategy;
|
||||
import org.apache.hudi.table.HoodieCopyOnWriteTable;
|
||||
import org.apache.hudi.table.HoodieTable;
|
||||
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 org.apache.spark.api.java.function.FlatMapFunction;
|
||||
import org.apache.spark.util.AccumulatorV2;
|
||||
import org.apache.spark.util.LongAccumulator;
|
||||
|
||||
/**
|
||||
* HoodieRealtimeTableCompactor compacts a hoodie table with merge on read storage. Computes all
|
||||
* possible compactions, passes it through a CompactionFilter and executes all the compactions and
|
||||
* writes a new version of base files and make a normal commit
|
||||
*
|
||||
* @see HoodieCompactor
|
||||
*/
|
||||
public class HoodieRealtimeTableCompactor implements HoodieCompactor {
|
||||
|
||||
private static Logger log = LogManager.getLogger(HoodieRealtimeTableCompactor.class);
|
||||
// Accumulator to keep track of total log files for a dataset
|
||||
private AccumulatorV2<Long, Long> totalLogFiles;
|
||||
// Accumulator to keep track of total log file slices for a dataset
|
||||
private AccumulatorV2<Long, Long> totalFileSlices;
|
||||
|
||||
@Override
|
||||
public JavaRDD<WriteStatus> compact(JavaSparkContext jsc,
|
||||
HoodieCompactionPlan compactionPlan, HoodieTable hoodieTable, HoodieWriteConfig config,
|
||||
String compactionInstantTime) throws IOException {
|
||||
if (compactionPlan == null || (compactionPlan.getOperations() == null)
|
||||
|| (compactionPlan.getOperations().isEmpty())) {
|
||||
return jsc.emptyRDD();
|
||||
}
|
||||
HoodieTableMetaClient metaClient = hoodieTable.getMetaClient();
|
||||
// Compacting is very similar to applying updates to existing file
|
||||
HoodieCopyOnWriteTable table = new HoodieCopyOnWriteTable(config, jsc);
|
||||
List<CompactionOperation> operations = compactionPlan.getOperations().stream().map(
|
||||
CompactionOperation::convertFromAvroRecordInstance).collect(toList());
|
||||
log.info("Compactor compacting " + operations + " files");
|
||||
|
||||
return jsc.parallelize(operations, operations.size())
|
||||
.map(s -> compact(table, metaClient, config, s, compactionInstantTime))
|
||||
.flatMap(List::iterator);
|
||||
}
|
||||
|
||||
private List<WriteStatus> compact(HoodieCopyOnWriteTable hoodieCopyOnWriteTable, HoodieTableMetaClient metaClient,
|
||||
HoodieWriteConfig config,
|
||||
CompactionOperation operation, String commitTime) throws IOException {
|
||||
FileSystem fs = metaClient.getFs();
|
||||
Schema readerSchema = HoodieAvroUtils
|
||||
.addMetadataFields(new Schema.Parser().parse(config.getSchema()));
|
||||
|
||||
log.info("Compacting base " + operation.getDataFilePath() + " with delta files " + operation
|
||||
.getDeltaFilePaths() + " for commit " + commitTime);
|
||||
// TODO - FIX THIS
|
||||
// Reads the entire avro file. Always only specific blocks should be read from the avro file
|
||||
// (failure recover).
|
||||
// Load all the delta commits since the last compaction commit and get all the blocks to be
|
||||
// loaded and load it using CompositeAvroLogReader
|
||||
// Since a DeltaCommit is not defined yet, reading all the records. revisit this soon.
|
||||
String maxInstantTime = metaClient.getActiveTimeline()
|
||||
.getTimelineOfActions(
|
||||
Sets.newHashSet(HoodieTimeline.COMMIT_ACTION, HoodieTimeline.ROLLBACK_ACTION,
|
||||
HoodieTimeline.DELTA_COMMIT_ACTION))
|
||||
.filterCompletedInstants().lastInstant().get().getTimestamp();
|
||||
log.info("MaxMemoryPerCompaction => " + config.getMaxMemoryPerCompaction());
|
||||
HoodieMergedLogRecordScanner scanner = new HoodieMergedLogRecordScanner(fs,
|
||||
metaClient.getBasePath(), operation.getDeltaFilePaths(), readerSchema, maxInstantTime,
|
||||
config.getMaxMemoryPerCompaction(), config.getCompactionLazyBlockReadEnabled(),
|
||||
config.getCompactionReverseLogReadEnabled(), config.getMaxDFSStreamBufferSize(),
|
||||
config.getSpillableMapBasePath());
|
||||
if (!scanner.iterator().hasNext()) {
|
||||
return Lists.<WriteStatus>newArrayList();
|
||||
}
|
||||
|
||||
Option<HoodieDataFile> oldDataFileOpt = operation.getBaseFile();
|
||||
|
||||
// Compacting is very similar to applying updates to existing file
|
||||
Iterator<List<WriteStatus>> result;
|
||||
// If the dataFile is present, there is a base parquet file present, perform updates else perform inserts into a
|
||||
// new base parquet file.
|
||||
if (oldDataFileOpt.isPresent()) {
|
||||
result = hoodieCopyOnWriteTable
|
||||
.handleUpdate(commitTime, operation.getFileId(), scanner.getRecords(), oldDataFileOpt.get());
|
||||
} else {
|
||||
result = hoodieCopyOnWriteTable
|
||||
.handleInsert(commitTime, operation.getPartitionPath(), operation.getFileId(), scanner.iterator());
|
||||
}
|
||||
Iterable<List<WriteStatus>> resultIterable = () -> result;
|
||||
return StreamSupport.stream(resultIterable.spliterator(), false).flatMap(Collection::stream)
|
||||
.peek(s -> {
|
||||
s.getStat().setTotalUpdatedRecordsCompacted(scanner.getNumMergedRecordsInLog());
|
||||
s.getStat().setTotalLogFilesCompacted(scanner.getTotalLogFiles());
|
||||
s.getStat().setTotalLogRecords(scanner.getTotalLogRecords());
|
||||
s.getStat().setPartitionPath(operation.getPartitionPath());
|
||||
s.getStat().setTotalLogSizeCompacted(operation.getMetrics().get(
|
||||
CompactionStrategy.TOTAL_LOG_FILE_SIZE).longValue());
|
||||
s.getStat().setTotalLogBlocks(scanner.getTotalLogBlocks());
|
||||
s.getStat().setTotalCorruptLogBlock(scanner.getTotalCorruptBlocks());
|
||||
s.getStat().setTotalRollbackBlocks(scanner.getTotalRollbacks());
|
||||
RuntimeStats runtimeStats = new RuntimeStats();
|
||||
runtimeStats.setTotalScanTime(scanner.getTotalTimeTakenToReadAndMergeBlocks());
|
||||
s.getStat().setRuntimeStats(runtimeStats);
|
||||
}).collect(toList());
|
||||
}
|
||||
|
||||
@Override
|
||||
public HoodieCompactionPlan generateCompactionPlan(JavaSparkContext jsc,
|
||||
HoodieTable hoodieTable, HoodieWriteConfig config, String compactionCommitTime,
|
||||
Set<HoodieFileGroupId> fgIdsInPendingCompactions) throws IOException {
|
||||
|
||||
totalLogFiles = new LongAccumulator();
|
||||
totalFileSlices = new LongAccumulator();
|
||||
jsc.sc().register(totalLogFiles);
|
||||
jsc.sc().register(totalFileSlices);
|
||||
|
||||
Preconditions
|
||||
.checkArgument(hoodieTable.getMetaClient().getTableType() == HoodieTableType.MERGE_ON_READ,
|
||||
"HoodieRealtimeTableCompactor can only compact table of type "
|
||||
+ HoodieTableType.MERGE_ON_READ + " and not " + hoodieTable.getMetaClient()
|
||||
.getTableType().name());
|
||||
|
||||
//TODO : check if maxMemory is not greater than JVM or spark.executor memory
|
||||
// TODO - rollback any compactions in flight
|
||||
HoodieTableMetaClient metaClient = hoodieTable.getMetaClient();
|
||||
log.info("Compacting " + metaClient.getBasePath() + " with commit " + compactionCommitTime);
|
||||
List<String> partitionPaths = FSUtils
|
||||
.getAllPartitionPaths(metaClient.getFs(), metaClient.getBasePath(),
|
||||
config.shouldAssumeDatePartitioning());
|
||||
|
||||
// filter the partition paths if needed to reduce list status
|
||||
partitionPaths = config.getCompactionStrategy().filterPartitionPaths(config, partitionPaths);
|
||||
|
||||
if (partitionPaths.isEmpty()) {
|
||||
// In case no partitions could be picked, return no compaction plan
|
||||
return null;
|
||||
}
|
||||
|
||||
RealtimeView fileSystemView = hoodieTable.getRTFileSystemView();
|
||||
log.info("Compaction looking for files to compact in " + partitionPaths + " partitions");
|
||||
List<HoodieCompactionOperation> operations =
|
||||
jsc.parallelize(partitionPaths, partitionPaths.size())
|
||||
.flatMap((FlatMapFunction<String, CompactionOperation>) partitionPath -> fileSystemView
|
||||
.getLatestFileSlices(partitionPath)
|
||||
.filter(slice ->
|
||||
!fgIdsInPendingCompactions.contains(slice.getFileGroupId()))
|
||||
.map(
|
||||
s -> {
|
||||
List<HoodieLogFile> logFiles = s.getLogFiles().sorted(HoodieLogFile
|
||||
.getLogFileComparator()).collect(Collectors.toList());
|
||||
totalLogFiles.add((long) logFiles.size());
|
||||
totalFileSlices.add(1L);
|
||||
// Avro generated classes are not inheriting Serializable. Using CompactionOperation POJO
|
||||
// for spark Map operations and collecting them finally in Avro generated classes for storing
|
||||
// into meta files.
|
||||
Option<HoodieDataFile> dataFile = s.getDataFile();
|
||||
return new CompactionOperation(dataFile, partitionPath, logFiles,
|
||||
config.getCompactionStrategy().captureMetrics(config, dataFile, partitionPath, logFiles));
|
||||
})
|
||||
.filter(c -> !c.getDeltaFilePaths().isEmpty())
|
||||
.collect(toList()).iterator()).collect().stream().map(CompactionUtils::buildHoodieCompactionOperation)
|
||||
.collect(toList());
|
||||
log.info("Total of " + operations.size() + " compactions are retrieved");
|
||||
log.info("Total number of latest files slices " + totalFileSlices.value());
|
||||
log.info("Total number of log files " + totalLogFiles.value());
|
||||
log.info("Total number of file slices " + totalFileSlices.value());
|
||||
// Filter the compactions with the passed in filter. This lets us choose most effective
|
||||
// compactions only
|
||||
HoodieCompactionPlan compactionPlan = config.getCompactionStrategy().generateCompactionPlan(config, operations,
|
||||
CompactionUtils.getAllPendingCompactionPlans(metaClient).stream().map(Pair::getValue).collect(toList()));
|
||||
Preconditions.checkArgument(compactionPlan.getOperations().stream().noneMatch(
|
||||
op -> fgIdsInPendingCompactions.contains(new HoodieFileGroupId(op.getPartitionPath(), op.getFileId()))),
|
||||
"Bad Compaction Plan. FileId MUST NOT have multiple pending compactions. "
|
||||
+ "Please fix your strategy implementation."
|
||||
+ "FileIdsWithPendingCompactions :" + fgIdsInPendingCompactions
|
||||
+ ", Selected workload :" + compactionPlan);
|
||||
if (compactionPlan.getOperations().isEmpty()) {
|
||||
log.warn("After filtering, Nothing to compact for " + metaClient.getBasePath());
|
||||
}
|
||||
return compactionPlan;
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,53 @@
|
||||
/*
|
||||
* 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.io.compact.strategy;
|
||||
|
||||
import com.google.common.collect.Lists;
|
||||
import java.util.List;
|
||||
import org.apache.hudi.avro.model.HoodieCompactionOperation;
|
||||
import org.apache.hudi.avro.model.HoodieCompactionPlan;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
|
||||
/**
|
||||
* CompactionStrategy which looks at total IO to be done for the compaction (read + write) and
|
||||
* limits the list of compactions to be under a configured limit on the IO
|
||||
*
|
||||
* @see CompactionStrategy
|
||||
*/
|
||||
public class BoundedIOCompactionStrategy extends CompactionStrategy {
|
||||
|
||||
@Override
|
||||
public List<HoodieCompactionOperation> orderAndFilter(HoodieWriteConfig writeConfig,
|
||||
List<HoodieCompactionOperation> operations, List<HoodieCompactionPlan> pendingCompactionPlans) {
|
||||
// Iterate through the operations in order and accept operations as long as we are within the
|
||||
// IO limit
|
||||
// Preserves the original ordering of compactions
|
||||
List<HoodieCompactionOperation> finalOperations = Lists.newArrayList();
|
||||
long targetIORemaining = writeConfig.getTargetIOPerCompactionInMB();
|
||||
for (HoodieCompactionOperation op : operations) {
|
||||
long opIo = op.getMetrics().get(TOTAL_IO_MB).longValue();
|
||||
targetIORemaining -= opIo;
|
||||
finalOperations.add(op);
|
||||
if (targetIORemaining <= 0) {
|
||||
return finalOperations;
|
||||
}
|
||||
}
|
||||
return finalOperations;
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,73 @@
|
||||
/*
|
||||
* 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.io.compact.strategy;
|
||||
|
||||
import java.text.SimpleDateFormat;
|
||||
import java.util.Comparator;
|
||||
import java.util.Date;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.stream.Collectors;
|
||||
import org.apache.commons.lang3.time.DateUtils;
|
||||
import org.apache.hudi.avro.model.HoodieCompactionOperation;
|
||||
import org.apache.hudi.avro.model.HoodieCompactionPlan;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
|
||||
/**
|
||||
* This strategy ensures that the last N partitions are picked up even if there are later partitions created for the
|
||||
* dataset. lastNPartitions is defined as the N partitions before the currentDate.
|
||||
* currentDay = 2018/01/01
|
||||
* The dataset has partitions for 2018/02/02 and 2018/03/03 beyond the currentDay
|
||||
* This strategy will pick up the following partitions for compaction :
|
||||
* (2018/01/01, allPartitionsInRange[(2018/01/01 - lastNPartitions) to 2018/01/01), 2018/02/02, 2018/03/03)
|
||||
*/
|
||||
public class BoundedPartitionAwareCompactionStrategy extends DayBasedCompactionStrategy {
|
||||
|
||||
SimpleDateFormat dateFormat = new SimpleDateFormat(datePartitionFormat);
|
||||
|
||||
@Override
|
||||
public List<HoodieCompactionOperation> orderAndFilter(HoodieWriteConfig writeConfig,
|
||||
List<HoodieCompactionOperation> operations, List<HoodieCompactionPlan> pendingCompactionPlans) {
|
||||
// The earliest partition to compact - current day minus the target partitions limit
|
||||
String earliestPartitionPathToCompact = dateFormat.format(DateUtils.addDays(new Date(), -1 * writeConfig
|
||||
.getTargetPartitionsPerDayBasedCompaction()));
|
||||
// Filter out all partitions greater than earliestPartitionPathToCompact
|
||||
List<HoodieCompactionOperation> eligibleCompactionOperations = operations.stream()
|
||||
.collect(Collectors.groupingBy(HoodieCompactionOperation::getPartitionPath)).entrySet().stream()
|
||||
.sorted(Map.Entry.comparingByKey(comparator))
|
||||
.filter(e -> comparator.compare(earliestPartitionPathToCompact, e.getKey()) >= 0)
|
||||
.flatMap(e -> e.getValue().stream())
|
||||
.collect(Collectors.toList());
|
||||
|
||||
return eligibleCompactionOperations;
|
||||
}
|
||||
|
||||
@Override
|
||||
public List<String> filterPartitionPaths(HoodieWriteConfig writeConfig, List<String> partitionPaths) {
|
||||
// The earliest partition to compact - current day minus the target partitions limit
|
||||
String earliestPartitionPathToCompact = dateFormat.format(DateUtils.addDays(new Date(), -1 * writeConfig
|
||||
.getTargetPartitionsPerDayBasedCompaction()));
|
||||
// Get all partitions and sort them
|
||||
List<String> filteredPartitionPaths = partitionPaths.stream().map(partition -> partition.replace("/", "-"))
|
||||
.sorted(Comparator.reverseOrder()).map(partitionPath -> partitionPath.replace("-", "/"))
|
||||
.filter(e -> comparator.compare(earliestPartitionPathToCompact, e) >= 0)
|
||||
.collect(Collectors.toList());
|
||||
return filteredPartitionPaths;
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,126 @@
|
||||
/*
|
||||
* 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.io.compact.strategy;
|
||||
|
||||
import com.google.common.collect.Maps;
|
||||
import java.io.Serializable;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import org.apache.hudi.avro.model.HoodieCompactionOperation;
|
||||
import org.apache.hudi.avro.model.HoodieCompactionPlan;
|
||||
import org.apache.hudi.common.model.HoodieDataFile;
|
||||
import org.apache.hudi.common.model.HoodieLogFile;
|
||||
import org.apache.hudi.common.util.FSUtils;
|
||||
import org.apache.hudi.common.util.Option;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
import org.apache.hudi.io.compact.HoodieRealtimeTableCompactor;
|
||||
|
||||
/**
|
||||
* Strategy for compaction. Pluggable implementation to define how compaction should be done. The
|
||||
* over-ridden implementations of this abstract class can capture the relevant metrics to order
|
||||
* and filter the final list of compaction operation to run in a single compaction.
|
||||
* Implementation of CompactionStrategy cannot hold any state. Difference instantiations can be
|
||||
* passed in every time
|
||||
*
|
||||
* @see HoodieRealtimeTableCompactor
|
||||
*/
|
||||
public abstract class CompactionStrategy implements Serializable {
|
||||
|
||||
public static final String TOTAL_IO_READ_MB = "TOTAL_IO_READ_MB";
|
||||
public static final String TOTAL_IO_WRITE_MB = "TOTAL_IO_WRITE_MB";
|
||||
public static final String TOTAL_IO_MB = "TOTAL_IO_MB";
|
||||
public static final String TOTAL_LOG_FILE_SIZE = "TOTAL_LOG_FILES_SIZE";
|
||||
public static final String TOTAL_LOG_FILES = "TOTAL_LOG_FILES";
|
||||
|
||||
/**
|
||||
* Callback hook when a HoodieCompactionOperation is created. Individual strategies can capture the
|
||||
* metrics they need to decide on the priority.
|
||||
*
|
||||
* @param dataFile - Base file to compact
|
||||
* @param partitionPath - Partition path
|
||||
* @param logFiles - List of log files to compact with the base file
|
||||
* @return Map[String, Object] - metrics captured
|
||||
*/
|
||||
public Map<String, Double> captureMetrics(HoodieWriteConfig writeConfig, Option<HoodieDataFile> dataFile,
|
||||
String partitionPath, List<HoodieLogFile> logFiles) {
|
||||
Map<String, Double> metrics = Maps.newHashMap();
|
||||
Long defaultMaxParquetFileSize = writeConfig.getParquetMaxFileSize();
|
||||
// Total size of all the log files
|
||||
Long totalLogFileSize = logFiles.stream().map(HoodieLogFile::getFileSize).filter(size -> size >= 0)
|
||||
.reduce((size1, size2) -> size1 + size2).orElse(0L);
|
||||
// Total read will be the base file + all the log files
|
||||
Long totalIORead = FSUtils.getSizeInMB((dataFile.isPresent() ? dataFile.get().getFileSize() : 0L)
|
||||
+ totalLogFileSize);
|
||||
// Total write will be similar to the size of the base file
|
||||
Long totalIOWrite = FSUtils
|
||||
.getSizeInMB(dataFile.isPresent() ? dataFile.get().getFileSize() : defaultMaxParquetFileSize);
|
||||
// Total IO will the the IO for read + write
|
||||
Long totalIO = totalIORead + totalIOWrite;
|
||||
// Save these metrics and we will use during the filter
|
||||
metrics.put(TOTAL_IO_READ_MB, totalIORead.doubleValue());
|
||||
metrics.put(TOTAL_IO_WRITE_MB, totalIOWrite.doubleValue());
|
||||
metrics.put(TOTAL_IO_MB, totalIO.doubleValue());
|
||||
metrics.put(TOTAL_LOG_FILE_SIZE, totalLogFileSize.doubleValue());
|
||||
metrics.put(TOTAL_LOG_FILES, (double) logFiles.size());
|
||||
return metrics;
|
||||
}
|
||||
|
||||
/**
|
||||
* Generate Compaction plan. Allows clients to order and filter the list of compactions to be set. The default
|
||||
* implementation takes care of setting compactor Id from configuration allowing subclasses to only worry about
|
||||
* ordering and filtering compaction operations
|
||||
*
|
||||
* @param writeConfig Hoodie Write Config
|
||||
* @param operations Compaction Operations to be ordered and filtered
|
||||
* @param pendingCompactionPlans Pending Compaction Plans for strategy to schedule next compaction plan
|
||||
* @return Compaction plan to be scheduled.
|
||||
*/
|
||||
public HoodieCompactionPlan generateCompactionPlan(HoodieWriteConfig writeConfig,
|
||||
List<HoodieCompactionOperation> operations, List<HoodieCompactionPlan> pendingCompactionPlans) {
|
||||
// Strategy implementation can overload this method to set specific compactor-id
|
||||
return HoodieCompactionPlan.newBuilder()
|
||||
.setOperations(orderAndFilter(writeConfig, operations, pendingCompactionPlans))
|
||||
.build();
|
||||
}
|
||||
|
||||
/**
|
||||
* Order and Filter the list of compactions. Use the metrics captured with the captureMetrics to order and filter out
|
||||
* compactions
|
||||
*
|
||||
* @param writeConfig config for this compaction is passed in
|
||||
* @param operations list of compactions collected
|
||||
* @param pendingCompactionPlans Pending Compaction Plans for strategy to schedule next compaction plan
|
||||
* @return list of compactions to perform in this run
|
||||
*/
|
||||
public List<HoodieCompactionOperation> orderAndFilter(HoodieWriteConfig writeConfig,
|
||||
List<HoodieCompactionOperation> operations,
|
||||
List<HoodieCompactionPlan> pendingCompactionPlans) {
|
||||
return operations;
|
||||
}
|
||||
|
||||
/**
|
||||
* Filter the partition paths based on compaction strategy
|
||||
* @param writeConfig
|
||||
* @param allPartitionPaths
|
||||
* @return
|
||||
*/
|
||||
public List<String> filterPartitionPaths(HoodieWriteConfig writeConfig, List<String> allPartitionPaths) {
|
||||
return allPartitionPaths;
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,83 @@
|
||||
/*
|
||||
* 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.io.compact.strategy;
|
||||
|
||||
import com.google.common.annotations.VisibleForTesting;
|
||||
import java.text.ParseException;
|
||||
import java.text.SimpleDateFormat;
|
||||
import java.util.Comparator;
|
||||
import java.util.Date;
|
||||
import java.util.List;
|
||||
import java.util.Locale;
|
||||
import java.util.Map;
|
||||
import java.util.stream.Collectors;
|
||||
import org.apache.hudi.avro.model.HoodieCompactionOperation;
|
||||
import org.apache.hudi.avro.model.HoodieCompactionPlan;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
import org.apache.hudi.exception.HoodieException;
|
||||
|
||||
/**
|
||||
* This strategy orders compactions in reverse order of creation of Hive Partitions. It helps to
|
||||
* compact data in latest partitions first and then older capped at the Total_IO allowed.
|
||||
*/
|
||||
public class DayBasedCompactionStrategy extends CompactionStrategy {
|
||||
|
||||
// For now, use SimpleDateFormat as default partition format
|
||||
protected static String datePartitionFormat = "yyyy/MM/dd";
|
||||
// Sorts compaction in LastInFirstCompacted order
|
||||
protected static Comparator<String> comparator = (String leftPartition,
|
||||
String rightPartition) -> {
|
||||
try {
|
||||
Date left = new SimpleDateFormat(datePartitionFormat, Locale.ENGLISH)
|
||||
.parse(leftPartition);
|
||||
Date right = new SimpleDateFormat(datePartitionFormat, Locale.ENGLISH)
|
||||
.parse(rightPartition);
|
||||
return left.after(right) ? -1 : right.after(left) ? 1 : 0;
|
||||
} catch (ParseException e) {
|
||||
throw new HoodieException("Invalid Partition Date Format", e);
|
||||
}
|
||||
};
|
||||
|
||||
@VisibleForTesting
|
||||
public Comparator<String> getComparator() {
|
||||
return comparator;
|
||||
}
|
||||
|
||||
@Override
|
||||
public List<HoodieCompactionOperation> orderAndFilter(HoodieWriteConfig writeConfig,
|
||||
List<HoodieCompactionOperation> operations, List<HoodieCompactionPlan> pendingCompactionPlans) {
|
||||
// Iterate through the operations and accept operations as long as we are within the configured target partitions
|
||||
// limit
|
||||
List<HoodieCompactionOperation> filteredList = operations.stream()
|
||||
.collect(Collectors.groupingBy(HoodieCompactionOperation::getPartitionPath)).entrySet().stream()
|
||||
.sorted(Map.Entry.comparingByKey(comparator)).limit(writeConfig.getTargetPartitionsPerDayBasedCompaction())
|
||||
.flatMap(e -> e.getValue().stream())
|
||||
.collect(Collectors.toList());
|
||||
return filteredList;
|
||||
}
|
||||
|
||||
@Override
|
||||
public List<String> filterPartitionPaths(HoodieWriteConfig writeConfig, List<String> allPartitionPaths) {
|
||||
List<String> filteredPartitionPaths = allPartitionPaths.stream().map(partition -> partition.replace("/", "-"))
|
||||
.sorted(Comparator.reverseOrder()).map(partitionPath -> partitionPath.replace("-", "/"))
|
||||
.collect(Collectors.toList()).subList(0, writeConfig.getTargetPartitionsPerDayBasedCompaction());
|
||||
return filteredPartitionPaths;
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,74 @@
|
||||
/*
|
||||
* 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.io.compact.strategy;
|
||||
|
||||
import java.util.Comparator;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.stream.Collectors;
|
||||
import org.apache.hudi.avro.model.HoodieCompactionOperation;
|
||||
import org.apache.hudi.avro.model.HoodieCompactionPlan;
|
||||
import org.apache.hudi.common.model.HoodieDataFile;
|
||||
import org.apache.hudi.common.model.HoodieLogFile;
|
||||
import org.apache.hudi.common.util.Option;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
|
||||
/**
|
||||
* LogFileSizeBasedCompactionStrategy orders the compactions based on the total log files size and
|
||||
* limits the compactions within a configured IO bound
|
||||
*
|
||||
* @see BoundedIOCompactionStrategy
|
||||
* @see CompactionStrategy
|
||||
*/
|
||||
public class LogFileSizeBasedCompactionStrategy extends BoundedIOCompactionStrategy implements
|
||||
Comparator<HoodieCompactionOperation> {
|
||||
|
||||
private static final String TOTAL_LOG_FILE_SIZE = "TOTAL_LOG_FILE_SIZE";
|
||||
|
||||
@Override
|
||||
public Map<String, Double> captureMetrics(HoodieWriteConfig config, Option<HoodieDataFile> dataFile,
|
||||
String partitionPath, List<HoodieLogFile> logFiles) {
|
||||
Map<String, Double> metrics = super.captureMetrics(config, dataFile, partitionPath, logFiles);
|
||||
|
||||
// Total size of all the log files
|
||||
Long totalLogFileSize = logFiles.stream().map(HoodieLogFile::getFileSize)
|
||||
.filter(size -> size >= 0).reduce((size1, size2) -> size1 + size2)
|
||||
.orElse(0L);
|
||||
// save the metrics needed during the order
|
||||
metrics.put(TOTAL_LOG_FILE_SIZE, totalLogFileSize.doubleValue());
|
||||
return metrics;
|
||||
}
|
||||
|
||||
@Override
|
||||
public List<HoodieCompactionOperation> orderAndFilter(HoodieWriteConfig writeConfig,
|
||||
List<HoodieCompactionOperation> operations, List<HoodieCompactionPlan> pendingCompactionPlans) {
|
||||
// Order the operations based on the reverse size of the logs and limit them by the IO
|
||||
return super
|
||||
.orderAndFilter(writeConfig,
|
||||
operations.stream().sorted(this).collect(Collectors.toList()), pendingCompactionPlans);
|
||||
}
|
||||
|
||||
@Override
|
||||
public int compare(HoodieCompactionOperation op1, HoodieCompactionOperation op2) {
|
||||
Long totalLogSize1 = op1.getMetrics().get(TOTAL_LOG_FILE_SIZE).longValue();
|
||||
Long totalLogSize2 = op2.getMetrics().get(TOTAL_LOG_FILE_SIZE).longValue();
|
||||
// Reverse the comparison order - so that larger log file size is compacted first
|
||||
return totalLogSize2.compareTo(totalLogSize1);
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,40 @@
|
||||
/*
|
||||
* 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.io.compact.strategy;
|
||||
|
||||
import java.util.List;
|
||||
import org.apache.hudi.avro.model.HoodieCompactionOperation;
|
||||
import org.apache.hudi.avro.model.HoodieCompactionPlan;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
|
||||
/**
|
||||
* UnBoundedCompactionStrategy will not change ordering or filter any compaction. It is a
|
||||
* pass-through and will compact all the base files which has a log file. This usually means
|
||||
* no-intelligence on compaction.
|
||||
*
|
||||
* @see CompactionStrategy
|
||||
*/
|
||||
public class UnBoundedCompactionStrategy extends CompactionStrategy {
|
||||
|
||||
@Override
|
||||
public List<HoodieCompactionOperation> orderAndFilter(HoodieWriteConfig config,
|
||||
List<HoodieCompactionOperation> operations, List<HoodieCompactionPlan> pendingCompactionWorkloads) {
|
||||
return operations;
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,65 @@
|
||||
/*
|
||||
* 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.io.compact.strategy;
|
||||
|
||||
import java.util.ArrayList;
|
||||
import java.util.Comparator;
|
||||
import java.util.List;
|
||||
import java.util.stream.Collectors;
|
||||
import org.apache.hudi.avro.model.HoodieCompactionOperation;
|
||||
import org.apache.hudi.avro.model.HoodieCompactionPlan;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
|
||||
/**
|
||||
* UnBoundedPartitionAwareCompactionStrategy is a custom UnBounded Strategy.
|
||||
* This will filter all the partitions that are eligible to be compacted by a
|
||||
* {@link BoundedPartitionAwareCompactionStrategy} and return the result.
|
||||
* This is done so that a long running UnBoundedPartitionAwareCompactionStrategy does not step over partitions
|
||||
* in a shorter running BoundedPartitionAwareCompactionStrategy. Essentially, this is an inverse of the
|
||||
* partitions chosen in BoundedPartitionAwareCompactionStrategy
|
||||
*
|
||||
* @see CompactionStrategy
|
||||
*/
|
||||
public class UnBoundedPartitionAwareCompactionStrategy extends CompactionStrategy {
|
||||
|
||||
@Override
|
||||
public List<HoodieCompactionOperation> orderAndFilter(HoodieWriteConfig config,
|
||||
final List<HoodieCompactionOperation> operations, final List<HoodieCompactionPlan> pendingCompactionWorkloads) {
|
||||
BoundedPartitionAwareCompactionStrategy boundedPartitionAwareCompactionStrategy
|
||||
= new BoundedPartitionAwareCompactionStrategy();
|
||||
List<HoodieCompactionOperation> operationsToExclude = boundedPartitionAwareCompactionStrategy
|
||||
.orderAndFilter(config, operations, pendingCompactionWorkloads);
|
||||
List<HoodieCompactionOperation> allOperations = new ArrayList<>(operations);
|
||||
allOperations.removeAll(operationsToExclude);
|
||||
return allOperations;
|
||||
}
|
||||
|
||||
@Override
|
||||
public List<String> filterPartitionPaths(HoodieWriteConfig writeConfig, List<String> partitionPaths) {
|
||||
List<String> allPartitionPaths = partitionPaths.stream().map(partition -> partition.replace("/", "-"))
|
||||
.sorted(Comparator.reverseOrder()).map(partitionPath -> partitionPath.replace("-", "/"))
|
||||
.collect(Collectors.toList());
|
||||
BoundedPartitionAwareCompactionStrategy boundedPartitionAwareCompactionStrategy
|
||||
= new BoundedPartitionAwareCompactionStrategy();
|
||||
List<String> partitionsToExclude = boundedPartitionAwareCompactionStrategy.filterPartitionPaths(writeConfig,
|
||||
partitionPaths);
|
||||
allPartitionPaths.removeAll(partitionsToExclude);
|
||||
return allPartitionPaths;
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,74 @@
|
||||
/*
|
||||
* 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.io.storage;
|
||||
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hudi.avro.HoodieAvroWriteSupport;
|
||||
import org.apache.parquet.hadoop.metadata.CompressionCodecName;
|
||||
|
||||
public class HoodieParquetConfig {
|
||||
|
||||
private HoodieAvroWriteSupport writeSupport;
|
||||
private CompressionCodecName compressionCodecName;
|
||||
private int blockSize;
|
||||
private int pageSize;
|
||||
private long maxFileSize;
|
||||
private Configuration hadoopConf;
|
||||
private double compressionRatio;
|
||||
|
||||
public HoodieParquetConfig(HoodieAvroWriteSupport writeSupport,
|
||||
CompressionCodecName compressionCodecName, int blockSize, int pageSize, long maxFileSize,
|
||||
Configuration hadoopConf, double compressionRatio) {
|
||||
this.writeSupport = writeSupport;
|
||||
this.compressionCodecName = compressionCodecName;
|
||||
this.blockSize = blockSize;
|
||||
this.pageSize = pageSize;
|
||||
this.maxFileSize = maxFileSize;
|
||||
this.hadoopConf = hadoopConf;
|
||||
this.compressionRatio = compressionRatio;
|
||||
}
|
||||
|
||||
public HoodieAvroWriteSupport getWriteSupport() {
|
||||
return writeSupport;
|
||||
}
|
||||
|
||||
public CompressionCodecName getCompressionCodecName() {
|
||||
return compressionCodecName;
|
||||
}
|
||||
|
||||
public int getBlockSize() {
|
||||
return blockSize;
|
||||
}
|
||||
|
||||
public int getPageSize() {
|
||||
return pageSize;
|
||||
}
|
||||
|
||||
public long getMaxFileSize() {
|
||||
return maxFileSize;
|
||||
}
|
||||
|
||||
public Configuration getHadoopConf() {
|
||||
return hadoopConf;
|
||||
}
|
||||
|
||||
public double getCompressionRatio() {
|
||||
return compressionRatio;
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,106 @@
|
||||
/*
|
||||
* 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.io.storage;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.concurrent.atomic.AtomicLong;
|
||||
import org.apache.avro.Schema;
|
||||
import org.apache.avro.generic.GenericRecord;
|
||||
import org.apache.avro.generic.IndexedRecord;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hudi.avro.HoodieAvroWriteSupport;
|
||||
import org.apache.hudi.common.io.storage.HoodieWrapperFileSystem;
|
||||
import org.apache.hudi.common.model.HoodieRecord;
|
||||
import org.apache.hudi.common.model.HoodieRecordPayload;
|
||||
import org.apache.hudi.common.util.FSUtils;
|
||||
import org.apache.hudi.common.util.HoodieAvroUtils;
|
||||
import org.apache.parquet.hadoop.ParquetFileWriter;
|
||||
import org.apache.parquet.hadoop.ParquetWriter;
|
||||
import org.apache.spark.TaskContext;
|
||||
|
||||
/**
|
||||
* HoodieParquetWriter extends the ParquetWriter to help limit the size of underlying file. Provides
|
||||
* a way to check if the current file can take more records with the <code>canWrite()</code>
|
||||
*/
|
||||
public class HoodieParquetWriter<T extends HoodieRecordPayload, R extends IndexedRecord> extends
|
||||
ParquetWriter<IndexedRecord> implements HoodieStorageWriter<R> {
|
||||
|
||||
private static AtomicLong recordIndex = new AtomicLong(1);
|
||||
|
||||
private final Path file;
|
||||
private final HoodieWrapperFileSystem fs;
|
||||
private final long maxFileSize;
|
||||
private final HoodieAvroWriteSupport writeSupport;
|
||||
private final String commitTime;
|
||||
private final Schema schema;
|
||||
|
||||
|
||||
public HoodieParquetWriter(String commitTime, Path file, HoodieParquetConfig parquetConfig,
|
||||
Schema schema) throws IOException {
|
||||
super(HoodieWrapperFileSystem.convertToHoodiePath(file, parquetConfig.getHadoopConf()),
|
||||
ParquetFileWriter.Mode.CREATE, parquetConfig.getWriteSupport(),
|
||||
parquetConfig.getCompressionCodecName(), parquetConfig.getBlockSize(),
|
||||
parquetConfig.getPageSize(), parquetConfig.getPageSize(),
|
||||
ParquetWriter.DEFAULT_IS_DICTIONARY_ENABLED, ParquetWriter.DEFAULT_IS_VALIDATING_ENABLED,
|
||||
ParquetWriter.DEFAULT_WRITER_VERSION,
|
||||
registerFileSystem(file, parquetConfig.getHadoopConf()));
|
||||
this.file = HoodieWrapperFileSystem.convertToHoodiePath(file, parquetConfig.getHadoopConf());
|
||||
this.fs = (HoodieWrapperFileSystem) this.file
|
||||
.getFileSystem(registerFileSystem(file, parquetConfig.getHadoopConf()));
|
||||
// We cannot accurately measure the snappy compressed output file size. We are choosing a
|
||||
// conservative 10%
|
||||
// TODO - compute this compression ratio dynamically by looking at the bytes written to the
|
||||
// stream and the actual file size reported by HDFS
|
||||
this.maxFileSize = parquetConfig.getMaxFileSize() + Math
|
||||
.round(parquetConfig.getMaxFileSize() * parquetConfig.getCompressionRatio());
|
||||
this.writeSupport = parquetConfig.getWriteSupport();
|
||||
this.commitTime = commitTime;
|
||||
this.schema = schema;
|
||||
}
|
||||
|
||||
public static Configuration registerFileSystem(Path file, Configuration conf) {
|
||||
Configuration returnConf = new Configuration(conf);
|
||||
String scheme = FSUtils.getFs(file.toString(), conf).getScheme();
|
||||
returnConf.set("fs." + HoodieWrapperFileSystem.getHoodieScheme(scheme) + ".impl",
|
||||
HoodieWrapperFileSystem.class.getName());
|
||||
return returnConf;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void writeAvroWithMetadata(R avroRecord, HoodieRecord record) throws IOException {
|
||||
String seqId = HoodieRecord.generateSequenceId(commitTime, TaskContext.getPartitionId(),
|
||||
recordIndex.getAndIncrement());
|
||||
HoodieAvroUtils.addHoodieKeyToRecord((GenericRecord) avroRecord, record.getRecordKey(),
|
||||
record.getPartitionPath(), file.getName());
|
||||
HoodieAvroUtils.addCommitMetadataToRecord((GenericRecord) avroRecord, commitTime, seqId);
|
||||
super.write(avroRecord);
|
||||
writeSupport.add(record.getRecordKey());
|
||||
}
|
||||
|
||||
public boolean canWrite() {
|
||||
return fs.getBytesWritten(file) < maxFileSize;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void writeAvro(String key, IndexedRecord object) throws IOException {
|
||||
super.write(object);
|
||||
writeSupport.add(key);
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,34 @@
|
||||
/*
|
||||
* 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.io.storage;
|
||||
|
||||
import java.io.IOException;
|
||||
import org.apache.avro.generic.IndexedRecord;
|
||||
import org.apache.hudi.common.model.HoodieRecord;
|
||||
|
||||
public interface HoodieStorageWriter<R extends IndexedRecord> {
|
||||
|
||||
void writeAvroWithMetadata(R newRecord, HoodieRecord record) throws IOException;
|
||||
|
||||
boolean canWrite();
|
||||
|
||||
void close() throws IOException;
|
||||
|
||||
void writeAvro(String key, R oldRecord) throws IOException;
|
||||
}
|
||||
@@ -0,0 +1,58 @@
|
||||
/*
|
||||
* 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.io.storage;
|
||||
|
||||
import java.io.IOException;
|
||||
import org.apache.avro.Schema;
|
||||
import org.apache.avro.generic.IndexedRecord;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hudi.avro.HoodieAvroWriteSupport;
|
||||
import org.apache.hudi.common.BloomFilter;
|
||||
import org.apache.hudi.common.model.HoodieRecordPayload;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
import org.apache.hudi.table.HoodieTable;
|
||||
import org.apache.parquet.avro.AvroSchemaConverter;
|
||||
|
||||
public class HoodieStorageWriterFactory {
|
||||
|
||||
public static <T extends HoodieRecordPayload, R extends IndexedRecord> HoodieStorageWriter<R> getStorageWriter(
|
||||
String commitTime, Path path, HoodieTable<T> hoodieTable,
|
||||
HoodieWriteConfig config, Schema schema) throws IOException {
|
||||
//TODO - based on the metadata choose the implementation of HoodieStorageWriter
|
||||
// Currently only parquet is supported
|
||||
return newParquetStorageWriter(commitTime, path, config, schema, hoodieTable);
|
||||
}
|
||||
|
||||
private static <T extends HoodieRecordPayload,
|
||||
R extends IndexedRecord> HoodieStorageWriter<R> newParquetStorageWriter(String commitTime, Path path,
|
||||
HoodieWriteConfig config, Schema schema, HoodieTable hoodieTable) throws IOException {
|
||||
BloomFilter filter = new BloomFilter(config.getBloomFilterNumEntries(),
|
||||
config.getBloomFilterFPP());
|
||||
HoodieAvroWriteSupport writeSupport = new HoodieAvroWriteSupport(
|
||||
new AvroSchemaConverter().convert(schema), schema, filter);
|
||||
|
||||
HoodieParquetConfig parquetConfig =
|
||||
new HoodieParquetConfig(writeSupport, config.getParquetCompressionCodec(),
|
||||
config.getParquetBlockSize(), config.getParquetPageSize(),
|
||||
config.getParquetMaxFileSize(), hoodieTable.getHadoopConf(),
|
||||
config.getParquetCompressionRatio());
|
||||
|
||||
return new HoodieParquetWriter<>(commitTime, path, parquetConfig, schema);
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,186 @@
|
||||
/*
|
||||
* 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.metrics;
|
||||
|
||||
import com.codahale.metrics.Timer;
|
||||
import com.google.common.annotations.VisibleForTesting;
|
||||
import org.apache.hudi.common.model.HoodieCommitMetadata;
|
||||
import org.apache.hudi.common.table.HoodieTimeline;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
import org.apache.log4j.LogManager;
|
||||
import org.apache.log4j.Logger;
|
||||
|
||||
/**
|
||||
* Wrapper for metrics-related operations.
|
||||
*/
|
||||
public class HoodieMetrics {
|
||||
|
||||
private static Logger logger = LogManager.getLogger(HoodieMetrics.class);
|
||||
// Some timers
|
||||
public String rollbackTimerName = null;
|
||||
public String cleanTimerName = null;
|
||||
public String commitTimerName = null;
|
||||
public String deltaCommitTimerName = null;
|
||||
public String finalizeTimerName = null;
|
||||
public String compactionTimerName = null;
|
||||
private HoodieWriteConfig config = null;
|
||||
private String tableName = null;
|
||||
private Timer rollbackTimer = null;
|
||||
private Timer cleanTimer = null;
|
||||
private Timer commitTimer = null;
|
||||
private Timer deltaCommitTimer = null;
|
||||
private Timer finalizeTimer = null;
|
||||
private Timer compactionTimer = null;
|
||||
|
||||
public HoodieMetrics(HoodieWriteConfig config, String tableName) {
|
||||
this.config = config;
|
||||
this.tableName = tableName;
|
||||
if (config.isMetricsOn()) {
|
||||
Metrics.init(config);
|
||||
this.rollbackTimerName = getMetricsName("timer", HoodieTimeline.ROLLBACK_ACTION);
|
||||
this.cleanTimerName = getMetricsName("timer", HoodieTimeline.CLEAN_ACTION);
|
||||
this.commitTimerName = getMetricsName("timer", HoodieTimeline.COMMIT_ACTION);
|
||||
this.deltaCommitTimerName = getMetricsName("timer", HoodieTimeline.DELTA_COMMIT_ACTION);
|
||||
this.finalizeTimerName = getMetricsName("timer", "finalize");
|
||||
this.compactionTimerName = getMetricsName("timer", HoodieTimeline.COMPACTION_ACTION);
|
||||
}
|
||||
}
|
||||
|
||||
private Timer createTimer(String name) {
|
||||
return config.isMetricsOn() ? Metrics.getInstance().getRegistry().timer(name) : null;
|
||||
}
|
||||
|
||||
public Timer.Context getRollbackCtx() {
|
||||
if (config.isMetricsOn() && rollbackTimer == null) {
|
||||
rollbackTimer = createTimer(rollbackTimerName);
|
||||
}
|
||||
return rollbackTimer == null ? null : rollbackTimer.time();
|
||||
}
|
||||
|
||||
public Timer.Context getCompactionCtx() {
|
||||
if (config.isMetricsOn() && compactionTimer == null) {
|
||||
compactionTimer = createTimer(commitTimerName);
|
||||
}
|
||||
return compactionTimer == null ? null : compactionTimer.time();
|
||||
}
|
||||
|
||||
public Timer.Context getCleanCtx() {
|
||||
if (config.isMetricsOn() && cleanTimer == null) {
|
||||
cleanTimer = createTimer(cleanTimerName);
|
||||
}
|
||||
return cleanTimer == null ? null : cleanTimer.time();
|
||||
}
|
||||
|
||||
public Timer.Context getCommitCtx() {
|
||||
if (config.isMetricsOn() && commitTimer == null) {
|
||||
commitTimer = createTimer(commitTimerName);
|
||||
}
|
||||
return commitTimer == null ? null : commitTimer.time();
|
||||
}
|
||||
|
||||
public Timer.Context getFinalizeCtx() {
|
||||
if (config.isMetricsOn() && finalizeTimer == null) {
|
||||
finalizeTimer = createTimer(finalizeTimerName);
|
||||
}
|
||||
return finalizeTimer == null ? null : finalizeTimer.time();
|
||||
}
|
||||
|
||||
public Timer.Context getDeltaCommitCtx() {
|
||||
if (config.isMetricsOn() && deltaCommitTimer == null) {
|
||||
deltaCommitTimer = createTimer(deltaCommitTimerName);
|
||||
}
|
||||
return deltaCommitTimer == null ? null : deltaCommitTimer.time();
|
||||
}
|
||||
|
||||
public void updateCommitMetrics(long commitEpochTimeInMs, long durationInMs,
|
||||
HoodieCommitMetadata metadata, String actionType) {
|
||||
if (config.isMetricsOn()) {
|
||||
long totalPartitionsWritten = metadata.fetchTotalPartitionsWritten();
|
||||
long totalFilesInsert = metadata.fetchTotalFilesInsert();
|
||||
long totalFilesUpdate = metadata.fetchTotalFilesUpdated();
|
||||
long totalRecordsWritten = metadata.fetchTotalRecordsWritten();
|
||||
long totalUpdateRecordsWritten = metadata.fetchTotalUpdateRecordsWritten();
|
||||
long totalInsertRecordsWritten = metadata.fetchTotalInsertRecordsWritten();
|
||||
long totalBytesWritten = metadata.fetchTotalBytesWritten();
|
||||
long totalTimeTakenByScanner = metadata.getTotalScanTime();
|
||||
long totalTimeTakenForInsert = metadata.getTotalCreateTime();
|
||||
long totalTimeTakenForUpsert = metadata.getTotalUpsertTime();
|
||||
long totalCompactedRecordsUpdated = metadata.getTotalCompactedRecordsUpdated();
|
||||
long totalLogFilesCompacted = metadata.getTotalLogFilesCompacted();
|
||||
long totalLogFilesSize = metadata.getTotalLogFilesSize();
|
||||
Metrics.registerGauge(getMetricsName(actionType, "duration"), durationInMs);
|
||||
Metrics.registerGauge(getMetricsName(actionType, "totalPartitionsWritten"), totalPartitionsWritten);
|
||||
Metrics.registerGauge(getMetricsName(actionType, "totalFilesInsert"), totalFilesInsert);
|
||||
Metrics.registerGauge(getMetricsName(actionType, "totalFilesUpdate"), totalFilesUpdate);
|
||||
Metrics.registerGauge(getMetricsName(actionType, "totalRecordsWritten"), totalRecordsWritten);
|
||||
Metrics.registerGauge(getMetricsName(actionType, "totalUpdateRecordsWritten"), totalUpdateRecordsWritten);
|
||||
Metrics.registerGauge(getMetricsName(actionType, "totalInsertRecordsWritten"), totalInsertRecordsWritten);
|
||||
Metrics.registerGauge(getMetricsName(actionType, "totalBytesWritten"), totalBytesWritten);
|
||||
Metrics.registerGauge(getMetricsName(actionType, "commitTime"), commitEpochTimeInMs);
|
||||
Metrics.registerGauge(getMetricsName(actionType, "totalScanTime"), totalTimeTakenByScanner);
|
||||
Metrics.registerGauge(getMetricsName(actionType, "totalCreateTime"), totalTimeTakenForInsert);
|
||||
Metrics.registerGauge(getMetricsName(actionType, "totalUpsertTime"), totalTimeTakenForUpsert);
|
||||
Metrics.registerGauge(getMetricsName(actionType, "totalCompactedRecordsUpdated"), totalCompactedRecordsUpdated);
|
||||
Metrics.registerGauge(getMetricsName(actionType, "totalLogFilesCompacted"), totalLogFilesCompacted);
|
||||
Metrics.registerGauge(getMetricsName(actionType, "totalLogFilesSize"), totalLogFilesSize);
|
||||
}
|
||||
}
|
||||
|
||||
public void updateRollbackMetrics(long durationInMs, long numFilesDeleted) {
|
||||
if (config.isMetricsOn()) {
|
||||
logger.info(String
|
||||
.format("Sending rollback metrics (duration=%d, numFilesDeleted=%d)", durationInMs,
|
||||
numFilesDeleted));
|
||||
Metrics.registerGauge(getMetricsName("rollback", "duration"), durationInMs);
|
||||
Metrics.registerGauge(getMetricsName("rollback", "numFilesDeleted"), numFilesDeleted);
|
||||
}
|
||||
}
|
||||
|
||||
public void updateCleanMetrics(long durationInMs, int numFilesDeleted) {
|
||||
if (config.isMetricsOn()) {
|
||||
logger.info(String
|
||||
.format("Sending clean metrics (duration=%d, numFilesDeleted=%d)", durationInMs,
|
||||
numFilesDeleted));
|
||||
Metrics.registerGauge(getMetricsName("clean", "duration"), durationInMs);
|
||||
Metrics.registerGauge(getMetricsName("clean", "numFilesDeleted"), numFilesDeleted);
|
||||
}
|
||||
}
|
||||
|
||||
public void updateFinalizeWriteMetrics(long durationInMs, long numFilesFinalized) {
|
||||
if (config.isMetricsOn()) {
|
||||
logger.info(String
|
||||
.format("Sending finalize write metrics (duration=%d, numFilesFinalized=%d)",
|
||||
durationInMs, numFilesFinalized));
|
||||
Metrics.registerGauge(getMetricsName("finalize", "duration"), durationInMs);
|
||||
Metrics.registerGauge(getMetricsName("finalize", "numFilesFinalized"), numFilesFinalized);
|
||||
}
|
||||
}
|
||||
|
||||
@VisibleForTesting
|
||||
String getMetricsName(String action, String metric) {
|
||||
return config == null ? null : String.format("%s.%s.%s", tableName, action, metric);
|
||||
}
|
||||
|
||||
/**
|
||||
* By default, the timer context returns duration with nano seconds. Convert it to millisecond.
|
||||
*/
|
||||
public long getDurationInMs(long ctxDuration) {
|
||||
return ctxDuration / 1000000;
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,40 @@
|
||||
/*
|
||||
* 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.metrics;
|
||||
|
||||
import java.io.Closeable;
|
||||
|
||||
/**
|
||||
* Used for testing.
|
||||
*/
|
||||
public class InMemoryMetricsReporter extends MetricsReporter {
|
||||
|
||||
@Override
|
||||
public void start() {
|
||||
}
|
||||
|
||||
@Override
|
||||
public void report() {
|
||||
}
|
||||
|
||||
@Override
|
||||
public Closeable getReporter() {
|
||||
return null;
|
||||
}
|
||||
}
|
||||
100
hudi-client/src/main/java/org/apache/hudi/metrics/Metrics.java
Normal file
100
hudi-client/src/main/java/org/apache/hudi/metrics/Metrics.java
Normal file
@@ -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.metrics;
|
||||
|
||||
import com.codahale.metrics.Gauge;
|
||||
import com.codahale.metrics.MetricRegistry;
|
||||
import com.google.common.io.Closeables;
|
||||
import java.io.Closeable;
|
||||
import org.apache.commons.configuration.ConfigurationException;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
import org.apache.hudi.exception.HoodieException;
|
||||
import org.apache.log4j.LogManager;
|
||||
import org.apache.log4j.Logger;
|
||||
|
||||
/**
|
||||
* This is the main class of the metrics system.
|
||||
*/
|
||||
public class Metrics {
|
||||
private static Logger logger = LogManager.getLogger(Metrics.class);
|
||||
|
||||
private static volatile boolean initialized = false;
|
||||
private static Metrics metrics = null;
|
||||
private final MetricRegistry registry;
|
||||
private MetricsReporter reporter = null;
|
||||
|
||||
private Metrics(HoodieWriteConfig metricConfig) throws ConfigurationException {
|
||||
registry = new MetricRegistry();
|
||||
|
||||
reporter = MetricsReporterFactory.createReporter(metricConfig, registry);
|
||||
if (reporter == null) {
|
||||
throw new RuntimeException("Cannot initialize Reporter.");
|
||||
}
|
||||
// reporter.start();
|
||||
|
||||
Runtime.getRuntime().addShutdownHook(new Thread() {
|
||||
@Override
|
||||
public void run() {
|
||||
try {
|
||||
reporter.report();
|
||||
Closeables.close(reporter.getReporter(), true);
|
||||
} catch (Exception e) {
|
||||
e.printStackTrace();
|
||||
}
|
||||
}
|
||||
});
|
||||
}
|
||||
|
||||
public static Metrics getInstance() {
|
||||
assert initialized;
|
||||
return metrics;
|
||||
}
|
||||
|
||||
public static synchronized void init(HoodieWriteConfig metricConfig) {
|
||||
if (initialized) {
|
||||
return;
|
||||
}
|
||||
try {
|
||||
metrics = new Metrics(metricConfig);
|
||||
} catch (ConfigurationException e) {
|
||||
throw new HoodieException(e);
|
||||
}
|
||||
initialized = true;
|
||||
}
|
||||
|
||||
public static void registerGauge(String metricName, final long value) {
|
||||
try {
|
||||
MetricRegistry registry = Metrics.getInstance().getRegistry();
|
||||
registry.register(metricName, (Gauge<Long>) () -> value);
|
||||
} catch (Exception e) {
|
||||
// Here we catch all exception, so the major upsert pipeline will not be affected if the
|
||||
// metrics system
|
||||
// has some issues.
|
||||
logger.error("Failed to send metrics: ", e);
|
||||
}
|
||||
}
|
||||
|
||||
public MetricRegistry getRegistry() {
|
||||
return registry;
|
||||
}
|
||||
|
||||
public Closeable getReporter() {
|
||||
return reporter.getReporter();
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,91 @@
|
||||
/*
|
||||
* 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.metrics;
|
||||
|
||||
import com.codahale.metrics.MetricFilter;
|
||||
import com.codahale.metrics.MetricRegistry;
|
||||
import com.codahale.metrics.graphite.Graphite;
|
||||
import com.codahale.metrics.graphite.GraphiteReporter;
|
||||
import java.io.Closeable;
|
||||
import java.net.InetSocketAddress;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
import org.apache.log4j.LogManager;
|
||||
import org.apache.log4j.Logger;
|
||||
|
||||
/**
|
||||
* Implementation of Graphite reporter, which connects to the Graphite server, and send metrics to
|
||||
* that server.
|
||||
*/
|
||||
public class MetricsGraphiteReporter extends MetricsReporter {
|
||||
|
||||
private static Logger logger = LogManager.getLogger(MetricsGraphiteReporter.class);
|
||||
private final MetricRegistry registry;
|
||||
private final GraphiteReporter graphiteReporter;
|
||||
private final HoodieWriteConfig config;
|
||||
private String serverHost;
|
||||
private int serverPort;
|
||||
|
||||
public MetricsGraphiteReporter(HoodieWriteConfig config, MetricRegistry registry) {
|
||||
this.registry = registry;
|
||||
this.config = config;
|
||||
|
||||
// Check the serverHost and serverPort here
|
||||
this.serverHost = config.getGraphiteServerHost();
|
||||
this.serverPort = config.getGraphiteServerPort();
|
||||
if (serverHost == null || serverPort == 0) {
|
||||
throw new RuntimeException(String
|
||||
.format("Graphite cannot be initialized with serverHost[%s] and serverPort[%s].",
|
||||
serverHost, serverPort));
|
||||
}
|
||||
|
||||
this.graphiteReporter = createGraphiteReport();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void start() {
|
||||
if (graphiteReporter != null) {
|
||||
graphiteReporter.start(30, TimeUnit.SECONDS);
|
||||
} else {
|
||||
logger.error("Cannot start as the graphiteReporter is null.");
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void report() {
|
||||
if (graphiteReporter != null) {
|
||||
graphiteReporter.report();
|
||||
} else {
|
||||
logger.error("Cannot report metrics as the graphiteReporter is null.");
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public Closeable getReporter() {
|
||||
return graphiteReporter;
|
||||
}
|
||||
|
||||
private GraphiteReporter createGraphiteReport() {
|
||||
Graphite graphite = new Graphite(new InetSocketAddress(serverHost, serverPort));
|
||||
String reporterPrefix = config.getGraphiteMetricPrefix();
|
||||
return GraphiteReporter.forRegistry(registry).prefixedWith(reporterPrefix)
|
||||
.convertRatesTo(TimeUnit.SECONDS).convertDurationsTo(TimeUnit.MILLISECONDS)
|
||||
.filter(MetricFilter.ALL).build(graphite);
|
||||
}
|
||||
}
|
||||
@@ -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.metrics;
|
||||
|
||||
import java.io.Closeable;
|
||||
|
||||
/**
|
||||
* Interface for implementing a Reporter.
|
||||
*/
|
||||
public abstract class MetricsReporter {
|
||||
|
||||
/**
|
||||
* Push out metrics at scheduled intervals
|
||||
*/
|
||||
public abstract void start();
|
||||
|
||||
/**
|
||||
* Deterministically push out metrics
|
||||
*/
|
||||
public abstract void report();
|
||||
|
||||
public abstract Closeable getReporter();
|
||||
}
|
||||
@@ -0,0 +1,49 @@
|
||||
/*
|
||||
* 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.metrics;
|
||||
|
||||
import com.codahale.metrics.MetricRegistry;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
import org.apache.log4j.LogManager;
|
||||
import org.apache.log4j.Logger;
|
||||
|
||||
/**
|
||||
* Factory class for creating MetricsReporter.
|
||||
*/
|
||||
public class MetricsReporterFactory {
|
||||
|
||||
private static Logger logger = LogManager.getLogger(MetricsReporterFactory.class);
|
||||
|
||||
public static MetricsReporter createReporter(HoodieWriteConfig config, MetricRegistry registry) {
|
||||
MetricsReporterType type = config.getMetricsReporterType();
|
||||
MetricsReporter reporter = null;
|
||||
switch (type) {
|
||||
case GRAPHITE:
|
||||
reporter = new MetricsGraphiteReporter(config, registry);
|
||||
break;
|
||||
case INMEMORY:
|
||||
reporter = new InMemoryMetricsReporter();
|
||||
break;
|
||||
default:
|
||||
logger.error("Reporter type[" + type + "] is not supported.");
|
||||
break;
|
||||
}
|
||||
return reporter;
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,27 @@
|
||||
/*
|
||||
* 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.metrics;
|
||||
|
||||
/**
|
||||
* Types of the reporter. Right now we only support Graphite. We can include JMX and CSV in the
|
||||
* future.
|
||||
*/
|
||||
public enum MetricsReporterType {
|
||||
GRAPHITE, INMEMORY
|
||||
}
|
||||
@@ -0,0 +1,806 @@
|
||||
/*
|
||||
* 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.table;
|
||||
|
||||
import com.google.common.hash.Hashing;
|
||||
import java.io.IOException;
|
||||
import java.io.Serializable;
|
||||
import java.nio.charset.StandardCharsets;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Collections;
|
||||
import java.util.HashMap;
|
||||
import java.util.Iterator;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
import java.util.stream.Collectors;
|
||||
import org.apache.avro.generic.GenericRecord;
|
||||
import org.apache.avro.generic.IndexedRecord;
|
||||
import org.apache.hadoop.fs.FileStatus;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.fs.PathFilter;
|
||||
import org.apache.hudi.WriteStatus;
|
||||
import org.apache.hudi.avro.model.HoodieCompactionPlan;
|
||||
import org.apache.hudi.common.HoodieCleanStat;
|
||||
import org.apache.hudi.common.HoodieRollbackStat;
|
||||
import org.apache.hudi.common.model.HoodieCommitMetadata;
|
||||
import org.apache.hudi.common.model.HoodieDataFile;
|
||||
import org.apache.hudi.common.model.HoodieKey;
|
||||
import org.apache.hudi.common.model.HoodieRecord;
|
||||
import org.apache.hudi.common.model.HoodieRecordLocation;
|
||||
import org.apache.hudi.common.model.HoodieRecordPayload;
|
||||
import org.apache.hudi.common.model.HoodieRollingStatMetadata;
|
||||
import org.apache.hudi.common.table.HoodieTimeline;
|
||||
import org.apache.hudi.common.table.timeline.HoodieActiveTimeline;
|
||||
import org.apache.hudi.common.table.timeline.HoodieInstant;
|
||||
import org.apache.hudi.common.util.FSUtils;
|
||||
import org.apache.hudi.common.util.Option;
|
||||
import org.apache.hudi.common.util.collection.Pair;
|
||||
import org.apache.hudi.common.util.queue.BoundedInMemoryExecutor;
|
||||
import org.apache.hudi.common.util.queue.BoundedInMemoryQueueConsumer;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
import org.apache.hudi.exception.HoodieException;
|
||||
import org.apache.hudi.exception.HoodieIOException;
|
||||
import org.apache.hudi.exception.HoodieNotSupportedException;
|
||||
import org.apache.hudi.exception.HoodieUpsertException;
|
||||
import org.apache.hudi.func.CopyOnWriteLazyInsertIterable;
|
||||
import org.apache.hudi.func.ParquetReaderIterator;
|
||||
import org.apache.hudi.func.SparkBoundedInMemoryExecutor;
|
||||
import org.apache.hudi.io.HoodieCleanHelper;
|
||||
import org.apache.hudi.io.HoodieCreateHandle;
|
||||
import org.apache.hudi.io.HoodieMergeHandle;
|
||||
import org.apache.log4j.LogManager;
|
||||
import org.apache.log4j.Logger;
|
||||
import org.apache.parquet.avro.AvroParquetReader;
|
||||
import org.apache.parquet.avro.AvroReadSupport;
|
||||
import org.apache.parquet.hadoop.ParquetReader;
|
||||
import org.apache.spark.Partitioner;
|
||||
import org.apache.spark.api.java.JavaRDD;
|
||||
import org.apache.spark.api.java.JavaSparkContext;
|
||||
import org.apache.spark.api.java.function.Function;
|
||||
import org.apache.spark.api.java.function.Function2;
|
||||
import org.apache.spark.api.java.function.PairFlatMapFunction;
|
||||
import scala.Tuple2;
|
||||
|
||||
/**
|
||||
* Implementation of a very heavily read-optimized Hoodie Table where
|
||||
* <p>
|
||||
* INSERTS - Produce new files, block aligned to desired size (or) Merge with the smallest existing
|
||||
* file, to expand it
|
||||
* <p>
|
||||
* UPDATES - Produce a new version of the file, just replacing the updated records with new values
|
||||
*/
|
||||
public class HoodieCopyOnWriteTable<T extends HoodieRecordPayload> extends HoodieTable<T> {
|
||||
|
||||
private static Logger logger = LogManager.getLogger(HoodieCopyOnWriteTable.class);
|
||||
|
||||
public HoodieCopyOnWriteTable(HoodieWriteConfig config, JavaSparkContext jsc) {
|
||||
super(config, jsc);
|
||||
}
|
||||
|
||||
private static PairFlatMapFunction<Iterator<Tuple2<String, String>>, String,
|
||||
PartitionCleanStat> deleteFilesFunc(
|
||||
HoodieTable table) {
|
||||
return (PairFlatMapFunction<Iterator<Tuple2<String, String>>, String, PartitionCleanStat>)
|
||||
iter -> {
|
||||
Map<String, PartitionCleanStat> partitionCleanStatMap = new HashMap<>();
|
||||
|
||||
FileSystem fs = table.getMetaClient().getFs();
|
||||
while (iter.hasNext()) {
|
||||
Tuple2<String, String> partitionDelFileTuple = iter.next();
|
||||
String partitionPath = partitionDelFileTuple._1();
|
||||
String deletePathStr = partitionDelFileTuple._2();
|
||||
Boolean deletedFileResult = deleteFileAndGetResult(fs, deletePathStr);
|
||||
if (!partitionCleanStatMap.containsKey(partitionPath)) {
|
||||
partitionCleanStatMap.put(partitionPath, new PartitionCleanStat(partitionPath));
|
||||
}
|
||||
PartitionCleanStat partitionCleanStat = partitionCleanStatMap.get(partitionPath);
|
||||
partitionCleanStat.addDeleteFilePatterns(deletePathStr);
|
||||
partitionCleanStat.addDeletedFileResult(deletePathStr, deletedFileResult);
|
||||
}
|
||||
|
||||
return partitionCleanStatMap.entrySet().stream()
|
||||
.map(e -> new Tuple2<>(e.getKey(), e.getValue()))
|
||||
.collect(Collectors.toList()).iterator();
|
||||
};
|
||||
}
|
||||
|
||||
private static PairFlatMapFunction<String, String, String> getFilesToDeleteFunc(HoodieTable table,
|
||||
HoodieWriteConfig config) {
|
||||
return (PairFlatMapFunction<String, String, String>) partitionPathToClean -> {
|
||||
HoodieCleanHelper cleaner = new HoodieCleanHelper(table, config);
|
||||
return cleaner.getDeletePaths(partitionPathToClean).stream()
|
||||
.map(deleteFile -> new Tuple2<>(partitionPathToClean, deleteFile.toString())).iterator();
|
||||
};
|
||||
}
|
||||
|
||||
private static Boolean deleteFileAndGetResult(FileSystem fs, String deletePathStr)
|
||||
throws IOException {
|
||||
Path deletePath = new Path(deletePathStr);
|
||||
logger.debug("Working on delete path :" + deletePath);
|
||||
boolean deleteResult = fs.delete(deletePath, false);
|
||||
if (deleteResult) {
|
||||
logger.debug("Cleaned file at path :" + deletePath);
|
||||
}
|
||||
return deleteResult;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Partitioner getUpsertPartitioner(WorkloadProfile profile) {
|
||||
if (profile == null) {
|
||||
throw new HoodieUpsertException("Need workload profile to construct the upsert partitioner.");
|
||||
}
|
||||
return new UpsertPartitioner(profile);
|
||||
}
|
||||
|
||||
@Override
|
||||
public Partitioner getInsertPartitioner(WorkloadProfile profile) {
|
||||
return getUpsertPartitioner(profile);
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean isWorkloadProfileNeeded() {
|
||||
return true;
|
||||
}
|
||||
|
||||
@Override
|
||||
public HoodieCompactionPlan scheduleCompaction(JavaSparkContext jsc, String commitTime) {
|
||||
throw new HoodieNotSupportedException("Compaction is not supported from a CopyOnWrite table");
|
||||
}
|
||||
|
||||
@Override
|
||||
public JavaRDD<WriteStatus> compact(JavaSparkContext jsc, String compactionInstantTime,
|
||||
HoodieCompactionPlan compactionPlan) {
|
||||
throw new HoodieNotSupportedException("Compaction is not supported from a CopyOnWrite table");
|
||||
}
|
||||
|
||||
public Iterator<List<WriteStatus>> handleUpdate(String commitTime, String fileId,
|
||||
Iterator<HoodieRecord<T>> recordItr) throws IOException {
|
||||
// This is needed since sometimes some buckets are never picked in getPartition() and end up with 0 records
|
||||
if (!recordItr.hasNext()) {
|
||||
logger.info("Empty partition with fileId => " + fileId);
|
||||
return Collections.singletonList((List<WriteStatus>) Collections.EMPTY_LIST).iterator();
|
||||
}
|
||||
// these are updates
|
||||
HoodieMergeHandle upsertHandle = getUpdateHandle(commitTime, fileId, recordItr);
|
||||
return handleUpdateInternal(upsertHandle, commitTime, fileId);
|
||||
}
|
||||
|
||||
public Iterator<List<WriteStatus>> handleUpdate(String commitTime, String fileId,
|
||||
Map<String, HoodieRecord<T>> keyToNewRecords, HoodieDataFile oldDataFile) throws IOException {
|
||||
// these are updates
|
||||
HoodieMergeHandle upsertHandle = getUpdateHandle(commitTime, fileId, keyToNewRecords, oldDataFile);
|
||||
return handleUpdateInternal(upsertHandle, commitTime, fileId);
|
||||
}
|
||||
|
||||
protected Iterator<List<WriteStatus>> handleUpdateInternal(HoodieMergeHandle upsertHandle,
|
||||
String commitTime, String fileId)
|
||||
throws IOException {
|
||||
if (upsertHandle.getOldFilePath() == null) {
|
||||
throw new HoodieUpsertException(
|
||||
"Error in finding the old file path at commit " + commitTime + " for fileId: " + fileId);
|
||||
} else {
|
||||
AvroReadSupport.setAvroReadSchema(getHadoopConf(), upsertHandle.getWriterSchema());
|
||||
BoundedInMemoryExecutor<GenericRecord, GenericRecord, Void> wrapper = null;
|
||||
try (ParquetReader<IndexedRecord> reader = AvroParquetReader.<IndexedRecord>builder(upsertHandle.getOldFilePath())
|
||||
.withConf(getHadoopConf()).build()) {
|
||||
wrapper = new SparkBoundedInMemoryExecutor(config, new ParquetReaderIterator(reader),
|
||||
new UpdateHandler(upsertHandle), x -> x);
|
||||
wrapper.execute();
|
||||
} catch (Exception e) {
|
||||
throw new HoodieException(e);
|
||||
} finally {
|
||||
upsertHandle.close();
|
||||
if (null != wrapper) {
|
||||
wrapper.shutdownNow();
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
//TODO(vc): This needs to be revisited
|
||||
if (upsertHandle.getWriteStatus().getPartitionPath() == null) {
|
||||
logger.info("Upsert Handle has partition path as null " + upsertHandle.getOldFilePath()
|
||||
+ ", " + upsertHandle.getWriteStatus());
|
||||
}
|
||||
return Collections.singletonList(Collections.singletonList(upsertHandle.getWriteStatus()))
|
||||
.iterator();
|
||||
}
|
||||
|
||||
protected HoodieMergeHandle getUpdateHandle(String commitTime, String fileId,
|
||||
Iterator<HoodieRecord<T>> recordItr) {
|
||||
return new HoodieMergeHandle<>(config, commitTime, this, recordItr, fileId);
|
||||
}
|
||||
|
||||
protected HoodieMergeHandle getUpdateHandle(String commitTime, String fileId,
|
||||
Map<String, HoodieRecord<T>> keyToNewRecords, HoodieDataFile dataFileToBeMerged) {
|
||||
return new HoodieMergeHandle<>(config, commitTime, this, keyToNewRecords, fileId, dataFileToBeMerged);
|
||||
}
|
||||
|
||||
public Iterator<List<WriteStatus>> handleInsert(String commitTime, String idPfx,
|
||||
Iterator<HoodieRecord<T>> recordItr) throws Exception {
|
||||
// This is needed since sometimes some buckets are never picked in getPartition() and end up with 0 records
|
||||
if (!recordItr.hasNext()) {
|
||||
logger.info("Empty partition");
|
||||
return Collections.singletonList((List<WriteStatus>) Collections.EMPTY_LIST).iterator();
|
||||
}
|
||||
return new CopyOnWriteLazyInsertIterable<>(recordItr, config, commitTime, this, idPfx);
|
||||
}
|
||||
|
||||
public Iterator<List<WriteStatus>> handleInsert(String commitTime, String partitionPath, String fileId,
|
||||
Iterator<HoodieRecord<T>> recordItr) {
|
||||
HoodieCreateHandle createHandle = new HoodieCreateHandle(config, commitTime, this, partitionPath, fileId,
|
||||
recordItr);
|
||||
createHandle.write();
|
||||
return Collections.singletonList(Collections.singletonList(createHandle.close())).iterator();
|
||||
}
|
||||
|
||||
@SuppressWarnings("unchecked")
|
||||
@Override
|
||||
public Iterator<List<WriteStatus>> handleUpsertPartition(String commitTime, Integer partition,
|
||||
Iterator recordItr, Partitioner partitioner) {
|
||||
UpsertPartitioner upsertPartitioner = (UpsertPartitioner) partitioner;
|
||||
BucketInfo binfo = upsertPartitioner.getBucketInfo(partition);
|
||||
BucketType btype = binfo.bucketType;
|
||||
try {
|
||||
if (btype.equals(BucketType.INSERT)) {
|
||||
return handleInsert(commitTime, binfo.fileIdPrefix, recordItr);
|
||||
} else if (btype.equals(BucketType.UPDATE)) {
|
||||
return handleUpdate(commitTime, binfo.fileIdPrefix, recordItr);
|
||||
} else {
|
||||
throw new HoodieUpsertException(
|
||||
"Unknown bucketType " + btype + " for partition :" + partition);
|
||||
}
|
||||
} catch (Throwable t) {
|
||||
String msg = "Error upserting bucketType " + btype + " for partition :" + partition;
|
||||
logger.error(msg, t);
|
||||
throw new HoodieUpsertException(msg, t);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public Iterator<List<WriteStatus>> handleInsertPartition(String commitTime, Integer partition,
|
||||
Iterator recordItr, Partitioner partitioner) {
|
||||
return handleUpsertPartition(commitTime, partition, recordItr, partitioner);
|
||||
}
|
||||
|
||||
/**
|
||||
* Performs cleaning of partition paths according to cleaning policy and returns the number of
|
||||
* files cleaned. Handles skews in partitions to clean by making files to clean as the unit of
|
||||
* task distribution.
|
||||
*
|
||||
* @throws IllegalArgumentException if unknown cleaning policy is provided
|
||||
*/
|
||||
@Override
|
||||
public List<HoodieCleanStat> clean(JavaSparkContext jsc) {
|
||||
try {
|
||||
FileSystem fs = getMetaClient().getFs();
|
||||
List<String> partitionsToClean = FSUtils
|
||||
.getAllPartitionPaths(fs, getMetaClient().getBasePath(),
|
||||
config.shouldAssumeDatePartitioning());
|
||||
logger.info("Partitions to clean up : " + partitionsToClean + ", with policy " + config
|
||||
.getCleanerPolicy());
|
||||
if (partitionsToClean.isEmpty()) {
|
||||
logger.info("Nothing to clean here mom. It is already clean");
|
||||
return Collections.emptyList();
|
||||
}
|
||||
return cleanPartitionPaths(partitionsToClean, jsc);
|
||||
} catch (IOException e) {
|
||||
throw new HoodieIOException("Failed to clean up after commit", e);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Common method used for cleaning out parquet files under a partition path during rollback of a
|
||||
* set of commits
|
||||
*/
|
||||
protected Map<FileStatus, Boolean> deleteCleanedFiles(Map<FileStatus, Boolean> results, String partitionPath,
|
||||
PathFilter filter)
|
||||
throws IOException {
|
||||
logger.info("Cleaning path " + partitionPath);
|
||||
FileSystem fs = getMetaClient().getFs();
|
||||
FileStatus[] toBeDeleted = fs.listStatus(FSUtils.getPartitionPath(config.getBasePath(), partitionPath), filter);
|
||||
for (FileStatus file : toBeDeleted) {
|
||||
boolean success = fs.delete(file.getPath(), false);
|
||||
results.put(file, success);
|
||||
logger.info("Delete file " + file.getPath() + "\t" + success);
|
||||
}
|
||||
return results;
|
||||
}
|
||||
|
||||
/**
|
||||
* Common method used for cleaning out parquet files under a partition path during rollback of a
|
||||
* set of commits
|
||||
*/
|
||||
protected Map<FileStatus, Boolean> deleteCleanedFiles(Map<FileStatus, Boolean> results, String commit, String
|
||||
partitionPath)
|
||||
throws IOException {
|
||||
logger.info("Cleaning path " + partitionPath);
|
||||
FileSystem fs = getMetaClient().getFs();
|
||||
PathFilter filter = (path) -> {
|
||||
if (path.toString().contains(".parquet")) {
|
||||
String fileCommitTime = FSUtils.getCommitTime(path.getName());
|
||||
return commit.equals(fileCommitTime);
|
||||
}
|
||||
return false;
|
||||
};
|
||||
FileStatus[] toBeDeleted = fs.listStatus(FSUtils.getPartitionPath(config.getBasePath(), partitionPath), filter);
|
||||
for (FileStatus file : toBeDeleted) {
|
||||
boolean success = fs.delete(file.getPath(), false);
|
||||
results.put(file, success);
|
||||
logger.info("Delete file " + file.getPath() + "\t" + success);
|
||||
}
|
||||
return results;
|
||||
}
|
||||
|
||||
@Override
|
||||
public List<HoodieRollbackStat> rollback(JavaSparkContext jsc, String commit, boolean deleteInstants)
|
||||
throws IOException {
|
||||
String actionType = metaClient.getCommitActionType();
|
||||
HoodieActiveTimeline activeTimeline = this.getActiveTimeline();
|
||||
List<String> inflights = this.getInflightCommitTimeline().getInstants()
|
||||
.map(HoodieInstant::getTimestamp).collect(Collectors.toList());
|
||||
// Atomically unpublish the commits
|
||||
if (!inflights.contains(commit)) {
|
||||
activeTimeline.revertToInflight(new HoodieInstant(false, actionType, commit));
|
||||
}
|
||||
logger.info("Unpublished " + commit);
|
||||
|
||||
// delete all the data files for this commit
|
||||
logger.info("Clean out all parquet files generated for commit: " + commit);
|
||||
List<HoodieRollbackStat> stats = jsc.parallelize(FSUtils
|
||||
.getAllPartitionPaths(metaClient.getFs(), getMetaClient().getBasePath(),
|
||||
config.shouldAssumeDatePartitioning()))
|
||||
.map((Function<String, HoodieRollbackStat>) partitionPath -> {
|
||||
// Scan all partitions files with this commit time
|
||||
final Map<FileStatus, Boolean> filesToDeletedStatus = new HashMap<>();
|
||||
deleteCleanedFiles(filesToDeletedStatus, commit, partitionPath);
|
||||
return HoodieRollbackStat.newBuilder().withPartitionPath(partitionPath)
|
||||
.withDeletedFileResults(filesToDeletedStatus).build();
|
||||
}).collect();
|
||||
|
||||
// Delete Inflight instant if enabled
|
||||
deleteInflightInstant(deleteInstants, activeTimeline,
|
||||
new HoodieInstant(true, actionType, commit));
|
||||
return stats;
|
||||
}
|
||||
|
||||
/**
|
||||
* Delete Inflight instant if enabled
|
||||
* @param deleteInstant Enable Deletion of Inflight instant
|
||||
* @param activeTimeline Hoodie active timeline
|
||||
* @param instantToBeDeleted Instant to be deleted
|
||||
*/
|
||||
protected void deleteInflightInstant(boolean deleteInstant, HoodieActiveTimeline activeTimeline,
|
||||
HoodieInstant instantToBeDeleted) {
|
||||
// Remove marker files always on rollback
|
||||
deleteMarkerDir(instantToBeDeleted.getTimestamp());
|
||||
|
||||
// Remove the rolled back inflight commits
|
||||
if (deleteInstant) {
|
||||
activeTimeline.deleteInflight(instantToBeDeleted);
|
||||
logger.info("Deleted inflight commit " + instantToBeDeleted);
|
||||
} else {
|
||||
logger.warn("Rollback finished without deleting inflight instant file. Instant=" + instantToBeDeleted);
|
||||
}
|
||||
}
|
||||
|
||||
private List<HoodieCleanStat> cleanPartitionPaths(List<String> partitionsToClean,
|
||||
JavaSparkContext jsc) {
|
||||
int cleanerParallelism = Math.min(partitionsToClean.size(), config.getCleanerParallelism());
|
||||
logger.info("Using cleanerParallelism: " + cleanerParallelism);
|
||||
List<Tuple2<String, PartitionCleanStat>> partitionCleanStats = jsc
|
||||
.parallelize(partitionsToClean, cleanerParallelism)
|
||||
.flatMapToPair(getFilesToDeleteFunc(this, config))
|
||||
.repartition(cleanerParallelism) // repartition to remove skews
|
||||
.mapPartitionsToPair(deleteFilesFunc(this)).reduceByKey(
|
||||
// merge partition level clean stats below
|
||||
(Function2<PartitionCleanStat, PartitionCleanStat, PartitionCleanStat>) (e1, e2) -> e1
|
||||
.merge(e2)).collect();
|
||||
|
||||
Map<String, PartitionCleanStat> partitionCleanStatsMap = partitionCleanStats.stream()
|
||||
.collect(Collectors.toMap(Tuple2::_1, Tuple2::_2));
|
||||
|
||||
HoodieCleanHelper cleaner = new HoodieCleanHelper(this, config);
|
||||
// Return PartitionCleanStat for each partition passed.
|
||||
return partitionsToClean.stream().map(partitionPath -> {
|
||||
PartitionCleanStat partitionCleanStat =
|
||||
(partitionCleanStatsMap.containsKey(partitionPath)) ? partitionCleanStatsMap
|
||||
.get(partitionPath) : new PartitionCleanStat(partitionPath);
|
||||
return HoodieCleanStat.newBuilder().withPolicy(config.getCleanerPolicy())
|
||||
.withPartitionPath(partitionPath)
|
||||
.withEarliestCommitRetained(cleaner.getEarliestCommitToRetain())
|
||||
.withDeletePathPattern(partitionCleanStat.deletePathPatterns)
|
||||
.withSuccessfulDeletes(partitionCleanStat.successDeleteFiles)
|
||||
.withFailedDeletes(partitionCleanStat.failedDeleteFiles).build();
|
||||
}).collect(Collectors.toList());
|
||||
}
|
||||
|
||||
enum BucketType {
|
||||
UPDATE, INSERT
|
||||
}
|
||||
|
||||
/**
|
||||
* Consumer that dequeues records from queue and sends to Merge Handle
|
||||
*/
|
||||
private static class UpdateHandler extends BoundedInMemoryQueueConsumer<GenericRecord, Void> {
|
||||
|
||||
private final HoodieMergeHandle upsertHandle;
|
||||
|
||||
private UpdateHandler(HoodieMergeHandle upsertHandle) {
|
||||
this.upsertHandle = upsertHandle;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected void consumeOneRecord(GenericRecord record) {
|
||||
upsertHandle.write(record);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected void finish() {
|
||||
}
|
||||
|
||||
@Override
|
||||
protected Void getResult() {
|
||||
return null;
|
||||
}
|
||||
}
|
||||
|
||||
private static class PartitionCleanStat implements Serializable {
|
||||
|
||||
private final String partitionPath;
|
||||
private final List<String> deletePathPatterns = new ArrayList<>();
|
||||
private final List<String> successDeleteFiles = new ArrayList<>();
|
||||
private final List<String> failedDeleteFiles = new ArrayList<>();
|
||||
|
||||
private PartitionCleanStat(String partitionPath) {
|
||||
this.partitionPath = partitionPath;
|
||||
}
|
||||
|
||||
private void addDeletedFileResult(String deletePathStr, Boolean deletedFileResult) {
|
||||
if (deletedFileResult) {
|
||||
successDeleteFiles.add(deletePathStr);
|
||||
} else {
|
||||
failedDeleteFiles.add(deletePathStr);
|
||||
}
|
||||
}
|
||||
|
||||
private void addDeleteFilePatterns(String deletePathStr) {
|
||||
deletePathPatterns.add(deletePathStr);
|
||||
}
|
||||
|
||||
private PartitionCleanStat merge(PartitionCleanStat other) {
|
||||
if (!this.partitionPath.equals(other.partitionPath)) {
|
||||
throw new RuntimeException(String
|
||||
.format("partitionPath is not a match: (%s, %s)", partitionPath, other.partitionPath));
|
||||
}
|
||||
successDeleteFiles.addAll(other.successDeleteFiles);
|
||||
deletePathPatterns.addAll(other.deletePathPatterns);
|
||||
failedDeleteFiles.addAll(other.failedDeleteFiles);
|
||||
return this;
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Helper class for a small file's location and its actual size on disk
|
||||
*/
|
||||
static class SmallFile implements Serializable {
|
||||
|
||||
HoodieRecordLocation location;
|
||||
long sizeBytes;
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
final StringBuilder sb = new StringBuilder("SmallFile {");
|
||||
sb.append("location=").append(location).append(", ");
|
||||
sb.append("sizeBytes=").append(sizeBytes);
|
||||
sb.append('}');
|
||||
return sb.toString();
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Helper class for an insert bucket along with the weight [0.0, 0.1] that defines the amount of
|
||||
* incoming inserts that should be allocated to the bucket
|
||||
*/
|
||||
class InsertBucket implements Serializable {
|
||||
|
||||
int bucketNumber;
|
||||
// fraction of total inserts, that should go into this bucket
|
||||
double weight;
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
final StringBuilder sb = new StringBuilder("WorkloadStat {");
|
||||
sb.append("bucketNumber=").append(bucketNumber).append(", ");
|
||||
sb.append("weight=").append(weight);
|
||||
sb.append('}');
|
||||
return sb.toString();
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Helper class for a bucket's type (INSERT and UPDATE) and its file location
|
||||
*/
|
||||
class BucketInfo implements Serializable {
|
||||
|
||||
BucketType bucketType;
|
||||
String fileIdPrefix;
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
final StringBuilder sb = new StringBuilder("BucketInfo {");
|
||||
sb.append("bucketType=").append(bucketType).append(", ");
|
||||
sb.append("fileIdPrefix=").append(fileIdPrefix);
|
||||
sb.append('}');
|
||||
return sb.toString();
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Packs incoming records to be upserted, into buckets (1 bucket = 1 RDD partition)
|
||||
*/
|
||||
class UpsertPartitioner extends Partitioner {
|
||||
|
||||
/**
|
||||
* List of all small files to be corrected
|
||||
*/
|
||||
List<SmallFile> smallFiles = new ArrayList<SmallFile>();
|
||||
/**
|
||||
* Total number of RDD partitions, is determined by total buckets we want to pack the incoming
|
||||
* workload into
|
||||
*/
|
||||
private int totalBuckets = 0;
|
||||
/**
|
||||
* Stat for the current workload. Helps in determining total inserts, upserts etc.
|
||||
*/
|
||||
private WorkloadStat globalStat;
|
||||
/**
|
||||
* Helps decide which bucket an incoming update should go to.
|
||||
*/
|
||||
private HashMap<String, Integer> updateLocationToBucket;
|
||||
/**
|
||||
* Helps us pack inserts into 1 or more buckets depending on number of incoming records.
|
||||
*/
|
||||
private HashMap<String, List<InsertBucket>> partitionPathToInsertBuckets;
|
||||
/**
|
||||
* Remembers what type each bucket is for later.
|
||||
*/
|
||||
private HashMap<Integer, BucketInfo> bucketInfoMap;
|
||||
|
||||
/**
|
||||
* Rolling stats for files
|
||||
*/
|
||||
protected HoodieRollingStatMetadata rollingStatMetadata;
|
||||
protected long averageRecordSize;
|
||||
|
||||
UpsertPartitioner(WorkloadProfile profile) {
|
||||
updateLocationToBucket = new HashMap<>();
|
||||
partitionPathToInsertBuckets = new HashMap<>();
|
||||
bucketInfoMap = new HashMap<>();
|
||||
globalStat = profile.getGlobalStat();
|
||||
rollingStatMetadata = getRollingStats();
|
||||
assignUpdates(profile);
|
||||
assignInserts(profile);
|
||||
|
||||
logger.info(
|
||||
"Total Buckets :" + totalBuckets + ", " + "buckets info => " + bucketInfoMap + ", \n"
|
||||
+ "Partition to insert buckets => " + partitionPathToInsertBuckets + ", \n"
|
||||
+ "UpdateLocations mapped to buckets =>" + updateLocationToBucket);
|
||||
}
|
||||
|
||||
private void assignUpdates(WorkloadProfile profile) {
|
||||
// each update location gets a partition
|
||||
WorkloadStat gStat = profile.getGlobalStat();
|
||||
for (Map.Entry<String, Pair<String, Long>> updateLocEntry : gStat.getUpdateLocationToCount()
|
||||
.entrySet()) {
|
||||
addUpdateBucket(updateLocEntry.getKey());
|
||||
}
|
||||
}
|
||||
|
||||
private int addUpdateBucket(String fileIdHint) {
|
||||
int bucket = totalBuckets;
|
||||
updateLocationToBucket.put(fileIdHint, bucket);
|
||||
BucketInfo bucketInfo = new BucketInfo();
|
||||
bucketInfo.bucketType = BucketType.UPDATE;
|
||||
bucketInfo.fileIdPrefix = fileIdHint;
|
||||
bucketInfoMap.put(totalBuckets, bucketInfo);
|
||||
totalBuckets++;
|
||||
return bucket;
|
||||
}
|
||||
|
||||
private void assignInserts(WorkloadProfile profile) {
|
||||
// for new inserts, compute buckets depending on how many records we have for each partition
|
||||
Set<String> partitionPaths = profile.getPartitionPaths();
|
||||
long averageRecordSize = averageBytesPerRecord();
|
||||
logger.info("AvgRecordSize => " + averageRecordSize);
|
||||
for (String partitionPath : partitionPaths) {
|
||||
WorkloadStat pStat = profile.getWorkloadStat(partitionPath);
|
||||
if (pStat.getNumInserts() > 0) {
|
||||
|
||||
List<SmallFile> smallFiles = getSmallFiles(partitionPath);
|
||||
logger.info("For partitionPath : " + partitionPath + " Small Files => " + smallFiles);
|
||||
|
||||
long totalUnassignedInserts = pStat.getNumInserts();
|
||||
List<Integer> bucketNumbers = new ArrayList<>();
|
||||
List<Long> recordsPerBucket = new ArrayList<>();
|
||||
|
||||
// first try packing this into one of the smallFiles
|
||||
for (SmallFile smallFile : smallFiles) {
|
||||
long recordsToAppend = Math
|
||||
.min((config.getParquetMaxFileSize() - smallFile.sizeBytes) / averageRecordSize,
|
||||
totalUnassignedInserts);
|
||||
if (recordsToAppend > 0 && totalUnassignedInserts > 0) {
|
||||
// create a new bucket or re-use an existing bucket
|
||||
int bucket;
|
||||
if (updateLocationToBucket.containsKey(smallFile.location.getFileId())) {
|
||||
bucket = updateLocationToBucket.get(smallFile.location.getFileId());
|
||||
logger.info("Assigning " + recordsToAppend + " inserts to existing update bucket "
|
||||
+ bucket);
|
||||
} else {
|
||||
bucket = addUpdateBucket(smallFile.location.getFileId());
|
||||
logger.info(
|
||||
"Assigning " + recordsToAppend + " inserts to new update bucket " + bucket);
|
||||
}
|
||||
bucketNumbers.add(bucket);
|
||||
recordsPerBucket.add(recordsToAppend);
|
||||
totalUnassignedInserts -= recordsToAppend;
|
||||
}
|
||||
}
|
||||
|
||||
// if we have anything more, create new insert buckets, like normal
|
||||
if (totalUnassignedInserts > 0) {
|
||||
long insertRecordsPerBucket = config.getCopyOnWriteInsertSplitSize();
|
||||
if (config.shouldAutoTuneInsertSplits()) {
|
||||
insertRecordsPerBucket = config.getParquetMaxFileSize() / averageRecordSize;
|
||||
}
|
||||
|
||||
int insertBuckets = (int) Math.ceil((1.0 * totalUnassignedInserts) / insertRecordsPerBucket);
|
||||
logger.info(
|
||||
"After small file assignment: unassignedInserts => " + totalUnassignedInserts
|
||||
+ ", totalInsertBuckets => " + insertBuckets + ", recordsPerBucket => "
|
||||
+ insertRecordsPerBucket);
|
||||
for (int b = 0; b < insertBuckets; b++) {
|
||||
bucketNumbers.add(totalBuckets);
|
||||
recordsPerBucket.add(totalUnassignedInserts / insertBuckets);
|
||||
BucketInfo bucketInfo = new BucketInfo();
|
||||
bucketInfo.bucketType = BucketType.INSERT;
|
||||
bucketInfo.fileIdPrefix = FSUtils.createNewFileIdPfx();
|
||||
bucketInfoMap.put(totalBuckets, bucketInfo);
|
||||
totalBuckets++;
|
||||
}
|
||||
}
|
||||
|
||||
// Go over all such buckets, and assign weights as per amount of incoming inserts.
|
||||
List<InsertBucket> insertBuckets = new ArrayList<>();
|
||||
for (int i = 0; i < bucketNumbers.size(); i++) {
|
||||
InsertBucket bkt = new InsertBucket();
|
||||
bkt.bucketNumber = bucketNumbers.get(i);
|
||||
bkt.weight = (1.0 * recordsPerBucket.get(i)) / pStat.getNumInserts();
|
||||
insertBuckets.add(bkt);
|
||||
}
|
||||
logger.info(
|
||||
"Total insert buckets for partition path " + partitionPath + " => " + insertBuckets);
|
||||
partitionPathToInsertBuckets.put(partitionPath, insertBuckets);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns a list of small files in the given partition path
|
||||
*/
|
||||
protected List<SmallFile> getSmallFiles(String partitionPath) {
|
||||
|
||||
// smallFiles only for partitionPath
|
||||
List<SmallFile> smallFileLocations = new ArrayList<>();
|
||||
|
||||
HoodieTimeline commitTimeline = getCompletedCommitsTimeline();
|
||||
|
||||
if (!commitTimeline.empty()) { // if we have some commits
|
||||
HoodieInstant latestCommitTime = commitTimeline.lastInstant().get();
|
||||
List<HoodieDataFile> allFiles = getROFileSystemView()
|
||||
.getLatestDataFilesBeforeOrOn(partitionPath, latestCommitTime.getTimestamp())
|
||||
.collect(Collectors.toList());
|
||||
|
||||
for (HoodieDataFile file : allFiles) {
|
||||
if (file.getFileSize() < config.getParquetSmallFileLimit()) {
|
||||
String filename = file.getFileName();
|
||||
SmallFile sf = new SmallFile();
|
||||
sf.location = new HoodieRecordLocation(FSUtils.getCommitTime(filename),
|
||||
FSUtils.getFileId(filename));
|
||||
sf.sizeBytes = file.getFileSize();
|
||||
smallFileLocations.add(sf);
|
||||
// Update the global small files list
|
||||
smallFiles.add(sf);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
return smallFileLocations;
|
||||
}
|
||||
|
||||
/**
|
||||
* Obtains the average record size based on records written during last commit. Used for
|
||||
* estimating how many records pack into one file.
|
||||
*/
|
||||
protected long averageBytesPerRecord() {
|
||||
long avgSize = 0L;
|
||||
HoodieTimeline commitTimeline = metaClient.getActiveTimeline().getCommitTimeline()
|
||||
.filterCompletedInstants();
|
||||
try {
|
||||
if (!commitTimeline.empty()) {
|
||||
HoodieInstant latestCommitTime = commitTimeline.lastInstant().get();
|
||||
HoodieCommitMetadata commitMetadata = HoodieCommitMetadata
|
||||
.fromBytes(commitTimeline.getInstantDetails(latestCommitTime).get(), HoodieCommitMetadata.class);
|
||||
avgSize = (long) Math.ceil(
|
||||
(1.0 * commitMetadata.fetchTotalBytesWritten()) / commitMetadata
|
||||
.fetchTotalRecordsWritten());
|
||||
}
|
||||
} catch (Throwable t) {
|
||||
// make this fail safe.
|
||||
logger.error("Error trying to compute average bytes/record ", t);
|
||||
}
|
||||
return avgSize <= 0L ? config.getCopyOnWriteRecordSizeEstimate() : avgSize;
|
||||
}
|
||||
|
||||
public BucketInfo getBucketInfo(int bucketNumber) {
|
||||
return bucketInfoMap.get(bucketNumber);
|
||||
}
|
||||
|
||||
public List<InsertBucket> getInsertBuckets(String partitionPath) {
|
||||
return partitionPathToInsertBuckets.get(partitionPath);
|
||||
}
|
||||
|
||||
@Override
|
||||
public int numPartitions() {
|
||||
return totalBuckets;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int getPartition(Object key) {
|
||||
Tuple2<HoodieKey, Option<HoodieRecordLocation>> keyLocation = (Tuple2<HoodieKey,
|
||||
Option<HoodieRecordLocation>>) key;
|
||||
if (keyLocation._2().isPresent()) {
|
||||
HoodieRecordLocation location = keyLocation._2().get();
|
||||
return updateLocationToBucket.get(location.getFileId());
|
||||
} else {
|
||||
List<InsertBucket> targetBuckets = partitionPathToInsertBuckets
|
||||
.get(keyLocation._1().getPartitionPath());
|
||||
// pick the target bucket to use based on the weights.
|
||||
double totalWeight = 0.0;
|
||||
final long totalInserts = Math.max(1, globalStat.getNumInserts());
|
||||
final long hashOfKey = Hashing.md5()
|
||||
.hashString(keyLocation._1().getRecordKey(), StandardCharsets.UTF_8).asLong();
|
||||
final double r = 1.0 * Math.floorMod(hashOfKey, totalInserts) / totalInserts;
|
||||
for (InsertBucket insertBucket : targetBuckets) {
|
||||
totalWeight += insertBucket.weight;
|
||||
if (r <= totalWeight) {
|
||||
return insertBucket.bucketNumber;
|
||||
}
|
||||
}
|
||||
// return first one, by default
|
||||
return targetBuckets.get(0).bucketNumber;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
protected HoodieRollingStatMetadata getRollingStats() {
|
||||
return null;
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,547 @@
|
||||
/*
|
||||
* 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.table;
|
||||
|
||||
import com.google.common.annotations.VisibleForTesting;
|
||||
import com.google.common.collect.Maps;
|
||||
import com.google.common.collect.Sets;
|
||||
import java.io.IOException;
|
||||
import java.io.UncheckedIOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Collections;
|
||||
import java.util.HashMap;
|
||||
import java.util.Iterator;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Objects;
|
||||
import java.util.Set;
|
||||
import java.util.stream.Collectors;
|
||||
import org.apache.hadoop.fs.FileStatus;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.fs.PathFilter;
|
||||
import org.apache.hudi.WriteStatus;
|
||||
import org.apache.hudi.avro.model.HoodieCompactionPlan;
|
||||
import org.apache.hudi.common.HoodieRollbackStat;
|
||||
import org.apache.hudi.common.model.FileSlice;
|
||||
import org.apache.hudi.common.model.HoodieCommitMetadata;
|
||||
import org.apache.hudi.common.model.HoodieLogFile;
|
||||
import org.apache.hudi.common.model.HoodieRecord;
|
||||
import org.apache.hudi.common.model.HoodieRecordLocation;
|
||||
import org.apache.hudi.common.model.HoodieRecordPayload;
|
||||
import org.apache.hudi.common.model.HoodieRollingStat;
|
||||
import org.apache.hudi.common.model.HoodieRollingStatMetadata;
|
||||
import org.apache.hudi.common.model.HoodieWriteStat;
|
||||
import org.apache.hudi.common.table.HoodieTimeline;
|
||||
import org.apache.hudi.common.table.SyncableFileSystemView;
|
||||
import org.apache.hudi.common.table.log.HoodieLogFormat;
|
||||
import org.apache.hudi.common.table.log.HoodieLogFormat.Writer;
|
||||
import org.apache.hudi.common.table.log.block.HoodieCommandBlock;
|
||||
import org.apache.hudi.common.table.log.block.HoodieCommandBlock.HoodieCommandBlockTypeEnum;
|
||||
import org.apache.hudi.common.table.log.block.HoodieLogBlock.HeaderMetadataType;
|
||||
import org.apache.hudi.common.table.timeline.HoodieActiveTimeline;
|
||||
import org.apache.hudi.common.table.timeline.HoodieInstant;
|
||||
import org.apache.hudi.common.util.FSUtils;
|
||||
import org.apache.hudi.common.util.Option;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
import org.apache.hudi.exception.HoodieCompactionException;
|
||||
import org.apache.hudi.exception.HoodieException;
|
||||
import org.apache.hudi.exception.HoodieIOException;
|
||||
import org.apache.hudi.exception.HoodieRollbackException;
|
||||
import org.apache.hudi.exception.HoodieUpsertException;
|
||||
import org.apache.hudi.func.MergeOnReadLazyInsertIterable;
|
||||
import org.apache.hudi.index.HoodieIndex;
|
||||
import org.apache.hudi.io.HoodieAppendHandle;
|
||||
import org.apache.hudi.io.compact.HoodieRealtimeTableCompactor;
|
||||
import org.apache.log4j.LogManager;
|
||||
import org.apache.log4j.Logger;
|
||||
import org.apache.spark.Partitioner;
|
||||
import org.apache.spark.api.java.JavaRDD;
|
||||
import org.apache.spark.api.java.JavaSparkContext;
|
||||
import org.apache.spark.api.java.function.Function;
|
||||
|
||||
/**
|
||||
* Implementation of a more real-time read-optimized Hoodie Table where <p> INSERTS - Same as
|
||||
* HoodieCopyOnWriteTable - Produce new files, block aligned to desired size (or) Merge with the
|
||||
* smallest existing file, to expand it </p> <p> UPDATES - Appends the changes to a rolling log file
|
||||
* maintained per file Id. Compaction merges the log file into the base file. </p> <p> WARNING - MOR
|
||||
* table type does not support nested rollbacks, every rollback must be followed by an attempted
|
||||
* commit action </p>
|
||||
*/
|
||||
public class HoodieMergeOnReadTable<T extends HoodieRecordPayload> extends
|
||||
HoodieCopyOnWriteTable<T> {
|
||||
|
||||
private static Logger logger = LogManager.getLogger(HoodieMergeOnReadTable.class);
|
||||
|
||||
// UpsertPartitioner for MergeOnRead table type
|
||||
private MergeOnReadUpsertPartitioner mergeOnReadUpsertPartitioner;
|
||||
|
||||
public HoodieMergeOnReadTable(HoodieWriteConfig config, JavaSparkContext jsc) {
|
||||
super(config, jsc);
|
||||
}
|
||||
|
||||
@Override
|
||||
public Partitioner getUpsertPartitioner(WorkloadProfile profile) {
|
||||
if (profile == null) {
|
||||
throw new HoodieUpsertException("Need workload profile to construct the upsert partitioner.");
|
||||
}
|
||||
mergeOnReadUpsertPartitioner = new MergeOnReadUpsertPartitioner(profile);
|
||||
return mergeOnReadUpsertPartitioner;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Iterator<List<WriteStatus>> handleUpdate(String commitTime, String fileId,
|
||||
Iterator<HoodieRecord<T>> recordItr) throws IOException {
|
||||
logger.info("Merging updates for commit " + commitTime + " for file " + fileId);
|
||||
|
||||
if (!index.canIndexLogFiles() && mergeOnReadUpsertPartitioner.getSmallFileIds().contains(fileId)) {
|
||||
logger.info(
|
||||
"Small file corrections for updates for commit " + commitTime + " for file " + fileId);
|
||||
return super.handleUpdate(commitTime, fileId, recordItr);
|
||||
} else {
|
||||
HoodieAppendHandle<T> appendHandle = new HoodieAppendHandle<>(config, commitTime, this,
|
||||
fileId, recordItr);
|
||||
appendHandle.doAppend();
|
||||
appendHandle.close();
|
||||
return Collections.singletonList(Collections.singletonList(appendHandle.getWriteStatus()))
|
||||
.iterator();
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public Iterator<List<WriteStatus>> handleInsert(String commitTime, String idPfx,
|
||||
Iterator<HoodieRecord<T>> recordItr) throws Exception {
|
||||
// If canIndexLogFiles, write inserts to log files else write inserts to parquet files
|
||||
if (index.canIndexLogFiles()) {
|
||||
return new MergeOnReadLazyInsertIterable<>(recordItr, config, commitTime, this, idPfx);
|
||||
} else {
|
||||
return super.handleInsert(commitTime, idPfx, recordItr);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public HoodieCompactionPlan scheduleCompaction(JavaSparkContext jsc, String instantTime) {
|
||||
logger.info("Checking if compaction needs to be run on " + config.getBasePath());
|
||||
Option<HoodieInstant> lastCompaction = getActiveTimeline().getCommitTimeline()
|
||||
.filterCompletedInstants().lastInstant();
|
||||
String deltaCommitsSinceTs = "0";
|
||||
if (lastCompaction.isPresent()) {
|
||||
deltaCommitsSinceTs = lastCompaction.get().getTimestamp();
|
||||
}
|
||||
|
||||
int deltaCommitsSinceLastCompaction = getActiveTimeline().getDeltaCommitTimeline()
|
||||
.findInstantsAfter(deltaCommitsSinceTs, Integer.MAX_VALUE).countInstants();
|
||||
if (config.getInlineCompactDeltaCommitMax() > deltaCommitsSinceLastCompaction) {
|
||||
logger.info("Not running compaction as only " + deltaCommitsSinceLastCompaction
|
||||
+ " delta commits was found since last compaction " + deltaCommitsSinceTs
|
||||
+ ". Waiting for " + config.getInlineCompactDeltaCommitMax());
|
||||
return new HoodieCompactionPlan();
|
||||
}
|
||||
|
||||
logger.info("Compacting merge on read table " + config.getBasePath());
|
||||
HoodieRealtimeTableCompactor compactor = new HoodieRealtimeTableCompactor();
|
||||
try {
|
||||
return compactor.generateCompactionPlan(jsc, this, config, instantTime,
|
||||
((SyncableFileSystemView)getRTFileSystemView()).getPendingCompactionOperations()
|
||||
.map(instantTimeCompactionopPair -> instantTimeCompactionopPair.getValue().getFileGroupId())
|
||||
.collect(Collectors.toSet()));
|
||||
|
||||
} catch (IOException e) {
|
||||
throw new HoodieCompactionException("Could not schedule compaction " + config.getBasePath(), e);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public JavaRDD<WriteStatus> compact(JavaSparkContext jsc, String compactionInstantTime,
|
||||
HoodieCompactionPlan compactionPlan) {
|
||||
HoodieRealtimeTableCompactor compactor = new HoodieRealtimeTableCompactor();
|
||||
try {
|
||||
return compactor.compact(jsc, compactionPlan, this, config, compactionInstantTime);
|
||||
} catch (IOException e) {
|
||||
throw new HoodieCompactionException("Could not compact " + config.getBasePath(), e);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public List<HoodieRollbackStat> rollback(JavaSparkContext jsc, String commit, boolean deleteInstants)
|
||||
throws IOException {
|
||||
|
||||
// At the moment, MOR table type does not support bulk nested rollbacks. Nested rollbacks is an experimental
|
||||
// feature that is expensive. To perform nested rollbacks, initiate multiple requests of client.rollback
|
||||
// (commitToRollback).
|
||||
// NOTE {@link HoodieCompactionConfig#withCompactionLazyBlockReadEnabled} needs to be set to TRUE. This is
|
||||
// required to avoid OOM when merging multiple LogBlocks performed during nested rollbacks.
|
||||
// Atomically un-publish all non-inflight commits
|
||||
Option<HoodieInstant> commitOrCompactionOption = Option.fromJavaOptional(this.getActiveTimeline()
|
||||
.getTimelineOfActions(Sets.newHashSet(HoodieActiveTimeline.COMMIT_ACTION,
|
||||
HoodieActiveTimeline.DELTA_COMMIT_ACTION, HoodieActiveTimeline.COMPACTION_ACTION)).getInstants()
|
||||
.filter(i -> commit.equals(i.getTimestamp()))
|
||||
.findFirst());
|
||||
HoodieInstant instantToRollback = commitOrCompactionOption.get();
|
||||
// Atomically un-publish all non-inflight commits
|
||||
if (!instantToRollback.isInflight()) {
|
||||
this.getActiveTimeline().revertToInflight(instantToRollback);
|
||||
}
|
||||
logger.info("Unpublished " + commit);
|
||||
Long startTime = System.currentTimeMillis();
|
||||
List<HoodieRollbackStat> allRollbackStats = jsc.parallelize(FSUtils
|
||||
.getAllPartitionPaths(this.metaClient.getFs(), this.getMetaClient().getBasePath(),
|
||||
config.shouldAssumeDatePartitioning()))
|
||||
.map((Function<String, HoodieRollbackStat>) partitionPath -> {
|
||||
HoodieActiveTimeline activeTimeline = this.getActiveTimeline().reload();
|
||||
HoodieRollbackStat hoodieRollbackStats = null;
|
||||
// Need to put the path filter here since Filter is not serializable
|
||||
// PathFilter to get all parquet files and log files that need to be deleted
|
||||
PathFilter filter = (path) -> {
|
||||
if (path.toString().contains(".parquet")) {
|
||||
String fileCommitTime = FSUtils.getCommitTime(path.getName());
|
||||
return commit.equals(fileCommitTime);
|
||||
} else if (path.toString().contains(".log")) {
|
||||
// Since the baseCommitTime is the only commit for new log files, it's okay here
|
||||
String fileCommitTime = FSUtils.getBaseCommitTimeFromLogPath(path);
|
||||
return commit.equals(fileCommitTime);
|
||||
}
|
||||
return false;
|
||||
};
|
||||
|
||||
final Map<FileStatus, Boolean> filesToDeletedStatus = new HashMap<>();
|
||||
|
||||
switch (instantToRollback.getAction()) {
|
||||
case HoodieTimeline.COMMIT_ACTION:
|
||||
try {
|
||||
// Rollback of a commit should delete the newly created parquet files along with any log
|
||||
// files created with this as baseCommit. This is required to support multi-rollbacks in a MOR table.
|
||||
super.deleteCleanedFiles(filesToDeletedStatus, partitionPath, filter);
|
||||
hoodieRollbackStats = HoodieRollbackStat.newBuilder()
|
||||
.withPartitionPath(partitionPath).withDeletedFileResults(filesToDeletedStatus).build();
|
||||
break;
|
||||
} catch (IOException io) {
|
||||
throw new UncheckedIOException("Failed to rollback for commit " + commit, io);
|
||||
}
|
||||
case HoodieTimeline.COMPACTION_ACTION:
|
||||
try {
|
||||
// If there is no delta commit present after the current commit (if compaction), no action, else we
|
||||
// need to make sure that a compaction commit rollback also deletes any log files written as part of the
|
||||
// succeeding deltacommit.
|
||||
boolean higherDeltaCommits = !activeTimeline.getDeltaCommitTimeline()
|
||||
.filterCompletedInstants().findInstantsAfter(commit, 1).empty();
|
||||
if (higherDeltaCommits) {
|
||||
// Rollback of a compaction action with no higher deltacommit means that the compaction is scheduled
|
||||
// and has not yet finished. In this scenario we should delete only the newly created parquet files
|
||||
// and not corresponding base commit log files created with this as baseCommit since updates would
|
||||
// have been written to the log files.
|
||||
super.deleteCleanedFiles(filesToDeletedStatus, commit, partitionPath);
|
||||
hoodieRollbackStats = HoodieRollbackStat.newBuilder()
|
||||
.withPartitionPath(partitionPath).withDeletedFileResults(filesToDeletedStatus).build();
|
||||
} else {
|
||||
// No deltacommits present after this compaction commit (inflight or requested). In this case, we
|
||||
// can also delete any log files that were created with this compaction commit as base
|
||||
// commit.
|
||||
super.deleteCleanedFiles(filesToDeletedStatus, partitionPath, filter);
|
||||
hoodieRollbackStats = HoodieRollbackStat.newBuilder()
|
||||
.withPartitionPath(partitionPath).withDeletedFileResults(filesToDeletedStatus).build();
|
||||
}
|
||||
break;
|
||||
} catch (IOException io) {
|
||||
throw new UncheckedIOException("Failed to rollback for commit " + commit, io);
|
||||
}
|
||||
case HoodieTimeline.DELTA_COMMIT_ACTION:
|
||||
// --------------------------------------------------------------------------------------------------
|
||||
// (A) The following cases are possible if index.canIndexLogFiles and/or index.isGlobal
|
||||
// --------------------------------------------------------------------------------------------------
|
||||
// (A.1) Failed first commit - Inserts were written to log files and HoodieWriteStat has no entries. In
|
||||
// this scenario we would want to delete these log files.
|
||||
// (A.2) Failed recurring commit - Inserts/Updates written to log files. In this scenario,
|
||||
// HoodieWriteStat will have the baseCommitTime for the first log file written, add rollback blocks.
|
||||
// (A.3) Rollback triggered for first commit - Inserts were written to the log files but the commit is
|
||||
// being reverted. In this scenario, HoodieWriteStat will be `null` for the attribute prevCommitTime and
|
||||
// and hence will end up deleting these log files. This is done so there are no orphan log files
|
||||
// lying around.
|
||||
// (A.4) Rollback triggered for recurring commits - Inserts/Updates are being rolled back, the actions
|
||||
// taken in this scenario is a combination of (A.2) and (A.3)
|
||||
// ---------------------------------------------------------------------------------------------------
|
||||
// (B) The following cases are possible if !index.canIndexLogFiles and/or !index.isGlobal
|
||||
// ---------------------------------------------------------------------------------------------------
|
||||
// (B.1) Failed first commit - Inserts were written to parquet files and HoodieWriteStat has no entries.
|
||||
// In this scenario, we delete all the parquet files written for the failed commit.
|
||||
// (B.2) Failed recurring commits - Inserts were written to parquet files and updates to log files. In
|
||||
// this scenario, perform (A.1) and for updates written to log files, write rollback blocks.
|
||||
// (B.3) Rollback triggered for first commit - Same as (B.1)
|
||||
// (B.4) Rollback triggered for recurring commits - Same as (B.2) plus we need to delete the log files
|
||||
// as well if the base parquet file gets deleted.
|
||||
try {
|
||||
HoodieCommitMetadata commitMetadata = HoodieCommitMetadata.fromBytes(
|
||||
metaClient.getCommitTimeline().getInstantDetails(
|
||||
new HoodieInstant(true, instantToRollback.getAction(), instantToRollback.getTimestamp()))
|
||||
.get(), HoodieCommitMetadata.class);
|
||||
|
||||
// read commit file and (either append delete blocks or delete file)
|
||||
Map<FileStatus, Long> filesToNumBlocksRollback = new HashMap<>();
|
||||
|
||||
// In case all data was inserts and the commit failed, delete the file belonging to that commit
|
||||
// We do not know fileIds for inserts (first inserts are either log files or parquet files),
|
||||
// delete all files for the corresponding failed commit, if present (same as COW)
|
||||
super.deleteCleanedFiles(filesToDeletedStatus, partitionPath, filter);
|
||||
final Set<String> deletedFiles = filesToDeletedStatus.entrySet().stream()
|
||||
.map(entry -> {
|
||||
Path filePath = entry.getKey().getPath();
|
||||
return FSUtils.getFileIdFromFilePath(filePath);
|
||||
}).collect(Collectors.toSet());
|
||||
|
||||
// append rollback blocks for updates
|
||||
if (commitMetadata.getPartitionToWriteStats().containsKey(partitionPath)) {
|
||||
hoodieRollbackStats = rollback(index, partitionPath, commit, commitMetadata, filesToDeletedStatus,
|
||||
filesToNumBlocksRollback, deletedFiles);
|
||||
}
|
||||
break;
|
||||
} catch (IOException io) {
|
||||
throw new UncheckedIOException("Failed to rollback for commit " + commit, io);
|
||||
}
|
||||
default:
|
||||
break;
|
||||
}
|
||||
return hoodieRollbackStats;
|
||||
}).filter(Objects::nonNull).collect();
|
||||
|
||||
// Delete Inflight instants if enabled
|
||||
deleteInflightInstant(deleteInstants, this.getActiveTimeline(), new HoodieInstant(true, instantToRollback
|
||||
.getAction(), instantToRollback.getTimestamp()));
|
||||
|
||||
logger.debug("Time(in ms) taken to finish rollback " + (System.currentTimeMillis() - startTime));
|
||||
|
||||
return allRollbackStats;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void finalizeWrite(JavaSparkContext jsc, String instantTs, List<HoodieWriteStat> stats)
|
||||
throws HoodieIOException {
|
||||
// delegate to base class for MOR tables
|
||||
super.finalizeWrite(jsc, instantTs, stats);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected HoodieRollingStatMetadata getRollingStats() {
|
||||
try {
|
||||
Option<HoodieInstant> lastInstant = this.getActiveTimeline().getDeltaCommitTimeline().filterCompletedInstants()
|
||||
.lastInstant();
|
||||
if (lastInstant.isPresent()) {
|
||||
HoodieCommitMetadata commitMetadata = HoodieCommitMetadata.fromBytes(
|
||||
this.getActiveTimeline().getInstantDetails(lastInstant.get()).get(), HoodieCommitMetadata.class);
|
||||
Option<String> lastRollingStat = Option.ofNullable(commitMetadata.getExtraMetadata()
|
||||
.get(HoodieRollingStatMetadata.ROLLING_STAT_METADATA_KEY));
|
||||
if (lastRollingStat.isPresent()) {
|
||||
return HoodieCommitMetadata
|
||||
.fromBytes(lastRollingStat.get().getBytes(), HoodieRollingStatMetadata.class);
|
||||
}
|
||||
}
|
||||
return null;
|
||||
} catch (IOException e) {
|
||||
throw new HoodieException();
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* UpsertPartitioner for MergeOnRead table type, this allows auto correction of small parquet
|
||||
* files to larger ones without the need for an index in the logFile.
|
||||
*/
|
||||
class MergeOnReadUpsertPartitioner extends HoodieCopyOnWriteTable.UpsertPartitioner {
|
||||
|
||||
MergeOnReadUpsertPartitioner(WorkloadProfile profile) {
|
||||
super(profile);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected List<SmallFile> getSmallFiles(String partitionPath) {
|
||||
|
||||
// smallFiles only for partitionPath
|
||||
List<SmallFile> smallFileLocations = new ArrayList<>();
|
||||
|
||||
// Init here since this class (and member variables) might not have been initialized
|
||||
HoodieTimeline commitTimeline = getCompletedCommitsTimeline();
|
||||
|
||||
// Find out all eligible small file slices
|
||||
if (!commitTimeline.empty()) {
|
||||
HoodieInstant latestCommitTime = commitTimeline.lastInstant().get();
|
||||
// find smallest file in partition and append to it
|
||||
List<FileSlice> allSmallFileSlices = new ArrayList<>();
|
||||
// If we cannot index log files, then we choose the smallest parquet file in the partition and add inserts to
|
||||
// it. Doing this overtime for a partition, we ensure that we handle small file issues
|
||||
if (!index.canIndexLogFiles()) {
|
||||
// TODO : choose last N small files since there can be multiple small files written to a single partition
|
||||
// by different spark partitions in a single batch
|
||||
Option<FileSlice> smallFileSlice = Option.fromJavaOptional(getRTFileSystemView()
|
||||
.getLatestFileSlicesBeforeOrOn(partitionPath, latestCommitTime.getTimestamp(), false).filter(
|
||||
fileSlice -> fileSlice.getLogFiles().count() < 1
|
||||
&& fileSlice.getDataFile().get().getFileSize() < config
|
||||
.getParquetSmallFileLimit()).sorted((FileSlice left, FileSlice right) ->
|
||||
left.getDataFile().get().getFileSize() < right.getDataFile().get().getFileSize()
|
||||
? -1 : 1).findFirst());
|
||||
if (smallFileSlice.isPresent()) {
|
||||
allSmallFileSlices.add(smallFileSlice.get());
|
||||
}
|
||||
} else {
|
||||
// If we can index log files, we can add more inserts to log files for fileIds including those under
|
||||
// pending compaction.
|
||||
List<FileSlice> allFileSlices = getRTFileSystemView()
|
||||
.getLatestFileSlicesBeforeOrOn(partitionPath, latestCommitTime.getTimestamp(), true)
|
||||
.collect(Collectors.toList());
|
||||
for (FileSlice fileSlice : allFileSlices) {
|
||||
if (isSmallFile(partitionPath, fileSlice)) {
|
||||
allSmallFileSlices.add(fileSlice);
|
||||
}
|
||||
}
|
||||
}
|
||||
// Create SmallFiles from the eligible file slices
|
||||
for (FileSlice smallFileSlice : allSmallFileSlices) {
|
||||
SmallFile sf = new SmallFile();
|
||||
if (smallFileSlice.getDataFile().isPresent()) {
|
||||
// TODO : Move logic of file name, file id, base commit time handling inside file slice
|
||||
String filename = smallFileSlice.getDataFile().get().getFileName();
|
||||
sf.location = new HoodieRecordLocation(FSUtils.getCommitTime(filename), FSUtils.getFileId(filename));
|
||||
sf.sizeBytes = getTotalFileSize(partitionPath, smallFileSlice);
|
||||
smallFileLocations.add(sf);
|
||||
// Update the global small files list
|
||||
smallFiles.add(sf);
|
||||
} else {
|
||||
HoodieLogFile logFile = smallFileSlice.getLogFiles().findFirst().get();
|
||||
sf.location = new HoodieRecordLocation(FSUtils.getBaseCommitTimeFromLogPath(logFile.getPath()),
|
||||
FSUtils.getFileIdFromLogPath(logFile.getPath()));
|
||||
sf.sizeBytes = getTotalFileSize(partitionPath, smallFileSlice);
|
||||
smallFileLocations.add(sf);
|
||||
// Update the global small files list
|
||||
smallFiles.add(sf);
|
||||
}
|
||||
}
|
||||
}
|
||||
return smallFileLocations;
|
||||
}
|
||||
|
||||
public List<String> getSmallFileIds() {
|
||||
return (List<String>) smallFiles.stream()
|
||||
.map(smallFile -> ((SmallFile) smallFile).location.getFileId())
|
||||
.collect(Collectors.toList());
|
||||
}
|
||||
|
||||
private long getTotalFileSize(String partitionPath, FileSlice fileSlice) {
|
||||
if (rollingStatMetadata != null) {
|
||||
Map<String, HoodieRollingStat> partitionRollingStats =
|
||||
rollingStatMetadata.getPartitionToRollingStats().get(partitionPath);
|
||||
if (partitionRollingStats != null) {
|
||||
HoodieRollingStat rollingStatForFile = partitionRollingStats.get(fileSlice.getFileId());
|
||||
if (rollingStatForFile != null) {
|
||||
long inserts = rollingStatForFile.getInserts();
|
||||
return averageRecordSize * inserts;
|
||||
}
|
||||
}
|
||||
}
|
||||
// In case Rolling Stats is not present, fall back to sizing log files based on heuristics
|
||||
if (!fileSlice.getDataFile().isPresent()) {
|
||||
return convertLogFilesSizeToExpectedParquetSize(fileSlice.getLogFiles().collect(Collectors.toList()));
|
||||
} else {
|
||||
return fileSlice.getDataFile().get().getFileSize() + convertLogFilesSizeToExpectedParquetSize(fileSlice
|
||||
.getLogFiles().collect(Collectors.toList()));
|
||||
}
|
||||
}
|
||||
|
||||
private boolean isSmallFile(String partitionPath, FileSlice fileSlice) {
|
||||
long totalSize = getTotalFileSize(partitionPath, fileSlice);
|
||||
return totalSize < config.getParquetMaxFileSize();
|
||||
}
|
||||
|
||||
// TODO (NA) : Make this static part of utility
|
||||
@VisibleForTesting
|
||||
public long convertLogFilesSizeToExpectedParquetSize(List<HoodieLogFile> hoodieLogFiles) {
|
||||
long totalSizeOfLogFiles = hoodieLogFiles.stream().map(hoodieLogFile -> hoodieLogFile.getFileSize())
|
||||
.filter(size -> size > 0)
|
||||
.reduce((a, b) -> (a + b)).orElse(0L);
|
||||
// Here we assume that if there is no base parquet file, all log files contain only inserts.
|
||||
// We can then just get the parquet equivalent size of these log files, compare that with
|
||||
// {@link config.getParquetMaxFileSize()} and decide if there is scope to insert more rows
|
||||
long logFilesEquivalentParquetFileSize = (long) (totalSizeOfLogFiles * config
|
||||
.getLogFileToParquetCompressionRatio());
|
||||
return logFilesEquivalentParquetFileSize;
|
||||
}
|
||||
}
|
||||
|
||||
private Map<HeaderMetadataType, String> generateHeader(String commit) {
|
||||
// generate metadata
|
||||
Map<HeaderMetadataType, String> header = Maps.newHashMap();
|
||||
header.put(HeaderMetadataType.INSTANT_TIME, metaClient.getActiveTimeline().lastInstant().get().getTimestamp());
|
||||
header.put(HeaderMetadataType.TARGET_INSTANT_TIME, commit);
|
||||
header.put(HeaderMetadataType.COMMAND_BLOCK_TYPE, String.valueOf(HoodieCommandBlockTypeEnum.ROLLBACK_PREVIOUS_BLOCK
|
||||
.ordinal()));
|
||||
return header;
|
||||
}
|
||||
|
||||
private HoodieRollbackStat rollback(HoodieIndex hoodieIndex, String partitionPath, String commit,
|
||||
HoodieCommitMetadata commitMetadata, final Map<FileStatus, Boolean> filesToDeletedStatus,
|
||||
Map<FileStatus, Long> filesToNumBlocksRollback, Set<String> deletedFiles) {
|
||||
// wStat.getPrevCommit() might not give the right commit time in the following
|
||||
// scenario : If a compaction was scheduled, the new commitTime associated with the requested compaction will be
|
||||
// used to write the new log files. In this case, the commit time for the log file is the compaction requested time.
|
||||
// But the index (global) might store the baseCommit of the parquet and not the requested, hence get the
|
||||
// baseCommit always by listing the file slice
|
||||
Map<String, String> fileIdToBaseCommitTimeForLogMap = this.getRTFileSystemView().getLatestFileSlices(partitionPath)
|
||||
.collect(Collectors.toMap(FileSlice::getFileId, FileSlice::getBaseInstantTime));
|
||||
commitMetadata.getPartitionToWriteStats().get(partitionPath).stream()
|
||||
.filter(wStat -> {
|
||||
// Filter out stats without prevCommit since they are all inserts
|
||||
return wStat != null && wStat.getPrevCommit() != HoodieWriteStat.NULL_COMMIT && wStat.getPrevCommit() != null
|
||||
&& !deletedFiles.contains(wStat.getFileId());
|
||||
}).forEach(wStat -> {
|
||||
Writer writer = null;
|
||||
String baseCommitTime = fileIdToBaseCommitTimeForLogMap.get(wStat.getFileId());
|
||||
boolean success = false;
|
||||
try {
|
||||
writer = HoodieLogFormat.newWriterBuilder().onParentPath(
|
||||
FSUtils.getPartitionPath(this.getMetaClient().getBasePath(), partitionPath))
|
||||
.withFileId(wStat.getFileId()).overBaseCommit(baseCommitTime)
|
||||
.withFs(this.metaClient.getFs())
|
||||
.withFileExtension(HoodieLogFile.DELTA_EXTENSION).build();
|
||||
// generate metadata
|
||||
Map<HeaderMetadataType, String> header = generateHeader(commit);
|
||||
// if update belongs to an existing log file
|
||||
writer = writer.appendBlock(new HoodieCommandBlock(header));
|
||||
success = true;
|
||||
} catch (IOException | InterruptedException io) {
|
||||
throw new HoodieRollbackException(
|
||||
"Failed to rollback for commit " + commit, io);
|
||||
} finally {
|
||||
try {
|
||||
if (writer != null) {
|
||||
writer.close();
|
||||
}
|
||||
if (success) {
|
||||
// This step is intentionally done after writer is closed. Guarantees that
|
||||
// getFileStatus would reflect correct stats and FileNotFoundException is not thrown in
|
||||
// cloud-storage : HUDI-168
|
||||
filesToNumBlocksRollback.put(this.getMetaClient().getFs()
|
||||
.getFileStatus(writer.getLogFile().getPath()), 1L);
|
||||
}
|
||||
} catch (IOException io) {
|
||||
throw new UncheckedIOException(io);
|
||||
}
|
||||
}
|
||||
});
|
||||
return HoodieRollbackStat.newBuilder()
|
||||
.withPartitionPath(partitionPath)
|
||||
.withDeletedFileResults(filesToDeletedStatus)
|
||||
.withRollbackBlockAppendResults(filesToNumBlocksRollback).build();
|
||||
}
|
||||
|
||||
}
|
||||
430
hudi-client/src/main/java/org/apache/hudi/table/HoodieTable.java
Normal file
430
hudi-client/src/main/java/org/apache/hudi/table/HoodieTable.java
Normal file
@@ -0,0 +1,430 @@
|
||||
/*
|
||||
* 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.table;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.io.Serializable;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Iterator;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.concurrent.TimeoutException;
|
||||
import java.util.stream.Collectors;
|
||||
import java.util.stream.Stream;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hudi.WriteStatus;
|
||||
import org.apache.hudi.avro.model.HoodieCompactionPlan;
|
||||
import org.apache.hudi.avro.model.HoodieSavepointMetadata;
|
||||
import org.apache.hudi.client.utils.ClientUtils;
|
||||
import org.apache.hudi.common.HoodieCleanStat;
|
||||
import org.apache.hudi.common.HoodieRollbackStat;
|
||||
import org.apache.hudi.common.SerializableConfiguration;
|
||||
import org.apache.hudi.common.model.HoodieRecord;
|
||||
import org.apache.hudi.common.model.HoodieRecordPayload;
|
||||
import org.apache.hudi.common.model.HoodieWriteStat;
|
||||
import org.apache.hudi.common.table.HoodieTableMetaClient;
|
||||
import org.apache.hudi.common.table.HoodieTimeline;
|
||||
import org.apache.hudi.common.table.SyncableFileSystemView;
|
||||
import org.apache.hudi.common.table.TableFileSystemView;
|
||||
import org.apache.hudi.common.table.timeline.HoodieActiveTimeline;
|
||||
import org.apache.hudi.common.table.timeline.HoodieInstant;
|
||||
import org.apache.hudi.common.table.view.FileSystemViewManager;
|
||||
import org.apache.hudi.common.table.view.HoodieTableFileSystemView;
|
||||
import org.apache.hudi.common.util.AvroUtils;
|
||||
import org.apache.hudi.common.util.ConsistencyGuard;
|
||||
import org.apache.hudi.common.util.ConsistencyGuard.FileVisibility;
|
||||
import org.apache.hudi.common.util.FSUtils;
|
||||
import org.apache.hudi.common.util.FailSafeConsistencyGuard;
|
||||
import org.apache.hudi.common.util.collection.Pair;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
import org.apache.hudi.exception.HoodieException;
|
||||
import org.apache.hudi.exception.HoodieIOException;
|
||||
import org.apache.hudi.exception.HoodieSavepointException;
|
||||
import org.apache.hudi.index.HoodieIndex;
|
||||
import org.apache.log4j.LogManager;
|
||||
import org.apache.log4j.Logger;
|
||||
import org.apache.spark.Partitioner;
|
||||
import org.apache.spark.api.java.JavaRDD;
|
||||
import org.apache.spark.api.java.JavaSparkContext;
|
||||
|
||||
/**
|
||||
* Abstract implementation of a HoodieTable
|
||||
*/
|
||||
public abstract class HoodieTable<T extends HoodieRecordPayload> implements Serializable {
|
||||
|
||||
private static Logger logger = LogManager.getLogger(HoodieTable.class);
|
||||
|
||||
protected final HoodieWriteConfig config;
|
||||
protected final HoodieTableMetaClient metaClient;
|
||||
protected final HoodieIndex<T> index;
|
||||
|
||||
private SerializableConfiguration hadoopConfiguration;
|
||||
private transient FileSystemViewManager viewManager;
|
||||
|
||||
protected HoodieTable(HoodieWriteConfig config, JavaSparkContext jsc) {
|
||||
this.config = config;
|
||||
this.hadoopConfiguration = new SerializableConfiguration(jsc.hadoopConfiguration());
|
||||
this.viewManager = FileSystemViewManager.createViewManager(
|
||||
new SerializableConfiguration(jsc.hadoopConfiguration()), config.getViewStorageConfig());
|
||||
this.metaClient = ClientUtils.createMetaClient(jsc, config, true);
|
||||
this.index = HoodieIndex.createIndex(config, jsc);
|
||||
}
|
||||
|
||||
private synchronized FileSystemViewManager getViewManager() {
|
||||
if (null == viewManager) {
|
||||
viewManager = FileSystemViewManager.createViewManager(hadoopConfiguration,
|
||||
config.getViewStorageConfig());
|
||||
}
|
||||
return viewManager;
|
||||
}
|
||||
|
||||
public static <T extends HoodieRecordPayload> HoodieTable<T> getHoodieTable(
|
||||
HoodieTableMetaClient metaClient, HoodieWriteConfig config, JavaSparkContext jsc) {
|
||||
switch (metaClient.getTableType()) {
|
||||
case COPY_ON_WRITE:
|
||||
return new HoodieCopyOnWriteTable<>(config, jsc);
|
||||
case MERGE_ON_READ:
|
||||
return new HoodieMergeOnReadTable<>(config, jsc);
|
||||
default:
|
||||
throw new HoodieException("Unsupported table type :" + metaClient.getTableType());
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Provides a partitioner to perform the upsert operation, based on the workload profile
|
||||
*/
|
||||
public abstract Partitioner getUpsertPartitioner(WorkloadProfile profile);
|
||||
|
||||
/**
|
||||
* Provides a partitioner to perform the insert operation, based on the workload profile
|
||||
*/
|
||||
public abstract Partitioner getInsertPartitioner(WorkloadProfile profile);
|
||||
|
||||
/**
|
||||
* Return whether this HoodieTable implementation can benefit from workload profiling
|
||||
*/
|
||||
public abstract boolean isWorkloadProfileNeeded();
|
||||
|
||||
public HoodieWriteConfig getConfig() {
|
||||
return config;
|
||||
}
|
||||
|
||||
public HoodieTableMetaClient getMetaClient() {
|
||||
return metaClient;
|
||||
}
|
||||
|
||||
public Configuration getHadoopConf() {
|
||||
return metaClient.getHadoopConf();
|
||||
}
|
||||
|
||||
/**
|
||||
* Get the view of the file system for this table
|
||||
*/
|
||||
public TableFileSystemView getFileSystemView() {
|
||||
return new HoodieTableFileSystemView(metaClient, getCompletedCommitsTimeline());
|
||||
}
|
||||
|
||||
/**
|
||||
* Get the read optimized view of the file system for this table
|
||||
*/
|
||||
public TableFileSystemView.ReadOptimizedView getROFileSystemView() {
|
||||
return getViewManager().getFileSystemView(metaClient.getBasePath());
|
||||
}
|
||||
|
||||
/**
|
||||
* Get the real time view of the file system for this table
|
||||
*/
|
||||
public TableFileSystemView.RealtimeView getRTFileSystemView() {
|
||||
return getViewManager().getFileSystemView(metaClient.getBasePath());
|
||||
}
|
||||
|
||||
/**
|
||||
* Get complete view of the file system for this table with ability to force sync
|
||||
*/
|
||||
public SyncableFileSystemView getHoodieView() {
|
||||
return getViewManager().getFileSystemView(metaClient.getBasePath());
|
||||
}
|
||||
|
||||
/**
|
||||
* Get only the completed (no-inflights) commit + deltacommit timeline
|
||||
*/
|
||||
public HoodieTimeline getCompletedCommitsTimeline() {
|
||||
return metaClient.getCommitsTimeline().filterCompletedInstants();
|
||||
}
|
||||
|
||||
/**
|
||||
* Get only the completed (no-inflights) commit timeline
|
||||
*/
|
||||
public HoodieTimeline getCompletedCommitTimeline() {
|
||||
return metaClient.getCommitTimeline().filterCompletedInstants();
|
||||
}
|
||||
|
||||
/**
|
||||
* Get only the inflights (no-completed) commit timeline
|
||||
*/
|
||||
public HoodieTimeline getInflightCommitTimeline() {
|
||||
return metaClient.getCommitsTimeline().filterInflightsExcludingCompaction();
|
||||
}
|
||||
|
||||
/**
|
||||
* Get only the completed (no-inflights) clean timeline
|
||||
*/
|
||||
public HoodieTimeline getCompletedCleanTimeline() {
|
||||
return getActiveTimeline().getCleanerTimeline().filterCompletedInstants();
|
||||
}
|
||||
|
||||
/**
|
||||
* Get only the completed (no-inflights) savepoint timeline
|
||||
*/
|
||||
public HoodieTimeline getCompletedSavepointTimeline() {
|
||||
return getActiveTimeline().getSavePointTimeline().filterCompletedInstants();
|
||||
}
|
||||
|
||||
/**
|
||||
* Get the list of savepoints in this table
|
||||
*/
|
||||
public List<String> getSavepoints() {
|
||||
return getCompletedSavepointTimeline().getInstants().map(HoodieInstant::getTimestamp)
|
||||
.collect(Collectors.toList());
|
||||
}
|
||||
|
||||
/**
|
||||
* Get the list of data file names savepointed
|
||||
*/
|
||||
public Stream<String> getSavepointedDataFiles(String savepointTime) {
|
||||
if (!getSavepoints().contains(savepointTime)) {
|
||||
throw new HoodieSavepointException(
|
||||
"Could not get data files for savepoint " + savepointTime + ". No such savepoint.");
|
||||
}
|
||||
HoodieInstant instant = new HoodieInstant(false, HoodieTimeline.SAVEPOINT_ACTION,
|
||||
savepointTime);
|
||||
HoodieSavepointMetadata metadata = null;
|
||||
try {
|
||||
metadata = AvroUtils
|
||||
.deserializeHoodieSavepointMetadata(getActiveTimeline().getInstantDetails(instant).get());
|
||||
} catch (IOException e) {
|
||||
throw new HoodieSavepointException(
|
||||
"Could not get savepointed data files for savepoint " + savepointTime, e);
|
||||
}
|
||||
return metadata.getPartitionMetadata().values().stream()
|
||||
.flatMap(s -> s.getSavepointDataFile().stream());
|
||||
}
|
||||
|
||||
public HoodieActiveTimeline getActiveTimeline() {
|
||||
return metaClient.getActiveTimeline();
|
||||
}
|
||||
|
||||
/**
|
||||
* Return the index
|
||||
*/
|
||||
public HoodieIndex<T> getIndex() {
|
||||
return index;
|
||||
}
|
||||
|
||||
/**
|
||||
* Perform the ultimate IO for a given upserted (RDD) partition
|
||||
*/
|
||||
public abstract Iterator<List<WriteStatus>> handleUpsertPartition(String commitTime,
|
||||
Integer partition, Iterator<HoodieRecord<T>> recordIterator, Partitioner partitioner);
|
||||
|
||||
/**
|
||||
* Perform the ultimate IO for a given inserted (RDD) partition
|
||||
*/
|
||||
public abstract Iterator<List<WriteStatus>> handleInsertPartition(String commitTime,
|
||||
Integer partition, Iterator<HoodieRecord<T>> recordIterator, Partitioner partitioner);
|
||||
|
||||
/**
|
||||
* Schedule compaction for the instant time
|
||||
* @param jsc Spark Context
|
||||
* @param instantTime Instant Time for scheduling compaction
|
||||
* @return
|
||||
*/
|
||||
public abstract HoodieCompactionPlan scheduleCompaction(JavaSparkContext jsc, String instantTime);
|
||||
|
||||
/**
|
||||
* Run Compaction on the table. Compaction arranges the data so that it is optimized for data
|
||||
* access
|
||||
*
|
||||
* @param jsc Spark Context
|
||||
* @param compactionInstantTime Instant Time
|
||||
* @param compactionPlan Compaction Plan
|
||||
*/
|
||||
public abstract JavaRDD<WriteStatus> compact(JavaSparkContext jsc, String compactionInstantTime,
|
||||
HoodieCompactionPlan compactionPlan);
|
||||
|
||||
/**
|
||||
* Clean partition paths according to cleaning policy and returns the number of files cleaned.
|
||||
*/
|
||||
public abstract List<HoodieCleanStat> clean(JavaSparkContext jsc);
|
||||
|
||||
/**
|
||||
* Rollback the (inflight/committed) record changes with the given commit time. Four steps: (1)
|
||||
* Atomically unpublish this commit (2) clean indexing data (3) clean new generated parquet files
|
||||
* / log blocks (4) Finally, delete .<action>.commit or .<action>.inflight file if deleteInstants = true
|
||||
*/
|
||||
public abstract List<HoodieRollbackStat> rollback(JavaSparkContext jsc, String commit, boolean deleteInstants)
|
||||
throws IOException;
|
||||
|
||||
/**
|
||||
* Finalize the written data onto storage. Perform any final cleanups
|
||||
*
|
||||
* @param jsc Spark Context
|
||||
* @param stats List of HoodieWriteStats
|
||||
* @throws HoodieIOException if some paths can't be finalized on storage
|
||||
*/
|
||||
public void finalizeWrite(JavaSparkContext jsc, String instantTs, List<HoodieWriteStat> stats)
|
||||
throws HoodieIOException {
|
||||
cleanFailedWrites(jsc, instantTs, stats, config.getConsistencyGuardConfig().isConsistencyCheckEnabled());
|
||||
}
|
||||
|
||||
/**
|
||||
* Delete Marker directory corresponding to an instant
|
||||
* @param instantTs Instant Time
|
||||
*/
|
||||
protected void deleteMarkerDir(String instantTs) {
|
||||
try {
|
||||
FileSystem fs = getMetaClient().getFs();
|
||||
Path markerDir = new Path(metaClient.getMarkerFolderPath(instantTs));
|
||||
if (fs.exists(markerDir)) {
|
||||
// For append only case, we do not write to marker dir. Hence, the above check
|
||||
logger.info("Removing marker directory=" + markerDir);
|
||||
fs.delete(markerDir, true);
|
||||
}
|
||||
} catch (IOException ioe) {
|
||||
throw new HoodieIOException(ioe.getMessage(), ioe);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Reconciles WriteStats and marker files to detect and safely delete duplicate data files created because of Spark
|
||||
* retries.
|
||||
*
|
||||
* @param jsc Spark Context
|
||||
* @param instantTs Instant Timestamp
|
||||
* @param stats Hoodie Write Stat
|
||||
* @param consistencyCheckEnabled Consistency Check Enabled
|
||||
* @throws HoodieIOException
|
||||
*/
|
||||
protected void cleanFailedWrites(JavaSparkContext jsc, String instantTs, List<HoodieWriteStat> stats,
|
||||
boolean consistencyCheckEnabled) throws HoodieIOException {
|
||||
try {
|
||||
// Reconcile marker and data files with WriteStats so that partially written data-files due to failed
|
||||
// (but succeeded on retry) tasks are removed.
|
||||
String basePath = getMetaClient().getBasePath();
|
||||
FileSystem fs = getMetaClient().getFs();
|
||||
Path markerDir = new Path(metaClient.getMarkerFolderPath(instantTs));
|
||||
|
||||
if (!fs.exists(markerDir)) {
|
||||
// Happens when all writes are appends
|
||||
return;
|
||||
}
|
||||
|
||||
List<String> invalidDataPaths = FSUtils.getAllDataFilesForMarkers(fs, basePath, instantTs, markerDir.toString());
|
||||
List<String> validDataPaths = stats.stream().map(w -> String.format("%s/%s", basePath, w.getPath()))
|
||||
.filter(p -> p.endsWith(".parquet")).collect(Collectors.toList());
|
||||
// Contains list of partially created files. These needs to be cleaned up.
|
||||
invalidDataPaths.removeAll(validDataPaths);
|
||||
if (!invalidDataPaths.isEmpty()) {
|
||||
logger.info("Removing duplicate data files created due to spark retries before committing. Paths="
|
||||
+ invalidDataPaths);
|
||||
}
|
||||
|
||||
Map<String, List<Pair<String, String>>> groupByPartition = invalidDataPaths.stream()
|
||||
.map(dp -> Pair.of(new Path(dp).getParent().toString(), dp))
|
||||
.collect(Collectors.groupingBy(Pair::getKey));
|
||||
|
||||
if (!groupByPartition.isEmpty()) {
|
||||
// Ensure all files in delete list is actually present. This is mandatory for an eventually consistent FS.
|
||||
// Otherwise, we may miss deleting such files. If files are not found even after retries, fail the commit
|
||||
if (consistencyCheckEnabled) {
|
||||
// This will either ensure all files to be deleted are present.
|
||||
waitForAllFiles(jsc, groupByPartition, FileVisibility.APPEAR);
|
||||
}
|
||||
|
||||
// Now delete partially written files
|
||||
jsc.parallelize(new ArrayList<>(groupByPartition.values()), config.getFinalizeWriteParallelism())
|
||||
.map(partitionWithFileList -> {
|
||||
final FileSystem fileSystem = metaClient.getFs();
|
||||
logger.info("Deleting invalid data files=" + partitionWithFileList);
|
||||
if (partitionWithFileList.isEmpty()) {
|
||||
return true;
|
||||
}
|
||||
// Delete
|
||||
partitionWithFileList.stream().map(Pair::getValue).forEach(file -> {
|
||||
try {
|
||||
fileSystem.delete(new Path(file), false);
|
||||
} catch (IOException e) {
|
||||
throw new HoodieIOException(e.getMessage(), e);
|
||||
}
|
||||
});
|
||||
|
||||
return true;
|
||||
}).collect();
|
||||
|
||||
// Now ensure the deleted files disappear
|
||||
if (consistencyCheckEnabled) {
|
||||
// This will either ensure all files to be deleted are absent.
|
||||
waitForAllFiles(jsc, groupByPartition, FileVisibility.DISAPPEAR);
|
||||
}
|
||||
}
|
||||
// Now delete the marker directory
|
||||
deleteMarkerDir(instantTs);
|
||||
} catch (IOException ioe) {
|
||||
throw new HoodieIOException(ioe.getMessage(), ioe);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Ensures all files passed either appear or disappear
|
||||
* @param jsc JavaSparkContext
|
||||
* @param groupByPartition Files grouped by partition
|
||||
* @param visibility Appear/Disappear
|
||||
*/
|
||||
private void waitForAllFiles(JavaSparkContext jsc, Map<String, List<Pair<String, String>>> groupByPartition,
|
||||
FileVisibility visibility) {
|
||||
// This will either ensure all files to be deleted are present.
|
||||
boolean checkPassed =
|
||||
jsc.parallelize(new ArrayList<>(groupByPartition.entrySet()), config.getFinalizeWriteParallelism())
|
||||
.map(partitionWithFileList -> waitForCondition(partitionWithFileList.getKey(),
|
||||
partitionWithFileList.getValue().stream(), visibility))
|
||||
.collect().stream().allMatch(x -> x);
|
||||
if (!checkPassed) {
|
||||
throw new HoodieIOException("Consistency check failed to ensure all files " + visibility);
|
||||
}
|
||||
}
|
||||
|
||||
private boolean waitForCondition(String partitionPath, Stream<Pair<String, String>> partitionFilePaths,
|
||||
FileVisibility visibility) {
|
||||
final FileSystem fileSystem = metaClient.getRawFs();
|
||||
List<String> fileList = partitionFilePaths.map(Pair::getValue).collect(Collectors.toList());
|
||||
try {
|
||||
getFailSafeConsistencyGuard(fileSystem).waitTill(partitionPath, fileList, visibility);
|
||||
} catch (IOException | TimeoutException ioe) {
|
||||
logger.error("Got exception while waiting for files to show up", ioe);
|
||||
return false;
|
||||
}
|
||||
return true;
|
||||
}
|
||||
|
||||
private ConsistencyGuard getFailSafeConsistencyGuard(FileSystem fileSystem) {
|
||||
return new FailSafeConsistencyGuard(fileSystem, config.getConsistencyGuardConfig());
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,35 @@
|
||||
/*
|
||||
* 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.table;
|
||||
|
||||
import org.apache.hudi.common.model.HoodieRecord;
|
||||
import org.apache.hudi.common.model.HoodieRecordPayload;
|
||||
import org.apache.spark.api.java.JavaRDD;
|
||||
|
||||
/**
|
||||
* Repartition input records into at least expected number of output spark partitions. It should
|
||||
* give below guarantees - Output spark partition will have records from only one hoodie partition.
|
||||
* - Average records per output spark partitions should be almost equal to (#inputRecords /
|
||||
* #outputSparkPartitions) to avoid possible skews.
|
||||
*/
|
||||
public interface UserDefinedBulkInsertPartitioner<T extends HoodieRecordPayload> {
|
||||
|
||||
JavaRDD<HoodieRecord<T>> repartitionRecords(JavaRDD<HoodieRecord<T>> records,
|
||||
int outputSparkPartitions);
|
||||
}
|
||||
@@ -0,0 +1,110 @@
|
||||
/*
|
||||
* 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.table;
|
||||
|
||||
import java.io.Serializable;
|
||||
import java.util.HashMap;
|
||||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
import org.apache.hudi.common.model.HoodieRecord;
|
||||
import org.apache.hudi.common.model.HoodieRecordLocation;
|
||||
import org.apache.hudi.common.model.HoodieRecordPayload;
|
||||
import org.apache.hudi.common.util.Option;
|
||||
import org.apache.spark.api.java.JavaRDD;
|
||||
import scala.Tuple2;
|
||||
|
||||
/**
|
||||
* Information about incoming records for upsert/insert obtained either via sampling or
|
||||
* introspecting the data fully
|
||||
* <p>
|
||||
* TODO(vc): Think about obtaining this directly from index.tagLocation
|
||||
*/
|
||||
public class WorkloadProfile<T extends HoodieRecordPayload> implements Serializable {
|
||||
|
||||
/**
|
||||
* Input workload
|
||||
*/
|
||||
private final JavaRDD<HoodieRecord<T>> taggedRecords;
|
||||
|
||||
/**
|
||||
* Computed workload profile
|
||||
*/
|
||||
private final HashMap<String, WorkloadStat> partitionPathStatMap;
|
||||
|
||||
|
||||
private final WorkloadStat globalStat;
|
||||
|
||||
|
||||
public WorkloadProfile(JavaRDD<HoodieRecord<T>> taggedRecords) {
|
||||
this.taggedRecords = taggedRecords;
|
||||
this.partitionPathStatMap = new HashMap<>();
|
||||
this.globalStat = new WorkloadStat();
|
||||
buildProfile();
|
||||
}
|
||||
|
||||
private void buildProfile() {
|
||||
|
||||
Map<Tuple2<String, Option<HoodieRecordLocation>>, Long> partitionLocationCounts = taggedRecords
|
||||
.mapToPair(record -> new Tuple2<>(
|
||||
new Tuple2<>(record.getPartitionPath(), Option.ofNullable(record.getCurrentLocation())),
|
||||
record)).countByKey();
|
||||
|
||||
for (Map.Entry<Tuple2<String, Option<HoodieRecordLocation>>, Long> e : partitionLocationCounts
|
||||
.entrySet()) {
|
||||
String partitionPath = e.getKey()._1();
|
||||
Long count = e.getValue();
|
||||
Option<HoodieRecordLocation> locOption = e.getKey()._2();
|
||||
|
||||
if (!partitionPathStatMap.containsKey(partitionPath)) {
|
||||
partitionPathStatMap.put(partitionPath, new WorkloadStat());
|
||||
}
|
||||
|
||||
if (locOption.isPresent()) {
|
||||
// update
|
||||
partitionPathStatMap.get(partitionPath).addUpdates(locOption.get(), count);
|
||||
globalStat.addUpdates(locOption.get(), count);
|
||||
} else {
|
||||
// insert
|
||||
partitionPathStatMap.get(partitionPath).addInserts(count);
|
||||
globalStat.addInserts(count);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
public WorkloadStat getGlobalStat() {
|
||||
return globalStat;
|
||||
}
|
||||
|
||||
public Set<String> getPartitionPaths() {
|
||||
return partitionPathStatMap.keySet();
|
||||
}
|
||||
|
||||
public WorkloadStat getWorkloadStat(String partitionPath) {
|
||||
return partitionPathStatMap.get(partitionPath);
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
final StringBuilder sb = new StringBuilder("WorkloadProfile {");
|
||||
sb.append("globalStat=").append(globalStat).append(", ");
|
||||
sb.append("partitionStat=").append(partitionPathStatMap);
|
||||
sb.append('}');
|
||||
return sb.toString();
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,70 @@
|
||||
/*
|
||||
* 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.table;
|
||||
|
||||
import java.io.Serializable;
|
||||
import java.util.HashMap;
|
||||
import org.apache.hudi.common.model.HoodieRecordLocation;
|
||||
import org.apache.hudi.common.util.collection.Pair;
|
||||
|
||||
/**
|
||||
* Wraps stats about a single partition path.
|
||||
*/
|
||||
public class WorkloadStat implements Serializable {
|
||||
|
||||
private long numInserts = 0L;
|
||||
|
||||
private long numUpdates = 0L;
|
||||
|
||||
private HashMap<String, Pair<String, Long>> updateLocationToCount;
|
||||
|
||||
public WorkloadStat() {
|
||||
updateLocationToCount = new HashMap<>();
|
||||
}
|
||||
|
||||
long addInserts(long numInserts) {
|
||||
return this.numInserts += numInserts;
|
||||
}
|
||||
|
||||
long addUpdates(HoodieRecordLocation location, long numUpdates) {
|
||||
updateLocationToCount.put(location.getFileId(), Pair.of(location.getInstantTime(), numUpdates));
|
||||
return this.numUpdates += numUpdates;
|
||||
}
|
||||
|
||||
public long getNumUpdates() {
|
||||
return numUpdates;
|
||||
}
|
||||
|
||||
public long getNumInserts() {
|
||||
return numInserts;
|
||||
}
|
||||
|
||||
public HashMap<String, Pair<String, Long>> getUpdateLocationToCount() {
|
||||
return updateLocationToCount;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
final StringBuilder sb = new StringBuilder("WorkloadStat {");
|
||||
sb.append("numInserts=").append(numInserts).append(", ");
|
||||
sb.append("numUpdates=").append(numUpdates);
|
||||
sb.append('}');
|
||||
return sb.toString();
|
||||
}
|
||||
}
|
||||
614
hudi-client/src/main/resources/META-INF/LICENSE.txt
Normal file
614
hudi-client/src/main/resources/META-INF/LICENSE.txt
Normal file
@@ -0,0 +1,614 @@
|
||||
|
||||
Apache License
|
||||
Version 2.0, January 2004
|
||||
http://www.apache.org/licenses/
|
||||
|
||||
TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION
|
||||
|
||||
1. Definitions.
|
||||
|
||||
"License" shall mean the terms and conditions for use, reproduction,
|
||||
and distribution as defined by Sections 1 through 9 of this document.
|
||||
|
||||
"Licensor" shall mean the copyright owner or entity authorized by
|
||||
the copyright owner that is granting the License.
|
||||
|
||||
"Legal Entity" shall mean the union of the acting entity and all
|
||||
other entities that control, are controlled by, or are under common
|
||||
control with that entity. For the purposes of this definition,
|
||||
"control" means (i) the power, direct or indirect, to cause the
|
||||
direction or management of such entity, whether by contract or
|
||||
otherwise, or (ii) ownership of fifty percent (50%) or more of the
|
||||
outstanding shares, or (iii) beneficial ownership of such entity.
|
||||
|
||||
"You" (or "Your") shall mean an individual or Legal Entity
|
||||
exercising permissions granted by this License.
|
||||
|
||||
"Source" form shall mean the preferred form for making modifications,
|
||||
including but not limited to software source code, documentation
|
||||
source, and configuration files.
|
||||
|
||||
"Object" form shall mean any form resulting from mechanical
|
||||
transformation or translation of a Source form, including but
|
||||
not limited to compiled object code, generated documentation,
|
||||
and conversions to other media types.
|
||||
|
||||
"Work" shall mean the work of authorship, whether in Source or
|
||||
Object form, made available under the License, as indicated by a
|
||||
copyright notice that is included in or attached to the work
|
||||
(an example is provided in the Appendix below).
|
||||
|
||||
"Derivative Works" shall mean any work, whether in Source or Object
|
||||
form, that is based on (or derived from) the Work and for which the
|
||||
editorial revisions, annotations, elaborations, or other modifications
|
||||
represent, as a whole, an original work of authorship. For the purposes
|
||||
of this License, Derivative Works shall not include works that remain
|
||||
separable from, or merely link (or bind by name) to the interfaces of,
|
||||
the Work and Derivative Works thereof.
|
||||
|
||||
"Contribution" shall mean any work of authorship, including
|
||||
the original version of the Work and any modifications or additions
|
||||
to that Work or Derivative Works thereof, that is intentionally
|
||||
submitted to Licensor for inclusion in the Work by the copyright owner
|
||||
or by an individual or Legal Entity authorized to submit on behalf of
|
||||
the copyright owner. For the purposes of this definition, "submitted"
|
||||
means any form of electronic, verbal, or written communication sent
|
||||
to the Licensor or its representatives, including but not limited to
|
||||
communication on electronic mailing lists, source code control systems,
|
||||
and issue tracking systems that are managed by, or on behalf of, the
|
||||
Licensor for the purpose of discussing and improving the Work, but
|
||||
excluding communication that is conspicuously marked or otherwise
|
||||
designated in writing by the copyright owner as "Not a Contribution."
|
||||
|
||||
"Contributor" shall mean Licensor and any individual or Legal Entity
|
||||
on behalf of whom a Contribution has been received by Licensor and
|
||||
subsequently incorporated within the Work.
|
||||
|
||||
2. Grant of Copyright License. Subject to the terms and conditions of
|
||||
this License, each Contributor hereby grants to You a perpetual,
|
||||
worldwide, non-exclusive, no-charge, royalty-free, irrevocable
|
||||
copyright license to reproduce, prepare Derivative Works of,
|
||||
publicly display, publicly perform, sublicense, and distribute the
|
||||
Work and such Derivative Works in Source or Object form.
|
||||
|
||||
3. Grant of Patent License. Subject to the terms and conditions of
|
||||
this License, each Contributor hereby grants to You a perpetual,
|
||||
worldwide, non-exclusive, no-charge, royalty-free, irrevocable
|
||||
(except as stated in this section) patent license to make, have made,
|
||||
use, offer to sell, sell, import, and otherwise transfer the Work,
|
||||
where such license applies only to those patent claims licensable
|
||||
by such Contributor that are necessarily infringed by their
|
||||
Contribution(s) alone or by combination of their Contribution(s)
|
||||
with the Work to which such Contribution(s) was submitted. If You
|
||||
institute patent litigation against any entity (including a
|
||||
cross-claim or counterclaim in a lawsuit) alleging that the Work
|
||||
or a Contribution incorporated within the Work constitutes direct
|
||||
or contributory patent infringement, then any patent licenses
|
||||
granted to You under this License for that Work shall terminate
|
||||
as of the date such litigation is filed.
|
||||
|
||||
4. Redistribution. You may reproduce and distribute copies of the
|
||||
Work or Derivative Works thereof in any medium, with or without
|
||||
modifications, and in Source or Object form, provided that You
|
||||
meet the following conditions:
|
||||
|
||||
(a) You must give any other recipients of the Work or
|
||||
Derivative Works a copy of this License; and
|
||||
|
||||
(b) You must cause any modified files to carry prominent notices
|
||||
stating that You changed the files; and
|
||||
|
||||
(c) You must retain, in the Source form of any Derivative Works
|
||||
that You distribute, all copyright, patent, trademark, and
|
||||
attribution notices from the Source form of the Work,
|
||||
excluding those notices that do not pertain to any part of
|
||||
the Derivative Works; and
|
||||
|
||||
(d) If the Work includes a "NOTICE" text file as part of its
|
||||
distribution, then any Derivative Works that You distribute must
|
||||
include a readable copy of the attribution notices contained
|
||||
within such NOTICE file, excluding those notices that do not
|
||||
pertain to any part of the Derivative Works, in at least one
|
||||
of the following places: within a NOTICE text file distributed
|
||||
as part of the Derivative Works; within the Source form or
|
||||
documentation, if provided along with the Derivative Works; or,
|
||||
within a display generated by the Derivative Works, if and
|
||||
wherever such third-party notices normally appear. The contents
|
||||
of the NOTICE file are for informational purposes only and
|
||||
do not modify the License. You may add Your own attribution
|
||||
notices within Derivative Works that You distribute, alongside
|
||||
or as an addendum to the NOTICE text from the Work, provided
|
||||
that such additional attribution notices cannot be construed
|
||||
as modifying the License.
|
||||
|
||||
You may add Your own copyright statement to Your modifications and
|
||||
may provide additional or different license terms and conditions
|
||||
for use, reproduction, or distribution of Your modifications, or
|
||||
for any such Derivative Works as a whole, provided Your use,
|
||||
reproduction, and distribution of the Work otherwise complies with
|
||||
the conditions stated in this License.
|
||||
|
||||
5. Submission of Contributions. Unless You explicitly state otherwise,
|
||||
any Contribution intentionally submitted for inclusion in the Work
|
||||
by You to the Licensor shall be under the terms and conditions of
|
||||
this License, without any additional terms or conditions.
|
||||
Notwithstanding the above, nothing herein shall supersede or modify
|
||||
the terms of any separate license agreement you may have executed
|
||||
with Licensor regarding such Contributions.
|
||||
|
||||
6. Trademarks. This License does not grant permission to use the trade
|
||||
names, trademarks, service marks, or product names of the Licensor,
|
||||
except as required for reasonable and customary use in describing the
|
||||
origin of the Work and reproducing the content of the NOTICE file.
|
||||
|
||||
7. Disclaimer of Warranty. Unless required by applicable law or
|
||||
agreed to in writing, Licensor provides the Work (and each
|
||||
Contributor provides its Contributions) on an "AS IS" BASIS,
|
||||
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or
|
||||
implied, including, without limitation, any warranties or conditions
|
||||
of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A
|
||||
PARTICULAR PURPOSE. You are solely responsible for determining the
|
||||
appropriateness of using or redistributing the Work and assume any
|
||||
risks associated with Your exercise of permissions under this License.
|
||||
|
||||
8. Limitation of Liability. In no event and under no legal theory,
|
||||
whether in tort (including negligence), contract, or otherwise,
|
||||
unless required by applicable law (such as deliberate and grossly
|
||||
negligent acts) or agreed to in writing, shall any Contributor be
|
||||
liable to You for damages, including any direct, indirect, special,
|
||||
incidental, or consequential damages of any character arising as a
|
||||
result of this License or out of the use or inability to use the
|
||||
Work (including but not limited to damages for loss of goodwill,
|
||||
work stoppage, computer failure or malfunction, or any and all
|
||||
other commercial damages or losses), even if such Contributor
|
||||
has been advised of the possibility of such damages.
|
||||
|
||||
9. Accepting Warranty or Additional Liability. While redistributing
|
||||
the Work or Derivative Works thereof, You may choose to offer,
|
||||
and charge a fee for, acceptance of support, warranty, indemnity,
|
||||
or other liability obligations and/or rights consistent with this
|
||||
License. However, in accepting such obligations, You may act only
|
||||
on Your own behalf and on Your sole responsibility, not on behalf
|
||||
of any other Contributor, and only if You agree to indemnify,
|
||||
defend, and hold each Contributor harmless for any liability
|
||||
incurred by, or claims asserted against, such Contributor by reason
|
||||
of your accepting any such warranty or additional liability.
|
||||
|
||||
END OF TERMS AND CONDITIONS
|
||||
|
||||
APPENDIX: How to apply the Apache License to your work.
|
||||
|
||||
To apply the Apache License to your work, attach the following
|
||||
boilerplate notice, with the fields enclosed by brackets "[]"
|
||||
replaced with your own identifying information. (Don't include
|
||||
the brackets!) The text should be enclosed in the appropriate
|
||||
comment syntax for the file format. We also recommend that a
|
||||
file or class name and description of purpose be included on the
|
||||
same "printed page" as the copyright notice for easier
|
||||
identification within third-party archives.
|
||||
|
||||
Copyright [yyyy] [name of copyright owner]
|
||||
|
||||
Licensed 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.
|
||||
|
||||
----
|
||||
This project bundles portions of the 'JQuery' project under the terms of the MIT license.
|
||||
|
||||
Copyright 2012 jQuery Foundation and other contributors
|
||||
http://jquery.com/
|
||||
|
||||
Permission is hereby granted, free of charge, to any person obtaining
|
||||
a copy of this software and associated documentation files (the
|
||||
"Software"), to deal in the Software without restriction, including
|
||||
without limitation the rights to use, copy, modify, merge, publish,
|
||||
distribute, sublicense, and/or sell copies of the Software, and to
|
||||
permit persons to whom the Software is furnished to do so, subject to
|
||||
the following conditions:
|
||||
|
||||
The above copyright notice and this permission notice shall be
|
||||
included in all copies or substantial portions of the Software.
|
||||
|
||||
THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND,
|
||||
EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF
|
||||
MERCHANTABILITY, FITNESS FOR A PARTICULAR PURPOSE AND
|
||||
NONINFRINGEMENT. IN NO EVENT SHALL THE AUTHORS OR COPYRIGHT HOLDERS BE
|
||||
LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER IN AN ACTION
|
||||
OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN CONNECTION
|
||||
WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE SOFTWARE.
|
||||
|
||||
----
|
||||
This project bundles a derivative of portions of the 'Asciidoctor' project
|
||||
under the terms of the MIT license.
|
||||
|
||||
The MIT License
|
||||
Copyright (C) 2012-2015 Dan Allen, Ryan Waldron and the Asciidoctor Project
|
||||
|
||||
Permission is hereby granted, free of charge, to any person obtaining a copy
|
||||
of this software and associated documentation files (the "Software"), to deal
|
||||
in the Software without restriction, including without limitation the rights
|
||||
to use, copy, modify, merge, publish, distribute, sublicense, and/or sell
|
||||
copies of the Software, and to permit persons to whom the Software is
|
||||
furnished to do so, subject to the following conditions:
|
||||
|
||||
The above copyright notice and this permission notice shall be included in
|
||||
all copies or substantial portions of the Software.
|
||||
|
||||
THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR
|
||||
IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY,
|
||||
FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
|
||||
AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER
|
||||
LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM,
|
||||
OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN
|
||||
THE SOFTWARE.
|
||||
|
||||
----
|
||||
This project incorporates portions of the 'Protocol Buffers' project avaialble
|
||||
under a '3-clause BSD' license.
|
||||
|
||||
Copyright 2008, Google Inc.
|
||||
All rights reserved.
|
||||
|
||||
Redistribution and use in source and binary forms, with or without
|
||||
modification, are permitted provided that the following conditions are
|
||||
met:
|
||||
|
||||
* Redistributions of source code must retain the above copyright
|
||||
notice, this list of conditions and the following disclaimer.
|
||||
* Redistributions in binary form must reproduce the above
|
||||
copyright notice, this list of conditions and the following disclaimer
|
||||
in the documentation and/or other materials provided with the
|
||||
distribution.
|
||||
* Neither the name of Google Inc. nor the names of its
|
||||
contributors may be used to endorse or promote products derived from
|
||||
this software without specific prior written permission.
|
||||
|
||||
THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS
|
||||
"AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT
|
||||
LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR
|
||||
A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT
|
||||
OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL,
|
||||
SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT
|
||||
LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE,
|
||||
DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY
|
||||
THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
|
||||
(INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE
|
||||
OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
|
||||
|
||||
Code generated by the Protocol Buffer compiler is owned by the owner
|
||||
of the input file used when generating it. This code is not
|
||||
standalone and requires a support library to be linked with it. This
|
||||
support library is itself covered by the above license.
|
||||
|
||||
----
|
||||
This project bundles a derivative image for our Orca Logo. This image is
|
||||
available under the Creative Commons By Attribution 3.0 License.
|
||||
|
||||
Creative Commons Legal Code
|
||||
|
||||
Attribution 3.0 Unported
|
||||
|
||||
CREATIVE COMMONS CORPORATION IS NOT A LAW FIRM AND DOES NOT PROVIDE
|
||||
LEGAL SERVICES. DISTRIBUTION OF THIS LICENSE DOES NOT CREATE AN
|
||||
ATTORNEY-CLIENT RELATIONSHIP. CREATIVE COMMONS PROVIDES THIS
|
||||
INFORMATION ON AN "AS-IS" BASIS. CREATIVE COMMONS MAKES NO WARRANTIES
|
||||
REGARDING THE INFORMATION PROVIDED, AND DISCLAIMS LIABILITY FOR
|
||||
DAMAGES RESULTING FROM ITS USE.
|
||||
|
||||
License
|
||||
|
||||
THE WORK (AS DEFINED BELOW) IS PROVIDED UNDER THE TERMS OF THIS CREATIVE
|
||||
COMMONS PUBLIC LICENSE ("CCPL" OR "LICENSE"). THE WORK IS PROTECTED BY
|
||||
COPYRIGHT AND/OR OTHER APPLICABLE LAW. ANY USE OF THE WORK OTHER THAN AS
|
||||
AUTHORIZED UNDER THIS LICENSE OR COPYRIGHT LAW IS PROHIBITED.
|
||||
|
||||
BY EXERCISING ANY RIGHTS TO THE WORK PROVIDED HERE, YOU ACCEPT AND AGREE
|
||||
TO BE BOUND BY THE TERMS OF THIS LICENSE. TO THE EXTENT THIS LICENSE MAY
|
||||
BE CONSIDERED TO BE A CONTRACT, THE LICENSOR GRANTS YOU THE RIGHTS
|
||||
CONTAINED HERE IN CONSIDERATION OF YOUR ACCEPTANCE OF SUCH TERMS AND
|
||||
CONDITIONS.
|
||||
|
||||
1. Definitions
|
||||
|
||||
a. "Adaptation" means a work based upon the Work, or upon the Work and
|
||||
other pre-existing works, such as a translation, adaptation,
|
||||
derivative work, arrangement of music or other alterations of a
|
||||
literary or artistic work, or phonogram or performance and includes
|
||||
cinematographic adaptations or any other form in which the Work may be
|
||||
recast, transformed, or adapted including in any form recognizably
|
||||
derived from the original, except that a work that constitutes a
|
||||
Collection will not be considered an Adaptation for the purpose of
|
||||
this License. For the avoidance of doubt, where the Work is a musical
|
||||
work, performance or phonogram, the synchronization of the Work in
|
||||
timed-relation with a moving image ("synching") will be considered an
|
||||
Adaptation for the purpose of this License.
|
||||
b. "Collection" means a collection of literary or artistic works, such as
|
||||
encyclopedias and anthologies, or performances, phonograms or
|
||||
broadcasts, or other works or subject matter other than works listed
|
||||
in Section 1(f) below, which, by reason of the selection and
|
||||
arrangement of their contents, constitute intellectual creations, in
|
||||
which the Work is included in its entirety in unmodified form along
|
||||
with one or more other contributions, each constituting separate and
|
||||
independent works in themselves, which together are assembled into a
|
||||
collective whole. A work that constitutes a Collection will not be
|
||||
considered an Adaptation (as defined above) for the purposes of this
|
||||
License.
|
||||
c. "Distribute" means to make available to the public the original and
|
||||
copies of the Work or Adaptation, as appropriate, through sale or
|
||||
other transfer of ownership.
|
||||
d. "Licensor" means the individual, individuals, entity or entities that
|
||||
offer(s) the Work under the terms of this License.
|
||||
e. "Original Author" means, in the case of a literary or artistic work,
|
||||
the individual, individuals, entity or entities who created the Work
|
||||
or if no individual or entity can be identified, the publisher; and in
|
||||
addition (i) in the case of a performance the actors, singers,
|
||||
musicians, dancers, and other persons who act, sing, deliver, declaim,
|
||||
play in, interpret or otherwise perform literary or artistic works or
|
||||
expressions of folklore; (ii) in the case of a phonogram the producer
|
||||
being the person or legal entity who first fixes the sounds of a
|
||||
performance or other sounds; and, (iii) in the case of broadcasts, the
|
||||
organization that transmits the broadcast.
|
||||
f. "Work" means the literary and/or artistic work offered under the terms
|
||||
of this License including without limitation any production in the
|
||||
literary, scientific and artistic domain, whatever may be the mode or
|
||||
form of its expression including digital form, such as a book,
|
||||
pamphlet and other writing; a lecture, address, sermon or other work
|
||||
of the same nature; a dramatic or dramatico-musical work; a
|
||||
choreographic work or entertainment in dumb show; a musical
|
||||
composition with or without words; a cinematographic work to which are
|
||||
assimilated works expressed by a process analogous to cinematography;
|
||||
a work of drawing, painting, architecture, sculpture, engraving or
|
||||
lithography; a photographic work to which are assimilated works
|
||||
expressed by a process analogous to photography; a work of applied
|
||||
art; an illustration, map, plan, sketch or three-dimensional work
|
||||
relative to geography, topography, architecture or science; a
|
||||
performance; a broadcast; a phonogram; a compilation of data to the
|
||||
extent it is protected as a copyrightable work; or a work performed by
|
||||
a variety or circus performer to the extent it is not otherwise
|
||||
considered a literary or artistic work.
|
||||
g. "You" means an individual or entity exercising rights under this
|
||||
License who has not previously violated the terms of this License with
|
||||
respect to the Work, or who has received express permission from the
|
||||
Licensor to exercise rights under this License despite a previous
|
||||
violation.
|
||||
h. "Publicly Perform" means to perform public recitations of the Work and
|
||||
to communicate to the public those public recitations, by any means or
|
||||
process, including by wire or wireless means or public digital
|
||||
performances; to make available to the public Works in such a way that
|
||||
members of the public may access these Works from a place and at a
|
||||
place individually chosen by them; to perform the Work to the public
|
||||
by any means or process and the communication to the public of the
|
||||
performances of the Work, including by public digital performance; to
|
||||
broadcast and rebroadcast the Work by any means including signs,
|
||||
sounds or images.
|
||||
i. "Reproduce" means to make copies of the Work by any means including
|
||||
without limitation by sound or visual recordings and the right of
|
||||
fixation and reproducing fixations of the Work, including storage of a
|
||||
protected performance or phonogram in digital form or other electronic
|
||||
medium.
|
||||
|
||||
2. Fair Dealing Rights. Nothing in this License is intended to reduce,
|
||||
limit, or restrict any uses free from copyright or rights arising from
|
||||
limitations or exceptions that are provided for in connection with the
|
||||
copyright protection under copyright law or other applicable laws.
|
||||
|
||||
3. License Grant. Subject to the terms and conditions of this License,
|
||||
Licensor hereby grants You a worldwide, royalty-free, non-exclusive,
|
||||
perpetual (for the duration of the applicable copyright) license to
|
||||
exercise the rights in the Work as stated below:
|
||||
|
||||
a. to Reproduce the Work, to incorporate the Work into one or more
|
||||
Collections, and to Reproduce the Work as incorporated in the
|
||||
Collections;
|
||||
b. to create and Reproduce Adaptations provided that any such Adaptation,
|
||||
including any translation in any medium, takes reasonable steps to
|
||||
clearly label, demarcate or otherwise identify that changes were made
|
||||
to the original Work. For example, a translation could be marked "The
|
||||
original work was translated from English to Spanish," or a
|
||||
modification could indicate "The original work has been modified.";
|
||||
c. to Distribute and Publicly Perform the Work including as incorporated
|
||||
in Collections; and,
|
||||
d. to Distribute and Publicly Perform Adaptations.
|
||||
e. For the avoidance of doubt:
|
||||
|
||||
i. Non-waivable Compulsory License Schemes. In those jurisdictions in
|
||||
which the right to collect royalties through any statutory or
|
||||
compulsory licensing scheme cannot be waived, the Licensor
|
||||
reserves the exclusive right to collect such royalties for any
|
||||
exercise by You of the rights granted under this License;
|
||||
ii. Waivable Compulsory License Schemes. In those jurisdictions in
|
||||
which the right to collect royalties through any statutory or
|
||||
compulsory licensing scheme can be waived, the Licensor waives the
|
||||
exclusive right to collect such royalties for any exercise by You
|
||||
of the rights granted under this License; and,
|
||||
iii. Voluntary License Schemes. The Licensor waives the right to
|
||||
collect royalties, whether individually or, in the event that the
|
||||
Licensor is a member of a collecting society that administers
|
||||
voluntary licensing schemes, via that society, from any exercise
|
||||
by You of the rights granted under this License.
|
||||
|
||||
The above rights may be exercised in all media and formats whether now
|
||||
known or hereafter devised. The above rights include the right to make
|
||||
such modifications as are technically necessary to exercise the rights in
|
||||
other media and formats. Subject to Section 8(f), all rights not expressly
|
||||
granted by Licensor are hereby reserved.
|
||||
|
||||
4. Restrictions. The license granted in Section 3 above is expressly made
|
||||
subject to and limited by the following restrictions:
|
||||
|
||||
a. You may Distribute or Publicly Perform the Work only under the terms
|
||||
of this License. You must include a copy of, or the Uniform Resource
|
||||
Identifier (URI) for, this License with every copy of the Work You
|
||||
Distribute or Publicly Perform. You may not offer or impose any terms
|
||||
on the Work that restrict the terms of this License or the ability of
|
||||
the recipient of the Work to exercise the rights granted to that
|
||||
recipient under the terms of the License. You may not sublicense the
|
||||
Work. You must keep intact all notices that refer to this License and
|
||||
to the disclaimer of warranties with every copy of the Work You
|
||||
Distribute or Publicly Perform. When You Distribute or Publicly
|
||||
Perform the Work, You may not impose any effective technological
|
||||
measures on the Work that restrict the ability of a recipient of the
|
||||
Work from You to exercise the rights granted to that recipient under
|
||||
the terms of the License. This Section 4(a) applies to the Work as
|
||||
incorporated in a Collection, but this does not require the Collection
|
||||
apart from the Work itself to be made subject to the terms of this
|
||||
License. If You create a Collection, upon notice from any Licensor You
|
||||
must, to the extent practicable, remove from the Collection any credit
|
||||
as required by Section 4(b), as requested. If You create an
|
||||
Adaptation, upon notice from any Licensor You must, to the extent
|
||||
practicable, remove from the Adaptation any credit as required by
|
||||
Section 4(b), as requested.
|
||||
b. If You Distribute, or Publicly Perform the Work or any Adaptations or
|
||||
Collections, You must, unless a request has been made pursuant to
|
||||
Section 4(a), keep intact all copyright notices for the Work and
|
||||
provide, reasonable to the medium or means You are utilizing: (i) the
|
||||
name of the Original Author (or pseudonym, if applicable) if supplied,
|
||||
and/or if the Original Author and/or Licensor designate another party
|
||||
or parties (e.g., a sponsor institute, publishing entity, journal) for
|
||||
attribution ("Attribution Parties") in Licensor's copyright notice,
|
||||
terms of service or by other reasonable means, the name of such party
|
||||
or parties; (ii) the title of the Work if supplied; (iii) to the
|
||||
extent reasonably practicable, the URI, if any, that Licensor
|
||||
specifies to be associated with the Work, unless such URI does not
|
||||
refer to the copyright notice or licensing information for the Work;
|
||||
and (iv) , consistent with Section 3(b), in the case of an Adaptation,
|
||||
a credit identifying the use of the Work in the Adaptation (e.g.,
|
||||
"French translation of the Work by Original Author," or "Screenplay
|
||||
based on original Work by Original Author"). The credit required by
|
||||
this Section 4 (b) may be implemented in any reasonable manner;
|
||||
provided, however, that in the case of a Adaptation or Collection, at
|
||||
a minimum such credit will appear, if a credit for all contributing
|
||||
authors of the Adaptation or Collection appears, then as part of these
|
||||
credits and in a manner at least as prominent as the credits for the
|
||||
other contributing authors. For the avoidance of doubt, You may only
|
||||
use the credit required by this Section for the purpose of attribution
|
||||
in the manner set out above and, by exercising Your rights under this
|
||||
License, You may not implicitly or explicitly assert or imply any
|
||||
connection with, sponsorship or endorsement by the Original Author,
|
||||
Licensor and/or Attribution Parties, as appropriate, of You or Your
|
||||
use of the Work, without the separate, express prior written
|
||||
permission of the Original Author, Licensor and/or Attribution
|
||||
Parties.
|
||||
c. Except as otherwise agreed in writing by the Licensor or as may be
|
||||
otherwise permitted by applicable law, if You Reproduce, Distribute or
|
||||
Publicly Perform the Work either by itself or as part of any
|
||||
Adaptations or Collections, You must not distort, mutilate, modify or
|
||||
take other derogatory action in relation to the Work which would be
|
||||
prejudicial to the Original Author's honor or reputation. Licensor
|
||||
agrees that in those jurisdictions (e.g. Japan), in which any exercise
|
||||
of the right granted in Section 3(b) of this License (the right to
|
||||
make Adaptations) would be deemed to be a distortion, mutilation,
|
||||
modification or other derogatory action prejudicial to the Original
|
||||
Author's honor and reputation, the Licensor will waive or not assert,
|
||||
as appropriate, this Section, to the fullest extent permitted by the
|
||||
applicable national law, to enable You to reasonably exercise Your
|
||||
right under Section 3(b) of this License (right to make Adaptations)
|
||||
but not otherwise.
|
||||
|
||||
5. Representations, Warranties and Disclaimer
|
||||
|
||||
UNLESS OTHERWISE MUTUALLY AGREED TO BY THE PARTIES IN WRITING, LICENSOR
|
||||
OFFERS THE WORK AS-IS AND MAKES NO REPRESENTATIONS OR WARRANTIES OF ANY
|
||||
KIND CONCERNING THE WORK, EXPRESS, IMPLIED, STATUTORY OR OTHERWISE,
|
||||
INCLUDING, WITHOUT LIMITATION, WARRANTIES OF TITLE, MERCHANTIBILITY,
|
||||
FITNESS FOR A PARTICULAR PURPOSE, NONINFRINGEMENT, OR THE ABSENCE OF
|
||||
LATENT OR OTHER DEFECTS, ACCURACY, OR THE PRESENCE OF ABSENCE OF ERRORS,
|
||||
WHETHER OR NOT DISCOVERABLE. SOME JURISDICTIONS DO NOT ALLOW THE EXCLUSION
|
||||
OF IMPLIED WARRANTIES, SO SUCH EXCLUSION MAY NOT APPLY TO YOU.
|
||||
|
||||
6. Limitation on Liability. EXCEPT TO THE EXTENT REQUIRED BY APPLICABLE
|
||||
LAW, IN NO EVENT WILL LICENSOR BE LIABLE TO YOU ON ANY LEGAL THEORY FOR
|
||||
ANY SPECIAL, INCIDENTAL, CONSEQUENTIAL, PUNITIVE OR EXEMPLARY DAMAGES
|
||||
ARISING OUT OF THIS LICENSE OR THE USE OF THE WORK, EVEN IF LICENSOR HAS
|
||||
BEEN ADVISED OF THE POSSIBILITY OF SUCH DAMAGES.
|
||||
|
||||
7. Termination
|
||||
|
||||
a. This License and the rights granted hereunder will terminate
|
||||
automatically upon any breach by You of the terms of this License.
|
||||
Individuals or entities who have received Adaptations or Collections
|
||||
from You under this License, however, will not have their licenses
|
||||
terminated provided such individuals or entities remain in full
|
||||
compliance with those licenses. Sections 1, 2, 5, 6, 7, and 8 will
|
||||
survive any termination of this License.
|
||||
b. Subject to the above terms and conditions, the license granted here is
|
||||
perpetual (for the duration of the applicable copyright in the Work).
|
||||
Notwithstanding the above, Licensor reserves the right to release the
|
||||
Work under different license terms or to stop distributing the Work at
|
||||
any time; provided, however that any such election will not serve to
|
||||
withdraw this License (or any other license that has been, or is
|
||||
required to be, granted under the terms of this License), and this
|
||||
License will continue in full force and effect unless terminated as
|
||||
stated above.
|
||||
|
||||
8. Miscellaneous
|
||||
|
||||
a. Each time You Distribute or Publicly Perform the Work or a Collection,
|
||||
the Licensor offers to the recipient a license to the Work on the same
|
||||
terms and conditions as the license granted to You under this License.
|
||||
b. Each time You Distribute or Publicly Perform an Adaptation, Licensor
|
||||
offers to the recipient a license to the original Work on the same
|
||||
terms and conditions as the license granted to You under this License.
|
||||
c. If any provision of this License is invalid or unenforceable under
|
||||
applicable law, it shall not affect the validity or enforceability of
|
||||
the remainder of the terms of this License, and without further action
|
||||
by the parties to this agreement, such provision shall be reformed to
|
||||
the minimum extent necessary to make such provision valid and
|
||||
enforceable.
|
||||
d. No term or provision of this License shall be deemed waived and no
|
||||
breach consented to unless such waiver or consent shall be in writing
|
||||
and signed by the party to be charged with such waiver or consent.
|
||||
e. This License constitutes the entire agreement between the parties with
|
||||
respect to the Work licensed here. There are no understandings,
|
||||
agreements or representations with respect to the Work not specified
|
||||
here. Licensor shall not be bound by any additional provisions that
|
||||
may appear in any communication from You. This License may not be
|
||||
modified without the mutual written agreement of the Licensor and You.
|
||||
f. The rights granted under, and the subject matter referenced, in this
|
||||
License were drafted utilizing the terminology of the Berne Convention
|
||||
for the Protection of Literary and Artistic Works (as amended on
|
||||
September 28, 1979), the Rome Convention of 1961, the WIPO Copyright
|
||||
Treaty of 1996, the WIPO Performances and Phonograms Treaty of 1996
|
||||
and the Universal Copyright Convention (as revised on July 24, 1971).
|
||||
These rights and subject matter take effect in the relevant
|
||||
jurisdiction in which the License terms are sought to be enforced
|
||||
according to the corresponding provisions of the implementation of
|
||||
those treaty provisions in the applicable national law. If the
|
||||
standard suite of rights granted under applicable copyright law
|
||||
includes additional rights not granted under this License, such
|
||||
additional rights are deemed to be included in the License; this
|
||||
License is not intended to restrict the license of any rights under
|
||||
applicable law.
|
||||
|
||||
|
||||
Creative Commons Notice
|
||||
|
||||
Creative Commons is not a party to this License, and makes no warranty
|
||||
whatsoever in connection with the Work. Creative Commons will not be
|
||||
liable to You or any party on any legal theory for any damages
|
||||
whatsoever, including without limitation any general, special,
|
||||
incidental or consequential damages arising in connection to this
|
||||
license. Notwithstanding the foregoing two (2) sentences, if Creative
|
||||
Commons has expressly identified itself as the Licensor hereunder, it
|
||||
shall have all rights and obligations of Licensor.
|
||||
|
||||
Except for the limited purpose of indicating to the public that the
|
||||
Work is licensed under the CCPL, Creative Commons does not authorize
|
||||
the use by either party of the trademark "Creative Commons" or any
|
||||
related trademark or logo of Creative Commons without the prior
|
||||
written consent of Creative Commons. Any permitted use will be in
|
||||
compliance with Creative Commons' then-current trademark usage
|
||||
guidelines, as may be published on its website or otherwise made
|
||||
available upon request from time to time. For the avoidance of doubt,
|
||||
this trademark restriction does not form part of this License.
|
||||
|
||||
Creative Commons may be contacted at https://creativecommons.org/.
|
||||
295
hudi-client/src/main/resources/META-INF/NOTICE.txt
Normal file
295
hudi-client/src/main/resources/META-INF/NOTICE.txt
Normal file
@@ -0,0 +1,295 @@
|
||||
Apache HUDI
|
||||
Copyright 2019 The Apache Software Foundation
|
||||
|
||||
Licensed 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.
|
||||
|
||||
This project includes:
|
||||
aircompressor under Apache License 2.0
|
||||
An open source Java toolkit for Amazon S3 under Apache License, Version 2.0
|
||||
Annotation 1.0 under The Apache Software License, Version 2.0
|
||||
ANTLR 3 Runtime under BSD licence
|
||||
ANTLR 4 Runtime under The BSD License
|
||||
ANTLR ST4 4.0.4 under BSD licence
|
||||
AOP alliance under Public Domain
|
||||
aopalliance version 1.0 repackaged as a module under CDDL + GPLv2 with classpath exception
|
||||
Apache Ant Core under The Apache Software License, Version 2.0
|
||||
Apache Ant Launcher under The Apache Software License, Version 2.0
|
||||
Apache Avro under The Apache Software License, Version 2.0
|
||||
Apache Avro IPC under The Apache Software License, Version 2.0
|
||||
Apache Avro Mapred API under The Apache Software License, Version 2.0
|
||||
Apache Calcite Avatica under Apache License, Version 2.0
|
||||
Apache Calcite Avatica Metrics under Apache License, Version 2.0
|
||||
Apache Commons Collections under Apache License, Version 2.0
|
||||
Apache Commons Crypto under Apache License, Version 2.0
|
||||
Apache Commons IO under Apache License, Version 2.0
|
||||
Apache Commons Lang under Apache License, Version 2.0
|
||||
Apache Commons Logging under The Apache Software License, Version 2.0
|
||||
Apache Curator under The Apache Software License, Version 2.0
|
||||
Apache Directory API ASN.1 API under The Apache Software License, Version 2.0
|
||||
Apache Directory LDAP API Utilities under The Apache Software License, Version 2.0
|
||||
Apache Groovy under The Apache Software License, Version 2.0
|
||||
Apache Hadoop Annotations under Apache License, Version 2.0
|
||||
Apache Hadoop Auth under Apache License, Version 2.0
|
||||
Apache Hadoop Client under Apache License, Version 2.0
|
||||
Apache Hadoop Common under Apache License, Version 2.0
|
||||
Apache Hadoop HDFS under Apache License, Version 2.0
|
||||
Apache Hadoop Mini-Cluster under The Apache Software License, Version 2.0
|
||||
Apache HBase - Annotations under Apache License, Version 2.0
|
||||
Apache HBase - Client under Apache License, Version 2.0
|
||||
Apache HBase - Common under Apache License, Version 2.0
|
||||
Apache HBase - Hadoop Compatibility under Apache License, Version 2.0
|
||||
Apache HBase - Hadoop Two Compatibility under Apache License, Version 2.0
|
||||
Apache HBase - Prefix Tree under Apache License, Version 2.0
|
||||
Apache HBase - Procedure under Apache License, Version 2.0
|
||||
Apache HBase - Protocol under Apache License, Version 2.0
|
||||
Apache HBase - Server under Apache License, Version 2.0
|
||||
Apache HBase - Testing Util under Apache License, Version 2.0
|
||||
Apache HttpClient under Apache License, Version 2.0
|
||||
Apache HttpCore under Apache License, Version 2.0
|
||||
Apache Ivy under The Apache Software License, Version 2.0
|
||||
Apache Log4j under The Apache Software License, Version 2.0
|
||||
Apache Log4j 1.x Compatibility API under The Apache Software License, Version 2.0
|
||||
Apache Log4j API under The Apache Software License, Version 2.0
|
||||
Apache Log4j Core under The Apache Software License, Version 2.0
|
||||
Apache Log4j SLF4J Binding under The Apache Software License, Version 2.0
|
||||
Apache Log4j Web under The Apache Software License, Version 2.0
|
||||
Apache Parquet Avro under The Apache Software License, Version 2.0
|
||||
Apache Parquet Avro (Incubating) under The Apache Software License, Version 2.0
|
||||
Apache Parquet Column under The Apache Software License, Version 2.0
|
||||
Apache Parquet Column (Incubating) under The Apache Software License, Version 2.0
|
||||
Apache Parquet Common under The Apache Software License, Version 2.0
|
||||
Apache Parquet Common (Incubating) under The Apache Software License, Version 2.0
|
||||
Apache Parquet Encodings under The Apache Software License, Version 2.0
|
||||
Apache Parquet Encodings (Incubating) under The Apache Software License, Version 2.0
|
||||
Apache Parquet Format (Incubating) under The Apache Software License, Version 2.0
|
||||
Apache Parquet Generator (Incubating) under The Apache Software License, Version 2.0
|
||||
Apache Parquet Hadoop under The Apache Software License, Version 2.0
|
||||
Apache Parquet Hadoop (Incubating) under The Apache Software License, Version 2.0
|
||||
Apache Parquet Hadoop Bundle under The Apache Software License, Version 2.0
|
||||
Apache Parquet Hadoop Bundle (Incubating) under The Apache Software License, Version 2.0
|
||||
Apache Parquet Jackson under The Apache Software License, Version 2.0
|
||||
Apache Parquet Jackson (Incubating) under The Apache Software License, Version 2.0
|
||||
Apache Thrift under The Apache Software License, Version 2.0
|
||||
Apache Velocity under The Apache Software License, Version 2.0
|
||||
Apache XBean :: ASM 5 shaded (repackaged) under null or null
|
||||
ApacheDS I18n under The Apache Software License, Version 2.0
|
||||
ApacheDS Protocol Kerberos Codec under The Apache Software License, Version 2.0
|
||||
ASM Commons under 3-Clause BSD License
|
||||
ASM Core under 3-Clause BSD License
|
||||
ASM Tree under 3-Clause BSD License
|
||||
Bean Validation API under The Apache Software License, Version 2.0
|
||||
Calcite Core under Apache License, Version 2.0
|
||||
Calcite Druid under Apache License, Version 2.0
|
||||
Calcite Linq4j under Apache License, Version 2.0
|
||||
chill under Apache 2
|
||||
chill-java under Apache 2
|
||||
com.twitter.common:objectsize under Apache License, Version 2.0
|
||||
Commons BeanUtils Core under The Apache Software License, Version 2.0
|
||||
Commons CLI under The Apache Software License, Version 2.0
|
||||
Commons Codec under The Apache Software License, Version 2.0
|
||||
Commons Compiler under New BSD License
|
||||
Commons Compress under The Apache Software License, Version 2.0
|
||||
Commons Configuration under The Apache Software License, Version 2.0
|
||||
Commons Daemon under The Apache Software License, Version 2.0
|
||||
Commons DBCP under The Apache Software License, Version 2.0
|
||||
Commons Lang under The Apache Software License, Version 2.0
|
||||
Commons Math under The Apache Software License, Version 2.0
|
||||
Commons Net under The Apache Software License, Version 2.0
|
||||
Commons Pool under The Apache Software License, Version 2.0
|
||||
commons-beanutils under Apache License
|
||||
Compress-LZF under Apache License 2.0
|
||||
Curator Client under The Apache Software License, Version 2.0
|
||||
Curator Framework under The Apache Software License, Version 2.0
|
||||
Curator Recipes under The Apache Software License, Version 2.0
|
||||
Data Mapper for Jackson under The Apache Software License, Version 2.0
|
||||
DataNucleus Core under The Apache Software License, Version 2.0
|
||||
Digester under The Apache Software License, Version 2.0
|
||||
Disruptor Framework under The Apache Software License, Version 2.0
|
||||
eigenbase-properties under Apache License, Version 2.0
|
||||
EL under The Apache Software License, Version 2.0
|
||||
empty under The Apache License, Version 2.0
|
||||
fastutil under Apache License, Version 2.0
|
||||
Findbugs Annotations under Apache License under Apache License, Version 2.0
|
||||
FindBugs-jsr305 under The Apache Software License, Version 2.0
|
||||
Fluent API for Apache HttpClient under Apache License, Version 2.0
|
||||
Glassfish Jasper under CDDL 1.0
|
||||
Glassfish Jasper API under Apache License Version 2.0
|
||||
Google Guice - Core Library under The Apache Software License, Version 2.0
|
||||
Google Guice - Extensions - Servlet under The Apache Software License, Version 2.0
|
||||
Graphite Integration for Metrics under Apache License 2.0
|
||||
Gson under The Apache Software License, Version 2.0
|
||||
Guava: Google Core Libraries for Java under The Apache Software License, Version 2.0
|
||||
Hadoop Metrics2 Reporter for Dropwizard Metrics under Apache License, Version 2.0
|
||||
hadoop-mapreduce-client-app under Apache License, Version 2.0
|
||||
hadoop-mapreduce-client-common under Apache License, Version 2.0
|
||||
hadoop-mapreduce-client-core under Apache License, Version 2.0
|
||||
hadoop-mapreduce-client-hs under The Apache Software License, Version 2.0
|
||||
hadoop-mapreduce-client-jobclient under Apache License, Version 2.0
|
||||
hadoop-mapreduce-client-shuffle under Apache License, Version 2.0
|
||||
hadoop-yarn-api under Apache License, Version 2.0
|
||||
hadoop-yarn-client under Apache License, Version 2.0
|
||||
hadoop-yarn-common under Apache License, Version 2.0
|
||||
hadoop-yarn-server-applicationhistoryservice under Apache License, Version 2.0
|
||||
hadoop-yarn-server-common under Apache License, Version 2.0
|
||||
hadoop-yarn-server-nodemanager under The Apache Software License, Version 2.0
|
||||
hadoop-yarn-server-resourcemanager under Apache License, Version 2.0
|
||||
hadoop-yarn-server-tests under The Apache Software License, Version 2.0
|
||||
hadoop-yarn-server-web-proxy under Apache License, Version 2.0
|
||||
Hamcrest Core under New BSD License
|
||||
Hive Common under The Apache Software License, Version 2.0
|
||||
Hive Llap Client under The Apache Software License, Version 2.0
|
||||
Hive Llap Common under The Apache Software License, Version 2.0
|
||||
Hive Llap Tez under The Apache Software License, Version 2.0
|
||||
Hive Query Language under The Apache Software License, Version 2.0
|
||||
Hive Serde under The Apache Software License, Version 2.0
|
||||
Hive Service RPC under The Apache Software License, Version 2.0
|
||||
Hive Shims under The Apache Software License, Version 2.0
|
||||
Hive Shims 0.23 under The Apache Software License, Version 2.0
|
||||
Hive Shims Common under The Apache Software License, Version 2.0
|
||||
Hive Shims Scheduler under The Apache Software License, Version 2.0
|
||||
Hive Storage API under Apache License, Version 2.0
|
||||
Hive Vector-Code-Gen Utilities under The Apache Software License, Version 2.0
|
||||
HK2 API module under CDDL + GPLv2 with classpath exception
|
||||
HK2 Implementation Utilities under CDDL + GPLv2 with classpath exception
|
||||
hoodie-client under Apache License, Version 2.0
|
||||
hoodie-common under Apache License, Version 2.0
|
||||
hoodie-hadoop-mr under Apache License, Version 2.0
|
||||
hoodie-timeline-service under Apache License, Version 2.0
|
||||
htrace-core under The Apache Software License, Version 2.0
|
||||
HttpClient under Apache License
|
||||
IntelliJ IDEA Annotations under The Apache Software License, Version 2.0
|
||||
Jackson under The Apache Software License, Version 2.0
|
||||
Jackson Integration for Metrics under Apache License 2.0
|
||||
Jackson-annotations under The Apache Software License, Version 2.0
|
||||
Jackson-core under The Apache Software License, Version 2.0
|
||||
jackson-databind under The Apache Software License, Version 2.0
|
||||
Jackson-module-paranamer under The Apache Software License, Version 2.0
|
||||
jackson-module-scala under The Apache Software License, Version 2.0
|
||||
jamon-runtime under Mozilla Public License Version 2.0
|
||||
Janino under New BSD License
|
||||
jasper-compiler under The Apache Software License, Version 2.0
|
||||
jasper-runtime under The Apache Software License, Version 2.0
|
||||
Java Authentication SPI for Containers under The Apache Software License, Version 2.0
|
||||
Java Servlet API under CDDL + GPLv2 with classpath exception
|
||||
java-xmlbuilder under Apache License, Version 2.0
|
||||
JavaBeans Activation Framework (JAF) under Common Development and Distribution License (CDDL) v1.0
|
||||
Javalin under The Apache Software License, Version 2.0
|
||||
Javassist under MPL 1.1 or LGPL 2.1 or Apache License 2.0
|
||||
javax.annotation API under CDDL + GPLv2 with classpath exception
|
||||
javax.inject under The Apache Software License, Version 2.0
|
||||
javax.inject:1 as OSGi bundle under CDDL + GPLv2 with classpath exception
|
||||
javax.ws.rs-api under CDDL 1.1 or GPL2 w/ CPE
|
||||
JAX-RS provider for JSON content type under The Apache Software License, Version 2.0 or GNU Lesser General Public License (LGPL), Version 2.1
|
||||
JAXB API bundle for GlassFish V3 under CDDL 1.1 or GPL2 w/ CPE
|
||||
JAXB RI under CDDL 1.1 or GPL2 w/ CPE
|
||||
JCL 1.1.1 implemented over SLF4J under MIT License
|
||||
JCodings under MIT License
|
||||
JCommander under The Apache Software License, Version 2.0
|
||||
jersey-client under CDDL 1.1 or GPL2 w/ CPE
|
||||
jersey-container-servlet under CDDL+GPL License
|
||||
jersey-container-servlet-core under CDDL+GPL License
|
||||
jersey-core under CDDL 1.1 or GPL2 w/ CPE
|
||||
jersey-core-client under CDDL+GPL License
|
||||
jersey-core-common under CDDL+GPL License
|
||||
jersey-core-server under CDDL+GPL License
|
||||
jersey-guice under CDDL 1.1 or GPL2 w/ CPE
|
||||
jersey-json under CDDL 1.1 or GPL2 w/ CPE
|
||||
jersey-media-jaxb under CDDL+GPL License
|
||||
jersey-repackaged-guava under CDDL+GPL License
|
||||
jersey-server under CDDL 1.1 or GPL2 w/ CPE
|
||||
Jettison under Apache License, Version 2.0
|
||||
Jetty :: Aggregate :: All core Jetty under Apache Software License - Version 2.0 or Eclipse Public License - Version 1.0
|
||||
Jetty :: Asynchronous HTTP Client under Apache Software License - Version 2.0 or Eclipse Public License - Version 1.0
|
||||
Jetty :: Http Utility under Apache Software License - Version 2.0 or Eclipse Public License - Version 1.0
|
||||
Jetty :: IO Utility under Apache Software License - Version 2.0 or Eclipse Public License - Version 1.0
|
||||
Jetty :: Security under Apache Software License - Version 2.0 or Eclipse Public License - Version 1.0
|
||||
Jetty :: Server Core under Apache Software License - Version 2.0 or Eclipse Public License - Version 1.0
|
||||
Jetty :: Servlet Handling under Apache Software License - Version 2.0 or Eclipse Public License - Version 1.0
|
||||
Jetty :: Utilities under Apache Software License - Version 2.0 or Eclipse Public License - Version 1.0
|
||||
Jetty :: Webapp Application Support under Apache Software License - Version 2.0 or Eclipse Public License - Version 1.0
|
||||
Jetty :: Websocket :: API under Apache Software License - Version 2.0 or Eclipse Public License - Version 1.0
|
||||
Jetty :: Websocket :: Client under Apache Software License - Version 2.0 or Eclipse Public License - Version 1.0
|
||||
Jetty :: Websocket :: Common under Apache Software License - Version 2.0 or Eclipse Public License - Version 1.0
|
||||
Jetty :: Websocket :: Server under Apache Software License - Version 2.0 or Eclipse Public License - Version 1.0
|
||||
Jetty :: Websocket :: Servlet Interface under Apache Software License - Version 2.0 or Eclipse Public License - Version 1.0
|
||||
Jetty :: XML utilities under Apache Software License - Version 2.0 or Eclipse Public License - Version 1.0
|
||||
Jetty Orbit :: Servlet API under Apache Software License - Version 2.0 or Eclipse Public License - Version 1.0
|
||||
Jetty Server under Apache Software License - Version 2.0 or Eclipse Public License - Version 1.0
|
||||
Jetty SSLEngine under Apache License Version 2
|
||||
Jetty Utilities under Apache Software License - Version 2.0 or Eclipse Public License - Version 1.0
|
||||
JLine under The BSD License
|
||||
Joda-Time under Apache 2
|
||||
Joni under MIT License
|
||||
JSch under BSD
|
||||
json4s-ast under ASL
|
||||
json4s-core under ASL
|
||||
json4s-jackson under ASL
|
||||
jsp-api under CDDL
|
||||
JTA 1.1 under The Apache Software License, Version 2.0
|
||||
JUL to SLF4J bridge under MIT License
|
||||
JUnit under Common Public License Version 1.0
|
||||
JVM Integration for Metrics under Apache License 2.0
|
||||
Kryo Shaded under 3-Clause BSD License
|
||||
leveldbjni-all under The BSD 3-Clause License
|
||||
LZ4 and xxHash under The Apache Software License, Version 2.0
|
||||
Metrics Core under Apache License 2.0
|
||||
Metrics Core Library under Apache License 2.0
|
||||
MinLog under New BSD License
|
||||
Mockito under The MIT License
|
||||
Netty/All-in-One under Apache License, Version 2.0
|
||||
Objenesis under Apache 2
|
||||
Open JSON under The Apache Software License, Version 2.0
|
||||
opencsv under Apache 2
|
||||
ORC Core under Apache License, Version 2.0
|
||||
org.jetbrains.kotlin:kotlin-stdlib under The Apache License, Version 2.0
|
||||
org.jetbrains.kotlin:kotlin-stdlib-common under The Apache License, Version 2.0
|
||||
org.jetbrains.kotlin:kotlin-stdlib-jdk7 under The Apache License, Version 2.0
|
||||
org.jetbrains.kotlin:kotlin-stdlib-jdk8 under The Apache License, Version 2.0
|
||||
org.pentaho:pentaho-aggdesigner-algorithm under Apache License, Version 2.0
|
||||
oro under Apache License, Version 2.0
|
||||
OSGi resource locator bundle - used by various API providers that rely on META-INF/services mechanism to locate providers. under CDDL + GPLv2 with classpath exception
|
||||
ParaNamer Core under BSD
|
||||
Protocol Buffer Java API under New BSD license
|
||||
Py4J under The New BSD License
|
||||
pyrolite under MIT License
|
||||
RabbitMQ Java Client under ASL 2.0 or GPL v2 or MPL 1.1
|
||||
RoaringBitmap under Apache 2
|
||||
RocksDB JNI under Apache License 2.0 or GNU General Public License, version 2
|
||||
Scala Compiler under BSD 3-Clause
|
||||
Scala Library under BSD 3-Clause
|
||||
scala-parser-combinators under BSD 3-clause
|
||||
scala-xml under BSD 3-clause
|
||||
Scalap under BSD 3-Clause
|
||||
scalatest under the Apache License, ASL Version 2.0
|
||||
ServiceLocator Default Implementation under CDDL + GPLv2 with classpath exception
|
||||
Servlet Specification 2.5 API under CDDL 1.0
|
||||
Servlet Specification API under Apache License Version 2.0
|
||||
servlet-api under CDDL
|
||||
SLF4J API Module under MIT License
|
||||
SLF4J LOG4J-12 Binding under MIT License
|
||||
snappy-java under The Apache Software License, Version 2.0
|
||||
Spark Project Catalyst under Apache 2.0 License
|
||||
Spark Project Core under Apache 2.0 License
|
||||
Spark Project Launcher under Apache 2.0 License
|
||||
Spark Project Networking under Apache 2.0 License
|
||||
Spark Project Shuffle Streaming Service under Apache 2.0 License
|
||||
Spark Project Sketch under Apache 2.0 License
|
||||
Spark Project SQL under Apache 2.0 License
|
||||
Spark Project Tags under Apache 2.0 License
|
||||
Spark Project Unsafe under Apache 2.0 License
|
||||
StAX API under The Apache Software License, Version 2.0
|
||||
stream-lib under Apache License, Version 2.0
|
||||
Streaming API for XML under GNU General Public Library or COMMON DEVELOPMENT AND DISTRIBUTION LICENSE (CDDL) Version 1.0
|
||||
The Netty Project under Apache License, Version 2.0
|
||||
univocity-parsers under Apache 2
|
||||
Xerces2 Java Parser under The Apache Software License, Version 2.0
|
||||
XML Commons External Components XML APIs under The Apache Software License, Version 2.0
|
||||
Xml Compatibility extensions for Jackson under The Apache Software License, Version 2.0 or GNU Lesser General Public License (LGPL), Version 2.1
|
||||
xmlenc Library under The BSD License
|
||||
XZ for Java under Public Domain
|
||||
zookeeper under Apache License, Version 2.0
|
||||
|
||||
23
hudi-client/src/main/resources/log4j.properties
Normal file
23
hudi-client/src/main/resources/log4j.properties
Normal file
@@ -0,0 +1,23 @@
|
||||
###
|
||||
# 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.
|
||||
###
|
||||
log4j.rootLogger=INFO, A1
|
||||
# A1 is set to be a ConsoleAppender.
|
||||
log4j.appender.A1=org.apache.log4j.ConsoleAppender
|
||||
# A1 uses PatternLayout.
|
||||
log4j.appender.A1.layout=org.apache.log4j.PatternLayout
|
||||
log4j.appender.A1.layout.ConversionPattern=%-4r [%t] %-5p %c %x - %m%n
|
||||
128
hudi-client/src/test/java/HoodieClientExample.java
Normal file
128
hudi-client/src/test/java/HoodieClientExample.java
Normal file
@@ -0,0 +1,128 @@
|
||||
/*
|
||||
* 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.
|
||||
*/
|
||||
|
||||
import com.beust.jcommander.JCommander;
|
||||
import com.beust.jcommander.Parameter;
|
||||
import java.util.List;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hudi.HoodieWriteClient;
|
||||
import org.apache.hudi.WriteStatus;
|
||||
import org.apache.hudi.common.HoodieTestDataGenerator;
|
||||
import org.apache.hudi.common.model.HoodieAvroPayload;
|
||||
import org.apache.hudi.common.model.HoodieRecord;
|
||||
import org.apache.hudi.common.model.HoodieTableType;
|
||||
import org.apache.hudi.common.table.HoodieTableMetaClient;
|
||||
import org.apache.hudi.common.util.FSUtils;
|
||||
import org.apache.hudi.common.util.Option;
|
||||
import org.apache.hudi.config.HoodieCompactionConfig;
|
||||
import org.apache.hudi.config.HoodieIndexConfig;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
import org.apache.hudi.index.HoodieIndex.IndexType;
|
||||
import org.apache.log4j.LogManager;
|
||||
import org.apache.log4j.Logger;
|
||||
import org.apache.spark.SparkConf;
|
||||
import org.apache.spark.api.java.JavaRDD;
|
||||
import org.apache.spark.api.java.JavaSparkContext;
|
||||
|
||||
/**
|
||||
* Driver program that uses the Hoodie client with synthetic workload, and performs basic operations. <p>
|
||||
*/
|
||||
public class HoodieClientExample {
|
||||
|
||||
private static Logger logger = LogManager.getLogger(HoodieClientExample.class);
|
||||
@Parameter(names = {"--help", "-h"}, help = true)
|
||||
public Boolean help = false;
|
||||
@Parameter(names = {"--table-path", "-p"}, description = "path for Hoodie sample table")
|
||||
private String tablePath = "file:///tmp/hoodie/sample-table";
|
||||
@Parameter(names = {"--table-name", "-n"}, description = "table name for Hoodie sample table")
|
||||
private String tableName = "hoodie_rt";
|
||||
@Parameter(names = {"--table-type", "-t"}, description = "One of COPY_ON_WRITE or MERGE_ON_READ")
|
||||
private String tableType = HoodieTableType.COPY_ON_WRITE.name();
|
||||
|
||||
public static void main(String[] args) throws Exception {
|
||||
HoodieClientExample cli = new HoodieClientExample();
|
||||
JCommander cmd = new JCommander(cli, args);
|
||||
|
||||
if (cli.help) {
|
||||
cmd.usage();
|
||||
System.exit(1);
|
||||
}
|
||||
cli.run();
|
||||
}
|
||||
|
||||
|
||||
public void run() throws Exception {
|
||||
|
||||
SparkConf sparkConf = new SparkConf().setAppName("hoodie-client-example");
|
||||
sparkConf.setMaster("local[1]");
|
||||
sparkConf.set("spark.serializer", "org.apache.spark.serializer.KryoSerializer");
|
||||
sparkConf.set("spark.kryoserializer.buffer.max", "512m");
|
||||
JavaSparkContext jsc = new JavaSparkContext(sparkConf);
|
||||
|
||||
// Generator of some records to be loaded in.
|
||||
HoodieTestDataGenerator dataGen = new HoodieTestDataGenerator();
|
||||
|
||||
// initialize the table, if not done already
|
||||
Path path = new Path(tablePath);
|
||||
FileSystem fs = FSUtils.getFs(tablePath, jsc.hadoopConfiguration());
|
||||
if (!fs.exists(path)) {
|
||||
HoodieTableMetaClient
|
||||
.initTableType(jsc.hadoopConfiguration(), tablePath, HoodieTableType.valueOf(tableType), tableName,
|
||||
HoodieAvroPayload.class.getName());
|
||||
}
|
||||
|
||||
// Create the write client to write some records in
|
||||
HoodieWriteConfig cfg = HoodieWriteConfig.newBuilder().withPath(tablePath)
|
||||
.withSchema(HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA).withParallelism(2, 2)
|
||||
.forTable(tableName)
|
||||
.withIndexConfig(HoodieIndexConfig.newBuilder().withIndexType(IndexType.BLOOM).build())
|
||||
.withCompactionConfig(
|
||||
HoodieCompactionConfig.newBuilder().archiveCommitsWith(2, 3).build()).build();
|
||||
HoodieWriteClient client = new HoodieWriteClient(jsc, cfg);
|
||||
|
||||
/**
|
||||
* Write 1 (only inserts)
|
||||
*/
|
||||
String newCommitTime = client.startCommit();
|
||||
logger.info("Starting commit " + newCommitTime);
|
||||
|
||||
List<HoodieRecord> records = dataGen.generateInserts(newCommitTime, 100);
|
||||
JavaRDD<HoodieRecord> writeRecords = jsc.<HoodieRecord>parallelize(records, 1);
|
||||
client.upsert(writeRecords, newCommitTime);
|
||||
|
||||
/**
|
||||
* Write 2 (updates)
|
||||
*/
|
||||
newCommitTime = client.startCommit();
|
||||
logger.info("Starting commit " + newCommitTime);
|
||||
records.addAll(dataGen.generateUpdates(newCommitTime, 100));
|
||||
writeRecords = jsc.<HoodieRecord>parallelize(records, 1);
|
||||
client.upsert(writeRecords, newCommitTime);
|
||||
|
||||
/**
|
||||
* Schedule a compaction and also perform compaction on a MOR dataset
|
||||
*/
|
||||
if (HoodieTableType.valueOf(tableType) == HoodieTableType.MERGE_ON_READ) {
|
||||
Option<String> instant = client.scheduleCompaction(Option.empty());
|
||||
JavaRDD<WriteStatus> writeStatues = client.compact(instant.get());
|
||||
client.commitCompaction(instant.get(), writeStatues, Option.empty());
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
@@ -0,0 +1,581 @@
|
||||
/*
|
||||
* 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;
|
||||
|
||||
import static org.apache.hudi.common.HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA;
|
||||
import static org.junit.Assert.assertEquals;
|
||||
import static org.junit.Assert.assertFalse;
|
||||
import static org.junit.Assert.assertTrue;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Arrays;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.stream.Collectors;
|
||||
import org.apache.hadoop.fs.FileStatus;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hudi.avro.model.HoodieCompactionOperation;
|
||||
import org.apache.hudi.avro.model.HoodieCompactionPlan;
|
||||
import org.apache.hudi.common.HoodieClientTestUtils;
|
||||
import org.apache.hudi.common.HoodieTestDataGenerator;
|
||||
import org.apache.hudi.common.model.FileSlice;
|
||||
import org.apache.hudi.common.model.HoodieDataFile;
|
||||
import org.apache.hudi.common.model.HoodieFileGroupId;
|
||||
import org.apache.hudi.common.model.HoodieRecord;
|
||||
import org.apache.hudi.common.model.HoodieTableType;
|
||||
import org.apache.hudi.common.model.HoodieTestUtils;
|
||||
import org.apache.hudi.common.table.HoodieTableMetaClient;
|
||||
import org.apache.hudi.common.table.HoodieTimeline;
|
||||
import org.apache.hudi.common.table.timeline.HoodieInstant;
|
||||
import org.apache.hudi.common.table.timeline.HoodieInstant.State;
|
||||
import org.apache.hudi.common.table.view.FileSystemViewStorageConfig;
|
||||
import org.apache.hudi.common.table.view.FileSystemViewStorageType;
|
||||
import org.apache.hudi.common.table.view.HoodieTableFileSystemView;
|
||||
import org.apache.hudi.common.util.AvroUtils;
|
||||
import org.apache.hudi.common.util.CompactionUtils;
|
||||
import org.apache.hudi.common.util.Option;
|
||||
import org.apache.hudi.common.util.collection.Pair;
|
||||
import org.apache.hudi.config.HoodieCompactionConfig;
|
||||
import org.apache.hudi.config.HoodieIndexConfig;
|
||||
import org.apache.hudi.config.HoodieStorageConfig;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
import org.apache.hudi.index.HoodieIndex;
|
||||
import org.apache.hudi.table.HoodieTable;
|
||||
import org.apache.spark.api.java.JavaRDD;
|
||||
import org.junit.Assert;
|
||||
import org.junit.Test;
|
||||
|
||||
/**
|
||||
* Test Cases for Async Compaction and Ingestion interaction
|
||||
*/
|
||||
public class TestAsyncCompaction extends TestHoodieClientBase {
|
||||
|
||||
private HoodieWriteConfig getConfig(Boolean autoCommit) {
|
||||
return getConfigBuilder(autoCommit).build();
|
||||
}
|
||||
|
||||
protected HoodieWriteConfig.Builder getConfigBuilder(Boolean autoCommit) {
|
||||
return HoodieWriteConfig.newBuilder().withPath(basePath).withSchema(TRIP_EXAMPLE_SCHEMA).withParallelism(2, 2)
|
||||
.withAutoCommit(autoCommit).withAssumeDatePartitioning(true).withCompactionConfig(
|
||||
HoodieCompactionConfig.newBuilder().compactionSmallFileSize(1024 * 1024 * 1024).withInlineCompaction(false)
|
||||
.withMaxNumDeltaCommitsBeforeCompaction(1).build())
|
||||
.withStorageConfig(HoodieStorageConfig.newBuilder().limitFileSize(1024 * 1024 * 1024).build())
|
||||
.forTable("test-trip-table")
|
||||
.withIndexConfig(HoodieIndexConfig.newBuilder().withIndexType(HoodieIndex.IndexType.BLOOM).build())
|
||||
.withEmbeddedTimelineServerEnabled(true).withFileSystemViewConfig(
|
||||
FileSystemViewStorageConfig.newBuilder().withStorageType(FileSystemViewStorageType.EMBEDDED_KV_STORE)
|
||||
.build());
|
||||
}
|
||||
|
||||
@Override
|
||||
public void tearDown() throws IOException {
|
||||
super.tearDown();
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testRollbackForInflightCompaction() throws Exception {
|
||||
// Rollback inflight compaction
|
||||
HoodieWriteConfig cfg = getConfig(false);
|
||||
HoodieWriteClient client = getHoodieWriteClient(cfg, true);
|
||||
|
||||
String firstInstantTime = "001";
|
||||
String secondInstantTime = "004";
|
||||
String compactionInstantTime = "005";
|
||||
|
||||
int numRecs = 2000;
|
||||
|
||||
List<HoodieRecord> records = dataGen.generateInserts(firstInstantTime, numRecs);
|
||||
runNextDeltaCommits(client, Arrays.asList(firstInstantTime, secondInstantTime),
|
||||
records, cfg, true, new ArrayList<>());
|
||||
|
||||
// Schedule compaction but do not run them
|
||||
scheduleCompaction(compactionInstantTime, client, cfg);
|
||||
|
||||
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), cfg.getBasePath());
|
||||
|
||||
HoodieInstant pendingCompactionInstant =
|
||||
metaClient.getActiveTimeline().filterPendingCompactionTimeline().firstInstant().get();
|
||||
assertTrue("Pending Compaction instant has expected instant time",
|
||||
pendingCompactionInstant.getTimestamp().equals(compactionInstantTime));
|
||||
assertTrue("Pending Compaction instant has expected state",
|
||||
pendingCompactionInstant.getState().equals(State.REQUESTED));
|
||||
|
||||
moveCompactionFromRequestedToInflight(compactionInstantTime, client, cfg);
|
||||
|
||||
// Reload and rollback inflight compaction
|
||||
metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), cfg.getBasePath());
|
||||
HoodieTable hoodieTable = HoodieTable.getHoodieTable(metaClient, cfg, jsc);
|
||||
hoodieTable.rollback(jsc, compactionInstantTime, false);
|
||||
|
||||
client.rollbackInflightCompaction(
|
||||
new HoodieInstant(State.INFLIGHT, HoodieTimeline.COMPACTION_ACTION, compactionInstantTime), hoodieTable);
|
||||
metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), cfg.getBasePath());
|
||||
pendingCompactionInstant = metaClient.getCommitsAndCompactionTimeline().filterPendingCompactionTimeline()
|
||||
.getInstants().findFirst().get();
|
||||
assertEquals("compaction", pendingCompactionInstant.getAction());
|
||||
assertEquals(State.REQUESTED, pendingCompactionInstant.getState());
|
||||
assertEquals(compactionInstantTime, pendingCompactionInstant.getTimestamp());
|
||||
|
||||
// We indirectly test for the race condition where a inflight instant was first deleted then created new. Every
|
||||
// time this happens, the pending compaction instant file in Hoodie Meta path becomes an empty file (Note: Hoodie
|
||||
// reads compaction plan from aux path which is untouched). TO test for regression, we simply get file status
|
||||
// and look at the file size
|
||||
FileStatus fstatus =
|
||||
metaClient.getFs().getFileStatus(new Path(metaClient.getMetaPath(), pendingCompactionInstant.getFileName()));
|
||||
assertTrue(fstatus.getLen() > 0);
|
||||
}
|
||||
|
||||
private Path getInstantPath(HoodieTableMetaClient metaClient, String timestamp, String action, State state) {
|
||||
HoodieInstant instant = new HoodieInstant(state, action, timestamp);
|
||||
return new Path(metaClient.getMetaPath(), instant.getFileName());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testRollbackInflightIngestionWithPendingCompaction() throws Exception {
|
||||
// Rollback inflight ingestion when there is pending compaction
|
||||
HoodieWriteConfig cfg = getConfig(false);
|
||||
HoodieWriteClient client = getHoodieWriteClient(cfg, true);
|
||||
|
||||
String firstInstantTime = "001";
|
||||
String secondInstantTime = "004";
|
||||
String compactionInstantTime = "005";
|
||||
String inflightInstantTime = "006";
|
||||
String nextInflightInstantTime = "007";
|
||||
|
||||
int numRecs = 2000;
|
||||
|
||||
List<HoodieRecord> records = dataGen.generateInserts(firstInstantTime, numRecs);
|
||||
records = runNextDeltaCommits(client, Arrays.asList(firstInstantTime, secondInstantTime),
|
||||
records, cfg, true, new ArrayList<>());
|
||||
|
||||
// Schedule compaction but do not run them
|
||||
scheduleCompaction(compactionInstantTime, client, cfg);
|
||||
|
||||
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), cfg.getBasePath());
|
||||
createNextDeltaCommit(inflightInstantTime, records, client, metaClient, cfg, true);
|
||||
|
||||
metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), cfg.getBasePath());
|
||||
HoodieInstant pendingCompactionInstant =
|
||||
metaClient.getActiveTimeline().filterPendingCompactionTimeline().firstInstant().get();
|
||||
assertTrue("Pending Compaction instant has expected instant time",
|
||||
pendingCompactionInstant.getTimestamp().equals(compactionInstantTime));
|
||||
HoodieInstant inflightInstant =
|
||||
metaClient.getActiveTimeline().filterInflightsExcludingCompaction().firstInstant().get();
|
||||
assertTrue("inflight instant has expected instant time",
|
||||
inflightInstant.getTimestamp().equals(inflightInstantTime));
|
||||
|
||||
//This should rollback
|
||||
client.startCommitWithTime(nextInflightInstantTime);
|
||||
|
||||
//Validate
|
||||
metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), cfg.getBasePath());
|
||||
inflightInstant =
|
||||
metaClient.getActiveTimeline().filterInflightsExcludingCompaction().firstInstant().get();
|
||||
assertTrue("inflight instant has expected instant time",
|
||||
inflightInstant.getTimestamp().equals(nextInflightInstantTime));
|
||||
assertTrue("Expect only one inflight instant",
|
||||
metaClient.getActiveTimeline().filterInflightsExcludingCompaction().getInstants().count() == 1);
|
||||
//Expect pending Compaction to be present
|
||||
pendingCompactionInstant =
|
||||
metaClient.getActiveTimeline().filterPendingCompactionTimeline().firstInstant().get();
|
||||
assertTrue("Pending Compaction instant has expected instant time",
|
||||
pendingCompactionInstant.getTimestamp().equals(compactionInstantTime));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testInflightCompaction() throws Exception {
|
||||
// There is inflight compaction. Subsequent compaction run must work correctly
|
||||
HoodieWriteConfig cfg = getConfig(true);
|
||||
HoodieWriteClient client = getHoodieWriteClient(cfg, true);
|
||||
|
||||
String firstInstantTime = "001";
|
||||
String secondInstantTime = "004";
|
||||
String compactionInstantTime = "005";
|
||||
String thirdInstantTime = "006";
|
||||
String fourthInstantTime = "007";
|
||||
|
||||
int numRecs = 2000;
|
||||
|
||||
List<HoodieRecord> records = dataGen.generateInserts(firstInstantTime, numRecs);
|
||||
records = runNextDeltaCommits(client, Arrays.asList(firstInstantTime, secondInstantTime),
|
||||
records, cfg, true, new ArrayList<>());
|
||||
|
||||
// Schedule and mark compaction instant as inflight
|
||||
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), cfg.getBasePath());
|
||||
HoodieTable hoodieTable = getHoodieTable(metaClient, cfg);
|
||||
scheduleCompaction(compactionInstantTime, client, cfg);
|
||||
moveCompactionFromRequestedToInflight(compactionInstantTime, client, cfg);
|
||||
|
||||
// Complete ingestions
|
||||
runNextDeltaCommits(client, Arrays.asList(thirdInstantTime, fourthInstantTime),
|
||||
records, cfg, false, Arrays.asList(compactionInstantTime));
|
||||
|
||||
// execute inflight compaction
|
||||
executeCompaction(compactionInstantTime, client, hoodieTable, cfg, numRecs, true);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testScheduleIngestionBeforePendingCompaction() throws Exception {
|
||||
// Case: Failure case. Latest pending compaction instant time must be earlier than this instant time
|
||||
HoodieWriteConfig cfg = getConfig(false);
|
||||
HoodieWriteClient client = getHoodieWriteClient(cfg, true);
|
||||
|
||||
String firstInstantTime = "001";
|
||||
String secondInstantTime = "004";
|
||||
String failedInstantTime = "005";
|
||||
String compactionInstantTime = "006";
|
||||
int numRecs = 2000;
|
||||
|
||||
List<HoodieRecord> records = dataGen.generateInserts(firstInstantTime, numRecs);
|
||||
records = runNextDeltaCommits(client, Arrays.asList(firstInstantTime, secondInstantTime),
|
||||
records, cfg, true, new ArrayList<>());
|
||||
|
||||
// Schedule compaction but do not run them
|
||||
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), cfg.getBasePath());
|
||||
scheduleCompaction(compactionInstantTime, client, cfg);
|
||||
metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), cfg.getBasePath());
|
||||
HoodieInstant pendingCompactionInstant =
|
||||
metaClient.getActiveTimeline().filterPendingCompactionTimeline().firstInstant().get();
|
||||
assertTrue("Pending Compaction instant has expected instant time",
|
||||
pendingCompactionInstant.getTimestamp().equals(compactionInstantTime));
|
||||
|
||||
boolean gotException = false;
|
||||
try {
|
||||
runNextDeltaCommits(client, Arrays.asList(failedInstantTime),
|
||||
records, cfg, false, Arrays.asList(compactionInstantTime));
|
||||
} catch (IllegalArgumentException iex) {
|
||||
// Latest pending compaction instant time must be earlier than this instant time. Should fail here
|
||||
gotException = true;
|
||||
}
|
||||
assertTrue("Latest pending compaction instant time must be earlier than this instant time", gotException);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testScheduleCompactionAfterPendingIngestion() throws Exception {
|
||||
// Case: Failure case. Earliest ingestion inflight instant time must be later than compaction time
|
||||
|
||||
HoodieWriteConfig cfg = getConfig(false);
|
||||
HoodieWriteClient client = getHoodieWriteClient(cfg, true);
|
||||
|
||||
String firstInstantTime = "001";
|
||||
String secondInstantTime = "004";
|
||||
String inflightInstantTime = "005";
|
||||
String compactionInstantTime = "006";
|
||||
int numRecs = 2000;
|
||||
|
||||
List<HoodieRecord> records = dataGen.generateInserts(firstInstantTime, numRecs);
|
||||
records = runNextDeltaCommits(client, Arrays.asList(firstInstantTime, secondInstantTime),
|
||||
records, cfg, true, new ArrayList<>());
|
||||
|
||||
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), cfg.getBasePath());
|
||||
createNextDeltaCommit(inflightInstantTime, records, client, metaClient, cfg, true);
|
||||
|
||||
metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), cfg.getBasePath());
|
||||
HoodieInstant inflightInstant =
|
||||
metaClient.getActiveTimeline().filterInflightsExcludingCompaction().firstInstant().get();
|
||||
assertTrue("inflight instant has expected instant time",
|
||||
inflightInstant.getTimestamp().equals(inflightInstantTime));
|
||||
|
||||
boolean gotException = false;
|
||||
try {
|
||||
// Schedule compaction but do not run them
|
||||
scheduleCompaction(compactionInstantTime, client, cfg);
|
||||
} catch (IllegalArgumentException iex) {
|
||||
// Earliest ingestion inflight instant time must be later than compaction time. Should fail here
|
||||
gotException = true;
|
||||
}
|
||||
assertTrue("Earliest ingestion inflight instant time must be later than compaction time", gotException);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testScheduleCompactionWithOlderOrSameTimestamp() throws Exception {
|
||||
// Case: Failure case. Earliest ingestion inflight instant time must be later than compaction time
|
||||
|
||||
HoodieWriteConfig cfg = getConfig(false);
|
||||
HoodieWriteClient client = getHoodieWriteClient(cfg, true);
|
||||
|
||||
String firstInstantTime = "001";
|
||||
String secondInstantTime = "004";
|
||||
String compactionInstantTime = "002";
|
||||
int numRecs = 2000;
|
||||
|
||||
List<HoodieRecord> records = dataGen.generateInserts(firstInstantTime, numRecs);
|
||||
records = runNextDeltaCommits(client, Arrays.asList(firstInstantTime, secondInstantTime),
|
||||
records, cfg, true, new ArrayList<>());
|
||||
|
||||
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), cfg.getBasePath());
|
||||
boolean gotException = false;
|
||||
try {
|
||||
// Schedule compaction but do not run them
|
||||
scheduleCompaction(compactionInstantTime, client, cfg);
|
||||
} catch (IllegalArgumentException iex) {
|
||||
gotException = true;
|
||||
}
|
||||
assertTrue("Compaction Instant to be scheduled cannot have older timestamp", gotException);
|
||||
|
||||
// Schedule with timestamp same as that of committed instant
|
||||
gotException = false;
|
||||
String dupCompactionInstantTime = secondInstantTime;
|
||||
try {
|
||||
// Schedule compaction but do not run them
|
||||
scheduleCompaction(dupCompactionInstantTime, client, cfg);
|
||||
} catch (IllegalArgumentException iex) {
|
||||
gotException = true;
|
||||
}
|
||||
assertTrue("Compaction Instant to be scheduled cannot have same timestamp as committed instant",
|
||||
gotException);
|
||||
|
||||
compactionInstantTime = "006";
|
||||
scheduleCompaction(compactionInstantTime, client, cfg);
|
||||
gotException = false;
|
||||
try {
|
||||
// Schedule compaction with the same times as a pending compaction
|
||||
scheduleCompaction(dupCompactionInstantTime, client, cfg);
|
||||
} catch (IllegalArgumentException iex) {
|
||||
gotException = true;
|
||||
}
|
||||
assertTrue("Compaction Instant to be scheduled cannot have same timestamp as a pending compaction",
|
||||
gotException);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testCompactionAfterTwoDeltaCommits() throws Exception {
|
||||
// No Delta Commits after compaction request
|
||||
HoodieWriteConfig cfg = getConfig(true);
|
||||
HoodieWriteClient client = getHoodieWriteClient(cfg, true);
|
||||
|
||||
String firstInstantTime = "001";
|
||||
String secondInstantTime = "004";
|
||||
String compactionInstantTime = "005";
|
||||
int numRecs = 2000;
|
||||
|
||||
List<HoodieRecord> records = dataGen.generateInserts(firstInstantTime, numRecs);
|
||||
records = runNextDeltaCommits(client, Arrays.asList(firstInstantTime, secondInstantTime),
|
||||
records, cfg, true, new ArrayList<>());
|
||||
|
||||
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), cfg.getBasePath());
|
||||
HoodieTable hoodieTable = getHoodieTable(metaClient, cfg);
|
||||
scheduleAndExecuteCompaction(compactionInstantTime, client, hoodieTable, cfg, numRecs, false);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testInterleavedCompaction() throws Exception {
|
||||
//Case: Two delta commits before and after compaction schedule
|
||||
HoodieWriteConfig cfg = getConfig(true);
|
||||
HoodieWriteClient client = getHoodieWriteClient(cfg, true);
|
||||
|
||||
String firstInstantTime = "001";
|
||||
String secondInstantTime = "004";
|
||||
String compactionInstantTime = "005";
|
||||
String thirdInstantTime = "006";
|
||||
String fourthInstantTime = "007";
|
||||
|
||||
int numRecs = 2000;
|
||||
|
||||
List<HoodieRecord> records = dataGen.generateInserts(firstInstantTime, numRecs);
|
||||
records = runNextDeltaCommits(client, Arrays.asList(firstInstantTime, secondInstantTime),
|
||||
records, cfg, true, new ArrayList<>());
|
||||
|
||||
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), cfg.getBasePath());
|
||||
HoodieTable hoodieTable = getHoodieTable(metaClient, cfg);
|
||||
scheduleCompaction(compactionInstantTime, client, cfg);
|
||||
|
||||
runNextDeltaCommits(client, Arrays.asList(thirdInstantTime, fourthInstantTime),
|
||||
records, cfg, false, Arrays.asList(compactionInstantTime));
|
||||
executeCompaction(compactionInstantTime, client, hoodieTable, cfg, numRecs, true);
|
||||
}
|
||||
|
||||
/**
|
||||
* HELPER METHODS FOR TESTING
|
||||
**/
|
||||
|
||||
private void validateDeltaCommit(String latestDeltaCommit,
|
||||
final Map<HoodieFileGroupId, Pair<String, HoodieCompactionOperation>> fgIdToCompactionOperation,
|
||||
HoodieWriteConfig cfg) throws IOException {
|
||||
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), cfg.getBasePath());
|
||||
HoodieTable table = getHoodieTable(metaClient, cfg);
|
||||
List<FileSlice> fileSliceList = getCurrentLatestFileSlices(table, cfg);
|
||||
fileSliceList.forEach(fileSlice -> {
|
||||
Pair<String, HoodieCompactionOperation> opPair = fgIdToCompactionOperation.get(fileSlice.getFileGroupId());
|
||||
if (opPair != null) {
|
||||
assertTrue("Expect baseInstant to match compaction Instant",
|
||||
fileSlice.getBaseInstantTime().equals(opPair.getKey()));
|
||||
assertTrue("Expect atleast one log file to be present where the latest delta commit was written",
|
||||
fileSlice.getLogFiles().count() > 0);
|
||||
assertFalse("Expect no data-file to be present", fileSlice.getDataFile().isPresent());
|
||||
} else {
|
||||
assertTrue("Expect baseInstant to be less than or equal to latestDeltaCommit",
|
||||
fileSlice.getBaseInstantTime().compareTo(latestDeltaCommit) <= 0);
|
||||
}
|
||||
});
|
||||
}
|
||||
|
||||
private List<HoodieRecord> runNextDeltaCommits(HoodieWriteClient client, List<String> deltaInstants,
|
||||
List<HoodieRecord> records, HoodieWriteConfig cfg, boolean insertFirst,
|
||||
List<String> expPendingCompactionInstants) throws Exception {
|
||||
|
||||
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), cfg.getBasePath());
|
||||
List<Pair<HoodieInstant, HoodieCompactionPlan>> pendingCompactions =
|
||||
CompactionUtils.getAllPendingCompactionPlans(metaClient);
|
||||
List<String> gotPendingCompactionInstants =
|
||||
pendingCompactions.stream().map(pc -> pc.getKey().getTimestamp()).sorted().collect(Collectors.toList());
|
||||
assertEquals(expPendingCompactionInstants, gotPendingCompactionInstants);
|
||||
|
||||
Map<HoodieFileGroupId, Pair<String, HoodieCompactionOperation>> fgIdToCompactionOperation =
|
||||
CompactionUtils.getAllPendingCompactionOperations(metaClient);
|
||||
|
||||
if (insertFirst) {
|
||||
// Use first instant for inserting records
|
||||
String firstInstant = deltaInstants.get(0);
|
||||
deltaInstants = deltaInstants.subList(1, deltaInstants.size());
|
||||
JavaRDD<HoodieRecord> writeRecords = jsc.parallelize(records, 1);
|
||||
client.startCommitWithTime(firstInstant);
|
||||
JavaRDD<WriteStatus> statuses = client.upsert(writeRecords, firstInstant);
|
||||
List<WriteStatus> statusList = statuses.collect();
|
||||
|
||||
if (!cfg.shouldAutoCommit()) {
|
||||
client.commit(firstInstant, statuses);
|
||||
}
|
||||
assertNoWriteErrors(statusList);
|
||||
metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), cfg.getBasePath());
|
||||
HoodieTable hoodieTable = getHoodieTable(metaClient, cfg);
|
||||
List<HoodieDataFile> dataFilesToRead = getCurrentLatestDataFiles(hoodieTable, cfg);
|
||||
assertTrue("RealtimeTableView should list the parquet files we wrote in the delta commit",
|
||||
dataFilesToRead.stream().findAny().isPresent());
|
||||
validateDeltaCommit(firstInstant, fgIdToCompactionOperation, cfg);
|
||||
}
|
||||
|
||||
int numRecords = records.size();
|
||||
for (String instantTime : deltaInstants) {
|
||||
records = dataGen.generateUpdates(instantTime, numRecords);
|
||||
metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), cfg.getBasePath());
|
||||
createNextDeltaCommit(instantTime, records, client, metaClient, cfg, false);
|
||||
validateDeltaCommit(instantTime, fgIdToCompactionOperation, cfg);
|
||||
}
|
||||
return records;
|
||||
}
|
||||
|
||||
private void moveCompactionFromRequestedToInflight(String compactionInstantTime, HoodieWriteClient client,
|
||||
HoodieWriteConfig cfg) throws IOException {
|
||||
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), cfg.getBasePath());
|
||||
HoodieInstant compactionInstant = HoodieTimeline.getCompactionRequestedInstant(compactionInstantTime);
|
||||
HoodieCompactionPlan workload = AvroUtils.deserializeCompactionPlan(
|
||||
metaClient.getActiveTimeline().getInstantAuxiliaryDetails(compactionInstant).get());
|
||||
metaClient.getActiveTimeline().transitionCompactionRequestedToInflight(compactionInstant);
|
||||
HoodieInstant instant = metaClient.getActiveTimeline().reload().filterPendingCompactionTimeline().getInstants()
|
||||
.filter(in -> in.getTimestamp().equals(compactionInstantTime)).findAny().get();
|
||||
assertTrue("Instant must be marked inflight", instant.isInflight());
|
||||
}
|
||||
|
||||
private void scheduleCompaction(String compactionInstantTime, HoodieWriteClient client, HoodieWriteConfig cfg)
|
||||
throws IOException {
|
||||
client.scheduleCompactionAtInstant(compactionInstantTime, Option.empty());
|
||||
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), cfg.getBasePath());
|
||||
HoodieInstant instant = metaClient.getActiveTimeline().filterPendingCompactionTimeline().lastInstant().get();
|
||||
assertEquals("Last compaction instant must be the one set",
|
||||
instant.getTimestamp(), compactionInstantTime);
|
||||
}
|
||||
|
||||
private void scheduleAndExecuteCompaction(String compactionInstantTime,
|
||||
HoodieWriteClient client, HoodieTable table, HoodieWriteConfig cfg, int expectedNumRecs,
|
||||
boolean hasDeltaCommitAfterPendingCompaction) throws IOException {
|
||||
scheduleCompaction(compactionInstantTime, client, cfg);
|
||||
executeCompaction(compactionInstantTime, client, table, cfg, expectedNumRecs, hasDeltaCommitAfterPendingCompaction);
|
||||
}
|
||||
|
||||
private void executeCompaction(String compactionInstantTime,
|
||||
HoodieWriteClient client, HoodieTable table, HoodieWriteConfig cfg, int expectedNumRecs,
|
||||
boolean hasDeltaCommitAfterPendingCompaction) throws IOException {
|
||||
|
||||
client.compact(compactionInstantTime);
|
||||
List<FileSlice> fileSliceList = getCurrentLatestFileSlices(table, cfg);
|
||||
assertTrue("Ensure latest file-slices are not empty", fileSliceList.stream().findAny().isPresent());
|
||||
assertFalse("Verify all file-slices have base-instant same as compaction instant",
|
||||
fileSliceList.stream().filter(fs -> !fs.getBaseInstantTime().equals(compactionInstantTime))
|
||||
.findAny().isPresent());
|
||||
assertFalse("Verify all file-slices have data-files",
|
||||
fileSliceList.stream().filter(fs -> !fs.getDataFile().isPresent()).findAny().isPresent());
|
||||
|
||||
if (hasDeltaCommitAfterPendingCompaction) {
|
||||
assertFalse("Verify all file-slices have atleast one log-file",
|
||||
fileSliceList.stream().filter(fs -> fs.getLogFiles().count() == 0).findAny().isPresent());
|
||||
} else {
|
||||
assertFalse("Verify all file-slices have no log-files",
|
||||
fileSliceList.stream().filter(fs -> fs.getLogFiles().count() > 0).findAny().isPresent());
|
||||
}
|
||||
|
||||
// verify that there is a commit
|
||||
table = getHoodieTable(
|
||||
new HoodieTableMetaClient(jsc.hadoopConfiguration(), cfg.getBasePath(), true), cfg);
|
||||
HoodieTimeline timeline = table.getMetaClient().getCommitTimeline().filterCompletedInstants();
|
||||
String latestCompactionCommitTime = timeline.lastInstant().get().getTimestamp();
|
||||
assertEquals("Expect compaction instant time to be the latest commit time",
|
||||
latestCompactionCommitTime, compactionInstantTime);
|
||||
Assert.assertEquals("Must contain expected records", expectedNumRecs,
|
||||
HoodieClientTestUtils.readSince(basePath, sqlContext, timeline, "000").count());
|
||||
|
||||
}
|
||||
|
||||
private List<WriteStatus> createNextDeltaCommit(String instantTime, List<HoodieRecord> records,
|
||||
HoodieWriteClient client, HoodieTableMetaClient metaClient, HoodieWriteConfig cfg, boolean skipCommit) {
|
||||
JavaRDD<HoodieRecord> writeRecords = jsc.parallelize(records, 1);
|
||||
|
||||
client.startCommitWithTime(instantTime);
|
||||
|
||||
JavaRDD<WriteStatus> statuses = client.upsert(writeRecords, instantTime);
|
||||
List<WriteStatus> statusList = statuses.collect();
|
||||
assertNoWriteErrors(statusList);
|
||||
if (!cfg.shouldAutoCommit() && !skipCommit) {
|
||||
client.commit(instantTime, statuses);
|
||||
}
|
||||
|
||||
Option<HoodieInstant> deltaCommit = metaClient.getActiveTimeline().reload().getDeltaCommitTimeline()
|
||||
.filterCompletedInstants().lastInstant();
|
||||
if (skipCommit && !cfg.shouldAutoCommit()) {
|
||||
assertTrue("Delta commit should not be latest instant",
|
||||
deltaCommit.get().getTimestamp().compareTo(instantTime) < 0);
|
||||
} else {
|
||||
assertTrue(deltaCommit.isPresent());
|
||||
assertEquals("Delta commit should be latest instant", instantTime, deltaCommit.get().getTimestamp());
|
||||
}
|
||||
return statusList;
|
||||
}
|
||||
|
||||
private List<HoodieDataFile> getCurrentLatestDataFiles(HoodieTable table, HoodieWriteConfig cfg) throws IOException {
|
||||
FileStatus[] allFiles = HoodieTestUtils.listAllDataFilesInPath(table.getMetaClient().getFs(), cfg.getBasePath());
|
||||
HoodieTableFileSystemView
|
||||
view = new HoodieTableFileSystemView(table.getMetaClient(), table.getCompletedCommitsTimeline(), allFiles);
|
||||
List<HoodieDataFile> dataFilesToRead = view.getLatestDataFiles().collect(Collectors.toList());
|
||||
return dataFilesToRead;
|
||||
}
|
||||
|
||||
private List<FileSlice> getCurrentLatestFileSlices(HoodieTable table, HoodieWriteConfig cfg) throws IOException {
|
||||
HoodieTableFileSystemView view = new HoodieTableFileSystemView(table.getMetaClient(),
|
||||
table.getMetaClient().getActiveTimeline().reload().getCommitsAndCompactionTimeline());
|
||||
List<FileSlice> fileSliceList =
|
||||
Arrays.asList(HoodieTestDataGenerator.DEFAULT_PARTITION_PATHS).stream().flatMap(partition ->
|
||||
view.getLatestFileSlices(partition)).collect(Collectors.toList());
|
||||
return fileSliceList;
|
||||
}
|
||||
|
||||
protected HoodieTableType getTableType() {
|
||||
return HoodieTableType.MERGE_ON_READ;
|
||||
}
|
||||
}
|
||||
1036
hudi-client/src/test/java/org/apache/hudi/TestCleaner.java
Normal file
1036
hudi-client/src/test/java/org/apache/hudi/TestCleaner.java
Normal file
File diff suppressed because it is too large
Load Diff
@@ -0,0 +1,328 @@
|
||||
/*
|
||||
* 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;
|
||||
|
||||
import static org.junit.Assert.assertEquals;
|
||||
import static org.junit.Assert.assertFalse;
|
||||
import static org.junit.Assert.assertTrue;
|
||||
import static org.junit.Assert.fail;
|
||||
|
||||
import java.io.File;
|
||||
import java.io.IOException;
|
||||
import java.util.List;
|
||||
import java.util.stream.Collectors;
|
||||
import org.apache.hudi.common.HoodieTestDataGenerator;
|
||||
import org.apache.hudi.common.model.HoodieCleaningPolicy;
|
||||
import org.apache.hudi.common.model.HoodieDataFile;
|
||||
import org.apache.hudi.common.model.HoodieRecord;
|
||||
import org.apache.hudi.common.model.HoodieTestUtils;
|
||||
import org.apache.hudi.common.table.HoodieTableMetaClient;
|
||||
import org.apache.hudi.common.table.TableFileSystemView.ReadOptimizedView;
|
||||
import org.apache.hudi.common.table.timeline.HoodieInstant;
|
||||
import org.apache.hudi.common.util.FSUtils;
|
||||
import org.apache.hudi.config.HoodieCompactionConfig;
|
||||
import org.apache.hudi.config.HoodieIndexConfig;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
import org.apache.hudi.exception.HoodieRollbackException;
|
||||
import org.apache.hudi.index.HoodieIndex;
|
||||
import org.apache.hudi.table.HoodieTable;
|
||||
import org.apache.spark.api.java.JavaRDD;
|
||||
import org.junit.Test;
|
||||
|
||||
/**
|
||||
* Test Cases for rollback of snapshots and commits
|
||||
*/
|
||||
public class TestClientRollback extends TestHoodieClientBase {
|
||||
|
||||
@Override
|
||||
public void tearDown() throws IOException {
|
||||
super.tearDown();
|
||||
}
|
||||
|
||||
/**
|
||||
* Test case for rollback-savepoint interaction
|
||||
*/
|
||||
@Test
|
||||
public void testSavepointAndRollback() throws Exception {
|
||||
HoodieWriteConfig cfg = getConfigBuilder().withCompactionConfig(
|
||||
HoodieCompactionConfig.newBuilder().withCleanerPolicy(HoodieCleaningPolicy.KEEP_LATEST_COMMITS).retainCommits(1)
|
||||
.build()).build();
|
||||
HoodieWriteClient client = getHoodieWriteClient(cfg);
|
||||
HoodieTestDataGenerator.writePartitionMetadata(fs, HoodieTestDataGenerator.DEFAULT_PARTITION_PATHS, basePath);
|
||||
|
||||
/**
|
||||
* Write 1 (only inserts)
|
||||
*/
|
||||
String newCommitTime = "001";
|
||||
client.startCommitWithTime(newCommitTime);
|
||||
|
||||
List<HoodieRecord> records = dataGen.generateInserts(newCommitTime, 200);
|
||||
JavaRDD<HoodieRecord> writeRecords = jsc.parallelize(records, 1);
|
||||
|
||||
List<WriteStatus> statuses = client.upsert(writeRecords, newCommitTime).collect();
|
||||
assertNoWriteErrors(statuses);
|
||||
|
||||
/**
|
||||
* Write 2 (updates)
|
||||
*/
|
||||
newCommitTime = "002";
|
||||
client.startCommitWithTime(newCommitTime);
|
||||
|
||||
records = dataGen.generateUpdates(newCommitTime, records);
|
||||
statuses = client.upsert(jsc.parallelize(records, 1), newCommitTime).collect();
|
||||
// Verify there are no errors
|
||||
assertNoWriteErrors(statuses);
|
||||
|
||||
client.savepoint("hoodie-unit-test", "test");
|
||||
|
||||
/**
|
||||
* Write 3 (updates)
|
||||
*/
|
||||
newCommitTime = "003";
|
||||
client.startCommitWithTime(newCommitTime);
|
||||
|
||||
records = dataGen.generateUpdates(newCommitTime, records);
|
||||
statuses = client.upsert(jsc.parallelize(records, 1), newCommitTime).collect();
|
||||
// Verify there are no errors
|
||||
assertNoWriteErrors(statuses);
|
||||
List<String> partitionPaths = FSUtils.getAllPartitionPaths(fs, cfg.getBasePath(),
|
||||
getConfig().shouldAssumeDatePartitioning());
|
||||
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath);
|
||||
HoodieTable table = HoodieTable.getHoodieTable(metaClient, getConfig(), jsc);
|
||||
final ReadOptimizedView view1 = table.getROFileSystemView();
|
||||
|
||||
List<HoodieDataFile> dataFiles = partitionPaths.stream().flatMap(s -> {
|
||||
return view1.getAllDataFiles(s).filter(f -> f.getCommitTime().equals("003"));
|
||||
}).collect(Collectors.toList());
|
||||
assertEquals("The data files for commit 003 should be present", 3, dataFiles.size());
|
||||
|
||||
dataFiles = partitionPaths.stream().flatMap(s -> {
|
||||
return view1.getAllDataFiles(s).filter(f -> f.getCommitTime().equals("002"));
|
||||
}).collect(Collectors.toList());
|
||||
assertEquals("The data files for commit 002 should be present", 3, dataFiles.size());
|
||||
|
||||
/**
|
||||
* Write 4 (updates)
|
||||
*/
|
||||
newCommitTime = "004";
|
||||
client.startCommitWithTime(newCommitTime);
|
||||
|
||||
records = dataGen.generateUpdates(newCommitTime, records);
|
||||
statuses = client.upsert(jsc.parallelize(records, 1), newCommitTime).collect();
|
||||
// Verify there are no errors
|
||||
assertNoWriteErrors(statuses);
|
||||
|
||||
metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath);
|
||||
table = HoodieTable.getHoodieTable(metaClient, getConfig(), jsc);
|
||||
final ReadOptimizedView view2 = table.getROFileSystemView();
|
||||
|
||||
dataFiles = partitionPaths.stream().flatMap(s -> {
|
||||
return view2.getAllDataFiles(s).filter(f -> f.getCommitTime().equals("004"));
|
||||
}).collect(Collectors.toList());
|
||||
assertEquals("The data files for commit 004 should be present", 3, dataFiles.size());
|
||||
|
||||
// rolling back to a non existent savepoint must not succeed
|
||||
try {
|
||||
client.rollbackToSavepoint("001");
|
||||
fail("Rolling back to non-existent savepoint should not be allowed");
|
||||
} catch (HoodieRollbackException e) {
|
||||
// this is good
|
||||
}
|
||||
|
||||
// rollback to savepoint 002
|
||||
HoodieInstant savepoint = table.getCompletedSavepointTimeline().getInstants().findFirst().get();
|
||||
client.rollbackToSavepoint(savepoint.getTimestamp());
|
||||
|
||||
metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath);
|
||||
table = HoodieTable.getHoodieTable(metaClient, getConfig(), jsc);
|
||||
final ReadOptimizedView view3 = table.getROFileSystemView();
|
||||
dataFiles = partitionPaths.stream().flatMap(s -> {
|
||||
return view3.getAllDataFiles(s).filter(f -> f.getCommitTime().equals("002"));
|
||||
}).collect(Collectors.toList());
|
||||
assertEquals("The data files for commit 002 be available", 3, dataFiles.size());
|
||||
|
||||
dataFiles = partitionPaths.stream().flatMap(s -> {
|
||||
return view3.getAllDataFiles(s).filter(f -> f.getCommitTime().equals("003"));
|
||||
}).collect(Collectors.toList());
|
||||
assertEquals("The data files for commit 003 should be rolled back", 0, dataFiles.size());
|
||||
|
||||
dataFiles = partitionPaths.stream().flatMap(s -> {
|
||||
return view3.getAllDataFiles(s).filter(f -> f.getCommitTime().equals("004"));
|
||||
}).collect(Collectors.toList());
|
||||
assertEquals("The data files for commit 004 should be rolled back", 0, dataFiles.size());
|
||||
}
|
||||
|
||||
/**
|
||||
* Test Cases for effects of rollbacking completed/inflight commits
|
||||
*/
|
||||
@Test
|
||||
public void testRollbackCommit() throws Exception {
|
||||
// Let's create some commit files and parquet files
|
||||
String commitTime1 = "20160501010101";
|
||||
String commitTime2 = "20160502020601";
|
||||
String commitTime3 = "20160506030611";
|
||||
new File(basePath + "/.hoodie").mkdirs();
|
||||
HoodieTestDataGenerator
|
||||
.writePartitionMetadata(fs, new String[]{"2016/05/01", "2016/05/02", "2016/05/06"}, basePath);
|
||||
|
||||
// Only first two have commit files
|
||||
HoodieTestUtils.createCommitFiles(basePath, commitTime1, commitTime2);
|
||||
// Third one has a .inflight intermediate commit file
|
||||
HoodieTestUtils.createInflightCommitFiles(basePath, commitTime3);
|
||||
|
||||
// Make commit1
|
||||
String file11 = HoodieTestUtils.createDataFile(basePath, "2016/05/01", commitTime1, "id11");
|
||||
String file12 = HoodieTestUtils.createDataFile(basePath, "2016/05/02", commitTime1, "id12");
|
||||
String file13 = HoodieTestUtils.createDataFile(basePath, "2016/05/06", commitTime1, "id13");
|
||||
|
||||
// Make commit2
|
||||
String file21 = HoodieTestUtils.createDataFile(basePath, "2016/05/01", commitTime2, "id21");
|
||||
String file22 = HoodieTestUtils.createDataFile(basePath, "2016/05/02", commitTime2, "id22");
|
||||
String file23 = HoodieTestUtils.createDataFile(basePath, "2016/05/06", commitTime2, "id23");
|
||||
|
||||
// Make commit3
|
||||
String file31 = HoodieTestUtils.createDataFile(basePath, "2016/05/01", commitTime3, "id31");
|
||||
String file32 = HoodieTestUtils.createDataFile(basePath, "2016/05/02", commitTime3, "id32");
|
||||
String file33 = HoodieTestUtils.createDataFile(basePath, "2016/05/06", commitTime3, "id33");
|
||||
|
||||
HoodieWriteConfig config = HoodieWriteConfig.newBuilder().withPath(basePath).withIndexConfig(
|
||||
HoodieIndexConfig.newBuilder().withIndexType(HoodieIndex.IndexType.INMEMORY).build()).build();
|
||||
|
||||
HoodieWriteClient client = getHoodieWriteClient(config, false);
|
||||
|
||||
// Rollback commit 1 (this should fail, since commit2 is still around)
|
||||
try {
|
||||
client.rollback(commitTime1);
|
||||
assertTrue("Should have thrown an exception ", false);
|
||||
} catch (HoodieRollbackException hrbe) {
|
||||
// should get here
|
||||
}
|
||||
|
||||
// Rollback commit3
|
||||
client.rollback(commitTime3);
|
||||
assertFalse(HoodieTestUtils.doesInflightExist(basePath, commitTime3));
|
||||
assertFalse(HoodieTestUtils.doesDataFileExist(basePath, "2016/05/01", commitTime3, file31)
|
||||
|| HoodieTestUtils.doesDataFileExist(basePath, "2016/05/02", commitTime3, file32)
|
||||
|| HoodieTestUtils.doesDataFileExist(basePath, "2016/05/06", commitTime3, file33));
|
||||
|
||||
// simulate partial failure, where .inflight was not deleted, but data files were.
|
||||
HoodieTestUtils.createInflightCommitFiles(basePath, commitTime3);
|
||||
client.rollback(commitTime3);
|
||||
assertFalse(HoodieTestUtils.doesInflightExist(basePath, commitTime3));
|
||||
|
||||
// Rollback commit2
|
||||
client.rollback(commitTime2);
|
||||
assertFalse(HoodieTestUtils.doesCommitExist(basePath, commitTime2));
|
||||
assertFalse(HoodieTestUtils.doesInflightExist(basePath, commitTime2));
|
||||
assertFalse(HoodieTestUtils.doesDataFileExist(basePath, "2016/05/01", commitTime2, file21)
|
||||
|| HoodieTestUtils.doesDataFileExist(basePath, "2016/05/02", commitTime2, file22)
|
||||
|| HoodieTestUtils.doesDataFileExist(basePath, "2016/05/06", commitTime2, file23));
|
||||
|
||||
// simulate partial failure, where only .commit => .inflight renaming succeeded, leaving a
|
||||
// .inflight commit and a bunch of data files around.
|
||||
HoodieTestUtils.createInflightCommitFiles(basePath, commitTime2);
|
||||
file21 = HoodieTestUtils.createDataFile(basePath, "2016/05/01", commitTime2, "id21");
|
||||
file22 = HoodieTestUtils.createDataFile(basePath, "2016/05/02", commitTime2, "id22");
|
||||
file23 = HoodieTestUtils.createDataFile(basePath, "2016/05/06", commitTime2, "id23");
|
||||
|
||||
client.rollback(commitTime2);
|
||||
assertFalse(HoodieTestUtils.doesCommitExist(basePath, commitTime2));
|
||||
assertFalse(HoodieTestUtils.doesInflightExist(basePath, commitTime2));
|
||||
assertFalse(HoodieTestUtils.doesDataFileExist(basePath, "2016/05/01", commitTime2, file21)
|
||||
|| HoodieTestUtils.doesDataFileExist(basePath, "2016/05/02", commitTime2, file22)
|
||||
|| HoodieTestUtils.doesDataFileExist(basePath, "2016/05/06", commitTime2, file23));
|
||||
|
||||
// Let's rollback commit1, Check results
|
||||
client.rollback(commitTime1);
|
||||
assertFalse(HoodieTestUtils.doesCommitExist(basePath, commitTime1));
|
||||
assertFalse(HoodieTestUtils.doesInflightExist(basePath, commitTime1));
|
||||
assertFalse(HoodieTestUtils.doesDataFileExist(basePath, "2016/05/01", commitTime1, file11)
|
||||
|| HoodieTestUtils.doesDataFileExist(basePath, "2016/05/02", commitTime1, file12)
|
||||
|| HoodieTestUtils.doesDataFileExist(basePath, "2016/05/06", commitTime1, file13));
|
||||
}
|
||||
|
||||
/**
|
||||
* Test auto-rollback of commits which are in flight
|
||||
*/
|
||||
@Test
|
||||
public void testAutoRollbackInflightCommit() throws Exception {
|
||||
// Let's create some commit files and parquet files
|
||||
String commitTime1 = "20160501010101";
|
||||
String commitTime2 = "20160502020601";
|
||||
String commitTime3 = "20160506030611";
|
||||
new File(basePath + "/.hoodie").mkdirs();
|
||||
HoodieTestDataGenerator
|
||||
.writePartitionMetadata(fs, new String[]{"2016/05/01", "2016/05/02", "2016/05/06"}, basePath);
|
||||
|
||||
// One good commit
|
||||
HoodieTestUtils.createCommitFiles(basePath, commitTime1);
|
||||
// Two inflight commits
|
||||
HoodieTestUtils.createInflightCommitFiles(basePath, commitTime2, commitTime3);
|
||||
|
||||
// Make commit1
|
||||
String file11 = HoodieTestUtils.createDataFile(basePath, "2016/05/01", commitTime1, "id11");
|
||||
String file12 = HoodieTestUtils.createDataFile(basePath, "2016/05/02", commitTime1, "id12");
|
||||
String file13 = HoodieTestUtils.createDataFile(basePath, "2016/05/06", commitTime1, "id13");
|
||||
|
||||
// Make commit2
|
||||
String file21 = HoodieTestUtils.createDataFile(basePath, "2016/05/01", commitTime2, "id21");
|
||||
String file22 = HoodieTestUtils.createDataFile(basePath, "2016/05/02", commitTime2, "id22");
|
||||
String file23 = HoodieTestUtils.createDataFile(basePath, "2016/05/06", commitTime2, "id23");
|
||||
|
||||
// Make commit3
|
||||
String file31 = HoodieTestUtils.createDataFile(basePath, "2016/05/01", commitTime3, "id31");
|
||||
String file32 = HoodieTestUtils.createDataFile(basePath, "2016/05/02", commitTime3, "id32");
|
||||
String file33 = HoodieTestUtils.createDataFile(basePath, "2016/05/06", commitTime3, "id33");
|
||||
|
||||
// Turn auto rollback off
|
||||
HoodieWriteConfig config = HoodieWriteConfig.newBuilder().withPath(basePath).withIndexConfig(
|
||||
HoodieIndexConfig.newBuilder().withIndexType(HoodieIndex.IndexType.INMEMORY).build()).build();
|
||||
|
||||
getHoodieWriteClient(config, false);
|
||||
|
||||
// Check results, nothing changed
|
||||
assertTrue(HoodieTestUtils.doesCommitExist(basePath, commitTime1));
|
||||
assertTrue(HoodieTestUtils.doesInflightExist(basePath, commitTime2));
|
||||
assertTrue(HoodieTestUtils.doesInflightExist(basePath, commitTime3));
|
||||
assertTrue(HoodieTestUtils.doesDataFileExist(basePath, "2016/05/01", commitTime3, file31)
|
||||
&& HoodieTestUtils.doesDataFileExist(basePath, "2016/05/02", commitTime3, file32)
|
||||
&& HoodieTestUtils.doesDataFileExist(basePath, "2016/05/06", commitTime3, file33));
|
||||
assertTrue(HoodieTestUtils.doesDataFileExist(basePath, "2016/05/01", commitTime2, file21)
|
||||
&& HoodieTestUtils.doesDataFileExist(basePath, "2016/05/02", commitTime2, file22)
|
||||
&& HoodieTestUtils.doesDataFileExist(basePath, "2016/05/06", commitTime2, file23));
|
||||
assertTrue(HoodieTestUtils.doesDataFileExist(basePath, "2016/05/01", commitTime1, file11)
|
||||
&& HoodieTestUtils.doesDataFileExist(basePath, "2016/05/02", commitTime1, file12)
|
||||
&& HoodieTestUtils.doesDataFileExist(basePath, "2016/05/06", commitTime1, file13));
|
||||
|
||||
// Turn auto rollback on
|
||||
getHoodieWriteClient(config, true).startCommit();
|
||||
assertTrue(HoodieTestUtils.doesCommitExist(basePath, commitTime1));
|
||||
assertFalse(HoodieTestUtils.doesInflightExist(basePath, commitTime2));
|
||||
assertFalse(HoodieTestUtils.doesInflightExist(basePath, commitTime3));
|
||||
assertFalse(HoodieTestUtils.doesDataFileExist(basePath, "2016/05/01", commitTime3, file31)
|
||||
|| HoodieTestUtils.doesDataFileExist(basePath, "2016/05/02", commitTime3, file32)
|
||||
|| HoodieTestUtils.doesDataFileExist(basePath, "2016/05/06", commitTime3, file33));
|
||||
assertFalse(HoodieTestUtils.doesDataFileExist(basePath, "2016/05/01", commitTime2, file21)
|
||||
|| HoodieTestUtils.doesDataFileExist(basePath, "2016/05/02", commitTime2, file22)
|
||||
|| HoodieTestUtils.doesDataFileExist(basePath, "2016/05/06", commitTime2, file23));
|
||||
assertTrue(HoodieTestUtils.doesDataFileExist(basePath, "2016/05/01", commitTime1, file11)
|
||||
&& HoodieTestUtils.doesDataFileExist(basePath, "2016/05/02", commitTime1, file12)
|
||||
&& HoodieTestUtils.doesDataFileExist(basePath, "2016/05/06", commitTime1, file13));
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,364 @@
|
||||
/*
|
||||
* 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;
|
||||
|
||||
import static org.apache.hudi.common.model.HoodieTableType.MERGE_ON_READ;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.Arrays;
|
||||
import java.util.HashSet;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
import java.util.stream.Collectors;
|
||||
import org.apache.hudi.CompactionAdminClient.ValidationOpResult;
|
||||
import org.apache.hudi.common.model.CompactionOperation;
|
||||
import org.apache.hudi.common.model.HoodieLogFile;
|
||||
import org.apache.hudi.common.model.HoodieTestUtils;
|
||||
import org.apache.hudi.common.table.HoodieTableMetaClient;
|
||||
import org.apache.hudi.common.table.view.HoodieTableFileSystemView;
|
||||
import org.apache.hudi.common.util.CompactionTestUtils;
|
||||
import org.apache.hudi.common.util.CompactionUtils;
|
||||
import org.apache.hudi.common.util.Option;
|
||||
import org.apache.hudi.common.util.collection.Pair;
|
||||
import org.apache.hudi.exception.HoodieException;
|
||||
import org.apache.hudi.exception.HoodieIOException;
|
||||
import org.junit.Assert;
|
||||
import org.junit.Before;
|
||||
import org.junit.Test;
|
||||
|
||||
public class TestCompactionAdminClient extends TestHoodieClientBase {
|
||||
|
||||
private HoodieTableMetaClient metaClient;
|
||||
private CompactionAdminClient client;
|
||||
|
||||
@Before
|
||||
public void init() throws IOException {
|
||||
super.init();
|
||||
metaClient = HoodieTestUtils.initTableType(HoodieTestUtils.getDefaultHadoopConf(), basePath, MERGE_ON_READ);
|
||||
client = new CompactionAdminClient(jsc, basePath);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void tearDown() throws IOException {
|
||||
super.tearDown();
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testUnscheduleCompactionPlan() throws Exception {
|
||||
int numEntriesPerInstant = 10;
|
||||
CompactionTestUtils
|
||||
.setupAndValidateCompactionOperations(metaClient, false, numEntriesPerInstant, numEntriesPerInstant,
|
||||
numEntriesPerInstant, numEntriesPerInstant);
|
||||
// THere are delta-commits after compaction instant
|
||||
validateUnSchedulePlan(client,
|
||||
"000", "001", numEntriesPerInstant, 2 * numEntriesPerInstant);
|
||||
// THere are delta-commits after compaction instant
|
||||
validateUnSchedulePlan(client,
|
||||
"002", "003", numEntriesPerInstant, 2 * numEntriesPerInstant);
|
||||
// THere are no delta-commits after compaction instant
|
||||
validateUnSchedulePlan(client,
|
||||
"004", "005", numEntriesPerInstant, 0);
|
||||
// THere are no delta-commits after compaction instant
|
||||
validateUnSchedulePlan(client,
|
||||
"006", "007", numEntriesPerInstant, 0);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testUnscheduleCompactionFileId() throws Exception {
|
||||
int numEntriesPerInstant = 10;
|
||||
CompactionTestUtils
|
||||
.setupAndValidateCompactionOperations(metaClient, false, numEntriesPerInstant, numEntriesPerInstant,
|
||||
numEntriesPerInstant, numEntriesPerInstant);
|
||||
Map<String, CompactionOperation> instantsWithOp =
|
||||
Arrays.asList("001", "003", "005", "007").stream().map(instant -> {
|
||||
try {
|
||||
return Pair.of(instant, CompactionUtils.getCompactionPlan(metaClient, instant));
|
||||
} catch (IOException ioe) {
|
||||
throw new HoodieException(ioe);
|
||||
}
|
||||
}).map(instantWithPlan -> instantWithPlan.getRight().getOperations().stream().map(op -> Pair.of(
|
||||
instantWithPlan.getLeft(), CompactionOperation.convertFromAvroRecordInstance(op))).findFirst().get())
|
||||
.collect(Collectors.toMap(Pair::getLeft, Pair::getRight));
|
||||
// THere are delta-commits after compaction instant
|
||||
validateUnScheduleFileId(client,
|
||||
"000", "001", instantsWithOp.get("001"), 2);
|
||||
// THere are delta-commits after compaction instant
|
||||
validateUnScheduleFileId(client,
|
||||
"002", "003", instantsWithOp.get("003"), 2);
|
||||
// THere are no delta-commits after compaction instant
|
||||
validateUnScheduleFileId(client,
|
||||
"004", "005", instantsWithOp.get("005"), 0);
|
||||
// THere are no delta-commits after compaction instant
|
||||
validateUnScheduleFileId(client,
|
||||
"006", "007", instantsWithOp.get("007"), 0);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testRepairCompactionPlan() throws Exception {
|
||||
int numEntriesPerInstant = 10;
|
||||
CompactionTestUtils
|
||||
.setupAndValidateCompactionOperations(metaClient,false, numEntriesPerInstant, numEntriesPerInstant,
|
||||
numEntriesPerInstant, numEntriesPerInstant);
|
||||
// THere are delta-commits after compaction instant
|
||||
validateRepair("000", "001", numEntriesPerInstant, 2 * numEntriesPerInstant);
|
||||
// THere are delta-commits after compaction instant
|
||||
validateRepair("002", "003", numEntriesPerInstant, 2 * numEntriesPerInstant);
|
||||
// THere are no delta-commits after compaction instant
|
||||
validateRepair("004", "005", numEntriesPerInstant, 0);
|
||||
// THere are no delta-commits after compaction instant
|
||||
validateRepair("006", "007", numEntriesPerInstant, 0);
|
||||
}
|
||||
|
||||
private void validateRepair(String ingestionInstant, String compactionInstant, int numEntriesPerInstant,
|
||||
int expNumRepairs) throws Exception {
|
||||
List<Pair<HoodieLogFile, HoodieLogFile>> renameFiles =
|
||||
validateUnSchedulePlan(client, ingestionInstant, compactionInstant, numEntriesPerInstant, expNumRepairs, true);
|
||||
metaClient = new HoodieTableMetaClient(metaClient.getHadoopConf(), basePath, true);
|
||||
List<ValidationOpResult> result = client.validateCompactionPlan(metaClient, compactionInstant, 1);
|
||||
if (expNumRepairs > 0) {
|
||||
Assert.assertTrue("Expect some failures in validation", result.stream().filter(r -> !r.isSuccess()).count() > 0);
|
||||
}
|
||||
// Now repair
|
||||
List<Pair<HoodieLogFile, HoodieLogFile>> undoFiles = result.stream().flatMap(r ->
|
||||
client.getRenamingActionsToAlignWithCompactionOperation(metaClient,
|
||||
compactionInstant, r.getOperation(), Option.empty()).stream())
|
||||
.map(rn -> {
|
||||
try {
|
||||
client.renameLogFile(metaClient, rn.getKey(), rn.getValue());
|
||||
} catch (IOException e) {
|
||||
throw new HoodieIOException(e.getMessage(), e);
|
||||
}
|
||||
return rn;
|
||||
}).collect(Collectors.toList());
|
||||
Map<String, String> renameFilesFromUndo =
|
||||
undoFiles.stream().collect(Collectors.toMap(p -> p.getRight().getPath().toString(),
|
||||
x -> x.getLeft().getPath().toString()));
|
||||
Map<String, String> expRenameFiles =
|
||||
renameFiles.stream().collect(Collectors.toMap(p -> p.getLeft().getPath().toString(),
|
||||
x -> x.getRight().getPath().toString()));
|
||||
if (expNumRepairs > 0) {
|
||||
Assert.assertFalse("Rename Files must be non-empty", renameFiles.isEmpty());
|
||||
} else {
|
||||
Assert.assertTrue("Rename Files must be empty", renameFiles.isEmpty());
|
||||
}
|
||||
expRenameFiles.entrySet().stream().forEach(r -> {
|
||||
logger.info("Key :" + r.getKey() + " renamed to " + r.getValue() + " rolled back to "
|
||||
+ renameFilesFromUndo.get(r.getKey()));
|
||||
});
|
||||
|
||||
Assert.assertEquals("Undo must completely rollback renames", expRenameFiles, renameFilesFromUndo);
|
||||
// Now expect validation to succeed
|
||||
result = client.validateCompactionPlan(metaClient, compactionInstant, 1);
|
||||
Assert.assertTrue("Expect no failures in validation", result.stream().filter(r -> !r.isSuccess()).count() == 0);
|
||||
Assert.assertEquals("Expected Num Repairs", expNumRepairs, undoFiles.size());
|
||||
}
|
||||
|
||||
/**
|
||||
* Enssure compaction plan is valid
|
||||
* @param compactionInstant Compaction Instant
|
||||
* @throws Exception
|
||||
*/
|
||||
private void ensureValidCompactionPlan(String compactionInstant) throws Exception {
|
||||
metaClient = new HoodieTableMetaClient(metaClient.getHadoopConf(), basePath, true);
|
||||
// Ensure compaction-plan is good to begin with
|
||||
List<ValidationOpResult> validationResults = client.validateCompactionPlan(metaClient,
|
||||
compactionInstant, 1);
|
||||
Assert.assertFalse("Some validations failed",
|
||||
validationResults.stream().filter(v -> !v.isSuccess()).findAny().isPresent());
|
||||
}
|
||||
|
||||
private void validateRenameFiles(List<Pair<HoodieLogFile, HoodieLogFile>> renameFiles,
|
||||
String ingestionInstant, String compactionInstant, HoodieTableFileSystemView fsView) {
|
||||
// Ensure new names of log-files are on expected lines
|
||||
Set<HoodieLogFile> uniqNewLogFiles = new HashSet<>();
|
||||
Set<HoodieLogFile> uniqOldLogFiles = new HashSet<>();
|
||||
|
||||
renameFiles.stream().forEach(lfPair -> {
|
||||
Assert.assertFalse("Old Log File Names do not collide", uniqOldLogFiles.contains(lfPair.getKey()));
|
||||
Assert.assertFalse("New Log File Names do not collide", uniqNewLogFiles.contains(lfPair.getValue()));
|
||||
uniqOldLogFiles.add(lfPair.getKey());
|
||||
uniqNewLogFiles.add(lfPair.getValue());
|
||||
});
|
||||
|
||||
renameFiles.stream().forEach(lfPair -> {
|
||||
HoodieLogFile oldLogFile = lfPair.getLeft();
|
||||
HoodieLogFile newLogFile = lfPair.getValue();
|
||||
Assert.assertEquals("Base Commit time is expected", ingestionInstant, newLogFile.getBaseCommitTime());
|
||||
Assert.assertEquals("Base Commit time is expected", compactionInstant, oldLogFile.getBaseCommitTime());
|
||||
Assert.assertEquals("File Id is expected", oldLogFile.getFileId(), newLogFile.getFileId());
|
||||
HoodieLogFile lastLogFileBeforeCompaction =
|
||||
fsView.getLatestMergedFileSlicesBeforeOrOn(HoodieTestUtils.DEFAULT_PARTITION_PATHS[0], ingestionInstant)
|
||||
.filter(fs -> fs.getFileId().equals(oldLogFile.getFileId()))
|
||||
.map(fs -> fs.getLogFiles().findFirst().get()).findFirst().get();
|
||||
Assert.assertEquals("Log Version expected",
|
||||
lastLogFileBeforeCompaction.getLogVersion() + oldLogFile.getLogVersion(),
|
||||
newLogFile.getLogVersion());
|
||||
Assert.assertTrue("Log version does not collide",
|
||||
newLogFile.getLogVersion() > lastLogFileBeforeCompaction.getLogVersion());
|
||||
});
|
||||
}
|
||||
|
||||
/**
|
||||
* Validate Unschedule operations
|
||||
*/
|
||||
private List<Pair<HoodieLogFile, HoodieLogFile>> validateUnSchedulePlan(CompactionAdminClient client,
|
||||
String ingestionInstant, String compactionInstant, int numEntriesPerInstant, int expNumRenames)
|
||||
throws Exception {
|
||||
return validateUnSchedulePlan(client, ingestionInstant, compactionInstant, numEntriesPerInstant,
|
||||
expNumRenames, false);
|
||||
}
|
||||
|
||||
/**
|
||||
* Validate Unschedule operations
|
||||
*/
|
||||
private List<Pair<HoodieLogFile, HoodieLogFile>> validateUnSchedulePlan(CompactionAdminClient client,
|
||||
String ingestionInstant, String compactionInstant, int numEntriesPerInstant, int expNumRenames,
|
||||
boolean skipUnSchedule) throws Exception {
|
||||
|
||||
ensureValidCompactionPlan(compactionInstant);
|
||||
|
||||
// Check suggested rename operations
|
||||
List<Pair<HoodieLogFile, HoodieLogFile>> renameFiles =
|
||||
client.getRenamingActionsForUnschedulingCompactionPlan(metaClient, compactionInstant, 1,
|
||||
Option.empty(), false);
|
||||
metaClient = new HoodieTableMetaClient(metaClient.getHadoopConf(), basePath, true);
|
||||
|
||||
// Log files belonging to file-slices created because of compaction request must be renamed
|
||||
|
||||
Set<HoodieLogFile> gotLogFilesToBeRenamed = renameFiles.stream().map(p -> p.getLeft()).collect(Collectors.toSet());
|
||||
final HoodieTableFileSystemView fsView =
|
||||
new HoodieTableFileSystemView(metaClient, metaClient.getCommitsAndCompactionTimeline());
|
||||
Set<HoodieLogFile> expLogFilesToBeRenamed = fsView.getLatestFileSlices(HoodieTestUtils.DEFAULT_PARTITION_PATHS[0])
|
||||
.filter(fs -> fs.getBaseInstantTime().equals(compactionInstant))
|
||||
.flatMap(fs -> fs.getLogFiles())
|
||||
.collect(Collectors.toSet());
|
||||
Assert.assertEquals("Log files belonging to file-slices created because of compaction request must be renamed",
|
||||
expLogFilesToBeRenamed, gotLogFilesToBeRenamed);
|
||||
|
||||
if (skipUnSchedule) {
|
||||
// Do the renaming only but do not touch the compaction plan - Needed for repair tests
|
||||
renameFiles.stream().forEach(lfPair -> {
|
||||
try {
|
||||
client.renameLogFile(metaClient, lfPair.getLeft(), lfPair.getRight());
|
||||
} catch (IOException e) {
|
||||
throw new HoodieIOException(e.getMessage(), e);
|
||||
}
|
||||
});
|
||||
} else {
|
||||
validateRenameFiles(renameFiles, ingestionInstant, compactionInstant, fsView);
|
||||
}
|
||||
|
||||
Map<String, Long> fileIdToCountsBeforeRenaming =
|
||||
fsView.getLatestMergedFileSlicesBeforeOrOn(HoodieTestUtils.DEFAULT_PARTITION_PATHS[0], compactionInstant)
|
||||
.filter(fs -> fs.getBaseInstantTime().equals(ingestionInstant))
|
||||
.map(fs -> Pair.of(fs.getFileId(), fs.getLogFiles().count()))
|
||||
.collect(Collectors.toMap(Pair::getKey, Pair::getValue));
|
||||
|
||||
// Call the main unschedule API
|
||||
|
||||
client.unscheduleCompactionPlan(compactionInstant, false, 1, false);
|
||||
|
||||
metaClient = new HoodieTableMetaClient(metaClient.getHadoopConf(), basePath, true);
|
||||
final HoodieTableFileSystemView newFsView =
|
||||
new HoodieTableFileSystemView(metaClient, metaClient.getCommitsAndCompactionTimeline());
|
||||
// Expect all file-slice whose base-commit is same as compaction commit to contain no new Log files
|
||||
newFsView.getLatestFileSlicesBeforeOrOn(HoodieTestUtils.DEFAULT_PARTITION_PATHS[0], compactionInstant, true)
|
||||
.filter(fs -> fs.getBaseInstantTime().equals(compactionInstant)).forEach(fs -> {
|
||||
Assert.assertFalse("No Data file must be present", fs.getDataFile().isPresent());
|
||||
Assert.assertTrue("No Log Files", fs.getLogFiles().count() == 0);
|
||||
});
|
||||
|
||||
// Ensure same number of log-files before and after renaming per fileId
|
||||
Map<String, Long> fileIdToCountsAfterRenaming =
|
||||
newFsView.getAllFileGroups(HoodieTestUtils.DEFAULT_PARTITION_PATHS[0]).flatMap(fg -> fg.getAllFileSlices())
|
||||
.filter(fs -> fs.getBaseInstantTime().equals(ingestionInstant))
|
||||
.map(fs -> Pair.of(fs.getFileId(), fs.getLogFiles().count()))
|
||||
.collect(Collectors.toMap(Pair::getKey, Pair::getValue));
|
||||
|
||||
Assert.assertEquals("Each File Id has same number of log-files",
|
||||
fileIdToCountsBeforeRenaming, fileIdToCountsAfterRenaming);
|
||||
Assert.assertEquals("Not Empty", numEntriesPerInstant, fileIdToCountsAfterRenaming.size());
|
||||
Assert.assertEquals("Expected number of renames", expNumRenames, renameFiles.size());
|
||||
return renameFiles;
|
||||
}
|
||||
|
||||
/**
|
||||
* Validate Unschedule operations
|
||||
*/
|
||||
private void validateUnScheduleFileId(CompactionAdminClient client, String ingestionInstant,
|
||||
String compactionInstant, CompactionOperation op, int expNumRenames) throws Exception {
|
||||
|
||||
ensureValidCompactionPlan(compactionInstant);
|
||||
|
||||
// Check suggested rename operations
|
||||
List<Pair<HoodieLogFile, HoodieLogFile>> renameFiles =
|
||||
client.getRenamingActionsForUnschedulingCompactionOperation(metaClient, compactionInstant, op,
|
||||
Option.empty(), false);
|
||||
metaClient = new HoodieTableMetaClient(metaClient.getHadoopConf(), basePath, true);
|
||||
|
||||
// Log files belonging to file-slices created because of compaction request must be renamed
|
||||
|
||||
Set<HoodieLogFile> gotLogFilesToBeRenamed = renameFiles.stream().map(p -> p.getLeft()).collect(Collectors.toSet());
|
||||
final HoodieTableFileSystemView fsView =
|
||||
new HoodieTableFileSystemView(metaClient, metaClient.getCommitsAndCompactionTimeline());
|
||||
Set<HoodieLogFile> expLogFilesToBeRenamed = fsView.getLatestFileSlices(HoodieTestUtils.DEFAULT_PARTITION_PATHS[0])
|
||||
.filter(fs -> fs.getBaseInstantTime().equals(compactionInstant))
|
||||
.filter(fs -> fs.getFileId().equals(op.getFileId()))
|
||||
.flatMap(fs -> fs.getLogFiles())
|
||||
.collect(Collectors.toSet());
|
||||
Assert.assertEquals("Log files belonging to file-slices created because of compaction request must be renamed",
|
||||
expLogFilesToBeRenamed, gotLogFilesToBeRenamed);
|
||||
validateRenameFiles(renameFiles, ingestionInstant, compactionInstant, fsView);
|
||||
|
||||
Map<String, Long> fileIdToCountsBeforeRenaming =
|
||||
fsView.getLatestMergedFileSlicesBeforeOrOn(HoodieTestUtils.DEFAULT_PARTITION_PATHS[0], compactionInstant)
|
||||
.filter(fs -> fs.getBaseInstantTime().equals(ingestionInstant))
|
||||
.filter(fs -> fs.getFileId().equals(op.getFileId()))
|
||||
.map(fs -> Pair.of(fs.getFileId(), fs.getLogFiles().count()))
|
||||
.collect(Collectors.toMap(Pair::getKey, Pair::getValue));
|
||||
|
||||
// Call the main unschedule API
|
||||
client.unscheduleCompactionFileId(op.getFileGroupId(), false, false);
|
||||
|
||||
metaClient = new HoodieTableMetaClient(metaClient.getHadoopConf(), basePath, true);
|
||||
final HoodieTableFileSystemView newFsView =
|
||||
new HoodieTableFileSystemView(metaClient, metaClient.getCommitsAndCompactionTimeline());
|
||||
// Expect all file-slice whose base-commit is same as compaction commit to contain no new Log files
|
||||
newFsView.getLatestFileSlicesBeforeOrOn(HoodieTestUtils.DEFAULT_PARTITION_PATHS[0], compactionInstant, true)
|
||||
.filter(fs -> fs.getBaseInstantTime().equals(compactionInstant))
|
||||
.filter(fs -> fs.getFileId().equals(op.getFileId())).forEach(fs -> {
|
||||
Assert.assertFalse("No Data file must be present", fs.getDataFile().isPresent());
|
||||
Assert.assertTrue("No Log Files", fs.getLogFiles().count() == 0);
|
||||
});
|
||||
|
||||
// Ensure same number of log-files before and after renaming per fileId
|
||||
Map<String, Long> fileIdToCountsAfterRenaming =
|
||||
newFsView.getAllFileGroups(HoodieTestUtils.DEFAULT_PARTITION_PATHS[0]).flatMap(fg -> fg.getAllFileSlices())
|
||||
.filter(fs -> fs.getBaseInstantTime().equals(ingestionInstant))
|
||||
.filter(fs -> fs.getFileId().equals(op.getFileId()))
|
||||
.map(fs -> Pair.of(fs.getFileId(), fs.getLogFiles().count()))
|
||||
.collect(Collectors.toMap(Pair::getKey, Pair::getValue));
|
||||
|
||||
Assert.assertEquals("Each File Id has same number of log-files",
|
||||
fileIdToCountsBeforeRenaming, fileIdToCountsAfterRenaming);
|
||||
Assert.assertEquals("Not Empty", 1, fileIdToCountsAfterRenaming.size());
|
||||
Assert.assertEquals("Expected number of renames", expNumRenames, renameFiles.size());
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,121 @@
|
||||
/*
|
||||
* 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;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.Arrays;
|
||||
import java.util.concurrent.TimeoutException;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.LocalFileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hudi.common.HoodieClientTestUtils;
|
||||
import org.apache.hudi.common.util.ConsistencyGuard;
|
||||
import org.apache.hudi.common.util.ConsistencyGuardConfig;
|
||||
import org.apache.hudi.common.util.FSUtils;
|
||||
import org.apache.hudi.common.util.FailSafeConsistencyGuard;
|
||||
import org.junit.Before;
|
||||
import org.junit.Test;
|
||||
import org.junit.rules.TemporaryFolder;
|
||||
|
||||
public class TestConsistencyGuard {
|
||||
private String basePath;
|
||||
protected transient FileSystem fs;
|
||||
|
||||
@Before
|
||||
public void setup() throws IOException {
|
||||
TemporaryFolder testFolder = new TemporaryFolder();
|
||||
testFolder.create();
|
||||
basePath = testFolder.getRoot().getAbsolutePath();
|
||||
fs = FSUtils.getFs(basePath, new Configuration());
|
||||
if (fs instanceof LocalFileSystem) {
|
||||
LocalFileSystem lfs = (LocalFileSystem) fs;
|
||||
// With LocalFileSystem, with checksum disabled, fs.open() returns an inputStream which is FSInputStream
|
||||
// This causes ClassCastExceptions in LogRecordScanner (and potentially other places) calling fs.open
|
||||
// So, for the tests, we enforce checksum verification to circumvent the problem
|
||||
lfs.setVerifyChecksum(true);
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testCheckPassingAppearAndDisAppear() throws Exception {
|
||||
HoodieClientTestUtils.fakeDataFile(basePath, "partition/path", "000", "f1");
|
||||
HoodieClientTestUtils.fakeDataFile(basePath, "partition/path", "000", "f2");
|
||||
HoodieClientTestUtils.fakeDataFile(basePath, "partition/path", "000", "f3");
|
||||
|
||||
ConsistencyGuard passing = new FailSafeConsistencyGuard(fs, getConsistencyGuardConfig(1, 1000, 1000));
|
||||
passing.waitTillFileAppears(new Path(basePath + "/partition/path/f1_1-0-1_000.parquet"));
|
||||
passing.waitTillFileAppears(new Path(basePath + "/partition/path/f2_1-0-1_000.parquet"));
|
||||
passing.waitTillAllFilesAppear(basePath + "/partition/path",
|
||||
Arrays.asList(basePath + "/partition/path/f1_1-0-1_000.parquet",
|
||||
basePath + "/partition/path/f2_1-0-1_000.parquet"));
|
||||
|
||||
fs.delete(new Path(basePath + "/partition/path/f1_1-0-1_000.parquet"), false);
|
||||
fs.delete(new Path(basePath + "/partition/path/f2_1-0-1_000.parquet"), false);
|
||||
passing.waitTillFileDisappears(new Path(basePath + "/partition/path/f1_1-0-1_000.parquet"));
|
||||
passing.waitTillFileDisappears(new Path(basePath + "/partition/path/f2_1-0-1_000.parquet"));
|
||||
passing.waitTillAllFilesDisappear(basePath + "/partition/path",
|
||||
Arrays.asList(basePath + "/partition/path/f1_1-0-1_000.parquet",
|
||||
basePath + "/partition/path/f2_1-0-1_000.parquet"));
|
||||
}
|
||||
|
||||
@Test(expected = TimeoutException.class)
|
||||
public void testCheckFailingAppear() throws Exception {
|
||||
HoodieClientTestUtils.fakeDataFile(basePath, "partition/path", "000", "f1");
|
||||
ConsistencyGuard passing = new FailSafeConsistencyGuard(fs, getConsistencyGuardConfig());
|
||||
passing.waitTillAllFilesAppear(basePath + "/partition/path",
|
||||
Arrays.asList(basePath + "/partition/path/f1_1-0-2_000.parquet",
|
||||
basePath + "/partition/path/f2_1-0-2_000.parquet"));
|
||||
}
|
||||
|
||||
|
||||
@Test(expected = TimeoutException.class)
|
||||
public void testCheckFailingAppears() throws Exception {
|
||||
HoodieClientTestUtils.fakeDataFile(basePath, "partition/path", "000", "f1");
|
||||
ConsistencyGuard passing = new FailSafeConsistencyGuard(fs, getConsistencyGuardConfig());
|
||||
passing.waitTillFileAppears(new Path(basePath + "/partition/path/f1_1-0-2_000.parquet"));
|
||||
}
|
||||
|
||||
@Test(expected = TimeoutException.class)
|
||||
public void testCheckFailingDisappear() throws Exception {
|
||||
HoodieClientTestUtils.fakeDataFile(basePath, "partition/path", "000", "f1");
|
||||
ConsistencyGuard passing = new FailSafeConsistencyGuard(fs, getConsistencyGuardConfig());
|
||||
passing.waitTillAllFilesDisappear(basePath + "/partition/path",
|
||||
Arrays.asList(basePath + "/partition/path/f1_1-0-1_000.parquet",
|
||||
basePath + "/partition/path/f2_1-0-2_000.parquet"));
|
||||
}
|
||||
|
||||
@Test(expected = TimeoutException.class)
|
||||
public void testCheckFailingDisappears() throws Exception {
|
||||
HoodieClientTestUtils.fakeDataFile(basePath, "partition/path", "000", "f1");
|
||||
HoodieClientTestUtils.fakeDataFile(basePath, "partition/path", "000", "f1");
|
||||
ConsistencyGuard passing = new FailSafeConsistencyGuard(fs, getConsistencyGuardConfig());
|
||||
passing.waitTillFileDisappears(new Path(basePath + "/partition/path/f1_1-0-1_000.parquet"));
|
||||
}
|
||||
|
||||
private ConsistencyGuardConfig getConsistencyGuardConfig() {
|
||||
return getConsistencyGuardConfig(3, 10, 10);
|
||||
}
|
||||
|
||||
private ConsistencyGuardConfig getConsistencyGuardConfig(int maxChecks, int initalSleep, int maxSleep) {
|
||||
return ConsistencyGuardConfig.newBuilder().withConsistencyCheckEnabled(true)
|
||||
.withInitialConsistencyCheckIntervalMs(initalSleep).withMaxConsistencyCheckIntervalMs(maxSleep)
|
||||
.withMaxConsistencyChecks(maxChecks).build();
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,543 @@
|
||||
/*
|
||||
* 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;
|
||||
|
||||
import static org.junit.Assert.assertEquals;
|
||||
import static org.junit.Assert.assertFalse;
|
||||
import static org.junit.Assert.assertTrue;
|
||||
|
||||
import java.io.File;
|
||||
import java.io.IOException;
|
||||
import java.io.Serializable;
|
||||
import java.util.ArrayList;
|
||||
import java.util.HashMap;
|
||||
import java.util.HashSet;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.LocalFileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hudi.common.HoodieCleanStat;
|
||||
import org.apache.hudi.common.HoodieClientTestUtils;
|
||||
import org.apache.hudi.common.HoodieTestDataGenerator;
|
||||
import org.apache.hudi.common.TestRawTripPayload.MetadataMergeWriteStatus;
|
||||
import org.apache.hudi.common.model.HoodiePartitionMetadata;
|
||||
import org.apache.hudi.common.model.HoodieRecord;
|
||||
import org.apache.hudi.common.model.HoodieTableType;
|
||||
import org.apache.hudi.common.model.HoodieTestUtils;
|
||||
import org.apache.hudi.common.table.HoodieTableMetaClient;
|
||||
import org.apache.hudi.common.table.HoodieTimeline;
|
||||
import org.apache.hudi.common.table.SyncableFileSystemView;
|
||||
import org.apache.hudi.common.table.timeline.HoodieActiveTimeline;
|
||||
import org.apache.hudi.common.table.view.FileSystemViewStorageConfig;
|
||||
import org.apache.hudi.common.table.view.FileSystemViewStorageType;
|
||||
import org.apache.hudi.common.util.ConsistencyGuardConfig;
|
||||
import org.apache.hudi.common.util.FSUtils;
|
||||
import org.apache.hudi.common.util.Option;
|
||||
import org.apache.hudi.config.HoodieCompactionConfig;
|
||||
import org.apache.hudi.config.HoodieIndexConfig;
|
||||
import org.apache.hudi.config.HoodieStorageConfig;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
import org.apache.hudi.index.HoodieIndex;
|
||||
import org.apache.hudi.index.HoodieIndex.IndexType;
|
||||
import org.apache.hudi.table.HoodieTable;
|
||||
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 org.apache.spark.sql.SQLContext;
|
||||
import org.junit.After;
|
||||
import org.junit.Assert;
|
||||
import org.junit.Before;
|
||||
import org.junit.rules.TemporaryFolder;
|
||||
|
||||
/**
|
||||
* Base Class providing setup/cleanup and utility methods for testing Hoodie Client facing tests
|
||||
*/
|
||||
public class TestHoodieClientBase implements Serializable {
|
||||
|
||||
protected static Logger logger = LogManager.getLogger(TestHoodieClientBase.class);
|
||||
|
||||
protected transient JavaSparkContext jsc = null;
|
||||
protected transient SQLContext sqlContext;
|
||||
protected transient FileSystem fs;
|
||||
protected String basePath = null;
|
||||
protected TemporaryFolder folder = null;
|
||||
protected transient HoodieTestDataGenerator dataGen = null;
|
||||
|
||||
private HoodieWriteClient writeClient;
|
||||
private HoodieReadClient readClient;
|
||||
|
||||
protected HoodieWriteClient getHoodieWriteClient(HoodieWriteConfig cfg) {
|
||||
return getHoodieWriteClient(cfg, false);
|
||||
}
|
||||
|
||||
protected HoodieWriteClient getHoodieWriteClient(HoodieWriteConfig cfg, boolean rollbackInflightCommit) {
|
||||
return getHoodieWriteClient(cfg, rollbackInflightCommit, HoodieIndex.createIndex(cfg, jsc));
|
||||
}
|
||||
|
||||
protected HoodieWriteClient getHoodieWriteClient(HoodieWriteConfig cfg, boolean rollbackInflightCommit,
|
||||
HoodieIndex index) {
|
||||
closeWriteClient();
|
||||
writeClient = new HoodieWriteClient(jsc, cfg, rollbackInflightCommit, index);
|
||||
return writeClient;
|
||||
}
|
||||
|
||||
protected HoodieReadClient getHoodieReadClient(String basePath) {
|
||||
closeReadClient();
|
||||
readClient = new HoodieReadClient(jsc, basePath);
|
||||
return readClient;
|
||||
}
|
||||
|
||||
private void closeWriteClient() {
|
||||
if (null != writeClient) {
|
||||
writeClient.close();
|
||||
writeClient = null;
|
||||
}
|
||||
}
|
||||
|
||||
private void closeReadClient() {
|
||||
if (null != readClient) {
|
||||
readClient.close();
|
||||
readClient = null;
|
||||
}
|
||||
}
|
||||
|
||||
@Before
|
||||
public void init() throws IOException {
|
||||
// Initialize a local spark env
|
||||
jsc = new JavaSparkContext(HoodieClientTestUtils.getSparkConfForTest("TestHoodieClient"));
|
||||
jsc.setLogLevel("ERROR");
|
||||
|
||||
//SQLContext stuff
|
||||
sqlContext = new SQLContext(jsc);
|
||||
|
||||
folder = new TemporaryFolder();
|
||||
folder.create();
|
||||
basePath = folder.getRoot().getAbsolutePath();
|
||||
|
||||
fs = FSUtils.getFs(basePath, jsc.hadoopConfiguration());
|
||||
if (fs instanceof LocalFileSystem) {
|
||||
LocalFileSystem lfs = (LocalFileSystem) fs;
|
||||
// With LocalFileSystem, with checksum disabled, fs.open() returns an inputStream which is FSInputStream
|
||||
// This causes ClassCastExceptions in LogRecordScanner (and potentially other places) calling fs.open
|
||||
// So, for the tests, we enforce checksum verification to circumvent the problem
|
||||
lfs.setVerifyChecksum(true);
|
||||
}
|
||||
HoodieTestUtils.initTableType(jsc.hadoopConfiguration(), basePath, getTableType());
|
||||
dataGen = new HoodieTestDataGenerator();
|
||||
}
|
||||
|
||||
@After
|
||||
/**
|
||||
* Properly release resources at end of each test
|
||||
*/
|
||||
public void tearDown() throws IOException {
|
||||
closeWriteClient();
|
||||
closeReadClient();
|
||||
|
||||
if (null != sqlContext) {
|
||||
logger.info("Clearing sql context cache of spark-session used in previous test-case");
|
||||
sqlContext.clearCache();
|
||||
}
|
||||
|
||||
if (null != jsc) {
|
||||
logger.info("Closing spark context used in previous test-case");
|
||||
jsc.close();
|
||||
}
|
||||
|
||||
// Create a temp folder as the base path
|
||||
if (null != folder) {
|
||||
logger.info("Explicitly removing workspace used in previously run test-case");
|
||||
folder.delete();
|
||||
}
|
||||
|
||||
if (null != fs) {
|
||||
logger.warn("Closing file-system instance used in previous test-run");
|
||||
fs.close();
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Get Default HoodieWriteConfig for tests
|
||||
*
|
||||
* @return Default Hoodie Write Config for tests
|
||||
*/
|
||||
protected HoodieWriteConfig getConfig() {
|
||||
return getConfigBuilder().build();
|
||||
}
|
||||
|
||||
/**
|
||||
* Get Config builder with default configs set
|
||||
*
|
||||
* @return Config Builder
|
||||
*/
|
||||
HoodieWriteConfig.Builder getConfigBuilder() {
|
||||
return HoodieWriteConfig.newBuilder().withPath(basePath).withSchema(HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA)
|
||||
.withParallelism(2, 2)
|
||||
.withBulkInsertParallelism(2).withFinalizeWriteParallelism(2)
|
||||
.withWriteStatusClass(MetadataMergeWriteStatus.class)
|
||||
.withConsistencyGuardConfig(ConsistencyGuardConfig.newBuilder().withConsistencyCheckEnabled(true).build())
|
||||
.withCompactionConfig(HoodieCompactionConfig.newBuilder().compactionSmallFileSize(1024 * 1024).build())
|
||||
.withStorageConfig(HoodieStorageConfig.newBuilder().limitFileSize(1024 * 1024).build())
|
||||
.forTable("test-trip-table")
|
||||
.withIndexConfig(HoodieIndexConfig.newBuilder().withIndexType(IndexType.BLOOM).build())
|
||||
.withEmbeddedTimelineServerEnabled(true).withFileSystemViewConfig(
|
||||
FileSystemViewStorageConfig.newBuilder().withStorageType(FileSystemViewStorageType.EMBEDDED_KV_STORE)
|
||||
.build());
|
||||
}
|
||||
|
||||
protected HoodieTable getHoodieTable(HoodieTableMetaClient metaClient, HoodieWriteConfig config) {
|
||||
HoodieTable table = HoodieTable.getHoodieTable(metaClient, config, jsc);
|
||||
((SyncableFileSystemView) (table.getRTFileSystemView())).reset();
|
||||
return table;
|
||||
}
|
||||
|
||||
/**
|
||||
* Assert no failures in writing hoodie files
|
||||
*
|
||||
* @param statuses List of Write Status
|
||||
*/
|
||||
static void assertNoWriteErrors(List<WriteStatus> statuses) {
|
||||
// Verify there are no errors
|
||||
for (WriteStatus status : statuses) {
|
||||
assertFalse("Errors found in write of " + status.getFileId(), status.hasErrors());
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Ensure presence of partition meta-data at known depth
|
||||
*
|
||||
* @param partitionPaths Partition paths to check
|
||||
* @param fs File System
|
||||
* @throws IOException in case of error
|
||||
*/
|
||||
void assertPartitionMetadata(String[] partitionPaths, FileSystem fs) throws IOException {
|
||||
for (String partitionPath : partitionPaths) {
|
||||
assertTrue(HoodiePartitionMetadata.hasPartitionMetadata(fs, new Path(basePath, partitionPath)));
|
||||
HoodiePartitionMetadata pmeta = new HoodiePartitionMetadata(fs, new Path(basePath, partitionPath));
|
||||
pmeta.readFromFS();
|
||||
Assert.assertEquals(HoodieTestDataGenerator.DEFAULT_PARTITION_DEPTH, pmeta.getPartitionDepth());
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Ensure records have location field set
|
||||
*
|
||||
* @param taggedRecords Tagged Records
|
||||
* @param commitTime Commit Timestamp
|
||||
*/
|
||||
void checkTaggedRecords(List<HoodieRecord> taggedRecords, String commitTime) {
|
||||
for (HoodieRecord rec : taggedRecords) {
|
||||
assertTrue("Record " + rec + " found with no location.", rec.isCurrentLocationKnown());
|
||||
assertEquals("All records should have commit time " + commitTime + ", since updates were made",
|
||||
rec.getCurrentLocation().getInstantTime(), commitTime);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Assert that there is no duplicate key at the partition level
|
||||
*
|
||||
* @param records List of Hoodie records
|
||||
*/
|
||||
void assertNodupesWithinPartition(List<HoodieRecord> records) {
|
||||
Map<String, Set<String>> partitionToKeys = new HashMap<>();
|
||||
for (HoodieRecord r : records) {
|
||||
String key = r.getRecordKey();
|
||||
String partitionPath = r.getPartitionPath();
|
||||
if (!partitionToKeys.containsKey(partitionPath)) {
|
||||
partitionToKeys.put(partitionPath, new HashSet<>());
|
||||
}
|
||||
assertTrue("key " + key + " is duplicate within partition " + partitionPath,
|
||||
!partitionToKeys.get(partitionPath).contains(key));
|
||||
partitionToKeys.get(partitionPath).add(key);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Helper to generate records generation function for testing Prepped version of API. Prepped APIs expect the records
|
||||
* to be already de-duped and have location set. This wrapper takes care of record-location setting. Uniqueness is
|
||||
* guaranteed by record-generation function itself.
|
||||
*
|
||||
* @param writeConfig Hoodie Write Config
|
||||
* @param recordGenFunction Records Generation function
|
||||
* @return Wrapped function
|
||||
*/
|
||||
private Function2<List<HoodieRecord>, String, Integer> wrapRecordsGenFunctionForPreppedCalls(
|
||||
final HoodieWriteConfig writeConfig,
|
||||
final Function2<List<HoodieRecord>, String, Integer> recordGenFunction) {
|
||||
return (commit, numRecords) -> {
|
||||
final HoodieIndex index = HoodieIndex.createIndex(writeConfig, jsc);
|
||||
List<HoodieRecord> records = recordGenFunction.apply(commit, numRecords);
|
||||
final HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath, true);
|
||||
HoodieTable table = HoodieTable.getHoodieTable(metaClient, writeConfig, jsc);
|
||||
JavaRDD<HoodieRecord> taggedRecords =
|
||||
index.tagLocation(jsc.parallelize(records, 1), jsc, table);
|
||||
return taggedRecords.collect();
|
||||
};
|
||||
}
|
||||
|
||||
/**
|
||||
* Generate wrapper for record generation function for testing Prepped APIs
|
||||
*
|
||||
* @param isPreppedAPI Flag to indicate if this is for testing prepped-version of APIs
|
||||
* @param writeConfig Hoodie Write Config
|
||||
* @param wrapped Actual Records Generation function
|
||||
* @return Wrapped Function
|
||||
*/
|
||||
Function2<List<HoodieRecord>, String, Integer> generateWrapRecordsFn(
|
||||
boolean isPreppedAPI,
|
||||
HoodieWriteConfig writeConfig,
|
||||
Function2<List<HoodieRecord>, String, Integer> wrapped) {
|
||||
if (isPreppedAPI) {
|
||||
return wrapRecordsGenFunctionForPreppedCalls(writeConfig, wrapped);
|
||||
} else {
|
||||
return wrapped;
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Helper to insert first batch of records and do regular assertions on the state after successful completion
|
||||
*
|
||||
* @param writeConfig Hoodie Write Config
|
||||
* @param client Hoodie Write Client
|
||||
* @param newCommitTime New Commit Timestamp to be used
|
||||
* @param initCommitTime Begin Timestamp (usually "000")
|
||||
* @param numRecordsInThisCommit Number of records to be added in the new commit
|
||||
* @param writeFn Write Function to be used for insertion
|
||||
* @param isPreppedAPI Boolean flag to indicate writeFn expects prepped records
|
||||
* @param assertForCommit Enable Assertion of Writes
|
||||
* @param expRecordsInThisCommit Expected number of records in this commit
|
||||
* @return RDD of write-status
|
||||
* @throws Exception in case of error
|
||||
*/
|
||||
JavaRDD<WriteStatus> insertFirstBatch(
|
||||
HoodieWriteConfig writeConfig,
|
||||
HoodieWriteClient client,
|
||||
String newCommitTime,
|
||||
String initCommitTime,
|
||||
int numRecordsInThisCommit,
|
||||
Function3<JavaRDD<WriteStatus>, HoodieWriteClient, JavaRDD<HoodieRecord>, String> writeFn,
|
||||
boolean isPreppedAPI,
|
||||
boolean assertForCommit,
|
||||
int expRecordsInThisCommit) throws Exception {
|
||||
final Function2<List<HoodieRecord>, String, Integer> recordGenFunction =
|
||||
generateWrapRecordsFn(isPreppedAPI, writeConfig, dataGen::generateInserts);
|
||||
|
||||
return writeBatch(client, newCommitTime, initCommitTime, Option.empty(), initCommitTime,
|
||||
numRecordsInThisCommit, recordGenFunction, writeFn, assertForCommit,
|
||||
expRecordsInThisCommit, expRecordsInThisCommit, 1);
|
||||
}
|
||||
|
||||
/**
|
||||
* Helper to upsert batch of records and do regular assertions on the state after successful completion
|
||||
*
|
||||
* @param writeConfig Hoodie Write Config
|
||||
* @param client Hoodie Write Client
|
||||
* @param newCommitTime New Commit Timestamp to be used
|
||||
* @param prevCommitTime Commit Timestamp used in previous commit
|
||||
* @param commitTimesBetweenPrevAndNew Sample of Timestamps between prevCommitTime and newCommitTime
|
||||
* @param initCommitTime Begin Timestamp (usually "000")
|
||||
* @param numRecordsInThisCommit Number of records to be added in the new commit
|
||||
* @param writeFn Write Function to be used for upsert
|
||||
* @param isPreppedAPI Boolean flag to indicate writeFn expects prepped records
|
||||
* @param assertForCommit Enable Assertion of Writes
|
||||
* @param expRecordsInThisCommit Expected number of records in this commit
|
||||
* @param expTotalRecords Expected number of records when scanned
|
||||
* @param expTotalCommits Expected number of commits (including this commit)
|
||||
* @return RDD of write-status
|
||||
* @throws Exception in case of error
|
||||
*/
|
||||
JavaRDD<WriteStatus> updateBatch(
|
||||
HoodieWriteConfig writeConfig,
|
||||
HoodieWriteClient client,
|
||||
String newCommitTime,
|
||||
String prevCommitTime,
|
||||
Option<List<String>> commitTimesBetweenPrevAndNew,
|
||||
String initCommitTime,
|
||||
int numRecordsInThisCommit,
|
||||
Function3<JavaRDD<WriteStatus>, HoodieWriteClient, JavaRDD<HoodieRecord>, String> writeFn,
|
||||
boolean isPreppedAPI,
|
||||
boolean assertForCommit,
|
||||
int expRecordsInThisCommit,
|
||||
int expTotalRecords,
|
||||
int expTotalCommits)
|
||||
throws Exception {
|
||||
final Function2<List<HoodieRecord>, String, Integer> recordGenFunction =
|
||||
generateWrapRecordsFn(isPreppedAPI, writeConfig, dataGen::generateUniqueUpdates);
|
||||
|
||||
return writeBatch(client, newCommitTime, prevCommitTime, commitTimesBetweenPrevAndNew, initCommitTime,
|
||||
numRecordsInThisCommit, recordGenFunction, writeFn, assertForCommit,
|
||||
expRecordsInThisCommit, expTotalRecords, expTotalCommits);
|
||||
}
|
||||
|
||||
/**
|
||||
* Helper to insert/upsert batch of records and do regular assertions on the state after successful completion
|
||||
*
|
||||
* @param client Hoodie Write Client
|
||||
* @param newCommitTime New Commit Timestamp to be used
|
||||
* @param prevCommitTime Commit Timestamp used in previous commit
|
||||
* @param commitTimesBetweenPrevAndNew Sample of Timestamps between prevCommitTime and newCommitTime
|
||||
* @param initCommitTime Begin Timestamp (usually "000")
|
||||
* @param numRecordsInThisCommit Number of records to be added in the new commit
|
||||
* @param recordGenFunction Records Generation Function
|
||||
* @param writeFn Write Function to be used for upsert
|
||||
* @param assertForCommit Enable Assertion of Writes
|
||||
* @param expRecordsInThisCommit Expected number of records in this commit
|
||||
* @param expTotalRecords Expected number of records when scanned
|
||||
* @param expTotalCommits Expected number of commits (including this commit)
|
||||
* @throws Exception in case of error
|
||||
*/
|
||||
JavaRDD<WriteStatus> writeBatch(
|
||||
HoodieWriteClient client,
|
||||
String newCommitTime,
|
||||
String prevCommitTime,
|
||||
Option<List<String>> commitTimesBetweenPrevAndNew,
|
||||
String initCommitTime,
|
||||
int numRecordsInThisCommit,
|
||||
Function2<List<HoodieRecord>, String, Integer> recordGenFunction,
|
||||
Function3<JavaRDD<WriteStatus>, HoodieWriteClient, JavaRDD<HoodieRecord>, String> writeFn,
|
||||
boolean assertForCommit,
|
||||
int expRecordsInThisCommit,
|
||||
int expTotalRecords,
|
||||
int expTotalCommits)
|
||||
throws Exception {
|
||||
|
||||
//Write 1 (only inserts)
|
||||
client.startCommitWithTime(newCommitTime);
|
||||
|
||||
List<HoodieRecord> records = recordGenFunction.apply(newCommitTime, numRecordsInThisCommit);
|
||||
JavaRDD<HoodieRecord> writeRecords = jsc.parallelize(records, 1);
|
||||
|
||||
JavaRDD<WriteStatus> result = writeFn.apply(client, writeRecords, newCommitTime);
|
||||
List<WriteStatus> statuses = result.collect();
|
||||
assertNoWriteErrors(statuses);
|
||||
|
||||
// check the partition metadata is written out
|
||||
assertPartitionMetadata(HoodieTestDataGenerator.DEFAULT_PARTITION_PATHS, fs);
|
||||
|
||||
// verify that there is a commit
|
||||
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath);
|
||||
HoodieTimeline timeline = new HoodieActiveTimeline(metaClient).getCommitTimeline();
|
||||
|
||||
if (assertForCommit) {
|
||||
assertEquals("Expecting " + expTotalCommits + " commits.", expTotalCommits,
|
||||
timeline.findInstantsAfter(initCommitTime, Integer.MAX_VALUE).countInstants());
|
||||
Assert.assertEquals("Latest commit should be " + newCommitTime, newCommitTime,
|
||||
timeline.lastInstant().get().getTimestamp());
|
||||
assertEquals("Must contain " + expRecordsInThisCommit + " records", expRecordsInThisCommit,
|
||||
HoodieClientTestUtils.readCommit(basePath, sqlContext, timeline, newCommitTime).count());
|
||||
|
||||
// Check the entire dataset has all records still
|
||||
String[] fullPartitionPaths = new String[dataGen.getPartitionPaths().length];
|
||||
for (int i = 0; i < fullPartitionPaths.length; i++) {
|
||||
fullPartitionPaths[i] = String.format("%s/%s/*", basePath, dataGen.getPartitionPaths()[i]);
|
||||
}
|
||||
assertEquals("Must contain " + expTotalRecords + " records", expTotalRecords,
|
||||
HoodieClientTestUtils.read(jsc, basePath, sqlContext, fs, fullPartitionPaths).count());
|
||||
|
||||
// Check that the incremental consumption from prevCommitTime
|
||||
assertEquals("Incremental consumption from " + prevCommitTime
|
||||
+ " should give all records in latest commit",
|
||||
HoodieClientTestUtils.readCommit(basePath, sqlContext, timeline, newCommitTime).count(),
|
||||
HoodieClientTestUtils.readSince(basePath, sqlContext, timeline, prevCommitTime).count());
|
||||
if (commitTimesBetweenPrevAndNew.isPresent()) {
|
||||
commitTimesBetweenPrevAndNew.get().forEach(ct -> {
|
||||
assertEquals("Incremental consumption from " + ct + " should give all records in latest commit",
|
||||
HoodieClientTestUtils.readCommit(basePath, sqlContext, timeline, newCommitTime).count(),
|
||||
HoodieClientTestUtils.readSince(basePath, sqlContext, timeline, ct).count());
|
||||
});
|
||||
}
|
||||
}
|
||||
return result;
|
||||
}
|
||||
|
||||
@After
|
||||
public void clean() {
|
||||
if (basePath != null) {
|
||||
new File(basePath).delete();
|
||||
}
|
||||
if (jsc != null) {
|
||||
jsc.stop();
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Get Cleaner state corresponding to a partition path
|
||||
*
|
||||
* @param hoodieCleanStatsTwo List of Clean Stats
|
||||
* @param partitionPath Partition path for filtering
|
||||
* @return Cleaner state corresponding to partition path
|
||||
*/
|
||||
HoodieCleanStat getCleanStat(List<HoodieCleanStat> hoodieCleanStatsTwo, String partitionPath) {
|
||||
return hoodieCleanStatsTwo.stream().filter(e -> e.getPartitionPath().equals(partitionPath)).findFirst().get();
|
||||
}
|
||||
|
||||
/**
|
||||
* Utility to simulate commit touching files in a partition
|
||||
*
|
||||
* @param files List of file-Ids to be touched
|
||||
* @param partitionPath Partition
|
||||
* @param commitTime Commit Timestamp
|
||||
* @throws IOException in case of error
|
||||
*/
|
||||
void updateAllFilesInPartition(List<String> files, String partitionPath, String commitTime)
|
||||
throws IOException {
|
||||
for (String fileId : files) {
|
||||
HoodieTestUtils.createDataFile(basePath, partitionPath, commitTime, fileId);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Helper methods to create new data files in a partition
|
||||
*
|
||||
* @param partitionPath Partition
|
||||
* @param commitTime Commit Timestamp
|
||||
* @param numFiles Number of files to be added
|
||||
* @return Created files
|
||||
* @throws IOException in case of error
|
||||
*/
|
||||
List<String> createFilesInPartition(String partitionPath, String commitTime, int numFiles)
|
||||
throws IOException {
|
||||
List<String> files = new ArrayList<>();
|
||||
for (int i = 0; i < numFiles; i++) {
|
||||
files.add(HoodieTestUtils.createNewDataFile(basePath, partitionPath, commitTime));
|
||||
}
|
||||
return files;
|
||||
}
|
||||
|
||||
// Functional Interfaces for passing lambda and Hoodie Write API contexts
|
||||
|
||||
@FunctionalInterface
|
||||
interface Function2<R, T1, T2> {
|
||||
|
||||
R apply(T1 v1, T2 v2) throws IOException;
|
||||
}
|
||||
|
||||
@FunctionalInterface
|
||||
interface Function3<R, T1, T2, T3> {
|
||||
|
||||
R apply(T1 v1, T2 v2, T3 v3) throws IOException;
|
||||
}
|
||||
|
||||
protected HoodieTableType getTableType() {
|
||||
return HoodieTableType.COPY_ON_WRITE;
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,735 @@
|
||||
/*
|
||||
* 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;
|
||||
|
||||
import static org.apache.hudi.common.util.ParquetUtils.readRowKeysFromParquet;
|
||||
import static org.junit.Assert.assertEquals;
|
||||
import static org.junit.Assert.assertFalse;
|
||||
import static org.junit.Assert.assertTrue;
|
||||
import static org.junit.Assert.fail;
|
||||
import static org.mockito.Mockito.mock;
|
||||
import static org.mockito.Mockito.when;
|
||||
|
||||
import java.io.FileInputStream;
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Arrays;
|
||||
import java.util.Collection;
|
||||
import java.util.HashMap;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
import java.util.UUID;
|
||||
import java.util.stream.Collectors;
|
||||
import org.apache.avro.generic.GenericRecord;
|
||||
import org.apache.commons.io.IOUtils;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hudi.common.HoodieClientTestUtils;
|
||||
import org.apache.hudi.common.HoodieTestDataGenerator;
|
||||
import org.apache.hudi.common.model.HoodieCommitMetadata;
|
||||
import org.apache.hudi.common.model.HoodieDataFile;
|
||||
import org.apache.hudi.common.model.HoodieKey;
|
||||
import org.apache.hudi.common.model.HoodieRecord;
|
||||
import org.apache.hudi.common.model.HoodieRollingStat;
|
||||
import org.apache.hudi.common.model.HoodieRollingStatMetadata;
|
||||
import org.apache.hudi.common.model.HoodieTestUtils;
|
||||
import org.apache.hudi.common.table.HoodieTableMetaClient;
|
||||
import org.apache.hudi.common.table.HoodieTimeline;
|
||||
import org.apache.hudi.common.table.TableFileSystemView.ReadOptimizedView;
|
||||
import org.apache.hudi.common.table.timeline.HoodieInstant;
|
||||
import org.apache.hudi.common.util.ConsistencyGuardConfig;
|
||||
import org.apache.hudi.common.util.FSUtils;
|
||||
import org.apache.hudi.common.util.Option;
|
||||
import org.apache.hudi.common.util.ParquetUtils;
|
||||
import org.apache.hudi.common.util.collection.Pair;
|
||||
import org.apache.hudi.config.HoodieCompactionConfig;
|
||||
import org.apache.hudi.config.HoodieStorageConfig;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
import org.apache.hudi.exception.HoodieCommitException;
|
||||
import org.apache.hudi.exception.HoodieIOException;
|
||||
import org.apache.hudi.index.HoodieIndex;
|
||||
import org.apache.hudi.table.HoodieTable;
|
||||
import org.apache.spark.api.java.JavaRDD;
|
||||
import org.junit.Assert;
|
||||
import org.junit.Test;
|
||||
|
||||
@SuppressWarnings("unchecked")
|
||||
public class TestHoodieClientOnCopyOnWriteStorage extends TestHoodieClientBase {
|
||||
|
||||
@Override
|
||||
public void tearDown() throws IOException {
|
||||
super.tearDown();
|
||||
}
|
||||
|
||||
/**
|
||||
* Test Auto Commit behavior for HoodieWriteClient insert API
|
||||
*/
|
||||
@Test
|
||||
public void testAutoCommitOnInsert() throws Exception {
|
||||
testAutoCommit(HoodieWriteClient::insert, false);
|
||||
}
|
||||
|
||||
/**
|
||||
* Test Auto Commit behavior for HoodieWriteClient insertPrepped API
|
||||
*/
|
||||
@Test
|
||||
public void testAutoCommitOnInsertPrepped() throws Exception {
|
||||
testAutoCommit(HoodieWriteClient::insertPreppedRecords, true);
|
||||
}
|
||||
|
||||
/**
|
||||
* Test Auto Commit behavior for HoodieWriteClient upsert API
|
||||
*/
|
||||
@Test
|
||||
public void testAutoCommitOnUpsert() throws Exception {
|
||||
testAutoCommit(HoodieWriteClient::upsert, false);
|
||||
}
|
||||
|
||||
/**
|
||||
* Test Auto Commit behavior for HoodieWriteClient upsert Prepped API
|
||||
*/
|
||||
@Test
|
||||
public void testAutoCommitOnUpsertPrepped() throws Exception {
|
||||
testAutoCommit(HoodieWriteClient::upsertPreppedRecords, true);
|
||||
}
|
||||
|
||||
/**
|
||||
* Test Auto Commit behavior for HoodieWriteClient bulk-insert API
|
||||
*/
|
||||
@Test
|
||||
public void testAutoCommitOnBulkInsert() throws Exception {
|
||||
testAutoCommit(HoodieWriteClient::bulkInsert, false);
|
||||
}
|
||||
|
||||
/**
|
||||
* Test Auto Commit behavior for HoodieWriteClient bulk-insert prepped API
|
||||
*/
|
||||
@Test
|
||||
public void testAutoCommitOnBulkInsertPrepped() throws Exception {
|
||||
testAutoCommit((writeClient, recordRDD, commitTime)
|
||||
-> writeClient.bulkInsertPreppedRecords(recordRDD, commitTime, Option.empty()), true);
|
||||
}
|
||||
|
||||
/**
|
||||
* Test auto-commit by applying write function
|
||||
*
|
||||
* @param writeFn One of HoodieWriteClient Write API
|
||||
* @throws Exception in case of failure
|
||||
*/
|
||||
private void testAutoCommit(
|
||||
Function3<JavaRDD<WriteStatus>, HoodieWriteClient, JavaRDD<HoodieRecord>, String> writeFn,
|
||||
boolean isPrepped) throws Exception {
|
||||
// Set autoCommit false
|
||||
HoodieWriteConfig cfg = getConfigBuilder().withAutoCommit(false).build();
|
||||
HoodieWriteClient client = getHoodieWriteClient(cfg);
|
||||
|
||||
String prevCommitTime = "000";
|
||||
String newCommitTime = "001";
|
||||
int numRecords = 200;
|
||||
JavaRDD<WriteStatus> result =
|
||||
insertFirstBatch(cfg, client, newCommitTime, prevCommitTime, numRecords, writeFn, isPrepped, false, numRecords);
|
||||
|
||||
assertFalse("If Autocommit is false, then commit should not be made automatically",
|
||||
HoodieTestUtils.doesCommitExist(basePath, newCommitTime));
|
||||
assertTrue("Commit should succeed", client.commit(newCommitTime, result));
|
||||
assertTrue("After explicit commit, commit file should be created",
|
||||
HoodieTestUtils.doesCommitExist(basePath, newCommitTime));
|
||||
}
|
||||
|
||||
/**
|
||||
* Test De-duplication behavior for HoodieWriteClient insert API
|
||||
*/
|
||||
@Test
|
||||
public void testDeduplicationOnInsert() throws Exception {
|
||||
testDeduplication(HoodieWriteClient::insert);
|
||||
}
|
||||
|
||||
/**
|
||||
* Test De-duplication behavior for HoodieWriteClient bulk-insert API
|
||||
*/
|
||||
@Test
|
||||
public void testDeduplicationOnBulkInsert() throws Exception {
|
||||
testDeduplication(HoodieWriteClient::bulkInsert);
|
||||
}
|
||||
|
||||
/**
|
||||
* Test De-duplication behavior for HoodieWriteClient upsert API
|
||||
*/
|
||||
@Test
|
||||
public void testDeduplicationOnUpsert() throws Exception {
|
||||
testDeduplication(HoodieWriteClient::upsert);
|
||||
}
|
||||
|
||||
/**
|
||||
* Test Deduplication Logic for write function
|
||||
*
|
||||
* @param writeFn One of HoddieWriteClient non-prepped write APIs
|
||||
* @throws Exception in case of failure
|
||||
*/
|
||||
private void testDeduplication(
|
||||
Function3<JavaRDD<WriteStatus>, HoodieWriteClient, JavaRDD<HoodieRecord>, String> writeFn) throws Exception {
|
||||
String newCommitTime = "001";
|
||||
|
||||
String recordKey = UUID.randomUUID().toString();
|
||||
HoodieKey keyOne = new HoodieKey(recordKey, "2018-01-01");
|
||||
HoodieRecord recordOne = new HoodieRecord(keyOne,
|
||||
HoodieTestDataGenerator.generateRandomValue(keyOne, newCommitTime));
|
||||
|
||||
HoodieKey keyTwo = new HoodieKey(recordKey, "2018-02-01");
|
||||
HoodieRecord recordTwo = new HoodieRecord(keyTwo,
|
||||
HoodieTestDataGenerator.generateRandomValue(keyTwo, newCommitTime));
|
||||
|
||||
// Same key and partition as keyTwo
|
||||
HoodieRecord recordThree = new HoodieRecord(keyTwo,
|
||||
HoodieTestDataGenerator.generateRandomValue(keyTwo, newCommitTime));
|
||||
|
||||
JavaRDD<HoodieRecord> records = jsc.parallelize(Arrays.asList(recordOne, recordTwo, recordThree), 1);
|
||||
|
||||
// dedup should be done based on recordKey only
|
||||
HoodieWriteClient clientWithDummyGlobalIndex = getWriteClientWithDummyIndex(true);
|
||||
List<HoodieRecord> dedupedRecs = clientWithDummyGlobalIndex.deduplicateRecords(records, 1).collect();
|
||||
assertEquals(1, dedupedRecs.size());
|
||||
assertNodupesWithinPartition(dedupedRecs);
|
||||
|
||||
// dedup should be done based on both recordKey and partitionPath
|
||||
HoodieWriteClient clientWithDummyNonGlobalIndex = getWriteClientWithDummyIndex(false);
|
||||
dedupedRecs =
|
||||
clientWithDummyNonGlobalIndex.deduplicateRecords(records, 1).collect();
|
||||
assertEquals(2, dedupedRecs.size());
|
||||
assertNodupesWithinPartition(dedupedRecs);
|
||||
|
||||
// Perform write-action and check
|
||||
HoodieWriteClient client = getHoodieWriteClient(
|
||||
getConfigBuilder().combineInput(true, true).build(), false);
|
||||
client.startCommitWithTime(newCommitTime);
|
||||
List<WriteStatus> statuses = writeFn.apply(client, records, newCommitTime).collect();
|
||||
assertNoWriteErrors(statuses);
|
||||
assertEquals(2, statuses.size());
|
||||
assertNodupesWithinPartition(
|
||||
statuses.stream().map(WriteStatus::getWrittenRecords)
|
||||
.flatMap(Collection::stream).collect(Collectors.toList()));
|
||||
}
|
||||
|
||||
/**
|
||||
* Build a test Hoodie WriteClient with dummy index to configure isGlobal flag
|
||||
*
|
||||
* @param isGlobal Flag to control HoodieIndex.isGlobal
|
||||
* @return Hoodie Write Client
|
||||
* @throws Exception in case of error
|
||||
*/
|
||||
private HoodieWriteClient getWriteClientWithDummyIndex(final boolean isGlobal) throws Exception {
|
||||
HoodieIndex index = mock(HoodieIndex.class);
|
||||
when(index.isGlobal()).thenReturn(isGlobal);
|
||||
return getHoodieWriteClient(getConfigBuilder().build(), false, index);
|
||||
}
|
||||
|
||||
/**
|
||||
* Test Upsert API
|
||||
*/
|
||||
@Test
|
||||
public void testUpserts() throws Exception {
|
||||
testUpsertsInternal(getConfig(),
|
||||
HoodieWriteClient::upsert, false);
|
||||
}
|
||||
|
||||
/**
|
||||
* Test UpsertPrepped API
|
||||
*/
|
||||
@Test
|
||||
public void testUpsertsPrepped() throws Exception {
|
||||
testUpsertsInternal(getConfig(),
|
||||
HoodieWriteClient::upsertPreppedRecords, true);
|
||||
}
|
||||
|
||||
/**
|
||||
* Test one of HoodieWriteClient upsert(Prepped) APIs
|
||||
*
|
||||
* @param hoodieWriteConfig Write Config
|
||||
* @param writeFn One of Hoodie Write Function API
|
||||
* @throws Exception in case of error
|
||||
*/
|
||||
private void testUpsertsInternal(HoodieWriteConfig hoodieWriteConfig,
|
||||
Function3<JavaRDD<WriteStatus>, HoodieWriteClient, JavaRDD<HoodieRecord>, String> writeFn,
|
||||
boolean isPrepped) throws Exception {
|
||||
HoodieWriteClient client = getHoodieWriteClient(hoodieWriteConfig, false);
|
||||
|
||||
//Write 1 (only inserts)
|
||||
String newCommitTime = "001";
|
||||
String initCommitTime = "000";
|
||||
int numRecords = 200;
|
||||
insertFirstBatch(hoodieWriteConfig,
|
||||
client, newCommitTime, initCommitTime, numRecords, HoodieWriteClient::insert, isPrepped, true, numRecords);
|
||||
|
||||
// Write 2 (updates)
|
||||
String prevCommitTime = newCommitTime;
|
||||
newCommitTime = "004";
|
||||
numRecords = 100;
|
||||
String commitTimeBetweenPrevAndNew = "002";
|
||||
updateBatch(hoodieWriteConfig, client, newCommitTime, prevCommitTime,
|
||||
Option.of(Arrays.asList(commitTimeBetweenPrevAndNew)),
|
||||
initCommitTime, numRecords, writeFn, isPrepped, true, numRecords, 200, 2);
|
||||
}
|
||||
|
||||
/**
|
||||
* Tesst deletion of records
|
||||
*/
|
||||
@Test
|
||||
public void testDeletes() throws Exception {
|
||||
HoodieWriteClient client = getHoodieWriteClient(getConfig(), false);
|
||||
|
||||
/**
|
||||
* Write 1 (inserts and deletes)
|
||||
* Write actual 200 insert records and ignore 100 delete records
|
||||
*/
|
||||
String initCommitTime = "000";
|
||||
String newCommitTime = "001";
|
||||
|
||||
final List<HoodieRecord> recordsInFirstBatch = new ArrayList<>();
|
||||
Function2<List<HoodieRecord>, String, Integer> recordGenFunction =
|
||||
(String commitTime, Integer numRecordsInThisCommit) -> {
|
||||
List<HoodieRecord> fewRecordsForInsert = dataGen.generateInserts(commitTime, 200);
|
||||
List<HoodieRecord> fewRecordsForDelete = dataGen.generateDeletes(commitTime, 100);
|
||||
|
||||
recordsInFirstBatch.addAll(fewRecordsForInsert);
|
||||
recordsInFirstBatch.addAll(fewRecordsForDelete);
|
||||
return recordsInFirstBatch;
|
||||
};
|
||||
writeBatch(client, newCommitTime, initCommitTime, Option.empty(), initCommitTime,
|
||||
//unused as genFn uses hard-coded number of inserts/updates/deletes
|
||||
-1,
|
||||
recordGenFunction, HoodieWriteClient::upsert, true,
|
||||
200, 200, 1);
|
||||
|
||||
/**
|
||||
* Write 2 (deletes+writes)
|
||||
*/
|
||||
String prevCommitTime = newCommitTime;
|
||||
newCommitTime = "004";
|
||||
final List<HoodieRecord> recordsInSecondBatch = new ArrayList<>();
|
||||
|
||||
recordGenFunction =
|
||||
(String commitTime, Integer numRecordsInThisCommit) -> {
|
||||
List<HoodieRecord> fewRecordsForDelete = recordsInFirstBatch.subList(0, 50);
|
||||
List<HoodieRecord> fewRecordsForUpdate = recordsInFirstBatch.subList(50, 100);
|
||||
recordsInSecondBatch.addAll(dataGen.generateDeletesFromExistingRecords(fewRecordsForDelete));
|
||||
recordsInSecondBatch.addAll(fewRecordsForUpdate);
|
||||
return recordsInSecondBatch;
|
||||
};
|
||||
writeBatch(client, newCommitTime, prevCommitTime, Option.empty(), initCommitTime,
|
||||
100, recordGenFunction, HoodieWriteClient::upsert, true,
|
||||
50, 150, 2);
|
||||
}
|
||||
|
||||
/**
|
||||
* Test scenario of new file-group getting added during upsert()
|
||||
*/
|
||||
@Test
|
||||
public void testSmallInsertHandlingForUpserts() throws Exception {
|
||||
final String testPartitionPath = "2016/09/26";
|
||||
final int insertSplitLimit = 100;
|
||||
// setup the small file handling params
|
||||
HoodieWriteConfig config = getSmallInsertWriteConfig(insertSplitLimit); // hold upto 200 records max
|
||||
dataGen = new HoodieTestDataGenerator(new String[]{testPartitionPath});
|
||||
|
||||
HoodieWriteClient client = getHoodieWriteClient(config, false);
|
||||
|
||||
// Inserts => will write file1
|
||||
String commitTime1 = "001";
|
||||
client.startCommitWithTime(commitTime1);
|
||||
List<HoodieRecord> inserts1 = dataGen.generateInserts(commitTime1, insertSplitLimit); // this writes ~500kb
|
||||
Set<String> keys1 = HoodieClientTestUtils.getRecordKeys(inserts1);
|
||||
|
||||
JavaRDD<HoodieRecord> insertRecordsRDD1 = jsc.parallelize(inserts1, 1);
|
||||
List<WriteStatus> statuses = client.upsert(insertRecordsRDD1, commitTime1).collect();
|
||||
|
||||
assertNoWriteErrors(statuses);
|
||||
|
||||
assertEquals("Just 1 file needs to be added.", 1, statuses.size());
|
||||
String file1 = statuses.get(0).getFileId();
|
||||
Assert.assertEquals("file should contain 100 records", readRowKeysFromParquet(jsc.hadoopConfiguration(),
|
||||
new Path(basePath, statuses.get(0).getStat().getPath())).size(), 100);
|
||||
|
||||
// Update + Inserts such that they just expand file1
|
||||
String commitTime2 = "002";
|
||||
client.startCommitWithTime(commitTime2);
|
||||
List<HoodieRecord> inserts2 = dataGen.generateInserts(commitTime2, 40);
|
||||
Set<String> keys2 = HoodieClientTestUtils.getRecordKeys(inserts2);
|
||||
List<HoodieRecord> insertsAndUpdates2 = new ArrayList<>();
|
||||
insertsAndUpdates2.addAll(inserts2);
|
||||
insertsAndUpdates2.addAll(dataGen.generateUpdates(commitTime2, inserts1));
|
||||
|
||||
JavaRDD<HoodieRecord> insertAndUpdatesRDD2 = jsc.parallelize(insertsAndUpdates2, 1);
|
||||
statuses = client.upsert(insertAndUpdatesRDD2, commitTime2).collect();
|
||||
assertNoWriteErrors(statuses);
|
||||
|
||||
assertEquals("Just 1 file needs to be updated.", 1, statuses.size());
|
||||
assertEquals("Existing file should be expanded", file1, statuses.get(0).getFileId());
|
||||
assertEquals("Existing file should be expanded", commitTime1, statuses.get(0).getStat().getPrevCommit());
|
||||
Path newFile = new Path(basePath, statuses.get(0).getStat().getPath());
|
||||
assertEquals("file should contain 140 records",
|
||||
readRowKeysFromParquet(jsc.hadoopConfiguration(), newFile).size(), 140);
|
||||
|
||||
List<GenericRecord> records = ParquetUtils.readAvroRecords(jsc.hadoopConfiguration(), newFile);
|
||||
for (GenericRecord record : records) {
|
||||
String recordKey = record.get(HoodieRecord.RECORD_KEY_METADATA_FIELD).toString();
|
||||
assertEquals("only expect commit2", commitTime2, record.get(HoodieRecord.COMMIT_TIME_METADATA_FIELD).toString());
|
||||
assertTrue("key expected to be part of commit2", keys2.contains(recordKey) || keys1.contains(recordKey));
|
||||
}
|
||||
|
||||
// update + inserts such that file1 is updated and expanded, a new file2 is created.
|
||||
String commitTime3 = "003";
|
||||
client.startCommitWithTime(commitTime3);
|
||||
List<HoodieRecord> insertsAndUpdates3 = dataGen.generateInserts(commitTime3, 200);
|
||||
Set<String> keys3 = HoodieClientTestUtils.getRecordKeys(insertsAndUpdates3);
|
||||
List<HoodieRecord> updates3 = dataGen.generateUpdates(commitTime3, inserts2);
|
||||
insertsAndUpdates3.addAll(updates3);
|
||||
|
||||
JavaRDD<HoodieRecord> insertAndUpdatesRDD3 = jsc.parallelize(insertsAndUpdates3, 1);
|
||||
statuses = client.upsert(insertAndUpdatesRDD3, commitTime3).collect();
|
||||
assertNoWriteErrors(statuses);
|
||||
|
||||
assertEquals("2 files needs to be committed.", 2, statuses.size());
|
||||
HoodieTableMetaClient metadata = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath);
|
||||
|
||||
HoodieTable table = getHoodieTable(metadata, config);
|
||||
ReadOptimizedView fileSystemView = table.getROFileSystemView();
|
||||
List<HoodieDataFile> files = fileSystemView.getLatestDataFilesBeforeOrOn(testPartitionPath, commitTime3)
|
||||
.collect(Collectors.toList());
|
||||
int numTotalInsertsInCommit3 = 0;
|
||||
int numTotalUpdatesInCommit3 = 0;
|
||||
for (HoodieDataFile file : files) {
|
||||
if (file.getFileName().contains(file1)) {
|
||||
assertEquals("Existing file should be expanded", commitTime3, file.getCommitTime());
|
||||
records = ParquetUtils.readAvroRecords(jsc.hadoopConfiguration(), new Path(file.getPath()));
|
||||
for (GenericRecord record : records) {
|
||||
String recordKey = record.get(HoodieRecord.RECORD_KEY_METADATA_FIELD).toString();
|
||||
String recordCommitTime = record.get(HoodieRecord.COMMIT_TIME_METADATA_FIELD).toString();
|
||||
if (recordCommitTime.equals(commitTime3)) {
|
||||
if (keys2.contains(recordKey)) {
|
||||
keys2.remove(recordKey);
|
||||
numTotalUpdatesInCommit3++;
|
||||
} else {
|
||||
numTotalInsertsInCommit3++;
|
||||
}
|
||||
}
|
||||
}
|
||||
assertEquals("All keys added in commit 2 must be updated in commit3 correctly", 0, keys2.size());
|
||||
} else {
|
||||
assertEquals("New file must be written for commit 3", commitTime3, file.getCommitTime());
|
||||
records = ParquetUtils.readAvroRecords(jsc.hadoopConfiguration(), new Path(file.getPath()));
|
||||
for (GenericRecord record : records) {
|
||||
String recordKey = record.get(HoodieRecord.RECORD_KEY_METADATA_FIELD).toString();
|
||||
assertEquals("only expect commit3", commitTime3,
|
||||
record.get(HoodieRecord.COMMIT_TIME_METADATA_FIELD).toString());
|
||||
assertTrue("key expected to be part of commit3", keys3.contains(recordKey));
|
||||
}
|
||||
numTotalInsertsInCommit3 += records.size();
|
||||
}
|
||||
}
|
||||
assertEquals("Total updates in commit3 must add up", inserts2.size(), numTotalUpdatesInCommit3);
|
||||
assertEquals("Total inserts in commit3 must add up", keys3.size(), numTotalInsertsInCommit3);
|
||||
}
|
||||
|
||||
/**
|
||||
* Test scenario of new file-group getting added during insert()
|
||||
*/
|
||||
@Test
|
||||
public void testSmallInsertHandlingForInserts() throws Exception {
|
||||
|
||||
final String testPartitionPath = "2016/09/26";
|
||||
final int insertSplitLimit = 100;
|
||||
// setup the small file handling params
|
||||
HoodieWriteConfig config = getSmallInsertWriteConfig(insertSplitLimit); // hold upto 200 records max
|
||||
dataGen = new HoodieTestDataGenerator(new String[]{testPartitionPath});
|
||||
HoodieWriteClient client = getHoodieWriteClient(config, false);
|
||||
|
||||
// Inserts => will write file1
|
||||
String commitTime1 = "001";
|
||||
client.startCommitWithTime(commitTime1);
|
||||
List<HoodieRecord> inserts1 = dataGen.generateInserts(commitTime1, insertSplitLimit); // this writes ~500kb
|
||||
Set<String> keys1 = HoodieClientTestUtils.getRecordKeys(inserts1);
|
||||
JavaRDD<HoodieRecord> insertRecordsRDD1 = jsc.parallelize(inserts1, 1);
|
||||
List<WriteStatus> statuses = client.insert(insertRecordsRDD1, commitTime1).collect();
|
||||
|
||||
assertNoWriteErrors(statuses);
|
||||
assertPartitionMetadata(new String[]{testPartitionPath}, fs);
|
||||
|
||||
assertEquals("Just 1 file needs to be added.", 1, statuses.size());
|
||||
String file1 = statuses.get(0).getFileId();
|
||||
assertEquals("file should contain 100 records", readRowKeysFromParquet(jsc.hadoopConfiguration(),
|
||||
new Path(basePath, statuses.get(0).getStat().getPath())).size(), 100);
|
||||
|
||||
// Second, set of Inserts should just expand file1
|
||||
String commitTime2 = "002";
|
||||
client.startCommitWithTime(commitTime2);
|
||||
List<HoodieRecord> inserts2 = dataGen.generateInserts(commitTime2, 40);
|
||||
Set<String> keys2 = HoodieClientTestUtils.getRecordKeys(inserts2);
|
||||
JavaRDD<HoodieRecord> insertRecordsRDD2 = jsc.parallelize(inserts2, 1);
|
||||
statuses = client.insert(insertRecordsRDD2, commitTime2).collect();
|
||||
assertNoWriteErrors(statuses);
|
||||
|
||||
assertEquals("Just 1 file needs to be updated.", 1, statuses.size());
|
||||
assertEquals("Existing file should be expanded", file1, statuses.get(0).getFileId());
|
||||
assertEquals("Existing file should be expanded", commitTime1, statuses.get(0).getStat().getPrevCommit());
|
||||
Path newFile = new Path(basePath, statuses.get(0).getStat().getPath());
|
||||
assertEquals("file should contain 140 records",
|
||||
readRowKeysFromParquet(jsc.hadoopConfiguration(), newFile).size(), 140);
|
||||
|
||||
List<GenericRecord> records = ParquetUtils.readAvroRecords(jsc.hadoopConfiguration(), newFile);
|
||||
for (GenericRecord record : records) {
|
||||
String recordKey = record.get(HoodieRecord.RECORD_KEY_METADATA_FIELD).toString();
|
||||
String recCommitTime = record.get(HoodieRecord.COMMIT_TIME_METADATA_FIELD).toString();
|
||||
assertTrue("Record expected to be part of commit 1 or commit2",
|
||||
commitTime1.equals(recCommitTime) || commitTime2.equals(recCommitTime));
|
||||
assertTrue("key expected to be part of commit 1 or commit2",
|
||||
keys2.contains(recordKey) || keys1.contains(recordKey));
|
||||
}
|
||||
|
||||
// Lots of inserts such that file1 is updated and expanded, a new file2 is created.
|
||||
String commitTime3 = "003";
|
||||
client.startCommitWithTime(commitTime3);
|
||||
List<HoodieRecord> insert3 = dataGen.generateInserts(commitTime3, 200);
|
||||
JavaRDD<HoodieRecord> insertRecordsRDD3 = jsc.parallelize(insert3, 1);
|
||||
statuses = client.insert(insertRecordsRDD3, commitTime3).collect();
|
||||
assertNoWriteErrors(statuses);
|
||||
assertEquals("2 files needs to be committed.", 2, statuses.size());
|
||||
|
||||
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath);
|
||||
HoodieTable table = getHoodieTable(metaClient, config);
|
||||
List<HoodieDataFile> files = table.getROFileSystemView()
|
||||
.getLatestDataFilesBeforeOrOn(testPartitionPath, commitTime3)
|
||||
.collect(Collectors.toList());
|
||||
assertEquals("Total of 2 valid data files", 2, files.size());
|
||||
|
||||
int totalInserts = 0;
|
||||
for (HoodieDataFile file : files) {
|
||||
assertEquals("All files must be at commit 3", commitTime3, file.getCommitTime());
|
||||
records = ParquetUtils.readAvroRecords(jsc.hadoopConfiguration(), new Path(file.getPath()));
|
||||
totalInserts += records.size();
|
||||
}
|
||||
assertEquals("Total number of records must add up", totalInserts,
|
||||
inserts1.size() + inserts2.size() + insert3.size());
|
||||
}
|
||||
|
||||
/**
|
||||
* Test to ensure commit metadata points to valid files
|
||||
*/
|
||||
@Test
|
||||
public void testCommitWritesRelativePaths() throws Exception {
|
||||
|
||||
HoodieWriteConfig cfg = getConfigBuilder().withAutoCommit(false).build();
|
||||
HoodieWriteClient client = getHoodieWriteClient(cfg);
|
||||
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath);
|
||||
HoodieTable table = HoodieTable.getHoodieTable(metaClient, cfg, jsc);
|
||||
|
||||
String commitTime = "000";
|
||||
client.startCommitWithTime(commitTime);
|
||||
|
||||
List<HoodieRecord> records = dataGen.generateInserts(commitTime, 200);
|
||||
JavaRDD<HoodieRecord> writeRecords = jsc.parallelize(records, 1);
|
||||
|
||||
JavaRDD<WriteStatus> result = client.bulkInsert(writeRecords, commitTime);
|
||||
|
||||
assertTrue("Commit should succeed", client.commit(commitTime, result));
|
||||
assertTrue("After explicit commit, commit file should be created",
|
||||
HoodieTestUtils.doesCommitExist(basePath, commitTime));
|
||||
|
||||
// Get parquet file paths from commit metadata
|
||||
String actionType = metaClient.getCommitActionType();
|
||||
HoodieInstant commitInstant = new HoodieInstant(false, actionType, commitTime);
|
||||
HoodieTimeline commitTimeline = metaClient.getCommitTimeline().filterCompletedInstants();
|
||||
HoodieCommitMetadata commitMetadata = HoodieCommitMetadata
|
||||
.fromBytes(commitTimeline.getInstantDetails(commitInstant).get(), HoodieCommitMetadata.class);
|
||||
String basePath = table.getMetaClient().getBasePath();
|
||||
Collection<String> commitPathNames = commitMetadata.getFileIdAndFullPaths(basePath).values();
|
||||
|
||||
// Read from commit file
|
||||
String filename = HoodieTestUtils.getCommitFilePath(basePath, commitTime);
|
||||
FileInputStream inputStream = new FileInputStream(filename);
|
||||
String everything = IOUtils.toString(inputStream);
|
||||
HoodieCommitMetadata metadata = HoodieCommitMetadata.fromJsonString(everything.toString(),
|
||||
HoodieCommitMetadata.class);
|
||||
HashMap<String, String> paths = metadata.getFileIdAndFullPaths(basePath);
|
||||
inputStream.close();
|
||||
|
||||
// Compare values in both to make sure they are equal.
|
||||
for (String pathName : paths.values()) {
|
||||
assertTrue(commitPathNames.contains(pathName));
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Test to ensure commit metadata points to valid files
|
||||
*/
|
||||
@Test
|
||||
public void testRollingStatsInMetadata() throws Exception {
|
||||
|
||||
HoodieWriteConfig cfg = getConfigBuilder().withAutoCommit(false).build();
|
||||
HoodieWriteClient client = getHoodieWriteClient(cfg);
|
||||
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath);
|
||||
HoodieTable table = HoodieTable.getHoodieTable(metaClient, cfg, jsc);
|
||||
|
||||
String commitTime = "000";
|
||||
client.startCommitWithTime(commitTime);
|
||||
|
||||
List<HoodieRecord> records = dataGen.generateInserts(commitTime, 200);
|
||||
JavaRDD<HoodieRecord> writeRecords = jsc.parallelize(records, 1);
|
||||
|
||||
JavaRDD<WriteStatus> result = client.bulkInsert(writeRecords, commitTime);
|
||||
|
||||
assertTrue("Commit should succeed", client.commit(commitTime, result));
|
||||
assertTrue("After explicit commit, commit file should be created",
|
||||
HoodieTestUtils.doesCommitExist(basePath, commitTime));
|
||||
|
||||
// Read from commit file
|
||||
String filename = HoodieTestUtils.getCommitFilePath(basePath, commitTime);
|
||||
FileInputStream inputStream = new FileInputStream(filename);
|
||||
String everything = IOUtils.toString(inputStream);
|
||||
HoodieCommitMetadata metadata = HoodieCommitMetadata.fromJsonString(everything.toString(),
|
||||
HoodieCommitMetadata.class);
|
||||
HoodieRollingStatMetadata rollingStatMetadata = HoodieCommitMetadata.fromJsonString(metadata.getExtraMetadata()
|
||||
.get(HoodieRollingStatMetadata.ROLLING_STAT_METADATA_KEY), HoodieRollingStatMetadata.class);
|
||||
int inserts = 0;
|
||||
for (Map.Entry<String, Map<String, HoodieRollingStat>> pstat : rollingStatMetadata.getPartitionToRollingStats()
|
||||
.entrySet()) {
|
||||
for (Map.Entry<String, HoodieRollingStat> stat : pstat.getValue().entrySet()) {
|
||||
inserts += stat.getValue().getInserts();
|
||||
}
|
||||
}
|
||||
Assert.assertEquals(inserts, 200);
|
||||
|
||||
// Update + Inserts such that they just expand file1
|
||||
commitTime = "001";
|
||||
client.startCommitWithTime(commitTime);
|
||||
|
||||
records = dataGen.generateUpdates(commitTime, records);
|
||||
writeRecords = jsc.parallelize(records, 1);
|
||||
result = client.upsert(writeRecords, commitTime);
|
||||
|
||||
assertTrue("Commit should succeed", client.commit(commitTime, result));
|
||||
assertTrue("After explicit commit, commit file should be created",
|
||||
HoodieTestUtils.doesCommitExist(basePath, commitTime));
|
||||
|
||||
// Read from commit file
|
||||
filename = HoodieTestUtils.getCommitFilePath(basePath, commitTime);
|
||||
inputStream = new FileInputStream(filename);
|
||||
everything = IOUtils.toString(inputStream);
|
||||
metadata = HoodieCommitMetadata.fromJsonString(everything.toString(), HoodieCommitMetadata.class);
|
||||
rollingStatMetadata = HoodieCommitMetadata.fromJsonString(metadata.getExtraMetadata()
|
||||
.get(HoodieRollingStatMetadata.ROLLING_STAT_METADATA_KEY), HoodieRollingStatMetadata.class);
|
||||
inserts = 0;
|
||||
int upserts = 0;
|
||||
for (Map.Entry<String, Map<String, HoodieRollingStat>> pstat : rollingStatMetadata.getPartitionToRollingStats()
|
||||
.entrySet()) {
|
||||
for (Map.Entry<String, HoodieRollingStat> stat : pstat.getValue().entrySet()) {
|
||||
inserts += stat.getValue().getInserts();
|
||||
upserts += stat.getValue().getUpserts();
|
||||
}
|
||||
}
|
||||
Assert.assertEquals(inserts, 200);
|
||||
Assert.assertEquals(upserts, 200);
|
||||
|
||||
}
|
||||
|
||||
/**
|
||||
* Tests behavior of committing only when consistency is verified
|
||||
*/
|
||||
@Test
|
||||
public void testConsistencyCheckDuringFinalize() throws Exception {
|
||||
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(),
|
||||
basePath);
|
||||
String commitTime = "000";
|
||||
HoodieWriteConfig cfg = getConfigBuilder().withAutoCommit(false).build();
|
||||
HoodieWriteClient client = getHoodieWriteClient(cfg);
|
||||
Pair<Path, JavaRDD<WriteStatus>> result = testConsistencyCheck(metaClient, commitTime);
|
||||
|
||||
// Delete orphan marker and commit should succeed
|
||||
metaClient.getFs().delete(result.getKey(), false);
|
||||
assertTrue("Commit should succeed", client.commit(commitTime, result.getRight()));
|
||||
assertTrue("After explicit commit, commit file should be created",
|
||||
HoodieTestUtils.doesCommitExist(basePath, commitTime));
|
||||
// Marker directory must be removed
|
||||
assertFalse(metaClient.getFs().exists(new Path(metaClient.getMarkerFolderPath(commitTime))));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testRollbackAfterConsistencyCheckFailure() throws Exception {
|
||||
String commitTime = "000";
|
||||
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath);
|
||||
HoodieWriteConfig cfg = getConfigBuilder().withAutoCommit(false).build();
|
||||
HoodieWriteClient client = getHoodieWriteClient(cfg);
|
||||
testConsistencyCheck(metaClient, commitTime);
|
||||
|
||||
// Rollback of this commit should succeed
|
||||
client.rollback(commitTime);
|
||||
assertFalse("After explicit rollback, commit file should not be present",
|
||||
HoodieTestUtils.doesCommitExist(basePath, commitTime));
|
||||
// Marker directory must be removed after rollback
|
||||
assertFalse(metaClient.getFs().exists(new Path(metaClient.getMarkerFolderPath(commitTime))));
|
||||
}
|
||||
|
||||
private Pair<Path, JavaRDD<WriteStatus>> testConsistencyCheck(HoodieTableMetaClient metaClient, String commitTime)
|
||||
throws Exception {
|
||||
HoodieWriteConfig cfg = getConfigBuilder().withAutoCommit(false)
|
||||
.withConsistencyGuardConfig(ConsistencyGuardConfig.newBuilder()
|
||||
.withConsistencyCheckEnabled(true)
|
||||
.withMaxConsistencyCheckIntervalMs(1)
|
||||
.withInitialConsistencyCheckIntervalMs(1)
|
||||
.build())
|
||||
.build();
|
||||
HoodieWriteClient client = getHoodieWriteClient(cfg);
|
||||
|
||||
client.startCommitWithTime(commitTime);
|
||||
JavaRDD<HoodieRecord> writeRecords = jsc.parallelize(dataGen.generateInserts(commitTime, 200), 1);
|
||||
JavaRDD<WriteStatus> result = client.bulkInsert(writeRecords, commitTime);
|
||||
result.collect();
|
||||
|
||||
// Create a dummy marker file to simulate the case that a marker file was created without data file.
|
||||
// This should fail the commit
|
||||
String partitionPath = Arrays.stream(fs.globStatus(new Path(String.format("%s/*/*/*/*",
|
||||
metaClient.getMarkerFolderPath(commitTime))),
|
||||
path -> path.toString().endsWith(HoodieTableMetaClient.MARKER_EXTN))).limit(1)
|
||||
.map(status -> status.getPath().getParent().toString()).collect(Collectors.toList()).get(0);
|
||||
Path markerFilePath = new Path(String.format("%s/%s", partitionPath,
|
||||
FSUtils.makeMarkerFile(commitTime, "1-0-1", UUID.randomUUID().toString())));
|
||||
metaClient.getFs().create(markerFilePath);
|
||||
logger.info("Created a dummy marker path=" + markerFilePath);
|
||||
|
||||
try {
|
||||
client.commit(commitTime, result);
|
||||
fail("Commit should fail due to consistency check");
|
||||
} catch (HoodieCommitException cme) {
|
||||
assertTrue(cme.getCause() instanceof HoodieIOException);
|
||||
}
|
||||
return Pair.of(markerFilePath, result);
|
||||
}
|
||||
|
||||
/**
|
||||
* Build Hoodie Write Config for small data file sizes
|
||||
*/
|
||||
private HoodieWriteConfig getSmallInsertWriteConfig(int insertSplitSize) {
|
||||
HoodieWriteConfig.Builder builder = getConfigBuilder();
|
||||
return builder.withCompactionConfig(
|
||||
HoodieCompactionConfig.newBuilder().compactionSmallFileSize(HoodieTestDataGenerator.SIZE_PER_RECORD * 15)
|
||||
.insertSplitSize(insertSplitSize).build()) // tolerate upto 15 records
|
||||
.withStorageConfig(
|
||||
HoodieStorageConfig.newBuilder().limitFileSize(HoodieTestDataGenerator.SIZE_PER_RECORD * 20).build())
|
||||
.build();
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,207 @@
|
||||
/*
|
||||
* 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;
|
||||
|
||||
import static org.junit.Assert.assertTrue;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.Arrays;
|
||||
import java.util.Collection;
|
||||
import java.util.List;
|
||||
import java.util.stream.Collectors;
|
||||
import org.apache.hudi.common.model.HoodieRecord;
|
||||
import org.apache.hudi.common.util.Option;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
import org.apache.spark.api.java.JavaRDD;
|
||||
import org.junit.Assert;
|
||||
import org.junit.Test;
|
||||
|
||||
@SuppressWarnings("unchecked")
|
||||
/**
|
||||
* Test-cases for covering HoodieReadClient APIs
|
||||
*/
|
||||
public class TestHoodieReadClient extends TestHoodieClientBase {
|
||||
|
||||
@Override
|
||||
public void tearDown() throws IOException {
|
||||
super.tearDown();
|
||||
}
|
||||
|
||||
/**
|
||||
* Test ReadFilter API after writing new records using HoodieWriteClient.insert
|
||||
*/
|
||||
@Test
|
||||
public void testReadFilterExistAfterInsert() throws Exception {
|
||||
testReadFilterExist(getConfig(), HoodieWriteClient::insert);
|
||||
}
|
||||
|
||||
/**
|
||||
* Test ReadFilter API after writing new records using HoodieWriteClient.insertPrepped
|
||||
*/
|
||||
@Test
|
||||
public void testReadFilterExistAfterInsertPrepped() throws Exception {
|
||||
testReadFilterExist(getConfig(), HoodieWriteClient::insertPreppedRecords);
|
||||
}
|
||||
|
||||
/**
|
||||
* Test ReadFilter API after writing new records using HoodieWriteClient.bulkInsert
|
||||
*/
|
||||
@Test
|
||||
public void testReadFilterExistAfterBulkInsert() throws Exception {
|
||||
testReadFilterExist(getConfigBuilder().withBulkInsertParallelism(1).build(), HoodieWriteClient::bulkInsert);
|
||||
}
|
||||
|
||||
/**
|
||||
* Test ReadFilter API after writing new records using HoodieWriteClient.bulkInsertPrepped
|
||||
*/
|
||||
@Test
|
||||
public void testReadFilterExistAfterBulkInsertPrepped() throws Exception {
|
||||
testReadFilterExist(getConfigBuilder().withBulkInsertParallelism(1).build(),
|
||||
(writeClient, recordRDD, commitTime) -> {
|
||||
return writeClient.bulkInsertPreppedRecords(recordRDD, commitTime, Option.empty());
|
||||
});
|
||||
}
|
||||
|
||||
/**
|
||||
* Helper to write new records using one of HoodieWriteClient's write API and use ReadClient to test filterExists()
|
||||
* API works correctly
|
||||
*
|
||||
* @param config Hoodie Write Config
|
||||
* @param writeFn Write Function for writing records
|
||||
* @throws Exception in case of error
|
||||
*/
|
||||
private void testReadFilterExist(HoodieWriteConfig config,
|
||||
Function3<JavaRDD<WriteStatus>, HoodieWriteClient, JavaRDD<HoodieRecord>, String> writeFn) throws Exception {
|
||||
HoodieWriteClient writeClient = getHoodieWriteClient(config);
|
||||
String newCommitTime = writeClient.startCommit();
|
||||
List<HoodieRecord> records = dataGen.generateInserts(newCommitTime, 100);
|
||||
JavaRDD<HoodieRecord> recordsRDD = jsc.parallelize(records, 1);
|
||||
|
||||
HoodieReadClient readClient = getHoodieReadClient(config.getBasePath());
|
||||
JavaRDD<HoodieRecord> filteredRDD = readClient.filterExists(recordsRDD);
|
||||
|
||||
// Should not find any files
|
||||
assertTrue(filteredRDD.collect().size() == 100);
|
||||
|
||||
JavaRDD<HoodieRecord> smallRecordsRDD = jsc.parallelize(records.subList(0, 75), 1);
|
||||
// We create three parquet file, each having one record. (3 different partitions)
|
||||
List<WriteStatus> statuses = writeFn.apply(writeClient, smallRecordsRDD, newCommitTime).collect();
|
||||
// Verify there are no errors
|
||||
assertNoWriteErrors(statuses);
|
||||
|
||||
readClient = getHoodieReadClient(config.getBasePath());
|
||||
filteredRDD = readClient.filterExists(recordsRDD);
|
||||
List<HoodieRecord> result = filteredRDD.collect();
|
||||
// Check results
|
||||
Assert.assertEquals(25, result.size());
|
||||
}
|
||||
|
||||
/**
|
||||
* Test tagLocation API after insert()
|
||||
*/
|
||||
@Test
|
||||
public void testTagLocationAfterInsert() throws Exception {
|
||||
testTagLocation(getConfig(), HoodieWriteClient::insert,
|
||||
HoodieWriteClient::upsert, false);
|
||||
}
|
||||
|
||||
/**
|
||||
* Test tagLocation API after insertPrepped()
|
||||
*/
|
||||
@Test
|
||||
public void testTagLocationAfterInsertPrepped() throws Exception {
|
||||
testTagLocation(getConfig(), HoodieWriteClient::insertPreppedRecords,
|
||||
HoodieWriteClient::upsertPreppedRecords, true);
|
||||
}
|
||||
|
||||
/**
|
||||
* Test tagLocation API after bulk-insert()
|
||||
*/
|
||||
@Test
|
||||
public void testTagLocationAfterBulkInsert() throws Exception {
|
||||
testTagLocation(getConfigBuilder().withBulkInsertParallelism(1).build(), HoodieWriteClient::bulkInsert,
|
||||
HoodieWriteClient::upsert, false);
|
||||
}
|
||||
|
||||
/**
|
||||
* Test tagLocation API after bulkInsertPrepped()
|
||||
*/
|
||||
@Test
|
||||
public void testTagLocationAfterBulkInsertPrepped() throws Exception {
|
||||
testTagLocation(getConfigBuilder().withBulkInsertParallelism(1).build(),
|
||||
(writeClient, recordRDD, commitTime)
|
||||
-> writeClient.bulkInsertPreppedRecords(recordRDD, commitTime, Option.empty()),
|
||||
HoodieWriteClient::upsertPreppedRecords, true);
|
||||
}
|
||||
|
||||
/**
|
||||
* Helper method to test tagLocation after using different HoodieWriteClient write APIS
|
||||
*
|
||||
* @param hoodieWriteConfig Write Config
|
||||
* @param insertFn Hoodie Write Client first Insert API
|
||||
* @param updateFn Hoodie Write Client upsert API
|
||||
* @param isPrepped isPrepped flag.
|
||||
* @throws Exception in case of error
|
||||
*/
|
||||
private void testTagLocation(
|
||||
HoodieWriteConfig hoodieWriteConfig,
|
||||
Function3<JavaRDD<WriteStatus>, HoodieWriteClient, JavaRDD<HoodieRecord>, String> insertFn,
|
||||
Function3<JavaRDD<WriteStatus>, HoodieWriteClient, JavaRDD<HoodieRecord>, String> updateFn,
|
||||
boolean isPrepped)
|
||||
throws Exception {
|
||||
HoodieWriteClient client = getHoodieWriteClient(hoodieWriteConfig);
|
||||
//Write 1 (only inserts)
|
||||
String newCommitTime = "001";
|
||||
String initCommitTime = "000";
|
||||
int numRecords = 200;
|
||||
JavaRDD<WriteStatus> result =
|
||||
insertFirstBatch(hoodieWriteConfig, client, newCommitTime, initCommitTime, numRecords, insertFn, isPrepped,
|
||||
true, numRecords);
|
||||
// Construct HoodieRecord from the WriteStatus but set HoodieKey, Data and HoodieRecordLocation accordingly
|
||||
// since they have been modified in the DAG
|
||||
JavaRDD<HoodieRecord> recordRDD =
|
||||
jsc.parallelize(
|
||||
result.collect().stream().map(WriteStatus::getWrittenRecords).flatMap(Collection::stream)
|
||||
.map(record -> new HoodieRecord(record.getKey(), null))
|
||||
.collect(Collectors.toList()));
|
||||
// Should have 100 records in table (check using Index), all in locations marked at commit
|
||||
HoodieReadClient readClient = getHoodieReadClient(hoodieWriteConfig.getBasePath());
|
||||
List<HoodieRecord> taggedRecords = readClient.tagLocation(recordRDD).collect();
|
||||
checkTaggedRecords(taggedRecords, newCommitTime);
|
||||
|
||||
// Write 2 (updates)
|
||||
String prevCommitTime = newCommitTime;
|
||||
newCommitTime = "004";
|
||||
numRecords = 100;
|
||||
String commitTimeBetweenPrevAndNew = "002";
|
||||
result = updateBatch(hoodieWriteConfig, client, newCommitTime, prevCommitTime,
|
||||
Option.of(Arrays.asList(commitTimeBetweenPrevAndNew)),
|
||||
initCommitTime, numRecords, updateFn, isPrepped,
|
||||
true, numRecords, 200, 2);
|
||||
recordRDD =
|
||||
jsc.parallelize(
|
||||
result.collect().stream().map(WriteStatus::getWrittenRecords).flatMap(Collection::stream)
|
||||
.map(record -> new HoodieRecord(record.getKey(), null))
|
||||
.collect(Collectors.toList()));
|
||||
// Index should be able to locate all updates in correct locations.
|
||||
readClient = getHoodieReadClient(hoodieWriteConfig.getBasePath());
|
||||
taggedRecords = readClient.tagLocation(recordRDD).collect();
|
||||
checkTaggedRecords(taggedRecords, newCommitTime);
|
||||
}
|
||||
}
|
||||
180
hudi-client/src/test/java/org/apache/hudi/TestMultiFS.java
Normal file
180
hudi-client/src/test/java/org/apache/hudi/TestMultiFS.java
Normal file
@@ -0,0 +1,180 @@
|
||||
/*
|
||||
* 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;
|
||||
|
||||
import static org.junit.Assert.assertEquals;
|
||||
|
||||
import java.io.Serializable;
|
||||
import java.util.List;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.hdfs.DistributedFileSystem;
|
||||
import org.apache.hadoop.hdfs.MiniDFSCluster;
|
||||
import org.apache.hudi.common.HoodieClientTestUtils;
|
||||
import org.apache.hudi.common.HoodieTestDataGenerator;
|
||||
import org.apache.hudi.common.minicluster.HdfsTestService;
|
||||
import org.apache.hudi.common.model.HoodieAvroPayload;
|
||||
import org.apache.hudi.common.model.HoodieRecord;
|
||||
import org.apache.hudi.common.model.HoodieTableType;
|
||||
import org.apache.hudi.common.model.HoodieTestUtils;
|
||||
import org.apache.hudi.common.table.HoodieTableMetaClient;
|
||||
import org.apache.hudi.common.table.HoodieTimeline;
|
||||
import org.apache.hudi.common.table.timeline.HoodieActiveTimeline;
|
||||
import org.apache.hudi.common.util.FSUtils;
|
||||
import org.apache.hudi.config.HoodieIndexConfig;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
import org.apache.hudi.index.HoodieIndex;
|
||||
import org.apache.log4j.LogManager;
|
||||
import org.apache.log4j.Logger;
|
||||
import org.apache.spark.SparkConf;
|
||||
import org.apache.spark.api.java.JavaRDD;
|
||||
import org.apache.spark.api.java.JavaSparkContext;
|
||||
import org.apache.spark.sql.Dataset;
|
||||
import org.apache.spark.sql.Row;
|
||||
import org.apache.spark.sql.SQLContext;
|
||||
import org.junit.After;
|
||||
import org.junit.AfterClass;
|
||||
import org.junit.BeforeClass;
|
||||
import org.junit.Test;
|
||||
|
||||
public class TestMultiFS implements Serializable {
|
||||
|
||||
private static String dfsBasePath;
|
||||
private static HdfsTestService hdfsTestService;
|
||||
private static MiniDFSCluster dfsCluster;
|
||||
private static DistributedFileSystem dfs;
|
||||
private static Logger logger = LogManager.getLogger(TestMultiFS.class);
|
||||
private static JavaSparkContext jsc;
|
||||
private static SQLContext sqlContext;
|
||||
private String tablePath = "file:///tmp/hoodie/sample-table";
|
||||
protected String tableName = "hoodie_rt";
|
||||
private HoodieWriteClient hdfsWriteClient;
|
||||
private String tableType = HoodieTableType.COPY_ON_WRITE.name();
|
||||
|
||||
@BeforeClass
|
||||
public static void initClass() throws Exception {
|
||||
hdfsTestService = new HdfsTestService();
|
||||
dfsCluster = hdfsTestService.start(true);
|
||||
|
||||
// Create a temp folder as the base path
|
||||
dfs = dfsCluster.getFileSystem();
|
||||
dfsBasePath = dfs.getWorkingDirectory().toString();
|
||||
dfs.mkdirs(new Path(dfsBasePath));
|
||||
|
||||
SparkConf sparkConf = new SparkConf().setAppName("hoodie-client-example");
|
||||
sparkConf.setMaster("local[1]");
|
||||
sparkConf.set("spark.serializer", "org.apache.spark.serializer.KryoSerializer");
|
||||
sparkConf.set("spark.kryoserializer.buffer.max", "512m");
|
||||
jsc = new JavaSparkContext(sparkConf);
|
||||
sqlContext = new SQLContext(jsc);
|
||||
}
|
||||
|
||||
private HoodieWriteClient getHoodieWriteClient(HoodieWriteConfig config) throws Exception {
|
||||
if (null != hdfsWriteClient) {
|
||||
hdfsWriteClient.close();
|
||||
}
|
||||
hdfsWriteClient = new HoodieWriteClient(jsc, config);
|
||||
return hdfsWriteClient;
|
||||
}
|
||||
|
||||
@After
|
||||
public void teardown() {
|
||||
if (null != hdfsWriteClient) {
|
||||
hdfsWriteClient.close();
|
||||
hdfsWriteClient = null;
|
||||
}
|
||||
}
|
||||
|
||||
@AfterClass
|
||||
public static void cleanupClass() throws Exception {
|
||||
if (jsc != null) {
|
||||
jsc.stop();
|
||||
}
|
||||
|
||||
if (hdfsTestService != null) {
|
||||
hdfsTestService.stop();
|
||||
dfsCluster.shutdown();
|
||||
}
|
||||
// Need to closeAll to clear FileSystem.Cache, required because DFS and LocalFS used in the
|
||||
// same JVM
|
||||
FileSystem.closeAll();
|
||||
}
|
||||
|
||||
|
||||
protected HoodieWriteConfig getHoodieWriteConfig(String basePath) {
|
||||
return HoodieWriteConfig.newBuilder().withPath(basePath).withEmbeddedTimelineServerEnabled(true)
|
||||
.withSchema(HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA).withParallelism(2, 2)
|
||||
.forTable(tableName).withIndexConfig(
|
||||
HoodieIndexConfig.newBuilder().withIndexType(HoodieIndex.IndexType.BLOOM).build()).build();
|
||||
}
|
||||
|
||||
@Test
|
||||
public void readLocalWriteHDFS() throws Exception {
|
||||
|
||||
// Generator of some records to be loaded in.
|
||||
HoodieTestDataGenerator dataGen = new HoodieTestDataGenerator();
|
||||
|
||||
// Initialize table and filesystem
|
||||
HoodieTableMetaClient
|
||||
.initTableType(jsc.hadoopConfiguration(), dfsBasePath, HoodieTableType.valueOf(tableType), tableName,
|
||||
HoodieAvroPayload.class.getName());
|
||||
|
||||
//Create write client to write some records in
|
||||
HoodieWriteConfig cfg = getHoodieWriteConfig(dfsBasePath);
|
||||
HoodieWriteClient hdfsWriteClient = getHoodieWriteClient(cfg);
|
||||
|
||||
// Write generated data to hdfs (only inserts)
|
||||
String readCommitTime = hdfsWriteClient.startCommit();
|
||||
logger.info("Starting commit " + readCommitTime);
|
||||
List<HoodieRecord> records = dataGen.generateInserts(readCommitTime, 100);
|
||||
JavaRDD<HoodieRecord> writeRecords = jsc.parallelize(records, 1);
|
||||
hdfsWriteClient.upsert(writeRecords, readCommitTime);
|
||||
|
||||
// Read from hdfs
|
||||
FileSystem fs = FSUtils.getFs(dfsBasePath, HoodieTestUtils.getDefaultHadoopConf());
|
||||
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(fs.getConf(), dfsBasePath);
|
||||
HoodieTimeline timeline = new HoodieActiveTimeline(metaClient).getCommitTimeline();
|
||||
Dataset<Row> readRecords = HoodieClientTestUtils.readCommit(dfsBasePath, sqlContext, timeline, readCommitTime);
|
||||
assertEquals("Should contain 100 records", readRecords.count(), records.size());
|
||||
|
||||
// Write to local
|
||||
HoodieTableMetaClient
|
||||
.initTableType(jsc.hadoopConfiguration(), tablePath, HoodieTableType.valueOf(tableType), tableName,
|
||||
HoodieAvroPayload.class.getName());
|
||||
HoodieWriteConfig localConfig = getHoodieWriteConfig(tablePath);
|
||||
HoodieWriteClient localWriteClient = getHoodieWriteClient(localConfig);
|
||||
|
||||
String writeCommitTime = localWriteClient.startCommit();
|
||||
logger.info("Starting write commit " + writeCommitTime);
|
||||
List<HoodieRecord> localRecords = dataGen.generateInserts(writeCommitTime, 100);
|
||||
JavaRDD<HoodieRecord> localWriteRecords = jsc.parallelize(localRecords, 1);
|
||||
logger.info("Writing to path: " + tablePath);
|
||||
localWriteClient.upsert(localWriteRecords, writeCommitTime);
|
||||
|
||||
logger.info("Reading from path: " + tablePath);
|
||||
fs = FSUtils.getFs(tablePath, HoodieTestUtils.getDefaultHadoopConf());
|
||||
metaClient = new HoodieTableMetaClient(fs.getConf(), tablePath);
|
||||
timeline = new HoodieActiveTimeline(metaClient).getCommitTimeline();
|
||||
Dataset<Row> localReadRecords = HoodieClientTestUtils.readCommit(tablePath, sqlContext, timeline, writeCommitTime);
|
||||
assertEquals("Should contain 100 records", localReadRecords.count(), localRecords.size());
|
||||
|
||||
hdfsWriteClient.close();
|
||||
localWriteClient.close();
|
||||
}
|
||||
}
|
||||
@@ -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;
|
||||
|
||||
import static org.junit.Assert.assertEquals;
|
||||
import static org.junit.Assert.assertTrue;
|
||||
|
||||
import java.io.IOException;
|
||||
import org.apache.hudi.common.model.HoodieRecord;
|
||||
import org.junit.Test;
|
||||
import org.mockito.Mockito;
|
||||
|
||||
public class TestWriteStatus {
|
||||
@Test
|
||||
public void testFailureFraction() throws IOException {
|
||||
WriteStatus status = new WriteStatus(true, 0.1);
|
||||
Throwable t = new Exception("some error in writing");
|
||||
for (int i = 0; i < 1000; i++) {
|
||||
status.markFailure(Mockito.mock(HoodieRecord.class), t, null);
|
||||
}
|
||||
assertTrue(status.getFailedRecords().size() > 0);
|
||||
assertTrue(status.getFailedRecords().size() < 150); //150 instead of 100, to prevent flaky test
|
||||
assertTrue(status.hasErrors());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testSuccessRecordTracking() {
|
||||
WriteStatus status = new WriteStatus(false, 1.0);
|
||||
Throwable t = new Exception("some error in writing");
|
||||
for (int i = 0; i < 1000; i++) {
|
||||
status.markSuccess(Mockito.mock(HoodieRecord.class), null);
|
||||
status.markFailure(Mockito.mock(HoodieRecord.class), t, null);
|
||||
}
|
||||
assertEquals(1000, status.getFailedRecords().size());
|
||||
assertTrue(status.hasErrors());
|
||||
assertTrue(status.getWrittenRecords().isEmpty());
|
||||
assertEquals(2000, status.getTotalRecords());
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,249 @@
|
||||
/*
|
||||
* 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.common;
|
||||
|
||||
import java.io.File;
|
||||
import java.io.IOException;
|
||||
import java.io.RandomAccessFile;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Arrays;
|
||||
import java.util.HashMap;
|
||||
import java.util.HashSet;
|
||||
import java.util.Iterator;
|
||||
import java.util.List;
|
||||
import java.util.Set;
|
||||
import java.util.UUID;
|
||||
import java.util.stream.Collectors;
|
||||
import org.apache.avro.Schema;
|
||||
import org.apache.avro.generic.GenericRecord;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hudi.HoodieReadClient;
|
||||
import org.apache.hudi.WriteStatus;
|
||||
import org.apache.hudi.avro.HoodieAvroWriteSupport;
|
||||
import org.apache.hudi.common.model.HoodieCommitMetadata;
|
||||
import org.apache.hudi.common.model.HoodieDataFile;
|
||||
import org.apache.hudi.common.model.HoodieRecord;
|
||||
import org.apache.hudi.common.model.HoodieTestUtils;
|
||||
import org.apache.hudi.common.table.HoodieTableMetaClient;
|
||||
import org.apache.hudi.common.table.HoodieTimeline;
|
||||
import org.apache.hudi.common.table.TableFileSystemView.ReadOptimizedView;
|
||||
import org.apache.hudi.common.table.timeline.HoodieInstant;
|
||||
import org.apache.hudi.common.table.view.HoodieTableFileSystemView;
|
||||
import org.apache.hudi.common.util.FSUtils;
|
||||
import org.apache.hudi.common.util.HoodieAvroUtils;
|
||||
import org.apache.hudi.config.HoodieStorageConfig;
|
||||
import org.apache.hudi.exception.HoodieException;
|
||||
import org.apache.hudi.io.storage.HoodieParquetConfig;
|
||||
import org.apache.hudi.io.storage.HoodieParquetWriter;
|
||||
import org.apache.log4j.LogManager;
|
||||
import org.apache.log4j.Logger;
|
||||
import org.apache.parquet.avro.AvroSchemaConverter;
|
||||
import org.apache.parquet.hadoop.ParquetWriter;
|
||||
import org.apache.parquet.hadoop.metadata.CompressionCodecName;
|
||||
import org.apache.spark.SparkConf;
|
||||
import org.apache.spark.api.java.JavaSparkContext;
|
||||
import org.apache.spark.sql.Dataset;
|
||||
import org.apache.spark.sql.Row;
|
||||
import org.apache.spark.sql.SQLContext;
|
||||
|
||||
/**
|
||||
* Utility methods to aid testing inside the HoodieClient module.
|
||||
*/
|
||||
public class HoodieClientTestUtils {
|
||||
|
||||
private static final transient Logger log = LogManager.getLogger(HoodieClientTestUtils.class);
|
||||
|
||||
public static List<WriteStatus> collectStatuses(Iterator<List<WriteStatus>> statusListItr) {
|
||||
List<WriteStatus> statuses = new ArrayList<>();
|
||||
while (statusListItr.hasNext()) {
|
||||
statuses.addAll(statusListItr.next());
|
||||
}
|
||||
return statuses;
|
||||
}
|
||||
|
||||
public static Set<String> getRecordKeys(List<HoodieRecord> hoodieRecords) {
|
||||
Set<String> keys = new HashSet<>();
|
||||
for (HoodieRecord rec : hoodieRecords) {
|
||||
keys.add(rec.getRecordKey());
|
||||
}
|
||||
return keys;
|
||||
}
|
||||
|
||||
private static void fakeMetaFile(String basePath, String commitTime, String suffix) throws IOException {
|
||||
String parentPath = basePath + "/" + HoodieTableMetaClient.METAFOLDER_NAME;
|
||||
new File(parentPath).mkdirs();
|
||||
new File(parentPath + "/" + commitTime + suffix).createNewFile();
|
||||
}
|
||||
|
||||
|
||||
public static void fakeCommitFile(String basePath, String commitTime) throws IOException {
|
||||
fakeMetaFile(basePath, commitTime, HoodieTimeline.COMMIT_EXTENSION);
|
||||
}
|
||||
|
||||
public static void fakeInFlightFile(String basePath, String commitTime) throws IOException {
|
||||
fakeMetaFile(basePath, commitTime, HoodieTimeline.INFLIGHT_EXTENSION);
|
||||
}
|
||||
|
||||
public static void fakeDataFile(String basePath, String partitionPath, String commitTime, String fileId)
|
||||
throws Exception {
|
||||
fakeDataFile(basePath, partitionPath, commitTime, fileId, 0);
|
||||
}
|
||||
|
||||
public static void fakeDataFile(String basePath, String partitionPath, String commitTime, String fileId, long length)
|
||||
throws Exception {
|
||||
String parentPath = String.format("%s/%s", basePath, partitionPath);
|
||||
new File(parentPath).mkdirs();
|
||||
String path = String.format("%s/%s", parentPath, FSUtils.makeDataFileName(commitTime, "1-0-1", fileId));
|
||||
new File(path).createNewFile();
|
||||
new RandomAccessFile(path, "rw").setLength(length);
|
||||
}
|
||||
|
||||
public static SparkConf getSparkConfForTest(String appName) {
|
||||
SparkConf sparkConf = new SparkConf().setAppName(appName)
|
||||
.set("spark.serializer", "org.apache.spark.serializer.KryoSerializer")
|
||||
.setMaster("local[8]");
|
||||
return HoodieReadClient.addHoodieSupport(sparkConf);
|
||||
}
|
||||
|
||||
public static HashMap<String, String> getLatestFileIDsToFullPath(String basePath, HoodieTimeline commitTimeline,
|
||||
List<HoodieInstant> commitsToReturn) throws IOException {
|
||||
HashMap<String, String> fileIdToFullPath = new HashMap<>();
|
||||
for (HoodieInstant commit : commitsToReturn) {
|
||||
HoodieCommitMetadata metadata = HoodieCommitMetadata.fromBytes(commitTimeline.getInstantDetails(commit).get(),
|
||||
HoodieCommitMetadata.class);
|
||||
fileIdToFullPath.putAll(metadata.getFileIdAndFullPaths(basePath));
|
||||
}
|
||||
return fileIdToFullPath;
|
||||
}
|
||||
|
||||
public static Dataset<Row> readCommit(String basePath, SQLContext sqlContext, HoodieTimeline commitTimeline,
|
||||
String commitTime) {
|
||||
HoodieInstant commitInstant = new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, commitTime);
|
||||
if (!commitTimeline.containsInstant(commitInstant)) {
|
||||
new HoodieException("No commit exists at " + commitTime);
|
||||
}
|
||||
try {
|
||||
HashMap<String, String> paths = getLatestFileIDsToFullPath(basePath, commitTimeline,
|
||||
Arrays.asList(commitInstant));
|
||||
log.info("Path :" + paths.values());
|
||||
return sqlContext.read().parquet(paths.values().toArray(new String[paths.size()]))
|
||||
.filter(String.format("%s ='%s'", HoodieRecord.COMMIT_TIME_METADATA_FIELD, commitTime));
|
||||
} catch (Exception e) {
|
||||
throw new HoodieException("Error reading commit " + commitTime, e);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Obtain all new data written into the Hoodie dataset since the given timestamp.
|
||||
*/
|
||||
public static Dataset<Row> readSince(String basePath, SQLContext sqlContext, HoodieTimeline commitTimeline,
|
||||
String lastCommitTime) {
|
||||
List<HoodieInstant> commitsToReturn = commitTimeline.findInstantsAfter(lastCommitTime, Integer.MAX_VALUE)
|
||||
.getInstants().collect(Collectors.toList());
|
||||
try {
|
||||
// Go over the commit metadata, and obtain the new files that need to be read.
|
||||
HashMap<String, String> fileIdToFullPath = getLatestFileIDsToFullPath(basePath, commitTimeline, commitsToReturn);
|
||||
return sqlContext.read().parquet(fileIdToFullPath.values().toArray(new String[fileIdToFullPath.size()]))
|
||||
.filter(String.format("%s >'%s'", HoodieRecord.COMMIT_TIME_METADATA_FIELD, lastCommitTime));
|
||||
} catch (IOException e) {
|
||||
throw new HoodieException("Error pulling data incrementally from commitTimestamp :" + lastCommitTime, e);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Reads the paths under the a hoodie dataset out as a DataFrame
|
||||
*/
|
||||
public static Dataset<Row> read(JavaSparkContext jsc, String basePath, SQLContext
|
||||
sqlContext,
|
||||
FileSystem
|
||||
fs, String...
|
||||
paths) {
|
||||
List<String> filteredPaths = new ArrayList<>();
|
||||
try {
|
||||
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(fs.getConf(), basePath, true);
|
||||
for (String path : paths) {
|
||||
ReadOptimizedView fileSystemView = new HoodieTableFileSystemView(
|
||||
metaClient, metaClient.getCommitsTimeline().filterCompletedInstants(), fs.globStatus(new Path(path)));
|
||||
List<HoodieDataFile> latestFiles = fileSystemView.getLatestDataFiles().collect(Collectors.toList());
|
||||
for (HoodieDataFile file : latestFiles) {
|
||||
filteredPaths.add(file.getPath());
|
||||
}
|
||||
}
|
||||
return sqlContext.read().parquet(filteredPaths.toArray(new String[filteredPaths.size()]));
|
||||
} catch (Exception e) {
|
||||
throw new HoodieException("Error reading hoodie dataset as a dataframe", e);
|
||||
}
|
||||
}
|
||||
|
||||
public static String writeParquetFile(String basePath,
|
||||
String partitionPath,
|
||||
String filename,
|
||||
List<HoodieRecord> records,
|
||||
Schema schema,
|
||||
BloomFilter filter,
|
||||
boolean createCommitTime) throws IOException {
|
||||
|
||||
if (filter == null) {
|
||||
filter = new BloomFilter(10000, 0.0000001);
|
||||
}
|
||||
HoodieAvroWriteSupport writeSupport = new HoodieAvroWriteSupport(new AvroSchemaConverter().convert(schema), schema,
|
||||
filter);
|
||||
String commitTime = FSUtils.getCommitTime(filename);
|
||||
HoodieParquetConfig config = new HoodieParquetConfig(writeSupport, CompressionCodecName.GZIP,
|
||||
ParquetWriter.DEFAULT_BLOCK_SIZE, ParquetWriter.DEFAULT_PAGE_SIZE, 120 * 1024 * 1024,
|
||||
HoodieTestUtils.getDefaultHadoopConf(),
|
||||
Double.valueOf(HoodieStorageConfig.DEFAULT_STREAM_COMPRESSION_RATIO));
|
||||
HoodieParquetWriter writer = new HoodieParquetWriter(
|
||||
commitTime,
|
||||
new Path(basePath + "/" + partitionPath + "/" + filename),
|
||||
config,
|
||||
schema);
|
||||
int seqId = 1;
|
||||
for (HoodieRecord record : records) {
|
||||
GenericRecord avroRecord = (GenericRecord) record.getData().getInsertValue(schema).get();
|
||||
HoodieAvroUtils.addCommitMetadataToRecord(avroRecord, commitTime, "" + seqId++);
|
||||
HoodieAvroUtils.addHoodieKeyToRecord(avroRecord, record.getRecordKey(), record.getPartitionPath(), filename);
|
||||
writer.writeAvro(record.getRecordKey(), avroRecord);
|
||||
filter.add(record.getRecordKey());
|
||||
}
|
||||
writer.close();
|
||||
|
||||
if (createCommitTime) {
|
||||
HoodieTestUtils.createMetadataFolder(basePath);
|
||||
HoodieTestUtils.createCommitFiles(basePath, commitTime);
|
||||
}
|
||||
return filename;
|
||||
}
|
||||
|
||||
public static String writeParquetFile(String basePath,
|
||||
String partitionPath,
|
||||
List<HoodieRecord> records,
|
||||
Schema schema,
|
||||
BloomFilter filter,
|
||||
boolean createCommitTime) throws IOException, InterruptedException {
|
||||
Thread.sleep(1000);
|
||||
String commitTime = HoodieTestUtils.makeNewCommitTime();
|
||||
String fileId = UUID.randomUUID().toString();
|
||||
String filename = FSUtils.makeDataFileName(commitTime, "1-0-1", fileId);
|
||||
HoodieTestUtils.createCommitFiles(basePath, commitTime);
|
||||
return HoodieClientTestUtils
|
||||
.writeParquetFile(basePath, partitionPath, filename, records, schema, filter, createCommitTime);
|
||||
}
|
||||
}
|
||||
@@ -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.common;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Arrays;
|
||||
import java.util.List;
|
||||
import java.util.stream.Collectors;
|
||||
import org.apache.avro.Schema;
|
||||
import org.apache.avro.generic.GenericRecord;
|
||||
import org.apache.avro.generic.GenericRecordBuilder;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.hive.serde2.ColumnProjectionUtils;
|
||||
import org.apache.hadoop.io.ArrayWritable;
|
||||
import org.apache.hadoop.io.Writable;
|
||||
import org.apache.hadoop.mapred.InputSplit;
|
||||
import org.apache.hadoop.mapred.JobConf;
|
||||
import org.apache.hadoop.mapred.RecordReader;
|
||||
import org.apache.hudi.common.model.HoodieTestUtils;
|
||||
import org.apache.hudi.common.util.HoodieAvroUtils;
|
||||
import org.apache.hudi.hadoop.realtime.HoodieRealtimeInputFormat;
|
||||
|
||||
/**
|
||||
* Utility methods to aid in testing MergeOnRead (workaround for HoodieReadClient for MOR)
|
||||
*/
|
||||
public class HoodieMergeOnReadTestUtils {
|
||||
|
||||
public static List<GenericRecord> getRecordsUsingInputFormat(List<String> inputPaths, String basePath)
|
||||
throws IOException {
|
||||
JobConf jobConf = new JobConf();
|
||||
Schema schema = HoodieAvroUtils.addMetadataFields(Schema.parse(HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA));
|
||||
HoodieRealtimeInputFormat inputFormat = new HoodieRealtimeInputFormat();
|
||||
setPropsForInputFormat(inputFormat, jobConf, schema, basePath);
|
||||
return inputPaths.stream().map(path -> {
|
||||
setInputPath(jobConf, path);
|
||||
List<GenericRecord> records = new ArrayList<>();
|
||||
try {
|
||||
List<InputSplit> splits = Arrays.asList(inputFormat.getSplits(jobConf, 1));
|
||||
RecordReader recordReader = inputFormat.getRecordReader(splits.get(0), jobConf, null);
|
||||
Void key = (Void) recordReader.createKey();
|
||||
ArrayWritable writable = (ArrayWritable) recordReader.createValue();
|
||||
while (recordReader.next(key, writable)) {
|
||||
GenericRecordBuilder newRecord = new GenericRecordBuilder(schema);
|
||||
// writable returns an array with [field1, field2, _hoodie_commit_time,
|
||||
// _hoodie_commit_seqno]
|
||||
Writable[] values = writable.get();
|
||||
schema.getFields().forEach(field -> {
|
||||
newRecord.set(field, values[2]);
|
||||
});
|
||||
records.add(newRecord.build());
|
||||
}
|
||||
} catch (IOException ie) {
|
||||
ie.printStackTrace();
|
||||
}
|
||||
return records;
|
||||
}).reduce((a, b) -> {
|
||||
a.addAll(b);
|
||||
return a;
|
||||
}).get();
|
||||
}
|
||||
|
||||
private static void setPropsForInputFormat(HoodieRealtimeInputFormat inputFormat, JobConf jobConf, Schema schema,
|
||||
String basePath) {
|
||||
List<Schema.Field> fields = schema.getFields();
|
||||
String names = fields.stream().map(f -> f.name().toString()).collect(Collectors.joining(","));
|
||||
String postions = fields.stream().map(f -> String.valueOf(f.pos())).collect(Collectors.joining(","));
|
||||
Configuration conf = HoodieTestUtils.getDefaultHadoopConf();
|
||||
jobConf.set(ColumnProjectionUtils.READ_COLUMN_NAMES_CONF_STR, names);
|
||||
jobConf.set(ColumnProjectionUtils.READ_COLUMN_IDS_CONF_STR, postions);
|
||||
jobConf.set("partition_columns", "datestr");
|
||||
conf.set(ColumnProjectionUtils.READ_COLUMN_NAMES_CONF_STR, names);
|
||||
conf.set(ColumnProjectionUtils.READ_COLUMN_IDS_CONF_STR, postions);
|
||||
conf.set("partition_columns", "datestr");
|
||||
inputFormat.setConf(conf);
|
||||
jobConf.addResource(conf);
|
||||
}
|
||||
|
||||
private static void setInputPath(JobConf jobConf, String inputPath) {
|
||||
jobConf.set("mapreduce.input.fileinputformat.inputdir", inputPath);
|
||||
jobConf.set("mapreduce.input.fileinputformat.inputdir", inputPath);
|
||||
jobConf.set("map.input.dir", inputPath);
|
||||
}
|
||||
}
|
||||
Some files were not shown because too many files have changed in this diff Show More
Reference in New Issue
Block a user