[HUDI-1089] Refactor hudi-client to support multi-engine (#1827)
- This change breaks `hudi-client` into `hudi-client-common` and `hudi-spark-client` modules - Simple usages of Spark using jsc.parallelize() has been redone using EngineContext#map, EngineContext#flatMap etc - Code changes in the PR, break classes into `BaseXYZ` parent classes with no spark dependencies living in `hudi-client-common` - Classes on `hudi-spark-client` are named `SparkXYZ` extending the parent classes with all the Spark dependencies - To simplify/cleanup, HoodieIndex#fetchRecordLocation has been removed and its usages in tests replaced with alternatives Co-authored-by: Vinoth Chandar <vinoth@apache.org>
This commit is contained in:
264
hudi-client/hudi-client-common/pom.xml
Normal file
264
hudi-client/hudi-client-common/pom.xml
Normal file
@@ -0,0 +1,264 @@
|
||||
<?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-client</artifactId>
|
||||
<groupId>org.apache.hudi</groupId>
|
||||
<version>0.6.1-SNAPSHOT</version>
|
||||
</parent>
|
||||
<modelVersion>4.0.0</modelVersion>
|
||||
|
||||
<artifactId>hudi-client-common</artifactId>
|
||||
<version>${parent.version}</version>
|
||||
|
||||
<name>hudi-client-common</name>
|
||||
<packaging>jar</packaging>
|
||||
|
||||
<dependencies>
|
||||
<!-- Scala -->
|
||||
<dependency>
|
||||
<groupId>org.scala-lang</groupId>
|
||||
<artifactId>scala-library</artifactId>
|
||||
<version>${scala.version}</version>
|
||||
</dependency>
|
||||
|
||||
<!-- Hoodie -->
|
||||
<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>
|
||||
|
||||
<!-- Logging -->
|
||||
<dependency>
|
||||
<groupId>log4j</groupId>
|
||||
<artifactId>log4j</artifactId>
|
||||
</dependency>
|
||||
|
||||
<!-- Parquet -->
|
||||
<dependency>
|
||||
<groupId>org.apache.parquet</groupId>
|
||||
<artifactId>parquet-avro</artifactId>
|
||||
</dependency>
|
||||
|
||||
<!-- Dropwizard Metrics -->
|
||||
<dependency>
|
||||
<groupId>io.dropwizard.metrics</groupId>
|
||||
<artifactId>metrics-graphite</artifactId>
|
||||
<exclusions>
|
||||
<exclusion>
|
||||
<groupId>com.rabbitmq</groupId>
|
||||
<artifactId>*</artifactId>
|
||||
</exclusion>
|
||||
</exclusions>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>io.dropwizard.metrics</groupId>
|
||||
<artifactId>metrics-core</artifactId>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>io.dropwizard.metrics</groupId>
|
||||
<artifactId>metrics-jmx</artifactId>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>io.prometheus</groupId>
|
||||
<artifactId>simpleclient</artifactId>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>io.prometheus</groupId>
|
||||
<artifactId>simpleclient_httpserver</artifactId>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>io.prometheus</groupId>
|
||||
<artifactId>simpleclient_dropwizard</artifactId>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>io.prometheus</groupId>
|
||||
<artifactId>simpleclient_pushgateway</artifactId>
|
||||
</dependency>
|
||||
|
||||
<!-- Hoodie - Tests -->
|
||||
<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>
|
||||
|
||||
<!-- Hadoop -->
|
||||
<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>
|
||||
|
||||
<!-- Test -->
|
||||
<dependency>
|
||||
<groupId>org.junit.jupiter</groupId>
|
||||
<artifactId>junit-jupiter-api</artifactId>
|
||||
<scope>test</scope>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.junit.jupiter</groupId>
|
||||
<artifactId>junit-jupiter-engine</artifactId>
|
||||
<scope>test</scope>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.junit.vintage</groupId>
|
||||
<artifactId>junit-vintage-engine</artifactId>
|
||||
<scope>test</scope>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.junit.jupiter</groupId>
|
||||
<artifactId>junit-jupiter-params</artifactId>
|
||||
<scope>test</scope>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.mockito</groupId>
|
||||
<artifactId>mockito-junit-jupiter</artifactId>
|
||||
<scope>test</scope>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.junit.platform</groupId>
|
||||
<artifactId>junit-platform-runner</artifactId>
|
||||
<scope>test</scope>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.junit.platform</groupId>
|
||||
<artifactId>junit-platform-suite-api</artifactId>
|
||||
<scope>test</scope>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.junit.platform</groupId>
|
||||
<artifactId>junit-platform-commons</artifactId>
|
||||
<scope>test</scope>
|
||||
</dependency>
|
||||
</dependencies>
|
||||
|
||||
<build>
|
||||
<plugins>
|
||||
<plugin>
|
||||
<groupId>org.jacoco</groupId>
|
||||
<artifactId>jacoco-maven-plugin</artifactId>
|
||||
</plugin>
|
||||
<plugin>
|
||||
<groupId>net.alchim31.maven</groupId>
|
||||
<artifactId>scala-maven-plugin</artifactId>
|
||||
<executions>
|
||||
<execution>
|
||||
<id>scala-compile-first</id>
|
||||
<phase>process-resources</phase>
|
||||
<goals>
|
||||
<goal>add-source</goal>
|
||||
<goal>compile</goal>
|
||||
</goals>
|
||||
</execution>
|
||||
<execution>
|
||||
<id>scala-test-compile</id>
|
||||
<phase>process-test-resources</phase>
|
||||
<goals>
|
||||
<goal>testCompile</goal>
|
||||
</goals>
|
||||
</execution>
|
||||
</executions>
|
||||
</plugin>
|
||||
<plugin>
|
||||
<groupId>org.apache.maven.plugins</groupId>
|
||||
<artifactId>maven-compiler-plugin</artifactId>
|
||||
<executions>
|
||||
<execution>
|
||||
<phase>compile</phase>
|
||||
<goals>
|
||||
<goal>compile</goal>
|
||||
</goals>
|
||||
</execution>
|
||||
</executions>
|
||||
</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>
|
||||
</project>
|
||||
@@ -0,0 +1,164 @@
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
* contributor license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright ownership.
|
||||
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||
* (the "License"); you may not use this file except in compliance with
|
||||
* the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hudi.async;
|
||||
|
||||
import org.apache.hudi.client.AbstractCompactor;
|
||||
import org.apache.hudi.client.AbstractHoodieWriteClient;
|
||||
import org.apache.hudi.client.common.EngineProperty;
|
||||
import org.apache.hudi.client.common.HoodieEngineContext;
|
||||
import org.apache.hudi.common.table.timeline.HoodieInstant;
|
||||
import org.apache.hudi.common.util.collection.Pair;
|
||||
import org.apache.hudi.exception.HoodieIOException;
|
||||
import org.apache.log4j.LogManager;
|
||||
import org.apache.log4j.Logger;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.concurrent.BlockingQueue;
|
||||
import java.util.concurrent.CompletableFuture;
|
||||
import java.util.concurrent.ExecutorService;
|
||||
import java.util.concurrent.Executors;
|
||||
import java.util.concurrent.LinkedBlockingQueue;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
import java.util.concurrent.locks.Condition;
|
||||
import java.util.concurrent.locks.ReentrantLock;
|
||||
import java.util.stream.IntStream;
|
||||
|
||||
/**
|
||||
* Async Compactor Service that runs in separate thread. Currently, only one compactor is allowed to run at any time.
|
||||
*/
|
||||
public abstract class AsyncCompactService extends HoodieAsyncService {
|
||||
|
||||
private static final long serialVersionUID = 1L;
|
||||
private static final Logger LOG = LogManager.getLogger(AsyncCompactService.class);
|
||||
|
||||
/**
|
||||
* This is the job pool used by async compaction.
|
||||
*/
|
||||
public static final String COMPACT_POOL_NAME = "hoodiecompact";
|
||||
|
||||
private final int maxConcurrentCompaction;
|
||||
private transient AbstractCompactor compactor;
|
||||
private transient HoodieEngineContext context;
|
||||
private transient BlockingQueue<HoodieInstant> pendingCompactions = new LinkedBlockingQueue<>();
|
||||
private transient ReentrantLock queueLock = new ReentrantLock();
|
||||
private transient Condition consumed = queueLock.newCondition();
|
||||
|
||||
public AsyncCompactService(HoodieEngineContext context, AbstractHoodieWriteClient client) {
|
||||
this(context, client, false);
|
||||
}
|
||||
|
||||
public AsyncCompactService(HoodieEngineContext context, AbstractHoodieWriteClient client, boolean runInDaemonMode) {
|
||||
super(runInDaemonMode);
|
||||
this.context = context;
|
||||
this.compactor = createCompactor(client);
|
||||
this.maxConcurrentCompaction = 1;
|
||||
}
|
||||
|
||||
protected abstract AbstractCompactor createCompactor(AbstractHoodieWriteClient client);
|
||||
|
||||
/**
|
||||
* Enqueues new Pending compaction.
|
||||
*/
|
||||
public void enqueuePendingCompaction(HoodieInstant instant) {
|
||||
pendingCompactions.add(instant);
|
||||
}
|
||||
|
||||
/**
|
||||
* Wait till outstanding pending compactions reduces to the passed in value.
|
||||
*
|
||||
* @param numPendingCompactions Maximum pending compactions allowed
|
||||
* @throws InterruptedException
|
||||
*/
|
||||
public void waitTillPendingCompactionsReducesTo(int numPendingCompactions) throws InterruptedException {
|
||||
try {
|
||||
queueLock.lock();
|
||||
while (!isShutdown() && (pendingCompactions.size() > numPendingCompactions)) {
|
||||
consumed.await();
|
||||
}
|
||||
} finally {
|
||||
queueLock.unlock();
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Fetch Next pending compaction if available.
|
||||
*
|
||||
* @return
|
||||
* @throws InterruptedException
|
||||
*/
|
||||
private HoodieInstant fetchNextCompactionInstant() throws InterruptedException {
|
||||
LOG.info("Compactor waiting for next instant for compaction upto 60 seconds");
|
||||
HoodieInstant instant = pendingCompactions.poll(10, TimeUnit.SECONDS);
|
||||
if (instant != null) {
|
||||
try {
|
||||
queueLock.lock();
|
||||
// Signal waiting thread
|
||||
consumed.signal();
|
||||
} finally {
|
||||
queueLock.unlock();
|
||||
}
|
||||
}
|
||||
return instant;
|
||||
}
|
||||
|
||||
/**
|
||||
* Start Compaction Service.
|
||||
*/
|
||||
@Override
|
||||
protected Pair<CompletableFuture, ExecutorService> startService() {
|
||||
ExecutorService executor = Executors.newFixedThreadPool(maxConcurrentCompaction,
|
||||
r -> {
|
||||
Thread t = new Thread(r, "async_compact_thread");
|
||||
t.setDaemon(isRunInDaemonMode());
|
||||
return t;
|
||||
});
|
||||
return Pair.of(CompletableFuture.allOf(IntStream.range(0, maxConcurrentCompaction).mapToObj(i -> CompletableFuture.supplyAsync(() -> {
|
||||
try {
|
||||
// Set Compactor Pool Name for allowing users to prioritize compaction
|
||||
LOG.info("Setting pool name for compaction to " + COMPACT_POOL_NAME);
|
||||
context.setProperty(EngineProperty.COMPACTION_POOL_NAME, COMPACT_POOL_NAME);
|
||||
|
||||
while (!isShutdownRequested()) {
|
||||
final HoodieInstant instant = fetchNextCompactionInstant();
|
||||
|
||||
if (null != instant) {
|
||||
LOG.info("Starting Compaction for instant " + instant);
|
||||
compactor.compact(instant);
|
||||
LOG.info("Finished Compaction for instant " + instant);
|
||||
}
|
||||
}
|
||||
LOG.info("Compactor shutting down properly!!");
|
||||
} catch (InterruptedException ie) {
|
||||
LOG.warn("Compactor executor thread got interrupted exception. Stopping", ie);
|
||||
} catch (IOException e) {
|
||||
LOG.error("Compactor executor failed", e);
|
||||
throw new HoodieIOException(e.getMessage(), e);
|
||||
}
|
||||
return true;
|
||||
}, executor)).toArray(CompletableFuture[]::new)), executor);
|
||||
}
|
||||
|
||||
|
||||
/**
|
||||
* Check whether compactor thread needs to be stopped.
|
||||
* @return
|
||||
*/
|
||||
protected boolean shouldStopCompactor() {
|
||||
return false;
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,168 @@
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hudi.async;
|
||||
|
||||
import org.apache.hudi.common.util.collection.Pair;
|
||||
|
||||
import org.apache.log4j.LogManager;
|
||||
import org.apache.log4j.Logger;
|
||||
|
||||
import java.io.Serializable;
|
||||
import java.util.concurrent.CompletableFuture;
|
||||
import java.util.concurrent.ExecutionException;
|
||||
import java.util.concurrent.ExecutorService;
|
||||
import java.util.concurrent.Executors;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
import java.util.function.Function;
|
||||
|
||||
/**
|
||||
* Base Class for running clean/delta-sync/compaction in separate thread and controlling their life-cycle.
|
||||
*/
|
||||
public abstract class HoodieAsyncService implements Serializable {
|
||||
|
||||
private static final Logger LOG = LogManager.getLogger(HoodieAsyncService.class);
|
||||
|
||||
// Flag to track if the service is started.
|
||||
private boolean started;
|
||||
// Flag indicating shutdown is externally requested
|
||||
private boolean shutdownRequested;
|
||||
// Flag indicating the service is shutdown
|
||||
private volatile boolean shutdown;
|
||||
// Executor Service for running delta-sync/compaction
|
||||
private transient ExecutorService executor;
|
||||
// Future tracking delta-sync/compaction
|
||||
private transient CompletableFuture future;
|
||||
// Run in daemon mode
|
||||
private final boolean runInDaemonMode;
|
||||
|
||||
protected HoodieAsyncService() {
|
||||
this(false);
|
||||
}
|
||||
|
||||
protected HoodieAsyncService(boolean runInDaemonMode) {
|
||||
shutdownRequested = false;
|
||||
this.runInDaemonMode = runInDaemonMode;
|
||||
}
|
||||
|
||||
protected boolean isShutdownRequested() {
|
||||
return shutdownRequested;
|
||||
}
|
||||
|
||||
protected boolean isShutdown() {
|
||||
return shutdown;
|
||||
}
|
||||
|
||||
/**
|
||||
* Wait till the service shutdown. If the service shutdown with exception, it will be thrown
|
||||
*
|
||||
* @throws ExecutionException
|
||||
* @throws InterruptedException
|
||||
*/
|
||||
public void waitForShutdown() throws ExecutionException, InterruptedException {
|
||||
try {
|
||||
future.get();
|
||||
} catch (ExecutionException ex) {
|
||||
LOG.error("Service shutdown with error", ex);
|
||||
throw ex;
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Request shutdown either forcefully or gracefully. Graceful shutdown allows the service to finish up the current
|
||||
* round of work and shutdown. For graceful shutdown, it waits till the service is shutdown
|
||||
*
|
||||
* @param force Forcefully shutdown
|
||||
*/
|
||||
public void shutdown(boolean force) {
|
||||
if (!shutdownRequested || force) {
|
||||
shutdownRequested = true;
|
||||
if (executor != null) {
|
||||
if (force) {
|
||||
executor.shutdownNow();
|
||||
} else {
|
||||
executor.shutdown();
|
||||
try {
|
||||
// Wait for some max time after requesting shutdown
|
||||
executor.awaitTermination(24, TimeUnit.HOURS);
|
||||
} catch (InterruptedException ie) {
|
||||
LOG.error("Interrupted while waiting for shutdown", ie);
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Start the service. Runs the service in a different thread and returns. Also starts a monitor thread to
|
||||
* run-callbacks in case of shutdown
|
||||
*
|
||||
* @param onShutdownCallback
|
||||
*/
|
||||
public void start(Function<Boolean, Boolean> onShutdownCallback) {
|
||||
Pair<CompletableFuture, ExecutorService> res = startService();
|
||||
future = res.getKey();
|
||||
executor = res.getValue();
|
||||
started = true;
|
||||
monitorThreads(onShutdownCallback);
|
||||
}
|
||||
|
||||
/**
|
||||
* Service implementation.
|
||||
*
|
||||
* @return
|
||||
*/
|
||||
protected abstract Pair<CompletableFuture, ExecutorService> startService();
|
||||
|
||||
/**
|
||||
* A monitor thread is started which would trigger a callback if the service is shutdown.
|
||||
*
|
||||
* @param onShutdownCallback
|
||||
*/
|
||||
private void monitorThreads(Function<Boolean, Boolean> onShutdownCallback) {
|
||||
LOG.info("Submitting monitor thread !!");
|
||||
Executors.newSingleThreadExecutor(r -> {
|
||||
Thread t = new Thread(r, "Monitor Thread");
|
||||
t.setDaemon(isRunInDaemonMode());
|
||||
return t;
|
||||
}).submit(() -> {
|
||||
boolean error = false;
|
||||
try {
|
||||
LOG.info("Monitoring thread(s) !!");
|
||||
future.get();
|
||||
} catch (ExecutionException ex) {
|
||||
LOG.error("Monitor noticed one or more threads failed. Requesting graceful shutdown of other threads", ex);
|
||||
error = true;
|
||||
} catch (InterruptedException ie) {
|
||||
LOG.error("Got interrupted Monitoring threads", ie);
|
||||
error = true;
|
||||
} finally {
|
||||
// Mark as shutdown
|
||||
shutdown = true;
|
||||
if (null != onShutdownCallback) {
|
||||
onShutdownCallback.apply(error);
|
||||
}
|
||||
shutdown(false);
|
||||
}
|
||||
});
|
||||
}
|
||||
|
||||
public boolean isRunInDaemonMode() {
|
||||
return runInDaemonMode;
|
||||
}
|
||||
}
|
||||
@@ -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.callback;
|
||||
|
||||
import org.apache.hudi.callback.common.HoodieWriteCommitCallbackMessage;
|
||||
|
||||
/**
|
||||
* A callback interface help to call back when a write commit completes successfully.
|
||||
*/
|
||||
public interface HoodieWriteCommitCallback {
|
||||
|
||||
/**
|
||||
* A callback method the user can implement to provide asynchronous handling of successful write.
|
||||
* This method will be called when a write operation is committed successfully.
|
||||
*
|
||||
* @param callbackMessage Callback msg, which will be sent to external system.
|
||||
*/
|
||||
void call(HoodieWriteCommitCallbackMessage callbackMessage);
|
||||
|
||||
}
|
||||
@@ -0,0 +1,108 @@
|
||||
/*
|
||||
* 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.callback.client.http;
|
||||
|
||||
import org.apache.http.HttpHeaders;
|
||||
import org.apache.http.client.config.RequestConfig;
|
||||
import org.apache.http.client.methods.CloseableHttpResponse;
|
||||
import org.apache.http.client.methods.HttpPost;
|
||||
import org.apache.http.entity.ContentType;
|
||||
import org.apache.http.entity.StringEntity;
|
||||
import org.apache.http.impl.client.CloseableHttpClient;
|
||||
import org.apache.http.impl.client.HttpClientBuilder;
|
||||
import org.apache.hudi.config.HoodieWriteCommitCallbackConfig;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
import org.apache.log4j.LogManager;
|
||||
import org.apache.log4j.Logger;
|
||||
|
||||
import java.io.Closeable;
|
||||
import java.io.IOException;
|
||||
import java.util.Properties;
|
||||
|
||||
/**
|
||||
* Write commit callback http client.
|
||||
*/
|
||||
public class HoodieWriteCommitHttpCallbackClient implements Closeable {
|
||||
|
||||
private static final Logger LOG = LogManager.getLogger(HoodieWriteCommitHttpCallbackClient.class);
|
||||
|
||||
public static final String HEADER_KEY_API_KEY = "HUDI-CALLBACK-KEY";
|
||||
|
||||
private final String apiKey;
|
||||
private final String url;
|
||||
private final CloseableHttpClient client;
|
||||
private Properties props;
|
||||
|
||||
public HoodieWriteCommitHttpCallbackClient(HoodieWriteConfig config) {
|
||||
this.props = config.getProps();
|
||||
this.apiKey = getApiKey();
|
||||
this.url = getUrl();
|
||||
this.client = getClient();
|
||||
}
|
||||
|
||||
public HoodieWriteCommitHttpCallbackClient(String apiKey, String url, CloseableHttpClient client) {
|
||||
this.apiKey = apiKey;
|
||||
this.url = url;
|
||||
this.client = client;
|
||||
}
|
||||
|
||||
public void send(String callbackMsg) {
|
||||
HttpPost request = new HttpPost(url);
|
||||
request.setHeader(HEADER_KEY_API_KEY, apiKey);
|
||||
request.setHeader(HttpHeaders.CONTENT_TYPE, ContentType.APPLICATION_JSON.toString());
|
||||
request.setEntity(new StringEntity(callbackMsg, ContentType.APPLICATION_JSON));
|
||||
try (CloseableHttpResponse response = client.execute(request)) {
|
||||
int statusCode = response.getStatusLine().getStatusCode();
|
||||
if (statusCode >= 300) {
|
||||
LOG.warn(String.format("Failed to send callback message. Response was %s", response));
|
||||
} else {
|
||||
LOG.info(String.format("Sent Callback data %s to %s successfully !", callbackMsg, url));
|
||||
}
|
||||
} catch (IOException e) {
|
||||
LOG.warn("Failed to send callback.", e);
|
||||
}
|
||||
}
|
||||
|
||||
private String getApiKey() {
|
||||
return props.getProperty(HoodieWriteCommitCallbackConfig.CALLBACK_HTTP_API_KEY);
|
||||
}
|
||||
|
||||
private String getUrl() {
|
||||
return props.getProperty(HoodieWriteCommitCallbackConfig.CALLBACK_HTTP_URL_PROP);
|
||||
}
|
||||
|
||||
private CloseableHttpClient getClient() {
|
||||
int timeoutSeconds = getHttpTimeoutSeconds() * 1000;
|
||||
return HttpClientBuilder.create()
|
||||
.setDefaultRequestConfig(RequestConfig.custom()
|
||||
.setConnectTimeout(timeoutSeconds)
|
||||
.setConnectionRequestTimeout(timeoutSeconds)
|
||||
.setSocketTimeout(timeoutSeconds).build())
|
||||
.build();
|
||||
}
|
||||
|
||||
private Integer getHttpTimeoutSeconds() {
|
||||
return Integer.parseInt(props.getProperty(HoodieWriteCommitCallbackConfig.CALLBACK_HTTP_TIMEOUT_SECONDS));
|
||||
}
|
||||
|
||||
@Override
|
||||
public void close() throws IOException {
|
||||
client.close();
|
||||
}
|
||||
}
|
||||
@@ -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.callback.common;
|
||||
|
||||
import java.io.Serializable;
|
||||
|
||||
/**
|
||||
* Base callback message, which contains commitTime and tableName only for now.
|
||||
*/
|
||||
public class HoodieWriteCommitCallbackMessage implements Serializable {
|
||||
|
||||
private static final long serialVersionUID = -3033643980627719561L;
|
||||
|
||||
/**
|
||||
* CommitTime for one batch write, this is required.
|
||||
*/
|
||||
private String commitTime;
|
||||
|
||||
/**
|
||||
* Table name this batch commit to.
|
||||
*/
|
||||
private String tableName;
|
||||
|
||||
/**
|
||||
* BathPath the table located.
|
||||
*/
|
||||
private String basePath;
|
||||
|
||||
public HoodieWriteCommitCallbackMessage() {
|
||||
}
|
||||
|
||||
public HoodieWriteCommitCallbackMessage(String commitTime, String tableName, String basePath) {
|
||||
this.commitTime = commitTime;
|
||||
this.tableName = tableName;
|
||||
this.basePath = basePath;
|
||||
}
|
||||
|
||||
public String getCommitTime() {
|
||||
return commitTime;
|
||||
}
|
||||
|
||||
public void setCommitTime(String commitTime) {
|
||||
this.commitTime = commitTime;
|
||||
}
|
||||
|
||||
public String getTableName() {
|
||||
return tableName;
|
||||
}
|
||||
|
||||
public void setTableName(String tableName) {
|
||||
this.tableName = tableName;
|
||||
}
|
||||
|
||||
public String getBasePath() {
|
||||
return basePath;
|
||||
}
|
||||
|
||||
public void setBasePath(String basePath) {
|
||||
this.basePath = basePath;
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,50 @@
|
||||
/*
|
||||
* 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.callback.impl;
|
||||
|
||||
import org.apache.hudi.callback.HoodieWriteCommitCallback;
|
||||
import org.apache.hudi.callback.client.http.HoodieWriteCommitHttpCallbackClient;
|
||||
import org.apache.hudi.callback.common.HoodieWriteCommitCallbackMessage;
|
||||
import org.apache.hudi.callback.util.HoodieWriteCommitCallbackUtil;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
|
||||
import org.apache.log4j.LogManager;
|
||||
import org.apache.log4j.Logger;
|
||||
|
||||
/**
|
||||
* A http implementation of {@link HoodieWriteCommitCallback}.
|
||||
*/
|
||||
public class HoodieWriteCommitHttpCallback implements HoodieWriteCommitCallback {
|
||||
|
||||
private static final Logger LOG = LogManager.getLogger(HoodieWriteCommitHttpCallback.class);
|
||||
|
||||
private final HoodieWriteCommitHttpCallbackClient client;
|
||||
|
||||
public HoodieWriteCommitHttpCallback(HoodieWriteConfig config) {
|
||||
this.client = new HoodieWriteCommitHttpCallbackClient(config);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void call(HoodieWriteCommitCallbackMessage callbackMessage) {
|
||||
// convert to json
|
||||
String callbackMsg = HoodieWriteCommitCallbackUtil.convertToJsonString(callbackMessage);
|
||||
LOG.info("Try to send callbackMsg, msg = " + callbackMsg);
|
||||
client.send(callbackMsg);
|
||||
}
|
||||
|
||||
}
|
||||
@@ -0,0 +1,46 @@
|
||||
/*
|
||||
* 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.callback.util;
|
||||
|
||||
import org.apache.hudi.callback.HoodieWriteCommitCallback;
|
||||
import org.apache.hudi.common.util.ReflectionUtils;
|
||||
import org.apache.hudi.common.util.StringUtils;
|
||||
import org.apache.hudi.config.HoodieWriteCommitCallbackConfig;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
import org.apache.hudi.exception.HoodieCommitCallbackException;
|
||||
|
||||
/**
|
||||
* Factory help to create {@link HoodieWriteCommitCallback}.
|
||||
*/
|
||||
public class HoodieCommitCallbackFactory {
|
||||
public static HoodieWriteCommitCallback create(HoodieWriteConfig config) {
|
||||
String callbackClass = config.getCallbackClass();
|
||||
if (!StringUtils.isNullOrEmpty(callbackClass)) {
|
||||
Object instance = ReflectionUtils.loadClass(callbackClass, config);
|
||||
if (!(instance instanceof HoodieWriteCommitCallback)) {
|
||||
throw new HoodieCommitCallbackException(callbackClass + " is not a subclass of "
|
||||
+ HoodieWriteCommitCallback.class.getSimpleName());
|
||||
}
|
||||
return (HoodieWriteCommitCallback) instance;
|
||||
} else {
|
||||
throw new HoodieCommitCallbackException(String.format("The value of the config option %s can not be null or "
|
||||
+ "empty", HoodieWriteCommitCallbackConfig.CALLBACK_CLASS_PROP));
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
@@ -0,0 +1,44 @@
|
||||
/*
|
||||
* 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.callback.util;
|
||||
|
||||
import org.apache.hudi.exception.HoodieCommitCallbackException;
|
||||
|
||||
import org.codehaus.jackson.map.ObjectMapper;
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
/**
|
||||
* Util helps to prepare callback message.
|
||||
*/
|
||||
public class HoodieWriteCommitCallbackUtil {
|
||||
|
||||
private static ObjectMapper mapper = new ObjectMapper();
|
||||
|
||||
/**
|
||||
* Convert data to json string format.
|
||||
*/
|
||||
public static String convertToJsonString(Object obj) {
|
||||
try {
|
||||
return mapper.writeValueAsString(obj);
|
||||
} catch (IOException e) {
|
||||
throw new HoodieCommitCallbackException("Callback service convert data to json failed", e);
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
@@ -0,0 +1,41 @@
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hudi.client;
|
||||
|
||||
import org.apache.hudi.common.model.HoodieRecordPayload;
|
||||
import org.apache.hudi.common.table.timeline.HoodieInstant;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.io.Serializable;
|
||||
|
||||
/**
|
||||
* Run one round of compaction.
|
||||
*/
|
||||
public abstract class AbstractCompactor<T extends HoodieRecordPayload, I, K, O> implements Serializable {
|
||||
|
||||
private static final long serialVersionUID = 1L;
|
||||
|
||||
protected transient AbstractHoodieWriteClient<T, I, K, O> compactionClient;
|
||||
|
||||
public AbstractCompactor(AbstractHoodieWriteClient<T, I, K, O> compactionClient) {
|
||||
this.compactionClient = compactionClient;
|
||||
}
|
||||
|
||||
public abstract void compact(HoodieInstant instant) throws IOException;
|
||||
}
|
||||
@@ -0,0 +1,132 @@
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hudi.client;
|
||||
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
|
||||
import org.apache.hudi.client.common.EngineProperty;
|
||||
import org.apache.hudi.client.embedded.EmbeddedTimelineService;
|
||||
import org.apache.hudi.client.common.HoodieEngineContext;
|
||||
import org.apache.hudi.common.fs.FSUtils;
|
||||
import org.apache.hudi.common.table.HoodieTableMetaClient;
|
||||
import org.apache.hudi.common.table.timeline.versioning.TimelineLayoutVersion;
|
||||
import org.apache.hudi.common.util.Option;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.log4j.LogManager;
|
||||
import org.apache.log4j.Logger;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.io.Serializable;
|
||||
|
||||
/**
|
||||
* 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, AutoCloseable {
|
||||
|
||||
private static final Logger LOG = LogManager.getLogger(AbstractHoodieClient.class);
|
||||
|
||||
protected final transient FileSystem fs;
|
||||
protected final transient HoodieEngineContext context;
|
||||
protected final transient Configuration hadoopConf;
|
||||
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(HoodieEngineContext context, HoodieWriteConfig clientConfig) {
|
||||
this(context, clientConfig, Option.empty());
|
||||
}
|
||||
|
||||
protected AbstractHoodieClient(HoodieEngineContext context, HoodieWriteConfig clientConfig,
|
||||
Option<EmbeddedTimelineService> timelineServer) {
|
||||
this.hadoopConf = context.getHadoopConf().get();
|
||||
this.fs = FSUtils.getFs(clientConfig.getBasePath(), hadoopConf);
|
||||
this.context = context;
|
||||
this.basePath = clientConfig.getBasePath();
|
||||
this.config = clientConfig;
|
||||
this.timelineServer = timelineServer;
|
||||
shouldStopTimelineServer = !timelineServer.isPresent();
|
||||
startEmbeddedServerView();
|
||||
}
|
||||
|
||||
/**
|
||||
* Releases any resources used by the client.
|
||||
*/
|
||||
@Override
|
||||
public void close() {
|
||||
stopEmbeddedServerView(true);
|
||||
}
|
||||
|
||||
private synchronized void stopEmbeddedServerView(boolean resetViewStorageConfig) {
|
||||
if (timelineServer.isPresent() && shouldStopTimelineServer) {
|
||||
// Stop only if owner
|
||||
LOG.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
|
||||
LOG.info("Starting Timeline service !!");
|
||||
Option<String> hostAddr = context.getProperty(EngineProperty.EMBEDDED_SERVER_HOST);
|
||||
timelineServer = Option.of(new EmbeddedTimelineService(context, hostAddr.orElse(null),
|
||||
config.getClientSpecifiedViewStorageConfig()));
|
||||
try {
|
||||
timelineServer.get().startServer();
|
||||
// Allow executor to find this newly instantiated timeline service
|
||||
config.setViewStorageConfig(timelineServer.get().getRemoteFileSystemViewConfig());
|
||||
} catch (IOException e) {
|
||||
LOG.warn("Unable to start timeline service. Proceeding as if embedded server is disabled", e);
|
||||
stopEmbeddedServerView(false);
|
||||
}
|
||||
} else {
|
||||
LOG.info("Timeline Server already running. Not restarting the service");
|
||||
}
|
||||
} else {
|
||||
LOG.info("Embedded Timeline Server is disabled. Not starting timeline service");
|
||||
}
|
||||
}
|
||||
|
||||
public HoodieWriteConfig getConfig() {
|
||||
return config;
|
||||
}
|
||||
|
||||
protected HoodieTableMetaClient createMetaClient(boolean loadActiveTimelineOnLoad) {
|
||||
return new HoodieTableMetaClient(hadoopConf, config.getBasePath(), loadActiveTimelineOnLoad,
|
||||
config.getConsistencyGuardConfig(),
|
||||
Option.of(new TimelineLayoutVersion(config.getTimelineLayoutVersion())));
|
||||
}
|
||||
}
|
||||
@@ -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.client;
|
||||
|
||||
import com.codahale.metrics.Timer;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hudi.avro.model.HoodieCleanMetadata;
|
||||
import org.apache.hudi.avro.model.HoodieCompactionPlan;
|
||||
import org.apache.hudi.avro.model.HoodieRestoreMetadata;
|
||||
import org.apache.hudi.avro.model.HoodieRollbackMetadata;
|
||||
import org.apache.hudi.callback.HoodieWriteCommitCallback;
|
||||
import org.apache.hudi.callback.common.HoodieWriteCommitCallbackMessage;
|
||||
import org.apache.hudi.callback.util.HoodieCommitCallbackFactory;
|
||||
import org.apache.hudi.client.embedded.EmbeddedTimelineService;
|
||||
import org.apache.hudi.client.common.HoodieEngineContext;
|
||||
import org.apache.hudi.common.model.HoodieCommitMetadata;
|
||||
import org.apache.hudi.common.model.HoodieKey;
|
||||
import org.apache.hudi.common.model.HoodieRecordPayload;
|
||||
import org.apache.hudi.common.model.HoodieWriteStat;
|
||||
import org.apache.hudi.common.model.WriteOperationType;
|
||||
import org.apache.hudi.common.table.HoodieTableMetaClient;
|
||||
import org.apache.hudi.common.table.timeline.HoodieActiveTimeline;
|
||||
import org.apache.hudi.common.table.timeline.HoodieInstant;
|
||||
import org.apache.hudi.common.table.timeline.HoodieTimeline;
|
||||
import org.apache.hudi.common.util.CommitUtils;
|
||||
import org.apache.hudi.common.util.Option;
|
||||
import org.apache.hudi.common.util.ValidationUtils;
|
||||
import org.apache.hudi.config.HoodieCompactionConfig;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
import org.apache.hudi.exception.HoodieCommitException;
|
||||
import org.apache.hudi.exception.HoodieIOException;
|
||||
import org.apache.hudi.exception.HoodieRestoreException;
|
||||
import org.apache.hudi.exception.HoodieRollbackException;
|
||||
import org.apache.hudi.exception.HoodieSavepointException;
|
||||
import org.apache.hudi.index.HoodieIndex;
|
||||
import org.apache.hudi.metrics.HoodieMetrics;
|
||||
import org.apache.hudi.table.BulkInsertPartitioner;
|
||||
import org.apache.hudi.table.HoodieTable;
|
||||
import org.apache.hudi.table.HoodieTimelineArchiveLog;
|
||||
import org.apache.hudi.table.MarkerFiles;
|
||||
import org.apache.hudi.table.action.HoodieWriteMetadata;
|
||||
import org.apache.hudi.table.action.savepoint.SavepointHelpers;
|
||||
import org.apache.log4j.LogManager;
|
||||
import org.apache.log4j.Logger;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.nio.charset.StandardCharsets;
|
||||
import java.text.ParseException;
|
||||
import java.util.Collection;
|
||||
import java.util.Collections;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.stream.Collectors;
|
||||
|
||||
/**
|
||||
* Abstract Write Client providing functionality for performing commit, index updates and rollback
|
||||
* Reused for regular write operations like upsert/insert/bulk-insert.. as well as bootstrap
|
||||
*
|
||||
* @param <T> Sub type of HoodieRecordPayload
|
||||
* @param <I> Type of inputs
|
||||
* @param <K> Type of keys
|
||||
* @param <O> Type of outputs
|
||||
*/
|
||||
public abstract class AbstractHoodieWriteClient<T extends HoodieRecordPayload, I, K, O> extends AbstractHoodieClient {
|
||||
|
||||
protected static final String LOOKUP_STR = "lookup";
|
||||
private static final long serialVersionUID = 1L;
|
||||
private static final Logger LOG = LogManager.getLogger(AbstractHoodieWriteClient.class);
|
||||
|
||||
protected final transient HoodieMetrics metrics;
|
||||
private final transient HoodieIndex<T, I, K, O> index;
|
||||
|
||||
protected transient Timer.Context writeTimer = null;
|
||||
protected transient Timer.Context compactionTimer;
|
||||
|
||||
private transient WriteOperationType operationType;
|
||||
private transient HoodieWriteCommitCallback commitCallback;
|
||||
protected final boolean rollbackPending;
|
||||
protected transient AsyncCleanerService asyncCleanerService;
|
||||
|
||||
/**
|
||||
* Create a write client, without cleaning up failed/inflight commits.
|
||||
*
|
||||
* @param context HoodieEngineContext
|
||||
* @param clientConfig instance of HoodieWriteConfig
|
||||
*/
|
||||
public AbstractHoodieWriteClient(HoodieEngineContext context, HoodieWriteConfig clientConfig) {
|
||||
this(context, clientConfig, false);
|
||||
}
|
||||
|
||||
/**
|
||||
* Create a write client, with new hudi index.
|
||||
*
|
||||
* @param context HoodieEngineContext
|
||||
* @param writeConfig instance of HoodieWriteConfig
|
||||
* @param rollbackPending whether need to cleanup pending commits
|
||||
*/
|
||||
public AbstractHoodieWriteClient(HoodieEngineContext context, HoodieWriteConfig writeConfig, boolean rollbackPending) {
|
||||
this(context, writeConfig, rollbackPending, Option.empty());
|
||||
}
|
||||
|
||||
/**
|
||||
* Create a write client, allows to specify all parameters.
|
||||
*
|
||||
* @param context HoodieEngineContext
|
||||
* @param writeConfig instance of HoodieWriteConfig
|
||||
* @param rollbackPending whether need to cleanup pending commits
|
||||
* @param timelineService Timeline Service that runs as part of write client.
|
||||
*/
|
||||
public AbstractHoodieWriteClient(HoodieEngineContext context, HoodieWriteConfig writeConfig, boolean rollbackPending,
|
||||
Option<EmbeddedTimelineService> timelineService) {
|
||||
super(context, writeConfig, timelineService);
|
||||
this.metrics = new HoodieMetrics(config, config.getTableName());
|
||||
this.rollbackPending = rollbackPending;
|
||||
this.index = createIndex(writeConfig);
|
||||
}
|
||||
|
||||
protected abstract HoodieIndex<T, I, K, O> createIndex(HoodieWriteConfig writeConfig);
|
||||
|
||||
public void setOperationType(WriteOperationType operationType) {
|
||||
this.operationType = operationType;
|
||||
}
|
||||
|
||||
public WriteOperationType getOperationType() {
|
||||
return this.operationType;
|
||||
}
|
||||
|
||||
/**
|
||||
* Commit changes performed at the given instantTime marker.
|
||||
*/
|
||||
public boolean commit(String instantTime, O writeStatuses) {
|
||||
return commit(instantTime, writeStatuses, Option.empty());
|
||||
}
|
||||
|
||||
/**
|
||||
*
|
||||
* Commit changes performed at the given instantTime marker.
|
||||
*/
|
||||
public boolean commit(String instantTime, O writeStatuses, Option<Map<String, String>> extraMetadata) {
|
||||
HoodieTableMetaClient metaClient = createMetaClient(false);
|
||||
String actionType = metaClient.getCommitActionType();
|
||||
return commit(instantTime, writeStatuses, extraMetadata, actionType, Collections.emptyMap());
|
||||
}
|
||||
|
||||
public abstract boolean commit(String instantTime, O writeStatuses, Option<Map<String, String>> extraMetadata,
|
||||
String commitActionType, Map<String, List<String>> partitionToReplacedFileIds);
|
||||
|
||||
public boolean commitStats(String instantTime, List<HoodieWriteStat> stats, Option<Map<String, String>> extraMetadata,
|
||||
String commitActionType) {
|
||||
return commitStats(instantTime, stats, extraMetadata, commitActionType, Collections.emptyMap());
|
||||
}
|
||||
|
||||
public boolean commitStats(String instantTime, List<HoodieWriteStat> stats, Option<Map<String, String>> extraMetadata,
|
||||
String commitActionType, Map<String, List<String>> partitionToReplaceFileIds) {
|
||||
LOG.info("Committing " + instantTime + " action " + commitActionType);
|
||||
// Create a Hoodie table which encapsulated the commits and files visible
|
||||
HoodieTable table = createTable(config, hadoopConf);
|
||||
|
||||
HoodieActiveTimeline activeTimeline = table.getActiveTimeline();
|
||||
HoodieCommitMetadata metadata = CommitUtils.buildMetadata(stats, partitionToReplaceFileIds, extraMetadata, operationType, config.getSchema(), commitActionType);
|
||||
// Finalize write
|
||||
finalizeWrite(table, instantTime, stats);
|
||||
|
||||
try {
|
||||
activeTimeline.saveAsComplete(new HoodieInstant(true, commitActionType, instantTime),
|
||||
Option.of(metadata.toJsonString().getBytes(StandardCharsets.UTF_8)));
|
||||
postCommit(table, metadata, instantTime, extraMetadata);
|
||||
emitCommitMetrics(instantTime, metadata, commitActionType);
|
||||
LOG.info("Committed " + instantTime);
|
||||
} catch (IOException e) {
|
||||
throw new HoodieCommitException("Failed to complete commit " + config.getBasePath() + " at time " + instantTime,
|
||||
e);
|
||||
}
|
||||
|
||||
// callback if needed.
|
||||
if (config.writeCommitCallbackOn()) {
|
||||
if (null == commitCallback) {
|
||||
commitCallback = HoodieCommitCallbackFactory.create(config);
|
||||
}
|
||||
commitCallback.call(new HoodieWriteCommitCallbackMessage(instantTime, config.getTableName(), config.getBasePath()));
|
||||
}
|
||||
return true;
|
||||
}
|
||||
|
||||
protected abstract HoodieTable<T, I, K, O> createTable(HoodieWriteConfig config, Configuration hadoopConf);
|
||||
|
||||
void emitCommitMetrics(String instantTime, HoodieCommitMetadata metadata, String actionType) {
|
||||
try {
|
||||
|
||||
if (writeTimer != null) {
|
||||
long durationInMs = metrics.getDurationInMs(writeTimer.stop());
|
||||
metrics.updateCommitMetrics(HoodieActiveTimeline.COMMIT_FORMATTER.parse(instantTime).getTime(), durationInMs,
|
||||
metadata, actionType);
|
||||
writeTimer = null;
|
||||
}
|
||||
} catch (ParseException e) {
|
||||
throw new HoodieCommitException("Failed to complete commit " + config.getBasePath() + " at time " + instantTime
|
||||
+ "Instant time is not of valid format", e);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Filter out HoodieRecords that already exists in the output folder. This is useful in deduplication.
|
||||
*
|
||||
* @param hoodieRecords Input Hoodie records.
|
||||
* @return A subset of hoodieRecords, with existing records filtered out.
|
||||
*/
|
||||
public abstract I filterExists(I hoodieRecords);
|
||||
|
||||
/**
|
||||
* Main API to run bootstrap to hudi.
|
||||
*/
|
||||
public void bootstrap(Option<Map<String, String>> extraMetadata) {
|
||||
if (rollbackPending) {
|
||||
rollBackInflightBootstrap();
|
||||
}
|
||||
HoodieTable<T, I, K, O> table = getTableAndInitCtx(WriteOperationType.UPSERT, HoodieTimeline.METADATA_BOOTSTRAP_INSTANT_TS);
|
||||
table.bootstrap(context, extraMetadata);
|
||||
}
|
||||
|
||||
/**
|
||||
* Main API to rollback pending bootstrap.
|
||||
*/
|
||||
protected void rollBackInflightBootstrap() {
|
||||
LOG.info("Rolling back pending bootstrap if present");
|
||||
HoodieTable<T, I, K, O> table = createTable(config, hadoopConf);
|
||||
HoodieTimeline inflightTimeline = table.getMetaClient().getCommitsTimeline().filterPendingExcludingCompaction();
|
||||
Option<String> instant = Option.fromJavaOptional(
|
||||
inflightTimeline.getReverseOrderedInstants().map(HoodieInstant::getTimestamp).findFirst());
|
||||
if (instant.isPresent() && HoodieTimeline.compareTimestamps(instant.get(), HoodieTimeline.LESSER_THAN_OR_EQUALS,
|
||||
HoodieTimeline.FULL_BOOTSTRAP_INSTANT_TS)) {
|
||||
LOG.info("Found pending bootstrap instants. Rolling them back");
|
||||
table.rollbackBootstrap(context, HoodieActiveTimeline.createNewInstantTime());
|
||||
LOG.info("Finished rolling back pending bootstrap");
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
/**
|
||||
* Upsert a batch of new records into Hoodie table at the supplied instantTime.
|
||||
*
|
||||
* @param records hoodieRecords to upsert
|
||||
* @param instantTime Instant time of the commit
|
||||
* @return WriteStatus to inspect errors and counts
|
||||
*/
|
||||
public abstract O upsert(I records, final String instantTime);
|
||||
|
||||
/**
|
||||
* Upserts the given prepared records into the Hoodie table, at the supplied instantTime.
|
||||
* <p>
|
||||
* This implementation requires that the input records are already tagged, and de-duped if needed.
|
||||
*
|
||||
* @param preppedRecords Prepared HoodieRecords to upsert
|
||||
* @param instantTime Instant time of the commit
|
||||
* @return JavaRDD[WriteStatus] - RDD of WriteStatus to inspect errors and counts
|
||||
*/
|
||||
public abstract O upsertPreppedRecords(I preppedRecords, final String instantTime);
|
||||
|
||||
/**
|
||||
* Inserts the given HoodieRecords, into the table. This API is intended to be used for normal writes.
|
||||
* <p>
|
||||
* This implementation skips the index check and is able to leverage benefits such as small file handling/blocking
|
||||
* alignment, as with upsert(), by profiling the workload
|
||||
*
|
||||
* @param records HoodieRecords to insert
|
||||
* @param instantTime Instant time of the commit
|
||||
* @return JavaRDD[WriteStatus] - RDD of WriteStatus to inspect errors and counts
|
||||
*/
|
||||
public abstract O insert(I records, final String instantTime);
|
||||
|
||||
/**
|
||||
* Inserts the given prepared records into the Hoodie table, at the supplied instantTime.
|
||||
* <p>
|
||||
* This implementation skips the index check, skips de-duping and is able to leverage benefits such as small file
|
||||
* handling/blocking alignment, as with insert(), by profiling the workload. The prepared HoodieRecords should be
|
||||
* de-duped if needed.
|
||||
*
|
||||
* @param preppedRecords HoodieRecords to insert
|
||||
* @param instantTime Instant time of the commit
|
||||
* @return JavaRDD[WriteStatus] - RDD of WriteStatus to inspect errors and counts
|
||||
*/
|
||||
public abstract O insertPreppedRecords(I preppedRecords, final String instantTime);
|
||||
|
||||
/**
|
||||
* Loads the given HoodieRecords, as inserts into the table. This is suitable for doing big bulk loads into a Hoodie
|
||||
* table for the very first time (e.g: converting an existing table to Hoodie).
|
||||
* <p>
|
||||
* This implementation uses sortBy (which does range partitioning based on reservoir sampling) and attempts to control
|
||||
* the numbers of files with less memory compared to the {@link AbstractHoodieWriteClient#insert(I, String)}
|
||||
*
|
||||
* @param records HoodieRecords to insert
|
||||
* @param instantTime Instant time of the commit
|
||||
* @return JavaRDD[WriteStatus] - RDD of WriteStatus to inspect errors and counts
|
||||
*/
|
||||
public abstract O bulkInsert(I records, final String instantTime);
|
||||
|
||||
/**
|
||||
* Loads the given HoodieRecords, as inserts into the table. This is suitable for doing big bulk loads into a Hoodie
|
||||
* table for the very first time (e.g: converting an existing table to Hoodie).
|
||||
* <p>
|
||||
* This implementation uses sortBy (which does range partitioning based on reservoir sampling) and attempts to control
|
||||
* the numbers of files with less memory compared to the {@link AbstractHoodieWriteClient#insert(I, String)}. Optionally
|
||||
* it allows users to specify their own partitioner. If specified then it will be used for repartitioning records. See
|
||||
* {@link BulkInsertPartitioner}.
|
||||
*
|
||||
* @param records HoodieRecords to insert
|
||||
* @param instantTime Instant time of the commit
|
||||
* @param userDefinedBulkInsertPartitioner If specified then it will be used to partition input records before they are inserted
|
||||
* into hoodie.
|
||||
* @return JavaRDD[WriteStatus] - RDD of WriteStatus to inspect errors and counts
|
||||
*/
|
||||
public abstract O bulkInsert(I records, final String instantTime,
|
||||
Option<BulkInsertPartitioner<I>> userDefinedBulkInsertPartitioner);
|
||||
|
||||
|
||||
/**
|
||||
* Loads the given HoodieRecords, as inserts into the table. This is suitable for doing big bulk loads into a Hoodie
|
||||
* table for the very first time (e.g: converting an existing table to Hoodie). The input records should contain no
|
||||
* duplicates if needed.
|
||||
* <p>
|
||||
* This implementation uses sortBy (which does range partitioning based on reservoir sampling) and attempts to control
|
||||
* the numbers of files with less memory compared to the {@link AbstractHoodieWriteClient#insert(I, String)}. Optionally
|
||||
* it allows users to specify their own partitioner. If specified then it will be used for repartitioning records. See
|
||||
* {@link BulkInsertPartitioner}.
|
||||
*
|
||||
* @param preppedRecords HoodieRecords to insert
|
||||
* @param instantTime Instant time of the commit
|
||||
* @param bulkInsertPartitioner If specified then it will be used to partition input records before they are inserted
|
||||
* into hoodie.
|
||||
* @return JavaRDD[WriteStatus] - RDD of WriteStatus to inspect errors and counts
|
||||
*/
|
||||
public abstract O bulkInsertPreppedRecords(I preppedRecords, final String instantTime,
|
||||
Option<BulkInsertPartitioner<I>> bulkInsertPartitioner);
|
||||
|
||||
/**
|
||||
* Deletes a list of {@link HoodieKey}s from the Hoodie table, at the supplied instantTime {@link HoodieKey}s will be
|
||||
* de-duped and non existent keys will be removed before deleting.
|
||||
*
|
||||
* @param keys {@link List} of {@link HoodieKey}s to be deleted
|
||||
* @param instantTime Commit time handle
|
||||
* @return JavaRDD[WriteStatus] - RDD of WriteStatus to inspect errors and counts
|
||||
*/
|
||||
public abstract O delete(K keys, final String instantTime);
|
||||
|
||||
/**
|
||||
* Common method containing steps to be performed after write (upsert/insert/..) operations including auto-commit.
|
||||
* @param result Commit Action Result
|
||||
* @param instantTime Instant Time
|
||||
* @param hoodieTable Hoodie Table
|
||||
* @return Write Status
|
||||
*/
|
||||
protected abstract O postWrite(HoodieWriteMetadata<O> result, String instantTime, HoodieTable<T, I, K, O> hoodieTable);
|
||||
|
||||
/**
|
||||
* Post Commit Hook. Derived classes use this method to perform post-commit processing
|
||||
*
|
||||
* @param table table to commit on
|
||||
* @param metadata Commit Metadata corresponding to committed instant
|
||||
* @param instantTime Instant Time
|
||||
* @param extraMetadata Additional Metadata passed by user
|
||||
*/
|
||||
protected void postCommit(HoodieTable<T, I, K, O> table, HoodieCommitMetadata metadata, String instantTime, Option<Map<String, String>> extraMetadata) {
|
||||
try {
|
||||
|
||||
// Delete the marker directory for the instant.
|
||||
new MarkerFiles(table, instantTime).quietDeleteMarkerDir(context, config.getMarkersDeleteParallelism());
|
||||
|
||||
// Do an inline compaction if enabled
|
||||
if (config.isInlineCompaction()) {
|
||||
runAnyPendingCompactions(table);
|
||||
metadata.addMetadata(HoodieCompactionConfig.INLINE_COMPACT_PROP, "true");
|
||||
inlineCompact(extraMetadata);
|
||||
} else {
|
||||
metadata.addMetadata(HoodieCompactionConfig.INLINE_COMPACT_PROP, "false");
|
||||
}
|
||||
// We cannot have unbounded commit files. Archive commits if we have to archive
|
||||
HoodieTimelineArchiveLog archiveLog = new HoodieTimelineArchiveLog(config, table);
|
||||
archiveLog.archiveIfRequired(context);
|
||||
autoCleanOnCommit(instantTime);
|
||||
} catch (IOException ioe) {
|
||||
throw new HoodieIOException(ioe.getMessage(), ioe);
|
||||
}
|
||||
}
|
||||
|
||||
protected void runAnyPendingCompactions(HoodieTable<T, I, K, O> table) {
|
||||
table.getActiveTimeline().getCommitsAndCompactionTimeline().filterPendingCompactionTimeline().getInstants()
|
||||
.forEach(instant -> {
|
||||
LOG.info("Running previously failed inflight compaction at instant " + instant);
|
||||
compact(instant.getTimestamp(), true);
|
||||
});
|
||||
}
|
||||
|
||||
/**
|
||||
* Handle auto clean during commit.
|
||||
*
|
||||
* @param instantTime
|
||||
*/
|
||||
protected void autoCleanOnCommit(String instantTime) {
|
||||
if (config.isAutoClean()) {
|
||||
// Call clean to cleanup if there is anything to cleanup after the commit,
|
||||
if (config.isAsyncClean()) {
|
||||
LOG.info("Cleaner has been spawned already. Waiting for it to finish");
|
||||
AsyncCleanerService.waitForCompletion(asyncCleanerService);
|
||||
LOG.info("Cleaner has finished");
|
||||
} else {
|
||||
LOG.info("Auto cleaning is enabled. Running cleaner now");
|
||||
clean(instantTime);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Create a savepoint based on the latest commit action on the timeline.
|
||||
*
|
||||
* @param user - User creating the savepoint
|
||||
* @param comment - Comment for the savepoint
|
||||
*/
|
||||
public void savepoint(String user, String comment) {
|
||||
HoodieTable<T, I, K, O> table = createTable(config, hadoopConf);
|
||||
if (table.getCompletedCommitsTimeline().empty()) {
|
||||
throw new HoodieSavepointException("Could not savepoint. Commit timeline is empty");
|
||||
}
|
||||
|
||||
String latestCommit = table.getCompletedCommitsTimeline().lastInstant().get().getTimestamp();
|
||||
LOG.info("Savepointing latest commit " + latestCommit);
|
||||
savepoint(latestCommit, user, comment);
|
||||
}
|
||||
|
||||
/**
|
||||
* Savepoint a specific commit instant time. Latest version of data files as of the passed in instantTime
|
||||
* will be referenced in the savepoint and will never be cleaned. The savepointed commit will never be rolledback or archived.
|
||||
* <p>
|
||||
* This gives an option to rollback the state to the savepoint anytime. Savepoint needs to be manually created and
|
||||
* deleted.
|
||||
* <p>
|
||||
* Savepoint should be on a commit that could not have been cleaned.
|
||||
*
|
||||
* @param instantTime - commit that should be savepointed
|
||||
* @param user - User creating the savepoint
|
||||
* @param comment - Comment for the savepoint
|
||||
*/
|
||||
public void savepoint(String instantTime, String user, String comment) {
|
||||
HoodieTable<T, I, K, O> table = createTable(config, hadoopConf);
|
||||
table.savepoint(context, instantTime, user, comment);
|
||||
}
|
||||
|
||||
/**
|
||||
* Delete a savepoint that was created. Once the savepoint is deleted, the commit can be rolledback and cleaner may
|
||||
* clean up data files.
|
||||
*
|
||||
* @param savepointTime - delete the savepoint
|
||||
* @return true if the savepoint was deleted successfully
|
||||
*/
|
||||
public void deleteSavepoint(String savepointTime) {
|
||||
HoodieTable<T, I, K, O> table = createTable(config, hadoopConf);
|
||||
SavepointHelpers.deleteSavepoint(table, savepointTime);
|
||||
}
|
||||
|
||||
/**
|
||||
* Restore the data to the savepoint.
|
||||
*
|
||||
* WARNING: This rolls back recent commits and deleted data files and also pending compactions after savepoint time.
|
||||
* Queries accessing the files will mostly fail. This is expected to be a manual operation and no concurrent write or
|
||||
* compaction is expected to be running
|
||||
*
|
||||
* @param savepointTime - savepoint time to rollback to
|
||||
* @return true if the savepoint was restored to successfully
|
||||
*/
|
||||
public void restoreToSavepoint(String savepointTime) {
|
||||
HoodieTable<T, I, K, O> table = createTable(config, hadoopConf);
|
||||
SavepointHelpers.validateSavepointPresence(table, savepointTime);
|
||||
restoreToInstant(savepointTime);
|
||||
SavepointHelpers.validateSavepointRestore(table, savepointTime);
|
||||
}
|
||||
|
||||
/**
|
||||
* Rollback the inflight record changes with the given commit time.
|
||||
*
|
||||
* @param commitInstantTime Instant time of the commit
|
||||
* @throws HoodieRollbackException if rollback cannot be performed successfully
|
||||
*/
|
||||
public boolean rollback(final String commitInstantTime) throws HoodieRollbackException {
|
||||
LOG.info("Begin rollback of instant " + commitInstantTime);
|
||||
final String rollbackInstantTime = HoodieActiveTimeline.createNewInstantTime();
|
||||
final Timer.Context timerContext = this.metrics.getRollbackCtx();
|
||||
try {
|
||||
HoodieTable<T, I, K, O> table = createTable(config, hadoopConf);
|
||||
Option<HoodieInstant> commitInstantOpt = Option.fromJavaOptional(table.getActiveTimeline().getCommitsTimeline().getInstants()
|
||||
.filter(instant -> HoodieActiveTimeline.EQUALS.test(instant.getTimestamp(), commitInstantTime))
|
||||
.findFirst());
|
||||
if (commitInstantOpt.isPresent()) {
|
||||
HoodieRollbackMetadata rollbackMetadata = table.rollback(context, rollbackInstantTime, commitInstantOpt.get(), true);
|
||||
if (timerContext != null) {
|
||||
long durationInMs = metrics.getDurationInMs(timerContext.stop());
|
||||
metrics.updateRollbackMetrics(durationInMs, rollbackMetadata.getTotalFilesDeleted());
|
||||
}
|
||||
return true;
|
||||
} else {
|
||||
LOG.warn("Cannot find instant " + commitInstantTime + " in the timeline, for rollback");
|
||||
return false;
|
||||
}
|
||||
} catch (Exception e) {
|
||||
throw new HoodieRollbackException("Failed to rollback " + config.getBasePath() + " commits " + commitInstantTime, e);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* NOTE : This action requires all writers (ingest and compact) to a table to be stopped before proceeding. Revert
|
||||
* the (inflight/committed) record changes for all commits after the provided instant time.
|
||||
*
|
||||
* @param instantTime Instant time to which restoration is requested
|
||||
*/
|
||||
public HoodieRestoreMetadata restoreToInstant(final String instantTime) throws HoodieRestoreException {
|
||||
LOG.info("Begin restore to instant " + instantTime);
|
||||
final String restoreInstantTime = HoodieActiveTimeline.createNewInstantTime();
|
||||
Timer.Context timerContext = metrics.getRollbackCtx();
|
||||
try {
|
||||
HoodieTable<T, I, K, O> table = createTable(config, hadoopConf);
|
||||
HoodieRestoreMetadata restoreMetadata = table.restore(context, restoreInstantTime, instantTime);
|
||||
if (timerContext != null) {
|
||||
final long durationInMs = metrics.getDurationInMs(timerContext.stop());
|
||||
final long totalFilesDeleted = restoreMetadata.getHoodieRestoreMetadata().values().stream()
|
||||
.flatMap(Collection::stream)
|
||||
.mapToLong(HoodieRollbackMetadata::getTotalFilesDeleted)
|
||||
.sum();
|
||||
metrics.updateRollbackMetrics(durationInMs, totalFilesDeleted);
|
||||
}
|
||||
return restoreMetadata;
|
||||
} catch (Exception e) {
|
||||
throw new HoodieRestoreException("Failed to restore to " + instantTime, e);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Clean up any stale/old files/data lying around (either on file storage or index storage) based on the
|
||||
* configurations and CleaningPolicy used. (typically files that no longer can be used by a running query can be
|
||||
* cleaned)
|
||||
*/
|
||||
public HoodieCleanMetadata clean(String cleanInstantTime) throws HoodieIOException {
|
||||
LOG.info("Cleaner started");
|
||||
final Timer.Context timerContext = metrics.getCleanCtx();
|
||||
HoodieCleanMetadata metadata = createTable(config, hadoopConf).clean(context, cleanInstantTime);
|
||||
if (timerContext != null && metadata != null) {
|
||||
long durationMs = metrics.getDurationInMs(timerContext.stop());
|
||||
metrics.updateCleanMetrics(durationMs, metadata.getTotalFilesDeleted());
|
||||
LOG.info("Cleaned " + metadata.getTotalFilesDeleted() + " files"
|
||||
+ " Earliest Retained Instant :" + metadata.getEarliestCommitToRetain()
|
||||
+ " cleanerElapsedMs" + durationMs);
|
||||
}
|
||||
return metadata;
|
||||
}
|
||||
|
||||
public HoodieCleanMetadata clean() {
|
||||
return clean(HoodieActiveTimeline.createNewInstantTime());
|
||||
}
|
||||
|
||||
/**
|
||||
* Provides a new commit time for a write operation (insert/update/delete).
|
||||
*/
|
||||
public String startCommit() {
|
||||
// NOTE : Need to ensure that rollback is done before a new commit is started
|
||||
if (rollbackPending) {
|
||||
// Only rollback pending commit/delta-commits. Do not touch compaction commits
|
||||
rollbackPendingCommits();
|
||||
}
|
||||
String instantTime = HoodieActiveTimeline.createNewInstantTime();
|
||||
startCommitWithTime(instantTime);
|
||||
return instantTime;
|
||||
}
|
||||
|
||||
/**
|
||||
* Provides a new commit time for a write operation (insert/update/delete).
|
||||
*
|
||||
* @param instantTime Instant time to be generated
|
||||
*/
|
||||
public void startCommitWithTime(String instantTime) {
|
||||
HoodieTableMetaClient metaClient = createMetaClient(true);
|
||||
startCommitWithTime(instantTime, metaClient.getCommitActionType(), metaClient);
|
||||
}
|
||||
|
||||
/**
|
||||
* Completes a new commit time for a write operation (insert/update/delete) with specified action.
|
||||
*/
|
||||
public void startCommitWithTime(String instantTime, String actionType) {
|
||||
HoodieTableMetaClient metaClient = createMetaClient(true);
|
||||
startCommitWithTime(instantTime, actionType, metaClient);
|
||||
}
|
||||
|
||||
/**
|
||||
* Completes a new commit time for a write operation (insert/update/delete) with specified action.
|
||||
*/
|
||||
private void startCommitWithTime(String instantTime, String actionType, HoodieTableMetaClient metaClient) {
|
||||
// NOTE : Need to ensure that rollback is done before a new commit is started
|
||||
if (rollbackPending) {
|
||||
// Only rollback inflight commit/delta-commits. Do not touch compaction commits
|
||||
rollbackPendingCommits();
|
||||
}
|
||||
startCommit(instantTime, actionType, metaClient);
|
||||
}
|
||||
|
||||
private void startCommit(String instantTime, String actionType, HoodieTableMetaClient metaClient) {
|
||||
LOG.info("Generate a new instant time: " + instantTime + " action: " + actionType);
|
||||
// if there are pending compactions, their instantTime must not be greater than that of this instant time
|
||||
metaClient.getActiveTimeline().filterPendingCompactionTimeline().lastInstant().ifPresent(latestPending ->
|
||||
ValidationUtils.checkArgument(
|
||||
HoodieTimeline.compareTimestamps(latestPending.getTimestamp(), HoodieTimeline.LESSER_THAN, instantTime),
|
||||
"Latest pending compaction instant time must be earlier than this instant time. Latest Compaction :"
|
||||
+ latestPending + ", Ingesting at " + instantTime));
|
||||
metaClient.getActiveTimeline().createNewInstant(new HoodieInstant(HoodieInstant.State.REQUESTED, actionType,
|
||||
instantTime));
|
||||
}
|
||||
|
||||
/**
|
||||
* Schedules a new compaction instant.
|
||||
*
|
||||
* @param extraMetadata Extra Metadata to be stored
|
||||
*/
|
||||
public Option<String> scheduleCompaction(Option<Map<String, String>> extraMetadata) throws HoodieIOException {
|
||||
String instantTime = HoodieActiveTimeline.createNewInstantTime();
|
||||
return scheduleCompactionAtInstant(instantTime, extraMetadata) ? Option.of(instantTime) : Option.empty();
|
||||
}
|
||||
|
||||
/**
|
||||
* Schedules a new compaction instant with passed-in instant time.
|
||||
*
|
||||
* @param instantTime Compaction Instant Time
|
||||
* @param extraMetadata Extra Metadata to be stored
|
||||
*/
|
||||
public boolean scheduleCompactionAtInstant(String instantTime, Option<Map<String, String>> extraMetadata) throws HoodieIOException {
|
||||
LOG.info("Scheduling compaction at instant time :" + instantTime);
|
||||
Option<HoodieCompactionPlan> plan = createTable(config, hadoopConf)
|
||||
.scheduleCompaction(context, instantTime, extraMetadata);
|
||||
return plan.isPresent();
|
||||
}
|
||||
|
||||
/**
|
||||
* Performs Compaction for the workload stored in instant-time.
|
||||
*
|
||||
* @param compactionInstantTime Compaction Instant Time
|
||||
* @return RDD of WriteStatus to inspect errors and counts
|
||||
*/
|
||||
public O compact(String compactionInstantTime) {
|
||||
return compact(compactionInstantTime, config.shouldAutoCommit());
|
||||
}
|
||||
|
||||
/**
|
||||
* Commit a compaction operation. Allow passing additional meta-data to be stored in commit instant file.
|
||||
*
|
||||
* @param compactionInstantTime Compaction Instant Time
|
||||
* @param writeStatuses RDD of WriteStatus to inspect errors and counts
|
||||
* @param extraMetadata Extra Metadata to be stored
|
||||
*/
|
||||
public abstract void commitCompaction(String compactionInstantTime, O writeStatuses,
|
||||
Option<Map<String, String>> extraMetadata) throws IOException;
|
||||
|
||||
/**
|
||||
* Commit Compaction and track metrics.
|
||||
*/
|
||||
protected abstract void completeCompaction(HoodieCommitMetadata metadata, O writeStatuses,
|
||||
HoodieTable<T, I, K, O> table, String compactionCommitTime);
|
||||
|
||||
|
||||
/**
|
||||
* Rollback failed compactions. Inflight rollbacks for compactions revert the .inflight file to the .requested file
|
||||
*
|
||||
* @param inflightInstant Inflight Compaction Instant
|
||||
* @param table Hoodie Table
|
||||
*/
|
||||
public void rollbackInflightCompaction(HoodieInstant inflightInstant, HoodieTable<T, I, K, O> table) {
|
||||
table.rollback(context, HoodieActiveTimeline.createNewInstantTime(), inflightInstant, false);
|
||||
table.getActiveTimeline().revertCompactionInflightToRequested(inflightInstant);
|
||||
}
|
||||
|
||||
/**
|
||||
* Cleanup all pending commits.
|
||||
*/
|
||||
private void rollbackPendingCommits() {
|
||||
HoodieTable<T, I, K, O> table = createTable(config, hadoopConf);
|
||||
HoodieTimeline inflightTimeline = table.getMetaClient().getCommitsTimeline().filterPendingExcludingCompaction();
|
||||
List<String> commits = inflightTimeline.getReverseOrderedInstants().map(HoodieInstant::getTimestamp)
|
||||
.collect(Collectors.toList());
|
||||
for (String commit : commits) {
|
||||
if (HoodieTimeline.compareTimestamps(commit, HoodieTimeline.LESSER_THAN_OR_EQUALS,
|
||||
HoodieTimeline.FULL_BOOTSTRAP_INSTANT_TS)) {
|
||||
rollBackInflightBootstrap();
|
||||
break;
|
||||
} else {
|
||||
rollback(commit);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Ensures compaction instant is in expected state and performs Compaction for the workload stored in instant-time.
|
||||
*
|
||||
* @param compactionInstantTime Compaction Instant Time
|
||||
* @return RDD of Write Status
|
||||
*/
|
||||
protected abstract O compact(String compactionInstantTime, boolean shouldComplete);
|
||||
|
||||
/**
|
||||
* Performs a compaction operation on a table, serially before or after an insert/upsert action.
|
||||
*/
|
||||
protected Option<String> inlineCompact(Option<Map<String, String>> extraMetadata) {
|
||||
Option<String> compactionInstantTimeOpt = scheduleCompaction(extraMetadata);
|
||||
compactionInstantTimeOpt.ifPresent(compactionInstantTime -> {
|
||||
// inline compaction should auto commit as the user is never given control
|
||||
compact(compactionInstantTime, true);
|
||||
});
|
||||
return compactionInstantTimeOpt;
|
||||
}
|
||||
|
||||
/**
|
||||
* Finalize Write operation.
|
||||
*
|
||||
* @param table HoodieTable
|
||||
* @param instantTime Instant Time
|
||||
* @param stats Hoodie Write Stat
|
||||
*/
|
||||
protected void finalizeWrite(HoodieTable<T, I, K, O> table, String instantTime, List<HoodieWriteStat> stats) {
|
||||
try {
|
||||
final Timer.Context finalizeCtx = metrics.getFinalizeCtx();
|
||||
table.finalizeWrite(context, instantTime, stats);
|
||||
if (finalizeCtx != null) {
|
||||
Option<Long> durationInMs = Option.of(metrics.getDurationInMs(finalizeCtx.stop()));
|
||||
durationInMs.ifPresent(duration -> {
|
||||
LOG.info("Finalize write elapsed time (milliseconds): " + duration);
|
||||
metrics.updateFinalizeWriteMetrics(duration, stats.size());
|
||||
});
|
||||
}
|
||||
} catch (HoodieIOException ioe) {
|
||||
throw new HoodieCommitException("Failed to complete commit " + instantTime + " due to finalize errors.", ioe);
|
||||
}
|
||||
}
|
||||
|
||||
public HoodieMetrics getMetrics() {
|
||||
return metrics;
|
||||
}
|
||||
|
||||
public HoodieIndex<T, I, K, O> getIndex() {
|
||||
return index;
|
||||
}
|
||||
|
||||
/**
|
||||
* Get HoodieTable and init {@link Timer.Context}.
|
||||
*
|
||||
* @param operationType write operation type
|
||||
* @param instantTime current inflight instant time
|
||||
* @return HoodieTable
|
||||
*/
|
||||
protected abstract HoodieTable<T, I, K, O> getTableAndInitCtx(WriteOperationType operationType, String instantTime);
|
||||
|
||||
/**
|
||||
* Sets write schema from last instant since deletes may not have schema set in the config.
|
||||
*/
|
||||
protected void setWriteSchemaForDeletes(HoodieTableMetaClient metaClient) {
|
||||
try {
|
||||
HoodieActiveTimeline activeTimeline = metaClient.getActiveTimeline();
|
||||
Option<HoodieInstant> lastInstant =
|
||||
activeTimeline.filterCompletedInstants().filter(s -> s.getAction().equals(metaClient.getCommitActionType()))
|
||||
.lastInstant();
|
||||
if (lastInstant.isPresent()) {
|
||||
HoodieCommitMetadata commitMetadata = HoodieCommitMetadata.fromBytes(
|
||||
activeTimeline.getInstantDetails(lastInstant.get()).get(), HoodieCommitMetadata.class);
|
||||
if (commitMetadata.getExtraMetadata().containsKey(HoodieCommitMetadata.SCHEMA_KEY)) {
|
||||
config.setSchema(commitMetadata.getExtraMetadata().get(HoodieCommitMetadata.SCHEMA_KEY));
|
||||
} else {
|
||||
throw new HoodieIOException("Latest commit does not have any schema in commit metadata");
|
||||
}
|
||||
} else {
|
||||
throw new HoodieIOException("Deletes issued without any prior commits");
|
||||
}
|
||||
} catch (IOException e) {
|
||||
throw new HoodieIOException("IOException thrown while reading last commit metadata", e);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void close() {
|
||||
// release AsyncCleanerService
|
||||
AsyncCleanerService.forceShutdown(asyncCleanerService);
|
||||
asyncCleanerService = null;
|
||||
|
||||
// Stop timeline-server if running
|
||||
super.close();
|
||||
// Calling this here releases any resources used by your index, so make sure to finish any related operations
|
||||
// before this point
|
||||
this.index.close();
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,85 @@
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hudi.client;
|
||||
|
||||
import org.apache.hudi.async.HoodieAsyncService;
|
||||
import org.apache.hudi.common.util.collection.Pair;
|
||||
import org.apache.hudi.exception.HoodieException;
|
||||
import org.apache.log4j.LogManager;
|
||||
import org.apache.log4j.Logger;
|
||||
|
||||
import java.util.concurrent.CompletableFuture;
|
||||
import java.util.concurrent.ExecutorService;
|
||||
import java.util.concurrent.Executors;
|
||||
|
||||
/**
|
||||
* Clean service running concurrently with write operation.
|
||||
*/
|
||||
class AsyncCleanerService extends HoodieAsyncService {
|
||||
|
||||
private static final Logger LOG = LogManager.getLogger(AsyncCleanerService.class);
|
||||
|
||||
private final AbstractHoodieWriteClient writeClient;
|
||||
private final String cleanInstantTime;
|
||||
private final transient ExecutorService executor = Executors.newSingleThreadExecutor();
|
||||
|
||||
protected AsyncCleanerService(AbstractHoodieWriteClient writeClient, String cleanInstantTime) {
|
||||
this.writeClient = writeClient;
|
||||
this.cleanInstantTime = cleanInstantTime;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected Pair<CompletableFuture, ExecutorService> startService() {
|
||||
return Pair.of(CompletableFuture.supplyAsync(() -> {
|
||||
writeClient.clean(cleanInstantTime);
|
||||
return true;
|
||||
}), executor);
|
||||
}
|
||||
|
||||
public static AsyncCleanerService startAsyncCleaningIfEnabled(AbstractHoodieWriteClient writeClient,
|
||||
String instantTime) {
|
||||
AsyncCleanerService asyncCleanerService = null;
|
||||
if (writeClient.getConfig().isAutoClean() && writeClient.getConfig().isAsyncClean()) {
|
||||
LOG.info("Auto cleaning is enabled. Running cleaner async to write operation");
|
||||
asyncCleanerService = new AsyncCleanerService(writeClient, instantTime);
|
||||
asyncCleanerService.start(null);
|
||||
} else {
|
||||
LOG.info("Auto cleaning is not enabled. Not running cleaner now");
|
||||
}
|
||||
return asyncCleanerService;
|
||||
}
|
||||
|
||||
public static void waitForCompletion(AsyncCleanerService asyncCleanerService) {
|
||||
if (asyncCleanerService != null) {
|
||||
LOG.info("Waiting for async cleaner to finish");
|
||||
try {
|
||||
asyncCleanerService.waitForShutdown();
|
||||
} catch (Exception e) {
|
||||
throw new HoodieException("Error waiting for async cleaning to finish", e);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
public static void forceShutdown(AsyncCleanerService asyncCleanerService) {
|
||||
if (asyncCleanerService != null) {
|
||||
LOG.info("Shutting down async cleaner");
|
||||
asyncCleanerService.shutdown(true);
|
||||
}
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,538 @@
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hudi.client;
|
||||
|
||||
import org.apache.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.common.HoodieEngineContext;
|
||||
import org.apache.hudi.common.fs.FSUtils;
|
||||
import org.apache.hudi.common.model.CompactionOperation;
|
||||
import org.apache.hudi.common.model.FileSlice;
|
||||
import org.apache.hudi.common.model.HoodieBaseFile;
|
||||
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.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.timeline.HoodieTimeline;
|
||||
import org.apache.hudi.common.table.timeline.TimelineMetadataUtils;
|
||||
import org.apache.hudi.common.table.view.HoodieTableFileSystemView;
|
||||
import org.apache.hudi.common.util.CompactionUtils;
|
||||
import org.apache.hudi.common.util.Option;
|
||||
import org.apache.hudi.common.util.ValidationUtils;
|
||||
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.table.action.compact.OperationResult;
|
||||
import org.apache.log4j.LogManager;
|
||||
import org.apache.log4j.Logger;
|
||||
|
||||
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 static org.apache.hudi.common.table.timeline.HoodieTimeline.COMPACTION_ACTION;
|
||||
|
||||
/**
|
||||
* Client to perform admin operations related to compaction.
|
||||
*/
|
||||
public class CompactionAdminClient extends AbstractHoodieClient {
|
||||
|
||||
private static final Logger LOG = LogManager.getLogger(CompactionAdminClient.class);
|
||||
|
||||
public CompactionAdminClient(HoodieEngineContext context, String basePath) {
|
||||
super(context, HoodieWriteConfig.newBuilder().withPath(basePath).build());
|
||||
}
|
||||
|
||||
/**
|
||||
* 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());
|
||||
context.setJobStatus(this.getClass().getSimpleName(), "Validate compaction operations");
|
||||
return context.map(ops, op -> {
|
||||
try {
|
||||
return validateCompactionOperation(metaClient, compactionInstant, op, Option.of(fsView));
|
||||
} catch (IOException e) {
|
||||
throw new HoodieIOException(e.getMessage(), e);
|
||||
}
|
||||
}, parallelism);
|
||||
}
|
||||
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);
|
||||
ValidationUtils.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);
|
||||
}
|
||||
// Overwrite compaction plan with updated info
|
||||
metaClient.getActiveTimeline().saveToCompactionRequested(
|
||||
new HoodieInstant(State.REQUESTED, COMPACTION_ACTION, compactionOperationWithInstant.getLeft()),
|
||||
TimelineMetadataUtils.serializeCompactionPlan(newPlan), true);
|
||||
}
|
||||
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 {
|
||||
return TimelineMetadataUtils.deserializeCompactionPlan(
|
||||
metaClient.getActiveTimeline().readCompactionPlanAsBytes(
|
||||
HoodieTimeline.getCompactionRequestedInstant(compactionInstant)).get());
|
||||
}
|
||||
|
||||
/**
|
||||
* 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.getDeltaFileNames().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 -> {
|
||||
ValidationUtils.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());
|
||||
ValidationUtils.checkArgument(statuses.length == 1, "Only one status must be present");
|
||||
ValidationUtils.checkArgument(statuses[0].isFile(), "Source File must exist");
|
||||
ValidationUtils.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<HoodieBaseFile> df = fs.getBaseFile();
|
||||
if (operation.getDataFileName().isPresent()) {
|
||||
String expPath = metaClient.getFs()
|
||||
.getFileStatus(
|
||||
new Path(FSUtils.getPartitionPath(metaClient.getBasePath(), operation.getPartitionPath()),
|
||||
new Path(operation.getDataFileName().get())))
|
||||
.getPath().toString();
|
||||
ValidationUtils.checkArgument(df.isPresent(),
|
||||
"Data File must be present. File Slice was : " + fs + ", operation :" + operation);
|
||||
ValidationUtils.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.getDeltaFileNames().stream().map(dp -> {
|
||||
try {
|
||||
FileStatus[] fileStatuses = metaClient.getFs().listStatus(new Path(
|
||||
FSUtils.getPartitionPath(metaClient.getBasePath(), operation.getPartitionPath()), new Path(dp)));
|
||||
ValidationUtils.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());
|
||||
ValidationUtils.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());
|
||||
ValidationUtils.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) {
|
||||
context.setJobStatus(this.getClass().getSimpleName(), "Execute unschedule operations");
|
||||
return context.map(renameActions, 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));
|
||||
}
|
||||
}, parallelism);
|
||||
} 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());
|
||||
context.setJobStatus(this.getClass().getSimpleName(), "Generate compaction unscheduling operations");
|
||||
return context.flatMap(ops, op -> {
|
||||
try {
|
||||
return getRenamingActionsForUnschedulingCompactionOperation(metaClient, compactionInstant, op,
|
||||
Option.of(fsView), skipValidation).stream();
|
||||
} catch (IOException ioe) {
|
||||
throw new HoodieIOException(ioe.getMessage(), ioe);
|
||||
} catch (CompactionValidationException ve) {
|
||||
throw new HoodieException(ve);
|
||||
}
|
||||
}, parallelism);
|
||||
}
|
||||
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.getBaseFile().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);
|
||||
}
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,150 @@
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hudi.client;
|
||||
|
||||
import org.apache.hudi.common.model.HoodieWriteStat;
|
||||
import org.apache.hudi.common.util.collection.Pair;
|
||||
|
||||
import java.io.Serializable;
|
||||
import java.util.ArrayList;
|
||||
import java.util.List;
|
||||
import java.util.Random;
|
||||
|
||||
/**
|
||||
* Hoodie's internal write status used in datasource implementation of bulk insert.
|
||||
*/
|
||||
public class HoodieInternalWriteStatus implements Serializable {
|
||||
|
||||
private static final long serialVersionUID = 1L;
|
||||
private static final long RANDOM_SEED = 9038412832L;
|
||||
|
||||
private String fileId;
|
||||
private String partitionPath;
|
||||
private List<String> successRecordKeys = new ArrayList<>();
|
||||
private List<Pair<String, Throwable>> failedRecordKeys = new ArrayList<>();
|
||||
|
||||
private HoodieWriteStat stat;
|
||||
|
||||
private long totalRecords = 0;
|
||||
private long totalErrorRecords = 0;
|
||||
private Throwable globalError = null;
|
||||
|
||||
private final double failureFraction;
|
||||
private final boolean trackSuccessRecords;
|
||||
private final transient Random random;
|
||||
|
||||
public HoodieInternalWriteStatus(Boolean trackSuccessRecords, Double failureFraction) {
|
||||
this.trackSuccessRecords = trackSuccessRecords;
|
||||
this.failureFraction = failureFraction;
|
||||
this.random = new Random(RANDOM_SEED);
|
||||
}
|
||||
|
||||
public void markSuccess(String recordKey) {
|
||||
if (trackSuccessRecords) {
|
||||
this.successRecordKeys.add(recordKey);
|
||||
}
|
||||
totalRecords++;
|
||||
}
|
||||
|
||||
public void markFailure(String recordKey, Throwable t) {
|
||||
if (failedRecordKeys.isEmpty() || (random.nextDouble() <= failureFraction)) {
|
||||
failedRecordKeys.add(Pair.of(recordKey, t));
|
||||
}
|
||||
totalRecords++;
|
||||
}
|
||||
|
||||
public boolean hasErrors() {
|
||||
return failedRecordKeys.size() != 0;
|
||||
}
|
||||
|
||||
public HoodieWriteStat getStat() {
|
||||
return stat;
|
||||
}
|
||||
|
||||
public void setStat(HoodieWriteStat stat) {
|
||||
this.stat = stat;
|
||||
}
|
||||
|
||||
public String getFileId() {
|
||||
return fileId;
|
||||
}
|
||||
|
||||
public void setFileId(String fileId) {
|
||||
this.fileId = fileId;
|
||||
}
|
||||
|
||||
public String getPartitionPath() {
|
||||
return partitionPath;
|
||||
}
|
||||
|
||||
public void setPartitionPath(String partitionPath) {
|
||||
this.partitionPath = partitionPath;
|
||||
}
|
||||
|
||||
public List<String> getSuccessRecordKeys() {
|
||||
return successRecordKeys;
|
||||
}
|
||||
|
||||
public long getFailedRowsSize() {
|
||||
return failedRecordKeys.size();
|
||||
}
|
||||
|
||||
public List<Pair<String, Throwable>> getFailedRecordKeys() {
|
||||
return failedRecordKeys;
|
||||
}
|
||||
|
||||
public void setFailedRecordKeys(List<Pair<String, Throwable>> failedRecordKeys) {
|
||||
this.failedRecordKeys = failedRecordKeys;
|
||||
}
|
||||
|
||||
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;
|
||||
}
|
||||
|
||||
public Throwable getGlobalError() {
|
||||
return globalError;
|
||||
}
|
||||
|
||||
public void setGlobalError(Throwable globalError) {
|
||||
this.globalError = globalError;
|
||||
}
|
||||
|
||||
public void setSuccessRecordKeys(List<String> successRecordKeys) {
|
||||
this.successRecordKeys = successRecordKeys;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
return "PartitionPath " + partitionPath + ", FileID " + fileId + ", Success records "
|
||||
+ totalRecords + ", errored Rows " + totalErrorRecords
|
||||
+ ", global error " + (globalError != null);
|
||||
}
|
||||
}
|
||||
@@ -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.client;
|
||||
|
||||
import com.fasterxml.jackson.databind.DeserializationFeature;
|
||||
import com.fasterxml.jackson.databind.ObjectMapper;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
|
||||
import org.apache.hudi.common.model.FileSlice;
|
||||
import org.apache.hudi.common.model.HoodieFileGroup;
|
||||
import org.apache.hudi.common.model.HoodieReplaceCommitMetadata;
|
||||
import org.apache.hudi.common.model.HoodieRollingStatMetadata;
|
||||
import org.apache.hudi.common.table.HoodieTableMetaClient;
|
||||
import org.apache.hudi.common.table.timeline.HoodieInstant;
|
||||
import org.apache.hudi.common.table.view.TableFileSystemView;
|
||||
import org.apache.hudi.client.common.HoodieEngineContext;
|
||||
|
||||
import org.apache.log4j.LogManager;
|
||||
import org.apache.log4j.Logger;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.io.Serializable;
|
||||
import java.util.List;
|
||||
import java.util.stream.Stream;
|
||||
|
||||
/**
|
||||
* Operates on marker files for a given write action (commit, delta commit, compaction).
|
||||
*/
|
||||
public class ReplaceArchivalHelper implements Serializable {
|
||||
|
||||
private static final Logger LOG = LogManager.getLogger(ReplaceArchivalHelper.class);
|
||||
|
||||
/**
|
||||
* Convert json metadata to avro format.
|
||||
*/
|
||||
public static org.apache.hudi.avro.model.HoodieReplaceCommitMetadata convertReplaceCommitMetadata(
|
||||
HoodieReplaceCommitMetadata hoodieReplaceCommitMetadata) {
|
||||
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.HoodieReplaceCommitMetadata avroMetaData =
|
||||
mapper.convertValue(hoodieReplaceCommitMetadata, org.apache.hudi.avro.model.HoodieReplaceCommitMetadata.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;
|
||||
}
|
||||
|
||||
/**
|
||||
* Delete all files represented by FileSlices in parallel. Return true if all files are deleted successfully.
|
||||
*/
|
||||
public static boolean deleteReplacedFileGroups(HoodieEngineContext context, HoodieTableMetaClient metaClient,
|
||||
TableFileSystemView fileSystemView,
|
||||
HoodieInstant instant, List<String> replacedPartitions) {
|
||||
|
||||
List<Boolean> f = context.map(replacedPartitions, partition -> {
|
||||
Stream<FileSlice> fileSlices = fileSystemView.getReplacedFileGroupsBeforeOrOn(instant.getTimestamp(), partition)
|
||||
.flatMap(HoodieFileGroup::getAllRawFileSlices);
|
||||
return fileSlices.allMatch(slice -> deleteFileSlice(slice, metaClient, instant));
|
||||
}, replacedPartitions.size());
|
||||
|
||||
return f.stream().reduce((x, y) -> x & y).orElse(true);
|
||||
}
|
||||
|
||||
private static boolean deleteFileSlice(FileSlice fileSlice, HoodieTableMetaClient metaClient, HoodieInstant instant) {
|
||||
boolean baseFileDeleteSuccess = fileSlice.getBaseFile().map(baseFile ->
|
||||
deletePath(new Path(baseFile.getPath()), metaClient, instant)).orElse(true);
|
||||
|
||||
boolean logFileSuccess = fileSlice.getLogFiles().map(logFile ->
|
||||
deletePath(logFile.getPath(), metaClient, instant)).allMatch(x -> x);
|
||||
return baseFileDeleteSuccess & logFileSuccess;
|
||||
}
|
||||
|
||||
private static boolean deletePath(Path path, HoodieTableMetaClient metaClient, HoodieInstant instant) {
|
||||
try {
|
||||
LOG.info("Deleting " + path + " before archiving " + instant);
|
||||
metaClient.getFs().delete(path);
|
||||
return true;
|
||||
} catch (IOException e) {
|
||||
LOG.error("unable to delete file groups that are replaced", e);
|
||||
return false;
|
||||
}
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,182 @@
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hudi.client;
|
||||
|
||||
import org.apache.hudi.common.model.HoodieKey;
|
||||
import org.apache.hudi.common.model.HoodieRecord;
|
||||
import org.apache.hudi.common.model.HoodieWriteStat;
|
||||
import org.apache.hudi.common.util.Option;
|
||||
|
||||
import java.io.Serializable;
|
||||
import java.util.ArrayList;
|
||||
import java.util.HashMap;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Random;
|
||||
|
||||
/**
|
||||
* Status of a write operation.
|
||||
*/
|
||||
public class WriteStatus implements Serializable {
|
||||
|
||||
private static final long serialVersionUID = 1L;
|
||||
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,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.client.bootstrap;
|
||||
|
||||
/**
|
||||
* Identifies different types of bootstrap.
|
||||
*/
|
||||
public enum BootstrapMode {
|
||||
/**
|
||||
* In this mode, record level metadata is generated for each source record and both original record and metadata
|
||||
* for each record copied.
|
||||
*/
|
||||
FULL_RECORD,
|
||||
|
||||
/**
|
||||
* In this mode, record level metadata alone is generated for each source record and stored in new bootstrap location.
|
||||
*/
|
||||
METADATA_ONLY
|
||||
}
|
||||
@@ -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.client.bootstrap;
|
||||
|
||||
import org.apache.hudi.common.model.HoodieRecordPayload;
|
||||
import org.apache.hudi.common.util.Option;
|
||||
|
||||
import org.apache.avro.Schema;
|
||||
import org.apache.avro.generic.GenericRecord;
|
||||
import org.apache.avro.generic.IndexedRecord;
|
||||
|
||||
public class BootstrapRecordPayload implements HoodieRecordPayload<BootstrapRecordPayload> {
|
||||
|
||||
private final GenericRecord record;
|
||||
|
||||
public BootstrapRecordPayload(GenericRecord record) {
|
||||
this.record = record;
|
||||
}
|
||||
|
||||
@Override
|
||||
public BootstrapRecordPayload preCombine(BootstrapRecordPayload another) {
|
||||
return this;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Option<IndexedRecord> combineAndGetUpdateValue(IndexedRecord currentValue, Schema schema) {
|
||||
return Option.ofNullable(record);
|
||||
}
|
||||
|
||||
@Override
|
||||
public Option<IndexedRecord> getInsertValue(Schema schema) {
|
||||
return Option.ofNullable(record);
|
||||
}
|
||||
|
||||
}
|
||||
@@ -0,0 +1,48 @@
|
||||
/*
|
||||
* 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.bootstrap;
|
||||
|
||||
import org.apache.hudi.client.WriteStatus;
|
||||
import org.apache.hudi.common.model.BootstrapFileMapping;
|
||||
import org.apache.hudi.common.model.HoodieWriteStat;
|
||||
import org.apache.hudi.common.util.collection.Pair;
|
||||
|
||||
/**
|
||||
* WriteStatus for Bootstrap.
|
||||
*/
|
||||
public class BootstrapWriteStatus extends WriteStatus {
|
||||
|
||||
private BootstrapFileMapping sourceFileMapping;
|
||||
|
||||
public BootstrapWriteStatus(Boolean trackSuccessRecords, Double failureFraction) {
|
||||
super(trackSuccessRecords, failureFraction);
|
||||
}
|
||||
|
||||
public BootstrapFileMapping getBootstrapSourceFileMapping() {
|
||||
return sourceFileMapping;
|
||||
}
|
||||
|
||||
public Pair<BootstrapFileMapping, HoodieWriteStat> getBootstrapSourceAndWriteStat() {
|
||||
return Pair.of(getBootstrapSourceFileMapping(), getStat());
|
||||
}
|
||||
|
||||
public void setBootstrapSourceFileMapping(BootstrapFileMapping sourceFileMapping) {
|
||||
this.sourceFileMapping = sourceFileMapping;
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,56 @@
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hudi.client.bootstrap;
|
||||
|
||||
import java.io.Serializable;
|
||||
import org.apache.hudi.avro.model.HoodieFileStatus;
|
||||
import org.apache.hudi.client.common.HoodieEngineContext;
|
||||
import org.apache.hudi.common.config.TypedProperties;
|
||||
import org.apache.hudi.common.util.collection.Pair;
|
||||
|
||||
import org.apache.log4j.LogManager;
|
||||
import org.apache.log4j.Logger;
|
||||
|
||||
import java.util.List;
|
||||
|
||||
/**
|
||||
* Creates Hoodie Records with complete record data, given a list of partitions to be bootstrapped.
|
||||
*/
|
||||
public abstract class FullRecordBootstrapDataProvider<I> implements Serializable {
|
||||
|
||||
protected static final Logger LOG = LogManager.getLogger(FullRecordBootstrapDataProvider.class);
|
||||
|
||||
protected final TypedProperties props;
|
||||
protected final transient HoodieEngineContext context;
|
||||
|
||||
public FullRecordBootstrapDataProvider(TypedProperties props, HoodieEngineContext context) {
|
||||
this.props = props;
|
||||
this.context = context;
|
||||
}
|
||||
|
||||
/**
|
||||
* Generates a list of input partition and files and returns a RDD representing source.
|
||||
* @param tableName Hudi Table Name
|
||||
* @param sourceBasePath Source Base Path
|
||||
* @param partitionPaths Partition Paths
|
||||
* @return input records
|
||||
*/
|
||||
public abstract I generateInputRecords(String tableName,
|
||||
String sourceBasePath, List<Pair<String, List<HoodieFileStatus>>> partitionPaths);
|
||||
}
|
||||
@@ -0,0 +1,69 @@
|
||||
/*
|
||||
* 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.bootstrap;
|
||||
|
||||
import org.apache.hudi.avro.HoodieAvroUtils;
|
||||
import org.apache.hudi.avro.model.HoodieFileStatus;
|
||||
import org.apache.hudi.client.common.HoodieEngineContext;
|
||||
import org.apache.hudi.common.util.collection.Pair;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
|
||||
import org.apache.avro.Schema;
|
||||
|
||||
import java.util.List;
|
||||
|
||||
/**
|
||||
* Bootstrap Schema Provider. Schema provided in config is used. If not available, use schema from Parquet
|
||||
*/
|
||||
public abstract class HoodieBootstrapSchemaProvider {
|
||||
|
||||
protected final HoodieWriteConfig writeConfig;
|
||||
|
||||
public HoodieBootstrapSchemaProvider(HoodieWriteConfig writeConfig) {
|
||||
this.writeConfig = writeConfig;
|
||||
}
|
||||
|
||||
/**
|
||||
* Main API to select avro schema for bootstrapping.
|
||||
* @param context HoodieEngineContext
|
||||
* @param partitions List of partitions with files within them
|
||||
* @return Avro Schema
|
||||
*/
|
||||
public final Schema getBootstrapSchema(HoodieEngineContext context, List<Pair<String, List<HoodieFileStatus>>> partitions) {
|
||||
if (writeConfig.getSchema() != null) {
|
||||
// Use schema specified by user if set
|
||||
Schema userSchema = Schema.parse(writeConfig.getSchema());
|
||||
if (!HoodieAvroUtils.getNullSchema().equals(userSchema)) {
|
||||
return userSchema;
|
||||
}
|
||||
}
|
||||
return getBootstrapSourceSchema(context, partitions);
|
||||
}
|
||||
|
||||
/**
|
||||
* Select a random file to be used to generate avro schema.
|
||||
* Override this method to get custom schema selection.
|
||||
* @param context HoodieEngineContext
|
||||
* @param partitions List of partitions with files within them
|
||||
* @return Avro Schema
|
||||
*/
|
||||
protected abstract Schema getBootstrapSourceSchema(HoodieEngineContext context,
|
||||
List<Pair<String, List<HoodieFileStatus>>> partitions);
|
||||
|
||||
}
|
||||
@@ -0,0 +1,48 @@
|
||||
/*
|
||||
* 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.bootstrap.selector;
|
||||
|
||||
import org.apache.hudi.avro.model.HoodieFileStatus;
|
||||
import org.apache.hudi.client.bootstrap.BootstrapMode;
|
||||
import org.apache.hudi.common.util.collection.Pair;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
|
||||
import java.io.Serializable;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
|
||||
/**
|
||||
* Pluggable Partition Selector for selecting partitions to perform full or metadata-only bootstrapping.
|
||||
*/
|
||||
public abstract class BootstrapModeSelector implements Serializable {
|
||||
|
||||
protected final HoodieWriteConfig writeConfig;
|
||||
|
||||
public BootstrapModeSelector(HoodieWriteConfig writeConfig) {
|
||||
this.writeConfig = writeConfig;
|
||||
}
|
||||
|
||||
/**
|
||||
* Classify partitions for the purpose of bootstrapping. For a non-partitioned source, input list will be one entry.
|
||||
*
|
||||
* @param partitions List of partitions with files present in each partitions
|
||||
* @return a partitions grouped by bootstrap mode
|
||||
*/
|
||||
public abstract Map<BootstrapMode, List<String>> select(List<Pair<String, List<HoodieFileStatus>>> partitions);
|
||||
}
|
||||
@@ -0,0 +1,56 @@
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hudi.client.bootstrap.selector;
|
||||
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.regex.Pattern;
|
||||
import java.util.stream.Collectors;
|
||||
import org.apache.hudi.avro.model.HoodieFileStatus;
|
||||
import org.apache.hudi.client.bootstrap.BootstrapMode;
|
||||
import org.apache.hudi.common.util.collection.Pair;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
import org.apache.log4j.LogManager;
|
||||
import org.apache.log4j.Logger;
|
||||
|
||||
public class BootstrapRegexModeSelector extends BootstrapModeSelector {
|
||||
|
||||
private static final long serialVersionUID = 1L;
|
||||
private static final Logger LOG = LogManager.getLogger(BootstrapRegexModeSelector.class);
|
||||
|
||||
private final Pattern pattern;
|
||||
private final BootstrapMode bootstrapModeOnMatch;
|
||||
private final BootstrapMode defaultMode;
|
||||
|
||||
public BootstrapRegexModeSelector(HoodieWriteConfig writeConfig) {
|
||||
super(writeConfig);
|
||||
this.pattern = Pattern.compile(writeConfig.getBootstrapModeSelectorRegex());
|
||||
this.bootstrapModeOnMatch = writeConfig.getBootstrapModeForRegexMatch();
|
||||
this.defaultMode = BootstrapMode.FULL_RECORD.equals(bootstrapModeOnMatch)
|
||||
? BootstrapMode.METADATA_ONLY : BootstrapMode.FULL_RECORD;
|
||||
LOG.info("Default Mode :" + defaultMode + ", on Match Mode :" + bootstrapModeOnMatch);
|
||||
}
|
||||
|
||||
@Override
|
||||
public Map<BootstrapMode, List<String>> select(List<Pair<String, List<HoodieFileStatus>>> partitions) {
|
||||
return partitions.stream()
|
||||
.map(p -> Pair.of(pattern.matcher(p.getKey()).matches() ? bootstrapModeOnMatch : defaultMode, p.getKey()))
|
||||
.collect(Collectors.groupingBy(Pair::getKey, Collectors.mapping(Pair::getValue, Collectors.toList())));
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,29 @@
|
||||
/*
|
||||
* 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.bootstrap.selector;
|
||||
|
||||
import org.apache.hudi.client.bootstrap.BootstrapMode;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
|
||||
public class FullRecordBootstrapModeSelector extends UniformBootstrapModeSelector {
|
||||
|
||||
public FullRecordBootstrapModeSelector(HoodieWriteConfig bootstrapConfig) {
|
||||
super(bootstrapConfig, BootstrapMode.FULL_RECORD);
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,29 @@
|
||||
/*
|
||||
* 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.bootstrap.selector;
|
||||
|
||||
import org.apache.hudi.client.bootstrap.BootstrapMode;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
|
||||
public class MetadataOnlyBootstrapModeSelector extends UniformBootstrapModeSelector {
|
||||
|
||||
public MetadataOnlyBootstrapModeSelector(HoodieWriteConfig bootstrapConfig) {
|
||||
super(bootstrapConfig, BootstrapMode.METADATA_ONLY);
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,48 @@
|
||||
/*
|
||||
* 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.bootstrap.selector;
|
||||
|
||||
import org.apache.hudi.avro.model.HoodieFileStatus;
|
||||
import org.apache.hudi.client.bootstrap.BootstrapMode;
|
||||
import org.apache.hudi.common.util.collection.Pair;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.stream.Collectors;
|
||||
|
||||
/**
|
||||
* A bootstrap selector which employs same bootstrap mode for all partitions.
|
||||
*/
|
||||
public abstract class UniformBootstrapModeSelector extends BootstrapModeSelector {
|
||||
|
||||
private final BootstrapMode bootstrapMode;
|
||||
|
||||
public UniformBootstrapModeSelector(HoodieWriteConfig bootstrapConfig, BootstrapMode bootstrapMode) {
|
||||
super(bootstrapConfig);
|
||||
this.bootstrapMode = bootstrapMode;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Map<BootstrapMode, List<String>> select(List<Pair<String, List<HoodieFileStatus>>> partitions) {
|
||||
return partitions.stream().map(p -> Pair.of(bootstrapMode, p))
|
||||
.collect(Collectors.groupingBy(Pair::getKey, Collectors.mapping(x -> x.getValue().getKey(),
|
||||
Collectors.toList())));
|
||||
}
|
||||
}
|
||||
@@ -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.bootstrap.translator;
|
||||
|
||||
import java.io.Serializable;
|
||||
import org.apache.hudi.common.config.TypedProperties;
|
||||
|
||||
public abstract class BootstrapPartitionPathTranslator implements Serializable {
|
||||
|
||||
private final TypedProperties properties;
|
||||
|
||||
public BootstrapPartitionPathTranslator(TypedProperties properties) {
|
||||
this.properties = properties;
|
||||
}
|
||||
|
||||
/**
|
||||
* Given a bootstrap partition path, translated partition path.
|
||||
*
|
||||
* @param bootStrapPartitionPath bootstrap Partition Path
|
||||
* @return Translated Path
|
||||
*/
|
||||
public abstract String getBootstrapTranslatedPath(String bootStrapPartitionPath);
|
||||
}
|
||||
@@ -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.client.bootstrap.translator;
|
||||
|
||||
import org.apache.hudi.common.config.TypedProperties;
|
||||
|
||||
/**
|
||||
* Return same path as bootstrap partition path.
|
||||
*/
|
||||
public class IdentityBootstrapPartitionPathTranslator extends BootstrapPartitionPathTranslator {
|
||||
|
||||
public IdentityBootstrapPartitionPathTranslator(TypedProperties properties) {
|
||||
super(properties);
|
||||
}
|
||||
|
||||
@Override
|
||||
public String getBootstrapTranslatedPath(String bootStrapPartitionPath) {
|
||||
return bootStrapPartitionPath;
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,33 @@
|
||||
/*
|
||||
* 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.common;
|
||||
|
||||
/**
|
||||
* Properties specific to each engine, that can be set/obtained from.
|
||||
*/
|
||||
public enum EngineProperty {
|
||||
// hostname to bind embedded timeline server to
|
||||
EMBEDDED_SERVER_HOST,
|
||||
// Pool/queue to use to run compaction.
|
||||
COMPACTION_POOL_NAME,
|
||||
// Amount of total memory available to each engine executor
|
||||
TOTAL_MEMORY_AVAILABLE,
|
||||
// Fraction of that memory, that is already in use by the engine
|
||||
MEMORY_FRACTION_IN_USE,
|
||||
}
|
||||
@@ -0,0 +1,71 @@
|
||||
/*
|
||||
* 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.common;
|
||||
|
||||
import org.apache.hudi.common.config.SerializableConfiguration;
|
||||
import org.apache.hudi.client.common.function.SerializableConsumer;
|
||||
import org.apache.hudi.client.common.function.SerializableFunction;
|
||||
import org.apache.hudi.client.common.function.SerializablePairFunction;
|
||||
import org.apache.hudi.common.util.Option;
|
||||
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.stream.Stream;
|
||||
|
||||
/**
|
||||
* Base class contains the context information needed by the engine at runtime. It will be extended by different
|
||||
* engine implementation if needed.
|
||||
*/
|
||||
public abstract class HoodieEngineContext {
|
||||
|
||||
/**
|
||||
* A wrapped hadoop configuration which can be serialized.
|
||||
*/
|
||||
private SerializableConfiguration hadoopConf;
|
||||
|
||||
private TaskContextSupplier taskContextSupplier;
|
||||
|
||||
public HoodieEngineContext(SerializableConfiguration hadoopConf, TaskContextSupplier taskContextSupplier) {
|
||||
this.hadoopConf = hadoopConf;
|
||||
this.taskContextSupplier = taskContextSupplier;
|
||||
}
|
||||
|
||||
public SerializableConfiguration getHadoopConf() {
|
||||
return hadoopConf;
|
||||
}
|
||||
|
||||
public TaskContextSupplier getTaskContextSupplier() {
|
||||
return taskContextSupplier;
|
||||
}
|
||||
|
||||
public abstract <I, O> List<O> map(List<I> data, SerializableFunction<I, O> func, int parallelism);
|
||||
|
||||
public abstract <I, O> List<O> flatMap(List<I> data, SerializableFunction<I, Stream<O>> func, int parallelism);
|
||||
|
||||
public abstract <I> void foreach(List<I> data, SerializableConsumer<I> consumer, int parallelism);
|
||||
|
||||
public abstract <I, K, V> Map<K, V> mapToPair(List<I> data, SerializablePairFunction<I, K, V> func, Integer parallelism);
|
||||
|
||||
public abstract void setProperty(EngineProperty key, String value);
|
||||
|
||||
public abstract Option<String> getProperty(EngineProperty key);
|
||||
|
||||
public abstract void setJobStatus(String activeModule, String activityDescription);
|
||||
|
||||
}
|
||||
@@ -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.client.common;
|
||||
|
||||
import org.apache.hudi.common.util.Option;
|
||||
|
||||
import java.io.Serializable;
|
||||
import java.util.function.Supplier;
|
||||
|
||||
/**
|
||||
* Base task context supplier.
|
||||
*/
|
||||
public abstract class TaskContextSupplier implements Serializable {
|
||||
|
||||
public abstract Supplier<Integer> getPartitionIdSupplier();
|
||||
|
||||
public abstract Supplier<Integer> getStageIdSupplier();
|
||||
|
||||
public abstract Supplier<Long> getAttemptIdSupplier();
|
||||
|
||||
public abstract Option<String> getProperty(EngineProperty prop);
|
||||
}
|
||||
@@ -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.client.common.function;
|
||||
|
||||
import java.io.Serializable;
|
||||
|
||||
/**
|
||||
* A wrapped {@link java.util.function.Consumer} which can be serialized.
|
||||
*
|
||||
* @param <I> input type
|
||||
*/
|
||||
@FunctionalInterface
|
||||
public interface SerializableConsumer<I> extends Serializable {
|
||||
void accept(I t) throws Exception;
|
||||
}
|
||||
@@ -0,0 +1,32 @@
|
||||
/*
|
||||
* 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.common.function;
|
||||
|
||||
import java.io.Serializable;
|
||||
|
||||
/**
|
||||
* A wrapped {@link java.util.function.Function} which can be serialized.
|
||||
*
|
||||
* @param <I> input data type
|
||||
* @param <O> output data type
|
||||
*/
|
||||
@FunctionalInterface
|
||||
public interface SerializableFunction<I, O> extends Serializable {
|
||||
O apply(I v1) throws Exception;
|
||||
}
|
||||
@@ -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.client.common.function;
|
||||
|
||||
import scala.Tuple2;
|
||||
|
||||
import java.io.Serializable;
|
||||
|
||||
/**
|
||||
* A function that returns key-value pairs (Tuple2<K, V>).
|
||||
*/
|
||||
@FunctionalInterface
|
||||
public interface SerializablePairFunction<I, K, V> extends Serializable {
|
||||
Tuple2<K, V> call(I t) throws Exception;
|
||||
}
|
||||
@@ -0,0 +1,107 @@
|
||||
/*
|
||||
* 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 org.apache.hudi.client.common.HoodieEngineContext;
|
||||
import org.apache.hudi.common.config.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 java.io.IOException;
|
||||
|
||||
/**
|
||||
* Timeline Service that runs as part of write client.
|
||||
*/
|
||||
public class EmbeddedTimelineService {
|
||||
|
||||
private static final Logger LOG = 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(HoodieEngineContext context, String embeddedTimelineServiceHostAddr, FileSystemViewStorageConfig config) {
|
||||
setHostAddr(embeddedTimelineServiceHostAddr);
|
||||
this.config = config;
|
||||
this.hadoopConf = context.getHadoopConf();
|
||||
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.newCopy());
|
||||
serverPort = server.startService();
|
||||
LOG.info("Started embedded timeline server at " + hostAddr + ":" + serverPort);
|
||||
}
|
||||
|
||||
private void setHostAddr(String embeddedTimelineServiceHostAddr) {
|
||||
if (embeddedTimelineServiceHostAddr != null) {
|
||||
LOG.info("Overriding hostIp to (" + embeddedTimelineServiceHostAddr + ") found in spark-conf. It was " + this.hostAddr);
|
||||
this.hostAddr = embeddedTimelineServiceHostAddr;
|
||||
} else {
|
||||
LOG.warn("Unable to find driver bind address from spark config");
|
||||
this.hostAddr = NetworkUtils.getHostname();
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Retrieves proper view storage configs for remote clients to access this service.
|
||||
*/
|
||||
public FileSystemViewStorageConfig getRemoteFileSystemViewConfig() {
|
||||
FileSystemViewStorageType viewStorageType = config.shouldEnableBackupForRemoteFileSystemView()
|
||||
? FileSystemViewStorageType.REMOTE_FIRST : FileSystemViewStorageType.REMOTE_ONLY;
|
||||
return FileSystemViewStorageConfig.newBuilder().withStorageType(viewStorageType)
|
||||
.withRemoteServerHost(hostAddr).withRemoteServerPort(serverPort).build();
|
||||
}
|
||||
|
||||
public FileSystemViewManager getViewManager() {
|
||||
return viewManager;
|
||||
}
|
||||
|
||||
public void stop() {
|
||||
if (null != server) {
|
||||
LOG.info("Closing Timeline server");
|
||||
this.server.close();
|
||||
this.server = null;
|
||||
this.viewManager = null;
|
||||
LOG.info("Closed Timeline server");
|
||||
}
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,129 @@
|
||||
/*
|
||||
* 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 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;
|
||||
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,52 @@
|
||||
/*
|
||||
* 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 java.util.Iterator;
|
||||
import java.util.function.Function;
|
||||
import org.apache.avro.generic.GenericRecord;
|
||||
|
||||
import org.apache.hudi.common.util.ValidationUtils;
|
||||
import org.apache.hudi.common.util.collection.Pair;
|
||||
|
||||
public class MergingIterator<T extends GenericRecord> implements Iterator<T> {
|
||||
|
||||
private final Iterator<T> leftIterator;
|
||||
private final Iterator<T> rightIterator;
|
||||
private final Function<Pair<T,T>, T> mergeFunction;
|
||||
|
||||
public MergingIterator(Iterator<T> leftIterator, Iterator<T> rightIterator, Function<Pair<T,T>, T> mergeFunction) {
|
||||
this.leftIterator = leftIterator;
|
||||
this.rightIterator = rightIterator;
|
||||
this.mergeFunction = mergeFunction;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean hasNext() {
|
||||
boolean leftHasNext = leftIterator.hasNext();
|
||||
boolean rightHasNext = rightIterator.hasNext();
|
||||
ValidationUtils.checkArgument(leftHasNext == rightHasNext);
|
||||
return leftHasNext;
|
||||
}
|
||||
|
||||
@Override
|
||||
public T next() {
|
||||
return mergeFunction.apply(Pair.of(leftIterator.next(), rightIterator.next()));
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,144 @@
|
||||
/*
|
||||
* 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 org.apache.hudi.client.bootstrap.BootstrapMode;
|
||||
import org.apache.hudi.client.bootstrap.selector.MetadataOnlyBootstrapModeSelector;
|
||||
import org.apache.hudi.client.bootstrap.translator.IdentityBootstrapPartitionPathTranslator;
|
||||
import org.apache.hudi.common.bootstrap.index.HFileBootstrapIndex;
|
||||
import org.apache.hudi.common.config.DefaultHoodieConfig;
|
||||
|
||||
import java.io.File;
|
||||
import java.io.FileReader;
|
||||
import java.io.IOException;
|
||||
import java.util.Properties;
|
||||
|
||||
/**
|
||||
* Bootstrap specific configs.
|
||||
*/
|
||||
public class HoodieBootstrapConfig extends DefaultHoodieConfig {
|
||||
|
||||
public static final String BOOTSTRAP_BASE_PATH_PROP = "hoodie.bootstrap.base.path";
|
||||
public static final String BOOTSTRAP_MODE_SELECTOR = "hoodie.bootstrap.mode.selector";
|
||||
public static final String FULL_BOOTSTRAP_INPUT_PROVIDER = "hoodie.bootstrap.full.input.provider";
|
||||
public static final String DEFAULT_FULL_BOOTSTRAP_INPUT_PROVIDER = "org.apache.hudi.bootstrap.SparkParquetBootstrapDataProvider";
|
||||
public static final String BOOTSTRAP_KEYGEN_CLASS = "hoodie.bootstrap.keygen.class";
|
||||
public static final String BOOTSTRAP_PARTITION_PATH_TRANSLATOR_CLASS =
|
||||
"hoodie.bootstrap.partitionpath.translator.class";
|
||||
public static final String DEFAULT_BOOTSTRAP_PARTITION_PATH_TRANSLATOR_CLASS =
|
||||
IdentityBootstrapPartitionPathTranslator.class.getName();
|
||||
|
||||
public static final String BOOTSTRAP_PARALLELISM = "hoodie.bootstrap.parallelism";
|
||||
public static final String DEFAULT_BOOTSTRAP_PARALLELISM = "1500";
|
||||
|
||||
// Used By BootstrapRegexModeSelector class. When a partition path matches the regex, the corresponding
|
||||
// mode will be used. Otherwise, the alternative mode will be used.
|
||||
public static final String BOOTSTRAP_MODE_SELECTOR_REGEX = "hoodie.bootstrap.mode.selector.regex";
|
||||
public static final String BOOTSTRAP_MODE_SELECTOR_REGEX_MODE = "hoodie.bootstrap.mode.selector.regex.mode";
|
||||
public static final String DEFAULT_BOOTSTRAP_MODE_SELECTOR_REGEX = ".*";
|
||||
public static final String DEFAULT_BOOTSTRAP_MODE_SELECTOR_REGEX_MODE = BootstrapMode.METADATA_ONLY.name();
|
||||
|
||||
public static final String BOOTSTRAP_INDEX_CLASS_PROP = "hoodie.bootstrap.index.class";
|
||||
public static final String DEFAULT_BOOTSTRAP_INDEX_CLASS = HFileBootstrapIndex.class.getName();
|
||||
|
||||
public HoodieBootstrapConfig(Properties props) {
|
||||
super(props);
|
||||
}
|
||||
|
||||
public static Builder newBuilder() {
|
||||
return new Builder();
|
||||
}
|
||||
|
||||
public static class Builder {
|
||||
|
||||
private final Properties props = new Properties();
|
||||
|
||||
public Builder fromFile(File propertiesFile) throws IOException {
|
||||
try (FileReader reader = new FileReader(propertiesFile)) {
|
||||
this.props.load(reader);
|
||||
return this;
|
||||
}
|
||||
}
|
||||
|
||||
public Builder withBootstrapBasePath(String basePath) {
|
||||
props.setProperty(BOOTSTRAP_BASE_PATH_PROP, basePath);
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder withBootstrapModeSelector(String partitionSelectorClass) {
|
||||
props.setProperty(BOOTSTRAP_MODE_SELECTOR, partitionSelectorClass);
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder withFullBootstrapInputProvider(String partitionSelectorClass) {
|
||||
props.setProperty(FULL_BOOTSTRAP_INPUT_PROVIDER, partitionSelectorClass);
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder withBootstrapKeyGenClass(String keyGenClass) {
|
||||
props.setProperty(BOOTSTRAP_KEYGEN_CLASS, keyGenClass);
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder withBootstrapPartitionPathTranslatorClass(String partitionPathTranslatorClass) {
|
||||
props.setProperty(BOOTSTRAP_PARTITION_PATH_TRANSLATOR_CLASS, partitionPathTranslatorClass);
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder withBootstrapParallelism(int parallelism) {
|
||||
props.setProperty(BOOTSTRAP_PARALLELISM, String.valueOf(parallelism));
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder withBootstrapModeSelectorRegex(String regex) {
|
||||
props.setProperty(BOOTSTRAP_MODE_SELECTOR_REGEX, regex);
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder withBootstrapModeForRegexMatch(BootstrapMode modeForRegexMatch) {
|
||||
props.setProperty(BOOTSTRAP_MODE_SELECTOR_REGEX_MODE, modeForRegexMatch.name());
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder fromProperties(Properties props) {
|
||||
this.props.putAll(props);
|
||||
return this;
|
||||
}
|
||||
|
||||
public HoodieBootstrapConfig build() {
|
||||
HoodieBootstrapConfig config = new HoodieBootstrapConfig(props);
|
||||
setDefaultOnCondition(props, !props.containsKey(BOOTSTRAP_PARALLELISM), BOOTSTRAP_PARALLELISM,
|
||||
DEFAULT_BOOTSTRAP_PARALLELISM);
|
||||
setDefaultOnCondition(props, !props.containsKey(BOOTSTRAP_PARTITION_PATH_TRANSLATOR_CLASS),
|
||||
BOOTSTRAP_PARTITION_PATH_TRANSLATOR_CLASS, DEFAULT_BOOTSTRAP_PARTITION_PATH_TRANSLATOR_CLASS);
|
||||
setDefaultOnCondition(props, !props.containsKey(BOOTSTRAP_MODE_SELECTOR), BOOTSTRAP_MODE_SELECTOR,
|
||||
MetadataOnlyBootstrapModeSelector.class.getCanonicalName());
|
||||
setDefaultOnCondition(props, !props.containsKey(BOOTSTRAP_MODE_SELECTOR_REGEX), BOOTSTRAP_MODE_SELECTOR_REGEX,
|
||||
DEFAULT_BOOTSTRAP_MODE_SELECTOR_REGEX);
|
||||
setDefaultOnCondition(props, !props.containsKey(BOOTSTRAP_MODE_SELECTOR_REGEX_MODE),
|
||||
BOOTSTRAP_MODE_SELECTOR_REGEX_MODE, DEFAULT_BOOTSTRAP_MODE_SELECTOR_REGEX_MODE);
|
||||
BootstrapMode.valueOf(props.getProperty(BOOTSTRAP_MODE_SELECTOR_REGEX_MODE));
|
||||
setDefaultOnCondition(props, !props.containsKey(BOOTSTRAP_INDEX_CLASS_PROP), BOOTSTRAP_INDEX_CLASS_PROP,
|
||||
DEFAULT_BOOTSTRAP_INDEX_CLASS);
|
||||
setDefaultOnCondition(props, !props.containsKey(FULL_BOOTSTRAP_INPUT_PROVIDER), FULL_BOOTSTRAP_INPUT_PROVIDER,
|
||||
DEFAULT_FULL_BOOTSTRAP_INPUT_PROVIDER);
|
||||
return config;
|
||||
}
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,334 @@
|
||||
/*
|
||||
* 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 org.apache.hudi.common.config.DefaultHoodieConfig;
|
||||
import org.apache.hudi.common.model.HoodieCleaningPolicy;
|
||||
import org.apache.hudi.common.model.OverwriteWithLatestAvroPayload;
|
||||
import org.apache.hudi.common.util.ValidationUtils;
|
||||
import org.apache.hudi.table.action.compact.strategy.CompactionStrategy;
|
||||
import org.apache.hudi.table.action.compact.strategy.LogFileSizeBasedCompactionStrategy;
|
||||
|
||||
import javax.annotation.concurrent.Immutable;
|
||||
|
||||
import java.io.File;
|
||||
import java.io.FileReader;
|
||||
import java.io.IOException;
|
||||
import java.util.Properties;
|
||||
|
||||
/**
|
||||
* 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";
|
||||
public static final String ASYNC_CLEAN_PROP = "hoodie.clean.async";
|
||||
|
||||
// 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 CLEANER_INCREMENTAL_MODE = "hoodie.cleaner.incremental.mode";
|
||||
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";
|
||||
// Set true to clean bootstrap source files when necessary
|
||||
public static final String CLEANER_BOOTSTRAP_BASE_FILE_ENABLED = "hoodie.cleaner.delete.bootstrap.base.file";
|
||||
// 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);
|
||||
// Hudi will use the previous commit to calculate the estimated record size by totalBytesWritten/totalRecordsWritten.
|
||||
// If the previous commit is too small to make an accurate estimation, Hudi will search commits in the reverse order,
|
||||
// until find a commit has totalBytesWritten larger than (PARQUET_SMALL_FILE_LIMIT_BYTES * RECORD_SIZE_ESTIMATION_THRESHOLD)
|
||||
public static final String RECORD_SIZE_ESTIMATION_THRESHOLD_PROP = "hoodie.record.size.estimation.threshold";
|
||||
public static final String DEFAULT_RECORD_SIZE_ESTIMATION_THRESHOLD = "1.0";
|
||||
|
||||
/**
|
||||
* 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 guesstimate 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 = OverwriteWithLatestAvroPayload.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_ASYNC_CLEAN = "false";
|
||||
private static final String DEFAULT_INLINE_COMPACT = "false";
|
||||
private static final String DEFAULT_INCREMENTAL_CLEANER = "true";
|
||||
private static final String DEFAULT_INLINE_COMPACT_NUM_DELTA_COMMITS = "5";
|
||||
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);
|
||||
private static final String DEFAULT_CLEANER_BOOTSTRAP_BASE_FILE_ENABLED = "false";
|
||||
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 {
|
||||
try (FileReader reader = new FileReader(propertiesFile)) {
|
||||
this.props.load(reader);
|
||||
return this;
|
||||
}
|
||||
}
|
||||
|
||||
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 withAsyncClean(Boolean asyncClean) {
|
||||
props.setProperty(ASYNC_CLEAN_PROP, String.valueOf(asyncClean));
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder withIncrementalCleaningMode(Boolean incrementalCleaningMode) {
|
||||
props.setProperty(CLEANER_INCREMENTAL_MODE, String.valueOf(incrementalCleaningMode));
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder withInlineCompaction(Boolean inlineCompaction) {
|
||||
props.setProperty(INLINE_COMPACT_PROP, String.valueOf(inlineCompaction));
|
||||
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 compactionRecordSizeEstimateThreshold(double threshold) {
|
||||
props.setProperty(RECORD_SIZE_ESTIMATION_THRESHOLD_PROP, String.valueOf(threshold));
|
||||
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 Builder withCleanBootstrapBaseFileEnabled(Boolean cleanBootstrapSourceFileEnabled) {
|
||||
props.setProperty(CLEANER_BOOTSTRAP_BASE_FILE_ENABLED, String.valueOf(cleanBootstrapSourceFileEnabled));
|
||||
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(ASYNC_CLEAN_PROP), ASYNC_CLEAN_PROP,
|
||||
DEFAULT_ASYNC_CLEAN);
|
||||
setDefaultOnCondition(props, !props.containsKey(CLEANER_INCREMENTAL_MODE), CLEANER_INCREMENTAL_MODE,
|
||||
DEFAULT_INCREMENTAL_CLEANER);
|
||||
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(RECORD_SIZE_ESTIMATION_THRESHOLD_PROP), RECORD_SIZE_ESTIMATION_THRESHOLD_PROP,
|
||||
DEFAULT_RECORD_SIZE_ESTIMATION_THRESHOLD);
|
||||
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);
|
||||
setDefaultOnCondition(props, !props.containsKey(CLEANER_BOOTSTRAP_BASE_FILE_ENABLED),
|
||||
CLEANER_BOOTSTRAP_BASE_FILE_ENABLED, DEFAULT_CLEANER_BOOTSTRAP_BASE_FILE_ENABLED);
|
||||
|
||||
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));
|
||||
ValidationUtils.checkArgument(maxInstantsToKeep > minInstantsToKeep,
|
||||
String.format(
|
||||
"Increase %s=%d to be greater than %s=%d.",
|
||||
HoodieCompactionConfig.MAX_COMMITS_TO_KEEP_PROP, maxInstantsToKeep,
|
||||
HoodieCompactionConfig.MIN_COMMITS_TO_KEEP_PROP, minInstantsToKeep));
|
||||
ValidationUtils.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,266 @@
|
||||
/*
|
||||
* 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 org.apache.hudi.common.config.DefaultHoodieConfig;
|
||||
import org.apache.hudi.index.hbase.DefaultHBaseQPSResourceAllocator;
|
||||
|
||||
import java.io.File;
|
||||
import java.io.FileReader;
|
||||
import java.io.IOException;
|
||||
import java.util.Properties;
|
||||
|
||||
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 final 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 HBASE_MAX_QPS_FRACTION_PROP = "hoodie.index.hbase.max.qps.fraction";
|
||||
|
||||
/**
|
||||
* 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 {
|
||||
try (FileReader reader = new FileReader(propertiesFile)) {
|
||||
this.props.load(reader);
|
||||
return this;
|
||||
}
|
||||
}
|
||||
|
||||
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 hbaseIndexSleepMsBetweenGetBatch(int sleepMsBetweenGetBatch) {
|
||||
props.setProperty(HBASE_SLEEP_MS_GET_BATCH_PROP, String.valueOf(sleepMsBetweenGetBatch));
|
||||
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, 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,
|
||||
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,282 @@
|
||||
/*
|
||||
* 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 org.apache.hudi.common.bloom.BloomFilterTypeCode;
|
||||
import org.apache.hudi.common.config.DefaultHoodieConfig;
|
||||
import org.apache.hudi.index.HoodieIndex;
|
||||
|
||||
import javax.annotation.concurrent.Immutable;
|
||||
|
||||
import java.io.File;
|
||||
import java.io.FileReader;
|
||||
import java.io.IOException;
|
||||
import java.util.Properties;
|
||||
|
||||
/**
|
||||
* 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();
|
||||
|
||||
public static final String INDEX_CLASS_PROP = "hoodie.index.class";
|
||||
public static final String DEFAULT_INDEX_CLASS = "";
|
||||
|
||||
// ***** 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";
|
||||
public static final String BLOOM_INDEX_FILTER_TYPE = "hoodie.bloom.index.filter.type";
|
||||
public static final String DEFAULT_BLOOM_INDEX_FILTER_TYPE = BloomFilterTypeCode.SIMPLE.name();
|
||||
public static final String HOODIE_BLOOM_INDEX_FILTER_DYNAMIC_MAX_ENTRIES = "hoodie.bloom.index.filter.dynamic.max.entries";
|
||||
public static final String DEFAULT_HOODIE_BLOOM_INDEX_FILTER_DYNAMIC_MAX_ENTRIES = "100000";
|
||||
public static final String SIMPLE_INDEX_USE_CACHING_PROP = "hoodie.simple.index.use.caching";
|
||||
public static final String DEFAULT_SIMPLE_INDEX_USE_CACHING = "true";
|
||||
public static final String SIMPLE_INDEX_PARALLELISM_PROP = "hoodie.simple.index.parallelism";
|
||||
public static final String DEFAULT_SIMPLE_INDEX_PARALLELISM = "50";
|
||||
public static final String GLOBAL_SIMPLE_INDEX_PARALLELISM_PROP = "hoodie.global.simple.index.parallelism";
|
||||
public static final String DEFAULT_GLOBAL_SIMPLE_INDEX_PARALLELISM = "100";
|
||||
|
||||
// 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";
|
||||
public static final String SIMPLE_INDEX_INPUT_STORAGE_LEVEL = "hoodie.simple.index.input.storage.level";
|
||||
public static final String DEFAULT_SIMPLE_INDEX_INPUT_STORAGE_LEVEL = "MEMORY_AND_DISK_SER";
|
||||
|
||||
/**
|
||||
* Only applies if index type is GLOBAL_BLOOM.
|
||||
* <p>
|
||||
* When set to true, an update to a record with a different partition from its existing one
|
||||
* will insert the record to the new partition and delete it from the old partition.
|
||||
* <p>
|
||||
* When set to false, a record will be updated to the old partition.
|
||||
*/
|
||||
public static final String BLOOM_INDEX_UPDATE_PARTITION_PATH = "hoodie.bloom.index.update.partition.path";
|
||||
public static final String DEFAULT_BLOOM_INDEX_UPDATE_PARTITION_PATH = "false";
|
||||
|
||||
public static final String SIMPLE_INDEX_UPDATE_PARTITION_PATH = "hoodie.simple.index.update.partition.path";
|
||||
public static final String DEFAULT_SIMPLE_INDEX_UPDATE_PARTITION_PATH = "false";
|
||||
|
||||
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 {
|
||||
try (FileReader reader = new FileReader(propertiesFile)) {
|
||||
this.props.load(reader);
|
||||
return this;
|
||||
}
|
||||
}
|
||||
|
||||
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 withIndexClass(String indexClass) {
|
||||
props.setProperty(INDEX_CLASS_PROP, indexClass);
|
||||
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 Builder withBloomIndexUpdatePartitionPath(boolean updatePartitionPath) {
|
||||
props.setProperty(BLOOM_INDEX_UPDATE_PARTITION_PATH, String.valueOf(updatePartitionPath));
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder withSimpleIndexParallelism(int parallelism) {
|
||||
props.setProperty(SIMPLE_INDEX_PARALLELISM_PROP, String.valueOf(parallelism));
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder simpleIndexUseCaching(boolean useCaching) {
|
||||
props.setProperty(SIMPLE_INDEX_USE_CACHING_PROP, String.valueOf(useCaching));
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder withSimpleIndexInputStorageLevel(String level) {
|
||||
props.setProperty(SIMPLE_INDEX_INPUT_STORAGE_LEVEL, level);
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder withGlobalSimpleIndexParallelism(int parallelism) {
|
||||
props.setProperty(GLOBAL_SIMPLE_INDEX_PARALLELISM_PROP, String.valueOf(parallelism));
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder withGlobalSimpleIndexUpdatePartitionPath(boolean updatePartitionPath) {
|
||||
props.setProperty(SIMPLE_INDEX_UPDATE_PARTITION_PATH, String.valueOf(updatePartitionPath));
|
||||
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(INDEX_CLASS_PROP), INDEX_CLASS_PROP, DEFAULT_INDEX_CLASS);
|
||||
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_UPDATE_PARTITION_PATH),
|
||||
BLOOM_INDEX_UPDATE_PARTITION_PATH, DEFAULT_BLOOM_INDEX_UPDATE_PARTITION_PATH);
|
||||
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);
|
||||
setDefaultOnCondition(props, !props.contains(BLOOM_INDEX_FILTER_TYPE),
|
||||
BLOOM_INDEX_FILTER_TYPE, DEFAULT_BLOOM_INDEX_FILTER_TYPE);
|
||||
setDefaultOnCondition(props, !props.contains(HOODIE_BLOOM_INDEX_FILTER_DYNAMIC_MAX_ENTRIES),
|
||||
HOODIE_BLOOM_INDEX_FILTER_DYNAMIC_MAX_ENTRIES, DEFAULT_HOODIE_BLOOM_INDEX_FILTER_DYNAMIC_MAX_ENTRIES);
|
||||
setDefaultOnCondition(props, !props.containsKey(SIMPLE_INDEX_PARALLELISM_PROP), SIMPLE_INDEX_PARALLELISM_PROP,
|
||||
DEFAULT_SIMPLE_INDEX_PARALLELISM);
|
||||
setDefaultOnCondition(props, !props.containsKey(SIMPLE_INDEX_USE_CACHING_PROP), SIMPLE_INDEX_USE_CACHING_PROP,
|
||||
DEFAULT_SIMPLE_INDEX_USE_CACHING);
|
||||
setDefaultOnCondition(props, !props.containsKey(SIMPLE_INDEX_INPUT_STORAGE_LEVEL), SIMPLE_INDEX_INPUT_STORAGE_LEVEL,
|
||||
DEFAULT_SIMPLE_INDEX_INPUT_STORAGE_LEVEL);
|
||||
setDefaultOnCondition(props, !props.containsKey(GLOBAL_SIMPLE_INDEX_PARALLELISM_PROP), GLOBAL_SIMPLE_INDEX_PARALLELISM_PROP,
|
||||
DEFAULT_GLOBAL_SIMPLE_INDEX_PARALLELISM);
|
||||
setDefaultOnCondition(props, !props.containsKey(SIMPLE_INDEX_UPDATE_PARTITION_PATH),
|
||||
SIMPLE_INDEX_UPDATE_PARTITION_PATH, DEFAULT_SIMPLE_INDEX_UPDATE_PARTITION_PATH);
|
||||
// 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,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.
|
||||
*/
|
||||
|
||||
package org.apache.hudi.config;
|
||||
|
||||
import org.apache.hudi.common.config.DefaultHoodieConfig;
|
||||
|
||||
import javax.annotation.concurrent.Immutable;
|
||||
|
||||
import java.io.File;
|
||||
import java.io.FileReader;
|
||||
import java.io.IOException;
|
||||
import java.util.Properties;
|
||||
|
||||
/**
|
||||
* 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 (1GB) 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;
|
||||
// Minimum memory size (100MB) for the spillable map.
|
||||
public static final long DEFAULT_MIN_MEMORY_FOR_SPILLABLE_MAP_IN_BYTES = 100 * 1024 * 1024L;
|
||||
// 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 {
|
||||
try (FileReader reader = new FileReader(propertiesFile)) {
|
||||
this.props.load(reader);
|
||||
return this;
|
||||
}
|
||||
}
|
||||
|
||||
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 withMaxMemoryMaxSize(long mergeMaxSize, long compactionMaxSize) {
|
||||
props.setProperty(MAX_MEMORY_FOR_MERGE_PROP, String.valueOf(mergeMaxSize));
|
||||
props.setProperty(MAX_MEMORY_FOR_COMPACTION_PROP, String.valueOf(compactionMaxSize));
|
||||
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;
|
||||
}
|
||||
|
||||
public HoodieMemoryConfig build() {
|
||||
HoodieMemoryConfig config = new HoodieMemoryConfig(props);
|
||||
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(MAX_MEMORY_FOR_MERGE_PROP), MAX_MEMORY_FOR_MERGE_PROP,
|
||||
String.valueOf(DEFAULT_MAX_MEMORY_FOR_SPILLABLE_MAP_IN_BYTES));
|
||||
setDefaultOnCondition(props, !props.containsKey(WRITESTATUS_FAILURE_FRACTION_PROP),
|
||||
WRITESTATUS_FAILURE_FRACTION_PROP, String.valueOf(DEFAULT_WRITESTATUS_FAILURE_FRACTION));
|
||||
return config;
|
||||
}
|
||||
}
|
||||
}
|
||||
@@ -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.config;
|
||||
|
||||
import org.apache.hudi.common.config.DefaultHoodieConfig;
|
||||
import org.apache.hudi.metrics.MetricsReporterType;
|
||||
|
||||
import javax.annotation.concurrent.Immutable;
|
||||
|
||||
import java.io.File;
|
||||
import java.io.FileReader;
|
||||
import java.io.IOException;
|
||||
import java.util.Properties;
|
||||
|
||||
/**
|
||||
* 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;
|
||||
|
||||
// Jmx
|
||||
public static final String JMX_PREFIX = METRIC_PREFIX + ".jmx";
|
||||
public static final String JMX_HOST = JMX_PREFIX + ".host";
|
||||
public static final String DEFAULT_JMX_HOST = "localhost";
|
||||
|
||||
public static final String JMX_PORT = JMX_PREFIX + ".port";
|
||||
public static final int DEFAULT_JMX_PORT = 9889;
|
||||
|
||||
public static final String GRAPHITE_METRIC_PREFIX = GRAPHITE_PREFIX + ".metric.prefix";
|
||||
|
||||
// User defined
|
||||
public static final String METRICS_REPORTER_CLASS = METRIC_PREFIX + ".reporter.class";
|
||||
public static final String DEFAULT_METRICS_REPORTER_CLASS = "";
|
||||
|
||||
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 {
|
||||
try (FileReader reader = new FileReader(propertiesFile)) {
|
||||
this.props.load(reader);
|
||||
return this;
|
||||
}
|
||||
}
|
||||
|
||||
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 toJmxHost(String host) {
|
||||
props.setProperty(JMX_HOST, host);
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder onJmxPort(String port) {
|
||||
props.setProperty(JMX_PORT, port);
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder usePrefix(String prefix) {
|
||||
props.setProperty(GRAPHITE_METRIC_PREFIX, prefix);
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder withReporterClass(String className) {
|
||||
props.setProperty(METRICS_REPORTER_CLASS, className);
|
||||
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(JMX_HOST), JMX_HOST,
|
||||
DEFAULT_JMX_HOST);
|
||||
setDefaultOnCondition(props, !props.containsKey(JMX_PORT), JMX_PORT,
|
||||
String.valueOf(DEFAULT_JMX_PORT));
|
||||
MetricsReporterType reporterType = MetricsReporterType.valueOf(props.getProperty(METRICS_REPORTER_TYPE));
|
||||
setDefaultOnCondition(props, reporterType == MetricsReporterType.DATADOG,
|
||||
HoodieMetricsDatadogConfig.newBuilder().fromProperties(props).build());
|
||||
setDefaultOnCondition(props, !props.containsKey(METRICS_REPORTER_CLASS),
|
||||
METRICS_REPORTER_CLASS, DEFAULT_METRICS_REPORTER_CLASS);
|
||||
setDefaultOnCondition(props, reporterType == MetricsReporterType.PROMETHEUS_PUSHGATEWAY,
|
||||
HoodieMetricsPrometheusConfig.newBuilder().fromProperties(props).build());
|
||||
setDefaultOnCondition(props, reporterType == MetricsReporterType.PROMETHEUS,
|
||||
HoodieMetricsPrometheusConfig.newBuilder().fromProperties(props).build());
|
||||
|
||||
return config;
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
@@ -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.config;
|
||||
|
||||
import org.apache.hudi.common.config.DefaultHoodieConfig;
|
||||
|
||||
import javax.annotation.concurrent.Immutable;
|
||||
|
||||
import java.util.Properties;
|
||||
|
||||
import static org.apache.hudi.config.HoodieMetricsConfig.METRIC_PREFIX;
|
||||
|
||||
/**
|
||||
* Configs for Datadog reporter type.
|
||||
* <p>
|
||||
* {@link org.apache.hudi.metrics.MetricsReporterType#DATADOG}
|
||||
*/
|
||||
@Immutable
|
||||
public class HoodieMetricsDatadogConfig extends DefaultHoodieConfig {
|
||||
|
||||
public static final String DATADOG_PREFIX = METRIC_PREFIX + ".datadog";
|
||||
public static final String DATADOG_REPORT_PERIOD_SECONDS = DATADOG_PREFIX + ".report.period.seconds";
|
||||
public static final int DEFAULT_DATADOG_REPORT_PERIOD_SECONDS = 30;
|
||||
public static final String DATADOG_API_SITE = DATADOG_PREFIX + ".api.site";
|
||||
public static final String DATADOG_API_KEY = DATADOG_PREFIX + ".api.key";
|
||||
public static final String DATADOG_API_KEY_SKIP_VALIDATION = DATADOG_PREFIX + ".api.key.skip.validation";
|
||||
public static final boolean DEFAULT_DATADOG_API_KEY_SKIP_VALIDATION = false;
|
||||
public static final String DATADOG_API_KEY_SUPPLIER = DATADOG_PREFIX + ".api.key.supplier";
|
||||
public static final String DATADOG_API_TIMEOUT_SECONDS = DATADOG_PREFIX + ".api.timeout.seconds";
|
||||
public static final int DEFAULT_DATADOG_API_TIMEOUT_SECONDS = 3;
|
||||
public static final String DATADOG_METRIC_PREFIX = DATADOG_PREFIX + ".metric.prefix";
|
||||
public static final String DATADOG_METRIC_HOST = DATADOG_PREFIX + ".metric.host";
|
||||
public static final String DATADOG_METRIC_TAGS = DATADOG_PREFIX + ".metric.tags";
|
||||
|
||||
private HoodieMetricsDatadogConfig(Properties props) {
|
||||
super(props);
|
||||
}
|
||||
|
||||
public static HoodieMetricsDatadogConfig.Builder newBuilder() {
|
||||
return new Builder();
|
||||
}
|
||||
|
||||
public static class Builder {
|
||||
|
||||
private final Properties props = new Properties();
|
||||
|
||||
public Builder fromProperties(Properties props) {
|
||||
this.props.putAll(props);
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder withDatadogReportPeriodSeconds(int period) {
|
||||
props.setProperty(DATADOG_REPORT_PERIOD_SECONDS, String.valueOf(period));
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder withDatadogApiSite(String apiSite) {
|
||||
props.setProperty(DATADOG_API_SITE, apiSite);
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder withDatadogApiKey(String apiKey) {
|
||||
props.setProperty(DATADOG_API_KEY, apiKey);
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder withDatadogApiKeySkipValidation(boolean skip) {
|
||||
props.setProperty(DATADOG_API_KEY_SKIP_VALIDATION, String.valueOf(skip));
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder withDatadogApiKeySupplier(String apiKeySupplier) {
|
||||
props.setProperty(DATADOG_API_KEY_SUPPLIER, apiKeySupplier);
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder withDatadogApiTimeoutSeconds(int timeout) {
|
||||
props.setProperty(DATADOG_API_TIMEOUT_SECONDS, String.valueOf(timeout));
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder withDatadogPrefix(String prefix) {
|
||||
props.setProperty(DATADOG_METRIC_PREFIX, prefix);
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder withDatadogHost(String host) {
|
||||
props.setProperty(DATADOG_METRIC_HOST, host);
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder withDatadogTags(String tags) {
|
||||
props.setProperty(DATADOG_METRIC_TAGS, tags);
|
||||
return this;
|
||||
}
|
||||
|
||||
public HoodieMetricsDatadogConfig build() {
|
||||
HoodieMetricsDatadogConfig config = new HoodieMetricsDatadogConfig(props);
|
||||
setDefaultOnCondition(props, !props.containsKey(DATADOG_REPORT_PERIOD_SECONDS),
|
||||
DATADOG_REPORT_PERIOD_SECONDS,
|
||||
String.valueOf(DEFAULT_DATADOG_REPORT_PERIOD_SECONDS));
|
||||
setDefaultOnCondition(props, !props.containsKey(DATADOG_API_KEY_SKIP_VALIDATION),
|
||||
DATADOG_API_KEY_SKIP_VALIDATION,
|
||||
String.valueOf(DEFAULT_DATADOG_API_KEY_SKIP_VALIDATION));
|
||||
setDefaultOnCondition(props, !props.containsKey(DATADOG_API_TIMEOUT_SECONDS),
|
||||
DATADOG_API_TIMEOUT_SECONDS,
|
||||
String.valueOf(DEFAULT_DATADOG_API_TIMEOUT_SECONDS));
|
||||
return config;
|
||||
}
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,102 @@
|
||||
/*
|
||||
* 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 org.apache.hudi.common.config.DefaultHoodieConfig;
|
||||
|
||||
import java.util.Properties;
|
||||
|
||||
import static org.apache.hudi.config.HoodieMetricsConfig.METRIC_PREFIX;
|
||||
|
||||
public class HoodieMetricsPrometheusConfig extends DefaultHoodieConfig {
|
||||
|
||||
// Prometheus PushGateWay
|
||||
public static final String PUSHGATEWAY_PREFIX = METRIC_PREFIX + ".pushgateway";
|
||||
|
||||
public static final String PUSHGATEWAY_HOST = PUSHGATEWAY_PREFIX + ".host";
|
||||
public static final String DEFAULT_PUSHGATEWAY_HOST = "localhost";
|
||||
|
||||
public static final String PUSHGATEWAY_PORT = PUSHGATEWAY_PREFIX + ".port";
|
||||
public static final int DEFAULT_PUSHGATEWAY_PORT = 9091;
|
||||
|
||||
public static final String PUSHGATEWAY_REPORT_PERIOD_SECONDS = PUSHGATEWAY_PREFIX + ".report.period.seconds";
|
||||
public static final int DEFAULT_PUSHGATEWAY_REPORT_PERIOD_SECONDS = 30;
|
||||
|
||||
public static final String PUSHGATEWAY_DELETE_ON_SHUTDOWN = PUSHGATEWAY_PREFIX + ".delete.on.shutdown";
|
||||
public static final boolean DEFAULT_PUSHGATEWAY_DELETE_ON_SHUTDOWN = true;
|
||||
|
||||
public static final String PUSHGATEWAY_JOB_NAME = PUSHGATEWAY_PREFIX + ".job.name";
|
||||
public static final String DEFAULT_PUSHGATEWAY_JOB_NAME = "";
|
||||
|
||||
public static final String PUSHGATEWAY_RANDOM_JOB_NAME_SUFFIX = PUSHGATEWAY_PREFIX + ".random.job.name.suffix";
|
||||
public static final boolean DEFAULT_PUSHGATEWAY_RANDOM_JOB_NAME_SUFFIX = true;
|
||||
|
||||
|
||||
// Prometheus HttpServer
|
||||
public static final String PROMETHEUS_PREFIX = METRIC_PREFIX + ".prometheus";
|
||||
public static final String PROMETHEUS_PORT = PROMETHEUS_PREFIX + ".port";
|
||||
public static final int DEFAULT_PROMETHEUS_PORT = 9090;
|
||||
|
||||
public HoodieMetricsPrometheusConfig(Properties props) {
|
||||
super(props);
|
||||
}
|
||||
|
||||
public static HoodieMetricsPrometheusConfig.Builder newBuilder() {
|
||||
return new HoodieMetricsPrometheusConfig.Builder();
|
||||
}
|
||||
|
||||
@Override
|
||||
public Properties getProps() {
|
||||
return super.getProps();
|
||||
}
|
||||
|
||||
public static class Builder {
|
||||
|
||||
private Properties props = new Properties();
|
||||
|
||||
public Builder fromProperties(Properties props) {
|
||||
this.props.putAll(props);
|
||||
return this;
|
||||
}
|
||||
|
||||
public HoodieMetricsPrometheusConfig build() {
|
||||
HoodieMetricsPrometheusConfig config = new HoodieMetricsPrometheusConfig(props);
|
||||
setDefaultOnCondition(props, !props.containsKey(PROMETHEUS_PORT), PROMETHEUS_PORT,
|
||||
String.valueOf(DEFAULT_PROMETHEUS_PORT));
|
||||
setDefaultOnCondition(props, !props.containsKey(PUSHGATEWAY_HOST),
|
||||
PUSHGATEWAY_HOST,
|
||||
DEFAULT_PUSHGATEWAY_HOST);
|
||||
setDefaultOnCondition(props, !props.containsKey(PUSHGATEWAY_PORT),
|
||||
PUSHGATEWAY_PORT,
|
||||
String.valueOf(DEFAULT_PUSHGATEWAY_PORT));
|
||||
setDefaultOnCondition(props, !props.containsKey(PUSHGATEWAY_REPORT_PERIOD_SECONDS),
|
||||
PUSHGATEWAY_REPORT_PERIOD_SECONDS,
|
||||
String.valueOf(DEFAULT_PUSHGATEWAY_REPORT_PERIOD_SECONDS));
|
||||
setDefaultOnCondition(props, !props.containsKey(PUSHGATEWAY_DELETE_ON_SHUTDOWN),
|
||||
PUSHGATEWAY_DELETE_ON_SHUTDOWN,
|
||||
String.valueOf(DEFAULT_PUSHGATEWAY_DELETE_ON_SHUTDOWN));
|
||||
setDefaultOnCondition(props, !props.containsKey(PUSHGATEWAY_JOB_NAME),
|
||||
PUSHGATEWAY_JOB_NAME, DEFAULT_PUSHGATEWAY_JOB_NAME);
|
||||
setDefaultOnCondition(props, !props.containsKey(PUSHGATEWAY_RANDOM_JOB_NAME_SUFFIX),
|
||||
PUSHGATEWAY_RANDOM_JOB_NAME_SUFFIX,
|
||||
String.valueOf(DEFAULT_PUSHGATEWAY_RANDOM_JOB_NAME_SUFFIX));
|
||||
return config;
|
||||
}
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,173 @@
|
||||
/*
|
||||
* 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 org.apache.hudi.common.config.DefaultHoodieConfig;
|
||||
|
||||
import javax.annotation.concurrent.Immutable;
|
||||
|
||||
import java.io.File;
|
||||
import java.io.FileReader;
|
||||
import java.io.IOException;
|
||||
import java.util.Properties;
|
||||
|
||||
/**
|
||||
* 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);
|
||||
public static final String HFILE_FILE_MAX_BYTES = "hoodie.hfile.max.file.size";
|
||||
public static final String HFILE_BLOCK_SIZE_BYTES = "hoodie.hfile.block.size";
|
||||
public static final String DEFAULT_HFILE_BLOCK_SIZE_BYTES = String.valueOf(1 * 1024 * 1024);
|
||||
public static final String DEFAULT_HFILE_FILE_MAX_BYTES = String.valueOf(120 * 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";
|
||||
public static final String HFILE_COMPRESSION_ALGORITHM = "hoodie.hfile.compression.algorithm";
|
||||
// Default compression codec for parquet
|
||||
public static final String DEFAULT_PARQUET_COMPRESSION_CODEC = "gzip";
|
||||
public static final String DEFAULT_HFILE_COMPRESSION_ALGORITHM = "GZ";
|
||||
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 {
|
||||
try (FileReader reader = new FileReader(propertiesFile)) {
|
||||
this.props.load(reader);
|
||||
return this;
|
||||
}
|
||||
}
|
||||
|
||||
public Builder fromProperties(Properties props) {
|
||||
this.props.putAll(props);
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder parquetMaxFileSize(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 hfileMaxFileSize(long maxFileSize) {
|
||||
props.setProperty(HFILE_FILE_MAX_BYTES, String.valueOf(maxFileSize));
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder hfileBlockSize(int blockSize) {
|
||||
props.setProperty(HFILE_BLOCK_SIZE_BYTES, String.valueOf(blockSize));
|
||||
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 hfileCompressionAlgorithm(String hfileCompressionAlgorithm) {
|
||||
props.setProperty(HFILE_COMPRESSION_ALGORITHM, hfileCompressionAlgorithm);
|
||||
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);
|
||||
|
||||
setDefaultOnCondition(props, !props.containsKey(HFILE_BLOCK_SIZE_BYTES), HFILE_BLOCK_SIZE_BYTES,
|
||||
DEFAULT_HFILE_BLOCK_SIZE_BYTES);
|
||||
setDefaultOnCondition(props, !props.containsKey(HFILE_COMPRESSION_ALGORITHM), HFILE_COMPRESSION_ALGORITHM,
|
||||
DEFAULT_HFILE_COMPRESSION_ALGORITHM);
|
||||
setDefaultOnCondition(props, !props.containsKey(HFILE_FILE_MAX_BYTES), HFILE_FILE_MAX_BYTES,
|
||||
DEFAULT_HFILE_FILE_MAX_BYTES);
|
||||
|
||||
return config;
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
@@ -0,0 +1,107 @@
|
||||
/*
|
||||
* 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 org.apache.hudi.common.config.DefaultHoodieConfig;
|
||||
|
||||
import java.io.File;
|
||||
import java.io.FileReader;
|
||||
import java.io.IOException;
|
||||
import java.util.Properties;
|
||||
|
||||
/**
|
||||
* Write callback related config.
|
||||
*/
|
||||
public class HoodieWriteCommitCallbackConfig extends DefaultHoodieConfig {
|
||||
|
||||
public static final String CALLBACK_PREFIX = "hoodie.write.commit.callback.";
|
||||
public static final String CALLBACK_ON = CALLBACK_PREFIX + "on";
|
||||
public static final boolean DEFAULT_CALLBACK_ON = false;
|
||||
|
||||
public static final String CALLBACK_CLASS_PROP = CALLBACK_PREFIX + "class";
|
||||
public static final String DEFAULT_CALLBACK_CLASS_PROP = "org.apache.hudi.callback.impl.HoodieWriteCommitHttpCallback";
|
||||
|
||||
// ***** HTTP callback configs *****
|
||||
public static final String CALLBACK_HTTP_URL_PROP = CALLBACK_PREFIX + "http.url";
|
||||
public static final String CALLBACK_HTTP_API_KEY = CALLBACK_PREFIX + "http.api.key";
|
||||
public static final String DEFAULT_CALLBACK_HTTP_API_KEY = "hudi_write_commit_http_callback";
|
||||
public static final String CALLBACK_HTTP_TIMEOUT_SECONDS = CALLBACK_PREFIX + "http.timeout.seconds";
|
||||
public static final int DEFAULT_CALLBACK_HTTP_TIMEOUT_SECONDS = 3;
|
||||
|
||||
private HoodieWriteCommitCallbackConfig(Properties props) {
|
||||
super(props);
|
||||
}
|
||||
|
||||
public static HoodieWriteCommitCallbackConfig.Builder newBuilder() {
|
||||
return new HoodieWriteCommitCallbackConfig.Builder();
|
||||
}
|
||||
|
||||
public static class Builder {
|
||||
|
||||
private final Properties props = new Properties();
|
||||
|
||||
public HoodieWriteCommitCallbackConfig.Builder fromFile(File propertiesFile) throws IOException {
|
||||
try (FileReader reader = new FileReader(propertiesFile)) {
|
||||
this.props.load(reader);
|
||||
return this;
|
||||
}
|
||||
}
|
||||
|
||||
public HoodieWriteCommitCallbackConfig.Builder fromProperties(Properties props) {
|
||||
this.props.putAll(props);
|
||||
return this;
|
||||
}
|
||||
|
||||
public HoodieWriteCommitCallbackConfig.Builder writeCommitCallbackOn(String callbackOn) {
|
||||
props.setProperty(CALLBACK_ON, callbackOn);
|
||||
return this;
|
||||
}
|
||||
|
||||
public HoodieWriteCommitCallbackConfig.Builder withCallbackClass(String callbackClass) {
|
||||
props.setProperty(CALLBACK_CLASS_PROP, callbackClass);
|
||||
return this;
|
||||
}
|
||||
|
||||
public HoodieWriteCommitCallbackConfig.Builder withCallbackHttpUrl(String url) {
|
||||
props.setProperty(CALLBACK_HTTP_URL_PROP, url);
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder withCallbackHttpTimeoutSeconds(String timeoutSeconds) {
|
||||
props.setProperty(CALLBACK_HTTP_TIMEOUT_SECONDS, timeoutSeconds);
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder withCallbackHttpApiKey(String apiKey) {
|
||||
props.setProperty(CALLBACK_HTTP_API_KEY, apiKey);
|
||||
return this;
|
||||
}
|
||||
|
||||
public HoodieWriteCommitCallbackConfig build() {
|
||||
HoodieWriteCommitCallbackConfig config = new HoodieWriteCommitCallbackConfig(props);
|
||||
setDefaultOnCondition(props, !props.containsKey(CALLBACK_ON), CALLBACK_ON, String.valueOf(DEFAULT_CALLBACK_ON));
|
||||
setDefaultOnCondition(props, !props.containsKey(CALLBACK_CLASS_PROP), CALLBACK_CLASS_PROP, DEFAULT_CALLBACK_CLASS_PROP);
|
||||
setDefaultOnCondition(props, !props.containsKey(CALLBACK_HTTP_API_KEY), CALLBACK_HTTP_API_KEY, DEFAULT_CALLBACK_HTTP_API_KEY);
|
||||
setDefaultOnCondition(props, !props.containsKey(CALLBACK_HTTP_TIMEOUT_SECONDS), CALLBACK_HTTP_TIMEOUT_SECONDS,
|
||||
String.valueOf(DEFAULT_CALLBACK_HTTP_TIMEOUT_SECONDS));
|
||||
|
||||
return config;
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
File diff suppressed because it is too large
Load Diff
@@ -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.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,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.exception;
|
||||
|
||||
import org.apache.hudi.callback.HoodieWriteCommitCallback;
|
||||
|
||||
/**
|
||||
* Exception thrown for any higher level errors when {@link HoodieWriteCommitCallback} is executing a callback.
|
||||
*/
|
||||
public class HoodieCommitCallbackException extends HoodieException {
|
||||
|
||||
public HoodieCommitCallbackException(String msg, Throwable e) {
|
||||
super(msg, e);
|
||||
}
|
||||
|
||||
public HoodieCommitCallbackException(String msg) {
|
||||
super(msg);
|
||||
}
|
||||
}
|
||||
@@ -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.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,37 @@
|
||||
/*
|
||||
* 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,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;
|
||||
|
||||
/**
|
||||
* <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,26 @@
|
||||
/*
|
||||
* 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 HoodieRestoreException extends HoodieException {
|
||||
|
||||
public HoodieRestoreException(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,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 HoodieUpgradeDowngradeException extends HoodieException {
|
||||
|
||||
public HoodieUpgradeDowngradeException(String msg, Throwable t) {
|
||||
super(msg, t);
|
||||
}
|
||||
|
||||
public HoodieUpgradeDowngradeException(int fromVersion, int toVersion, boolean upgrade) {
|
||||
super(String.format("Cannot %s from version %s -> %s", upgrade ? "upgrade" : "downgrade", fromVersion, toVersion), null);
|
||||
}
|
||||
}
|
||||
@@ -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.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,116 @@
|
||||
/*
|
||||
* 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.execution;
|
||||
|
||||
import org.apache.hudi.client.WriteStatus;
|
||||
import org.apache.hudi.client.common.TaskContextSupplier;
|
||||
import org.apache.hudi.common.model.HoodieRecord;
|
||||
import org.apache.hudi.common.model.HoodieRecordPayload;
|
||||
import org.apache.hudi.common.util.queue.BoundedInMemoryQueueConsumer;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
import org.apache.hudi.execution.HoodieLazyInsertIterable.HoodieInsertValueGenResult;
|
||||
import org.apache.hudi.io.HoodieWriteHandle;
|
||||
import org.apache.hudi.io.WriteHandleFactory;
|
||||
import org.apache.hudi.table.HoodieTable;
|
||||
|
||||
import java.util.ArrayList;
|
||||
import java.util.HashMap;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
|
||||
/**
|
||||
* Consumes stream of hoodie records from in-memory queue and writes to one or more create-handles.
|
||||
*/
|
||||
public class CopyOnWriteInsertHandler<T extends HoodieRecordPayload>
|
||||
extends BoundedInMemoryQueueConsumer<HoodieInsertValueGenResult<HoodieRecord>, List<WriteStatus>> {
|
||||
|
||||
private HoodieWriteConfig config;
|
||||
private String instantTime;
|
||||
private boolean areRecordsSorted;
|
||||
private HoodieTable hoodieTable;
|
||||
private String idPrefix;
|
||||
private TaskContextSupplier taskContextSupplier;
|
||||
private WriteHandleFactory writeHandleFactory;
|
||||
|
||||
private final List<WriteStatus> statuses = new ArrayList<>();
|
||||
// Stores the open HoodieWriteHandle for each table partition path
|
||||
// If the records are consumed in order, there should be only one open handle in this mapping.
|
||||
// Otherwise, there may be multiple handles.
|
||||
private Map<String, HoodieWriteHandle> handles = new HashMap<>();
|
||||
|
||||
public CopyOnWriteInsertHandler(HoodieWriteConfig config, String instantTime,
|
||||
boolean areRecordsSorted, HoodieTable hoodieTable, String idPrefix,
|
||||
TaskContextSupplier taskContextSupplier,
|
||||
WriteHandleFactory writeHandleFactory) {
|
||||
this.config = config;
|
||||
this.instantTime = instantTime;
|
||||
this.areRecordsSorted = areRecordsSorted;
|
||||
this.hoodieTable = hoodieTable;
|
||||
this.idPrefix = idPrefix;
|
||||
this.taskContextSupplier = taskContextSupplier;
|
||||
this.writeHandleFactory = writeHandleFactory;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void consumeOneRecord(HoodieInsertValueGenResult<HoodieRecord> payload) {
|
||||
final HoodieRecord insertPayload = payload.record;
|
||||
String partitionPath = insertPayload.getPartitionPath();
|
||||
HoodieWriteHandle handle = handles.get(partitionPath);
|
||||
if (handle == null) {
|
||||
// If the records are sorted, this means that we encounter a new partition path
|
||||
// and the records for the previous partition path are all written,
|
||||
// so we can safely closely existing open handle to reduce memory footprint.
|
||||
if (areRecordsSorted) {
|
||||
closeOpenHandles();
|
||||
}
|
||||
// Lazily initialize the handle, for the first time
|
||||
handle = writeHandleFactory.create(config, instantTime, hoodieTable,
|
||||
insertPayload.getPartitionPath(), idPrefix, taskContextSupplier);
|
||||
handles.put(partitionPath, handle);
|
||||
}
|
||||
|
||||
if (!handle.canWrite(payload.record)) {
|
||||
// Handle is full. Close the handle and add the WriteStatus
|
||||
statuses.add(handle.close());
|
||||
// Open new handle
|
||||
handle = writeHandleFactory.create(config, instantTime, hoodieTable,
|
||||
insertPayload.getPartitionPath(), idPrefix, taskContextSupplier);
|
||||
handles.put(partitionPath, handle);
|
||||
}
|
||||
handle.write(insertPayload, payload.insertValue, payload.exception);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void finish() {
|
||||
closeOpenHandles();
|
||||
assert statuses.size() > 0;
|
||||
}
|
||||
|
||||
@Override
|
||||
public List<WriteStatus> getResult() {
|
||||
return statuses;
|
||||
}
|
||||
|
||||
private void closeOpenHandles() {
|
||||
for (HoodieWriteHandle handle : handles.values()) {
|
||||
statuses.add(handle.close());
|
||||
}
|
||||
handles.clear();
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,114 @@
|
||||
/*
|
||||
* 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.execution;
|
||||
|
||||
import org.apache.hudi.client.WriteStatus;
|
||||
import org.apache.hudi.client.utils.LazyIterableIterator;
|
||||
import org.apache.hudi.client.common.TaskContextSupplier;
|
||||
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.config.HoodieWriteConfig;
|
||||
import org.apache.hudi.io.CreateHandleFactory;
|
||||
import org.apache.hudi.io.WriteHandleFactory;
|
||||
import org.apache.hudi.table.HoodieTable;
|
||||
|
||||
import org.apache.avro.Schema;
|
||||
import org.apache.avro.generic.IndexedRecord;
|
||||
|
||||
import java.util.Iterator;
|
||||
import java.util.List;
|
||||
import java.util.function.Function;
|
||||
|
||||
/**
|
||||
* Lazy Iterable, that writes a stream of HoodieRecords sorted by the partitionPath, into new files.
|
||||
*/
|
||||
public abstract class HoodieLazyInsertIterable<T extends HoodieRecordPayload>
|
||||
extends LazyIterableIterator<HoodieRecord<T>, List<WriteStatus>> {
|
||||
|
||||
protected final HoodieWriteConfig hoodieConfig;
|
||||
protected final String instantTime;
|
||||
protected boolean areRecordsSorted;
|
||||
protected final HoodieTable hoodieTable;
|
||||
protected final String idPrefix;
|
||||
protected TaskContextSupplier taskContextSupplier;
|
||||
protected WriteHandleFactory writeHandleFactory;
|
||||
|
||||
public HoodieLazyInsertIterable(Iterator<HoodieRecord<T>> recordItr,
|
||||
boolean areRecordsSorted,
|
||||
HoodieWriteConfig config,
|
||||
String instantTime,
|
||||
HoodieTable hoodieTable,
|
||||
String idPrefix,
|
||||
TaskContextSupplier taskContextSupplier) {
|
||||
this(recordItr, areRecordsSorted, config, instantTime, hoodieTable, idPrefix, taskContextSupplier,
|
||||
new CreateHandleFactory<>());
|
||||
}
|
||||
|
||||
public HoodieLazyInsertIterable(Iterator<HoodieRecord<T>> recordItr, boolean areRecordsSorted,
|
||||
HoodieWriteConfig config, String instantTime, HoodieTable hoodieTable,
|
||||
String idPrefix, TaskContextSupplier taskContextSupplier,
|
||||
WriteHandleFactory writeHandleFactory) {
|
||||
super(recordItr);
|
||||
this.areRecordsSorted = areRecordsSorted;
|
||||
this.hoodieConfig = config;
|
||||
this.instantTime = instantTime;
|
||||
this.hoodieTable = hoodieTable;
|
||||
this.idPrefix = idPrefix;
|
||||
this.taskContextSupplier = taskContextSupplier;
|
||||
this.writeHandleFactory = writeHandleFactory;
|
||||
}
|
||||
|
||||
// Used for caching HoodieRecord along with insertValue. We need this to offload computation work to buffering thread.
|
||||
public 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 void end() {}
|
||||
|
||||
protected CopyOnWriteInsertHandler getInsertHandler() {
|
||||
return new CopyOnWriteInsertHandler(hoodieConfig, instantTime, areRecordsSorted, hoodieTable, idPrefix,
|
||||
taskContextSupplier, writeHandleFactory);
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,28 @@
|
||||
/*
|
||||
* 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.execution.bulkinsert;
|
||||
|
||||
/**
|
||||
* Bulk insert sort mode.
|
||||
*/
|
||||
public enum BulkInsertSortMode {
|
||||
NONE,
|
||||
GLOBAL_SORT,
|
||||
PARTITION_SORT
|
||||
}
|
||||
@@ -0,0 +1,109 @@
|
||||
/*
|
||||
* 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 org.apache.hudi.ApiMaturityLevel;
|
||||
import org.apache.hudi.PublicAPIClass;
|
||||
import org.apache.hudi.PublicAPIMethod;
|
||||
import org.apache.hudi.client.common.HoodieEngineContext;
|
||||
import org.apache.hudi.common.model.FileSlice;
|
||||
import org.apache.hudi.common.model.HoodieKey;
|
||||
import org.apache.hudi.common.model.HoodieRecordPayload;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
import org.apache.hudi.exception.HoodieIndexException;
|
||||
import org.apache.hudi.table.HoodieTable;
|
||||
|
||||
import java.io.Serializable;
|
||||
|
||||
/**
|
||||
* Base class for different types of indexes to determine the mapping from uuid.
|
||||
*
|
||||
* @param <T> Sub type of HoodieRecordPayload
|
||||
* @param <I> Type of inputs
|
||||
* @param <K> Type of keys
|
||||
* @param <O> Type of outputs
|
||||
*/
|
||||
@PublicAPIClass(maturity = ApiMaturityLevel.EVOLVING)
|
||||
public abstract class HoodieIndex<T extends HoodieRecordPayload, I, K, O> implements Serializable {
|
||||
|
||||
protected final HoodieWriteConfig config;
|
||||
|
||||
protected HoodieIndex(HoodieWriteConfig config) {
|
||||
this.config = config;
|
||||
}
|
||||
|
||||
/**
|
||||
* Looks up the index and tags each incoming record with a location of a file that contains the row (if it is actually
|
||||
* present).
|
||||
*/
|
||||
@PublicAPIMethod(maturity = ApiMaturityLevel.STABLE)
|
||||
public abstract I tagLocation(I records, HoodieEngineContext context,
|
||||
HoodieTable<T, I, K, O> 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
|
||||
*/
|
||||
@PublicAPIMethod(maturity = ApiMaturityLevel.STABLE)
|
||||
public abstract O updateLocation(O writeStatusRDD, HoodieEngineContext context,
|
||||
HoodieTable<T, I, K, O> hoodieTable) throws HoodieIndexException;
|
||||
|
||||
/**
|
||||
* Rollback the effects of the commit made at instantTime.
|
||||
*/
|
||||
@PublicAPIMethod(maturity = ApiMaturityLevel.STABLE)
|
||||
public abstract boolean rollbackCommit(String instantTime);
|
||||
|
||||
/**
|
||||
* 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
|
||||
*/
|
||||
@PublicAPIMethod(maturity = ApiMaturityLevel.STABLE)
|
||||
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
|
||||
*/
|
||||
@PublicAPIMethod(maturity = ApiMaturityLevel.EVOLVING)
|
||||
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.
|
||||
*/
|
||||
@PublicAPIMethod(maturity = ApiMaturityLevel.STABLE)
|
||||
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, SIMPLE, GLOBAL_SIMPLE
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,88 @@
|
||||
/*
|
||||
* 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 org.apache.hudi.client.common.HoodieEngineContext;
|
||||
import org.apache.hudi.common.model.HoodieBaseFile;
|
||||
import org.apache.hudi.common.model.HoodieRecord;
|
||||
import org.apache.hudi.common.model.HoodieRecordLocation;
|
||||
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.table.HoodieTable;
|
||||
|
||||
import java.util.ArrayList;
|
||||
import java.util.List;
|
||||
|
||||
import static java.util.stream.Collectors.toList;
|
||||
|
||||
/**
|
||||
* Hoodie Index Utilities.
|
||||
*/
|
||||
public class HoodieIndexUtils {
|
||||
|
||||
/**
|
||||
* Fetches Pair of partition path and {@link HoodieBaseFile}s for interested partitions.
|
||||
*
|
||||
* @param partitions list of partitions of interest
|
||||
* @param context instance of {@link HoodieEngineContext} to use
|
||||
* @param hoodieTable instance of {@link HoodieTable} of interest
|
||||
* @return the list of Pairs of partition path and fileId
|
||||
*/
|
||||
public static List<Pair<String, HoodieBaseFile>> getLatestBaseFilesForAllPartitions(final List<String> partitions,
|
||||
final HoodieEngineContext context,
|
||||
final HoodieTable hoodieTable) {
|
||||
context.setJobStatus(HoodieIndexUtils.class.getSimpleName(), "Load latest base files from all partitions");
|
||||
return context.flatMap(partitions, partitionPath -> {
|
||||
Option<HoodieInstant> latestCommitTime = hoodieTable.getMetaClient().getCommitsTimeline()
|
||||
.filterCompletedInstants().lastInstant();
|
||||
List<Pair<String, HoodieBaseFile>> filteredFiles = new ArrayList<>();
|
||||
if (latestCommitTime.isPresent()) {
|
||||
filteredFiles = hoodieTable.getBaseFileOnlyView()
|
||||
.getLatestBaseFilesBeforeOrOn(partitionPath, latestCommitTime.get().getTimestamp())
|
||||
.map(f -> Pair.of(partitionPath, f))
|
||||
.collect(toList());
|
||||
}
|
||||
return filteredFiles.stream();
|
||||
}, Math.max(partitions.size(), 1));
|
||||
}
|
||||
|
||||
/**
|
||||
* Get tagged record for the passed in {@link HoodieRecord}.
|
||||
*
|
||||
* @param inputRecord instance of {@link HoodieRecord} for which tagging is requested
|
||||
* @param location {@link HoodieRecordLocation} for the passed in {@link HoodieRecord}
|
||||
* @return the tagged {@link HoodieRecord}
|
||||
*/
|
||||
public static HoodieRecord getTaggedRecord(HoodieRecord inputRecord, Option<HoodieRecordLocation> location) {
|
||||
HoodieRecord 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.unseal();
|
||||
record.setCurrentLocation(location.get());
|
||||
record.seal();
|
||||
}
|
||||
return record;
|
||||
}
|
||||
}
|
||||
@@ -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.index.bloom;
|
||||
|
||||
import java.io.Serializable;
|
||||
import java.util.Objects;
|
||||
|
||||
/**
|
||||
* 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 Objects.requireNonNull(minRecordKey).compareTo(recordKey) <= 0
|
||||
&& Objects.requireNonNull(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.equals(that.fileId, fileId) && Objects.equals(that.minRecordKey, minRecordKey)
|
||||
&& Objects.equals(that.maxRecordKey, maxRecordKey);
|
||||
|
||||
}
|
||||
|
||||
@Override
|
||||
public int hashCode() {
|
||||
return Objects.hash(fileId, minRecordKey, maxRecordKey);
|
||||
}
|
||||
|
||||
@Override
|
||||
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,41 @@
|
||||
/*
|
||||
* 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 org.apache.hudi.common.util.collection.Pair;
|
||||
|
||||
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 and partition pair for a given record key and partition path.
|
||||
*
|
||||
* @param partitionPath the partition path of interest
|
||||
* @param recordKey the record key to be looked up
|
||||
* @return the {@link Set} of matching <Partition path, file name> pairs where the record could potentially be
|
||||
* present.
|
||||
*/
|
||||
Set<Pair<String, String>> getMatchingFilesAndPartition(String partitionPath, String recordKey);
|
||||
|
||||
}
|
||||
@@ -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.index.bloom;
|
||||
|
||||
import org.apache.hudi.common.util.collection.Pair;
|
||||
|
||||
import java.util.ArrayList;
|
||||
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 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<>();
|
||||
private final Map<String, String> fileIdToPartitionPathMap = new HashMap<>();
|
||||
|
||||
/**
|
||||
* Instantiates {@link IntervalTreeBasedGlobalIndexFileFilter}.
|
||||
*
|
||||
* @param partitionToFileIndexInfo Map of partition to List of {@link BloomIndexFileInfo}s
|
||||
*/
|
||||
IntervalTreeBasedGlobalIndexFileFilter(final Map<String, List<BloomIndexFileInfo>> partitionToFileIndexInfo) {
|
||||
List<BloomIndexFileInfo> allIndexFiles = new ArrayList<>();
|
||||
|
||||
partitionToFileIndexInfo.forEach((parition, bloomIndexFileInfoList) -> bloomIndexFileInfoList.forEach(file -> {
|
||||
fileIdToPartitionPathMap.put(file.getFileId(), parition);
|
||||
allIndexFiles.add(file);
|
||||
}));
|
||||
|
||||
// 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<Pair<String, String>> getMatchingFilesAndPartition(String partitionPath, String recordKey) {
|
||||
Set<String> matchingFiles = new HashSet<>();
|
||||
matchingFiles.addAll(indexLookUpTree.getMatchingIndexFiles(recordKey));
|
||||
matchingFiles.addAll(filesWithNoRanges);
|
||||
Set<Pair<String, String>> toReturn = new HashSet<>();
|
||||
matchingFiles.forEach(file -> toReturn.add(Pair.of(fileIdToPartitionPathMap.get(file), file)));
|
||||
return toReturn;
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,80 @@
|
||||
/*
|
||||
* 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 org.apache.hudi.common.util.collection.Pair;
|
||||
|
||||
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<Pair<String, String>> getMatchingFilesAndPartition(String partitionPath, String recordKey) {
|
||||
Set<Pair<String, 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)) {
|
||||
partitionToFileIndexLookUpTree.get(partitionPath).getMatchingIndexFiles(recordKey).forEach(file ->
|
||||
toReturn.add(Pair.of(partitionPath, file)));
|
||||
}
|
||||
if (partitionToFilesWithNoRanges.containsKey(partitionPath)) {
|
||||
partitionToFilesWithNoRanges.get(partitionPath).forEach(file ->
|
||||
toReturn.add(Pair.of(partitionPath, file)));
|
||||
}
|
||||
return toReturn;
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,147 @@
|
||||
/*
|
||||
* 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,148 @@
|
||||
/*
|
||||
* 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,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 org.apache.hudi.common.util.collection.Pair;
|
||||
|
||||
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<Pair<String, String>> getMatchingFilesAndPartition(String partitionPath, String recordKey) {
|
||||
Set<Pair<String, String>> toReturn = new HashSet<>();
|
||||
partitionToFileIndexInfo.forEach((partition, bloomIndexFileInfoList) -> bloomIndexFileInfoList.forEach(file -> {
|
||||
if (shouldCompareWithFile(file, recordKey)) {
|
||||
toReturn.add(Pair.of(partition, file.getFileId()));
|
||||
}
|
||||
}));
|
||||
return toReturn;
|
||||
}
|
||||
}
|
||||
@@ -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 org.apache.hudi.common.util.collection.Pair;
|
||||
|
||||
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<Pair<String, String>> getMatchingFilesAndPartition(String partitionPath, String recordKey) {
|
||||
List<BloomIndexFileInfo> indexInfos = partitionToFileIndexInfo.get(partitionPath);
|
||||
Set<Pair<String, 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(Pair.of(partitionPath, 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,52 @@
|
||||
/*
|
||||
* 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 final Logger LOG = 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
|
||||
LOG.info(String.format("Release QPS resources called for %s with default implementation, do nothing",
|
||||
this.hoodieWriteConfig.getHbaseTableName()));
|
||||
}
|
||||
}
|
||||
@@ -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,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.io;
|
||||
|
||||
import org.apache.hudi.client.common.TaskContextSupplier;
|
||||
import org.apache.hudi.common.model.HoodieRecordPayload;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
import org.apache.hudi.table.HoodieTable;
|
||||
|
||||
public class AppendHandleFactory<T extends HoodieRecordPayload, I, K, O> extends WriteHandleFactory<T, I, K, O> {
|
||||
|
||||
@Override
|
||||
public HoodieAppendHandle<T, I, K, O> create(final HoodieWriteConfig hoodieConfig, final String commitTime,
|
||||
final HoodieTable<T, I, K, O> hoodieTable, final String partitionPath,
|
||||
final String fileIdPrefix, final TaskContextSupplier sparkTaskContextSupplier) {
|
||||
|
||||
return new HoodieAppendHandle(hoodieConfig, commitTime, hoodieTable, partitionPath,
|
||||
getNextFileId(fileIdPrefix), sparkTaskContextSupplier);
|
||||
}
|
||||
}
|
||||
@@ -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.io;
|
||||
|
||||
import org.apache.hudi.client.common.TaskContextSupplier;
|
||||
import org.apache.hudi.common.model.HoodieRecordPayload;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
import org.apache.hudi.table.HoodieTable;
|
||||
|
||||
public class CreateHandleFactory<T extends HoodieRecordPayload, I, K, O> extends WriteHandleFactory<T, I, K, O> {
|
||||
|
||||
@Override
|
||||
public HoodieWriteHandle<T, I, K, O> create(final HoodieWriteConfig hoodieConfig, final String commitTime,
|
||||
final HoodieTable<T, I, K, O> hoodieTable, final String partitionPath,
|
||||
final String fileIdPrefix, TaskContextSupplier taskContextSupplier) {
|
||||
|
||||
return new HoodieCreateHandle(hoodieConfig, commitTime, hoodieTable, partitionPath,
|
||||
getNextFileId(fileIdPrefix), taskContextSupplier);
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,348 @@
|
||||
/*
|
||||
* 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.hudi.avro.HoodieAvroUtils;
|
||||
import org.apache.hudi.client.WriteStatus;
|
||||
import org.apache.hudi.client.common.TaskContextSupplier;
|
||||
import org.apache.hudi.common.fs.FSUtils;
|
||||
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.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.model.IOType;
|
||||
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.HoodieDataBlock;
|
||||
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.table.view.TableFileSystemView.SliceView;
|
||||
import org.apache.hudi.common.util.DefaultSizeEstimator;
|
||||
import org.apache.hudi.common.util.Option;
|
||||
import org.apache.hudi.common.util.SizeEstimator;
|
||||
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.avro.generic.GenericRecord;
|
||||
import org.apache.avro.generic.IndexedRecord;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.log4j.LogManager;
|
||||
import org.apache.log4j.Logger;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.HashMap;
|
||||
import java.util.Iterator;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.concurrent.atomic.AtomicLong;
|
||||
|
||||
/**
|
||||
* IO Operation to append data onto an existing file.
|
||||
*/
|
||||
public class HoodieAppendHandle<T extends HoodieRecordPayload, I, K, O> extends HoodieWriteHandle<T, I, K, O> {
|
||||
|
||||
private static final Logger LOG = 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 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;
|
||||
// Total number of bytes written to file
|
||||
private long sizeInBytes = 0;
|
||||
// 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 = new HashMap<>();
|
||||
// Total number of new records inserted into the delta file
|
||||
private long insertRecordsWritten = 0;
|
||||
|
||||
private SizeEstimator<HoodieRecord> sizeEstimator;
|
||||
|
||||
public HoodieAppendHandle(HoodieWriteConfig config, String instantTime, HoodieTable<T, I, K, O> hoodieTable,
|
||||
String partitionPath, String fileId, Iterator<HoodieRecord<T>> recordItr, TaskContextSupplier taskContextSupplier) {
|
||||
super(config, instantTime, partitionPath, fileId, hoodieTable, taskContextSupplier);
|
||||
writeStatus.setStat(new HoodieDeltaWriteStat());
|
||||
this.fileId = fileId;
|
||||
this.recordItr = recordItr;
|
||||
sizeEstimator = new DefaultSizeEstimator();
|
||||
}
|
||||
|
||||
public HoodieAppendHandle(HoodieWriteConfig config, String instantTime, HoodieTable<T, I, K, O> hoodieTable,
|
||||
String partitionPath, String fileId, TaskContextSupplier sparkTaskContextSupplier) {
|
||||
this(config, instantTime, hoodieTable, partitionPath, fileId, null, sparkTaskContextSupplier);
|
||||
}
|
||||
|
||||
private void init(HoodieRecord record) {
|
||||
if (doInit) {
|
||||
// extract some information from the first record
|
||||
SliceView rtView = hoodieTable.getSliceView();
|
||||
Option<FileSlice> fileSlice = rtView.getLatestFileSlice(partitionPath, fileId);
|
||||
// Set the base commit time as the current instantTime for new inserts into log files
|
||||
String baseInstantTime;
|
||||
if (fileSlice.isPresent()) {
|
||||
baseInstantTime = fileSlice.get().getBaseInstantTime();
|
||||
} else {
|
||||
baseInstantTime = instantTime;
|
||||
// This means there is no base data file, start appending to a new log file
|
||||
fileSlice = Option.of(new FileSlice(partitionPath, baseInstantTime, this.fileId));
|
||||
LOG.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.sizeEstimate(record);
|
||||
try {
|
||||
//save hoodie partition meta in the partition path
|
||||
HoodiePartitionMetadata partitionMetadata = new HoodiePartitionMetadata(fs, baseInstantTime,
|
||||
new Path(config.getBasePath()), FSUtils.getPartitionPath(config.getBasePath(), partitionPath));
|
||||
partitionMetadata.trySave(getPartitionId());
|
||||
|
||||
// Since the actual log file written to can be different based on when rollover happens, we use the
|
||||
// base file to denote some log appends happened on a slice. writeToken will still fence concurrent
|
||||
// writers.
|
||||
createMarkerFile(partitionPath, FSUtils.makeDataFileName(baseInstantTime, writeToken, fileId, hoodieTable.getBaseFileExtension()));
|
||||
|
||||
this.writer = createLogWriter(fileSlice, baseInstantTime);
|
||||
this.currentLogFile = writer.getLogFile();
|
||||
((HoodieDeltaWriteStat) writeStatus.getStat()).setLogVersion(currentLogFile.getLogVersion());
|
||||
((HoodieDeltaWriteStat) writeStatus.getStat()).setLogOffset(writer.getCurrentSize());
|
||||
} catch (Exception e) {
|
||||
LOG.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 = partitionPath.length() == 0 ? new Path(writer.getLogFile().getFileName())
|
||||
: 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(writerSchema);
|
||||
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, 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) {
|
||||
LOG.error("Error writing record " + hoodieRecord, e);
|
||||
writeStatus.markFailure(hoodieRecord, e, recordMetadata);
|
||||
}
|
||||
return Option.empty();
|
||||
}
|
||||
|
||||
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, writerSchemaWithMetafields.toString());
|
||||
if (recordList.size() > 0) {
|
||||
writer = writer.appendBlock(HoodieDataBlock.getBlock(hoodieTable.getLogDataBlockFormat(), recordList, header));
|
||||
recordList.clear();
|
||||
}
|
||||
if (keysToDelete.size() > 0) {
|
||||
writer = writer.appendBlock(new HoodieDeleteBlock(keysToDelete.toArray(new HoodieKey[keysToDelete.size()]), 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);
|
||||
LOG.error("Error writing record " + record, t);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public WriteStatus close() {
|
||||
try {
|
||||
// flush any remaining records to disk
|
||||
doAppend(header);
|
||||
|
||||
if (writer != null) {
|
||||
sizeInBytes = writer.getCurrentSize();
|
||||
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);
|
||||
|
||||
LOG.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;
|
||||
}
|
||||
|
||||
@Override
|
||||
public IOType getIOType() {
|
||||
return IOType.APPEND;
|
||||
}
|
||||
|
||||
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) {
|
||||
if (!partitionPath.equals(record.getPartitionPath())) {
|
||||
HoodieUpsertException failureEx = new HoodieUpsertException("mismatched partition path, record partition: "
|
||||
+ record.getPartitionPath() + " but trying to insert into partition: " + partitionPath);
|
||||
writeStatus.markFailure(record, failureEx, record.getData().getMetadata());
|
||||
return;
|
||||
}
|
||||
|
||||
// update the new location of the record, so we know where to find it next
|
||||
record.unseal();
|
||||
record.setNewLocation(new HoodieRecordLocation(instantTime, fileId));
|
||||
record.seal();
|
||||
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
|
||||
LOG.info("AvgRecordSize => " + averageRecordSize);
|
||||
averageRecordSize = (averageRecordSize + sizeEstimator.sizeEstimate(record)) / 2;
|
||||
doAppend(header);
|
||||
estimatedNumberOfBytesWritten += averageRecordSize * numberOfRecords;
|
||||
numberOfRecords = 0;
|
||||
}
|
||||
}
|
||||
}
|
||||
@@ -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.io;
|
||||
|
||||
import org.apache.hudi.avro.HoodieAvroUtils;
|
||||
import org.apache.hudi.client.common.TaskContextSupplier;
|
||||
import org.apache.hudi.common.model.HoodieRecord;
|
||||
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;
|
||||
|
||||
/**
|
||||
* This class is essentially same as Create Handle but overrides two things
|
||||
* 1) Schema : Metadata bootstrap writes only metadata fields as part of write. So, setup the writer schema accordingly.
|
||||
* 2) canWrite is overridden to always return true so that skeleton file and bootstrap file is aligned and we don't end up
|
||||
* writing more than 1 skeleton file for the same bootstrap file.
|
||||
* @param <T> HoodieRecordPayload
|
||||
*/
|
||||
public class HoodieBootstrapHandle<T extends HoodieRecordPayload, I, K, O> extends HoodieCreateHandle<T, I, K, O> {
|
||||
|
||||
public HoodieBootstrapHandle(HoodieWriteConfig config, String commitTime, HoodieTable<T, I, K, O> hoodieTable,
|
||||
String partitionPath, String fileId, TaskContextSupplier taskContextSupplier) {
|
||||
super(config, commitTime, hoodieTable, partitionPath, fileId,
|
||||
Pair.of(HoodieAvroUtils.RECORD_KEY_SCHEMA,
|
||||
HoodieAvroUtils.addMetadataFields(HoodieAvroUtils.RECORD_KEY_SCHEMA)), taskContextSupplier);
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean canWrite(HoodieRecord record) {
|
||||
return true;
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,211 @@
|
||||
/*
|
||||
* 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.avro.Schema;
|
||||
import org.apache.hudi.client.WriteStatus;
|
||||
import org.apache.hudi.client.common.TaskContextSupplier;
|
||||
import org.apache.hudi.common.fs.FSUtils;
|
||||
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.model.IOType;
|
||||
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.HoodieInsertException;
|
||||
import org.apache.hudi.io.storage.HoodieFileWriter;
|
||||
import org.apache.hudi.io.storage.HoodieFileWriterFactory;
|
||||
import org.apache.hudi.table.HoodieTable;
|
||||
|
||||
import org.apache.avro.generic.GenericRecord;
|
||||
import org.apache.avro.generic.IndexedRecord;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.log4j.LogManager;
|
||||
import org.apache.log4j.Logger;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.Iterator;
|
||||
import java.util.Map;
|
||||
|
||||
public class HoodieCreateHandle<T extends HoodieRecordPayload, I, K, O> extends HoodieWriteHandle<T, I, K, O> {
|
||||
|
||||
private static final Logger LOG = LogManager.getLogger(HoodieCreateHandle.class);
|
||||
|
||||
private final HoodieFileWriter<IndexedRecord> fileWriter;
|
||||
private final Path path;
|
||||
private long recordsWritten = 0;
|
||||
private long insertRecordsWritten = 0;
|
||||
private long recordsDeleted = 0;
|
||||
private Map<String, HoodieRecord<T>> recordMap;
|
||||
private boolean useWriterSchema = false;
|
||||
|
||||
public HoodieCreateHandle(HoodieWriteConfig config, String instantTime, HoodieTable<T, I, K, O> hoodieTable,
|
||||
String partitionPath, String fileId, TaskContextSupplier taskContextSupplier) {
|
||||
this(config, instantTime, hoodieTable, partitionPath, fileId, getWriterSchemaIncludingAndExcludingMetadataPair(config),
|
||||
taskContextSupplier);
|
||||
}
|
||||
|
||||
public HoodieCreateHandle(HoodieWriteConfig config, String instantTime, HoodieTable<T, I, K, O> hoodieTable,
|
||||
String partitionPath, String fileId, Pair<Schema, Schema> writerSchemaIncludingAndExcludingMetadataPair,
|
||||
TaskContextSupplier taskContextSupplier) {
|
||||
super(config, instantTime, partitionPath, fileId, hoodieTable, writerSchemaIncludingAndExcludingMetadataPair,
|
||||
taskContextSupplier);
|
||||
writeStatus.setFileId(fileId);
|
||||
writeStatus.setPartitionPath(partitionPath);
|
||||
|
||||
this.path = makeNewPath(partitionPath);
|
||||
|
||||
try {
|
||||
HoodiePartitionMetadata partitionMetadata = new HoodiePartitionMetadata(fs, instantTime,
|
||||
new Path(config.getBasePath()), FSUtils.getPartitionPath(config.getBasePath(), partitionPath));
|
||||
partitionMetadata.trySave(getPartitionId());
|
||||
createMarkerFile(partitionPath, FSUtils.makeDataFileName(this.instantTime, this.writeToken, this.fileId, hoodieTable.getBaseFileExtension()));
|
||||
this.fileWriter = HoodieFileWriterFactory.getFileWriter(instantTime, path, hoodieTable, config, writerSchemaWithMetafields, this.taskContextSupplier);
|
||||
} catch (IOException e) {
|
||||
throw new HoodieInsertException("Failed to initialize HoodieStorageWriter for path " + path, e);
|
||||
}
|
||||
LOG.info("New CreateHandle for partition :" + partitionPath + " with fileId " + fileId);
|
||||
}
|
||||
|
||||
/**
|
||||
* Called by the compactor code path.
|
||||
*/
|
||||
public HoodieCreateHandle(HoodieWriteConfig config, String instantTime, HoodieTable<T, I, K, O> hoodieTable,
|
||||
String partitionPath, String fileId, Map<String, HoodieRecord<T>> recordMap,
|
||||
TaskContextSupplier taskContextSupplier) {
|
||||
this(config, instantTime, hoodieTable, partitionPath, fileId, taskContextSupplier);
|
||||
this.recordMap = recordMap;
|
||||
this.useWriterSchema = true;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean canWrite(HoodieRecord record) {
|
||||
return fileWriter.canWrite() && record.getPartitionPath().equals(writeStatus.getPartitionPath());
|
||||
}
|
||||
|
||||
/**
|
||||
* Perform the actual writing of the given record into the backing file.
|
||||
*/
|
||||
@Override
|
||||
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());
|
||||
fileWriter.writeAvroWithMetadata(recordWithMetadataInSchema, record);
|
||||
// update the new location of record, so we know where to find it next
|
||||
record.unseal();
|
||||
record.setNewLocation(new HoodieRecordLocation(instantTime, writeStatus.getFileId()));
|
||||
record.seal();
|
||||
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);
|
||||
LOG.error("Error writing record " + record, t);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Writes all records passed.
|
||||
*/
|
||||
public void write() {
|
||||
Iterator<String> keyIterator;
|
||||
if (hoodieTable.requireSortedRecords()) {
|
||||
// Sorting the keys limits the amount of extra memory required for writing sorted records
|
||||
keyIterator = recordMap.keySet().stream().sorted().iterator();
|
||||
} else {
|
||||
keyIterator = recordMap.keySet().stream().iterator();
|
||||
}
|
||||
try {
|
||||
while (keyIterator.hasNext()) {
|
||||
final String key = keyIterator.next();
|
||||
HoodieRecord<T> record = recordMap.get(key);
|
||||
if (useWriterSchema) {
|
||||
write(record, record.getData().getInsertValue(writerSchemaWithMetafields));
|
||||
} else {
|
||||
write(record, record.getData().getInsertValue(writerSchema));
|
||||
}
|
||||
}
|
||||
} catch (IOException io) {
|
||||
throw new HoodieInsertException("Failed to insert records for path " + path, io);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public WriteStatus getWriteStatus() {
|
||||
return writeStatus;
|
||||
}
|
||||
|
||||
@Override
|
||||
public IOType getIOType() {
|
||||
return IOType.CREATE;
|
||||
}
|
||||
|
||||
/**
|
||||
* Performs actions to durably, persist the current changes and returns a WriteStatus object.
|
||||
*/
|
||||
@Override
|
||||
public WriteStatus close() {
|
||||
LOG
|
||||
.info("Closing the file " + writeStatus.getFileId() + " as we are done with all the records " + recordsWritten);
|
||||
try {
|
||||
|
||||
fileWriter.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);
|
||||
|
||||
LOG.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.hudi.common.model.HoodieRecordPayload;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
import org.apache.hudi.table.HoodieTable;
|
||||
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
|
||||
public abstract class HoodieIOHandle<T extends HoodieRecordPayload, I, K, O> {
|
||||
|
||||
protected final String instantTime;
|
||||
protected final HoodieWriteConfig config;
|
||||
protected final FileSystem fs;
|
||||
protected final HoodieTable<T, I, K, O> hoodieTable;
|
||||
|
||||
HoodieIOHandle(HoodieWriteConfig config, String instantTime, HoodieTable<T, I, K, O> hoodieTable) {
|
||||
this.instantTime = instantTime;
|
||||
this.config = config;
|
||||
this.hoodieTable = hoodieTable;
|
||||
this.fs = getFileSystem();
|
||||
}
|
||||
|
||||
protected abstract FileSystem getFileSystem();
|
||||
}
|
||||
@@ -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.io;
|
||||
|
||||
import org.apache.hudi.common.model.HoodieBaseFile;
|
||||
import org.apache.hudi.common.model.HoodieKey;
|
||||
import org.apache.hudi.common.model.HoodieRecordLocation;
|
||||
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;
|
||||
|
||||
import org.apache.hadoop.fs.Path;
|
||||
|
||||
import java.util.Iterator;
|
||||
|
||||
import scala.Tuple2;
|
||||
|
||||
/**
|
||||
* {@link HoodieRecordLocation} fetch handle for all records from {@link HoodieBaseFile} of interest.
|
||||
*
|
||||
* @param <T>
|
||||
*/
|
||||
public class HoodieKeyLocationFetchHandle<T extends HoodieRecordPayload, I, K, O> extends HoodieReadHandle<T, I, K, O> {
|
||||
|
||||
private final Pair<String, HoodieBaseFile> partitionPathBaseFilePair;
|
||||
|
||||
public HoodieKeyLocationFetchHandle(HoodieWriteConfig config, HoodieTable<T, I, K, O> hoodieTable,
|
||||
Pair<String, HoodieBaseFile> partitionPathBaseFilePair) {
|
||||
super(config, null, hoodieTable, Pair.of(partitionPathBaseFilePair.getLeft(), partitionPathBaseFilePair.getRight().getFileId()));
|
||||
this.partitionPathBaseFilePair = partitionPathBaseFilePair;
|
||||
}
|
||||
|
||||
public Iterator<Tuple2<HoodieKey, HoodieRecordLocation>> locations() {
|
||||
HoodieBaseFile baseFile = partitionPathBaseFilePair.getRight();
|
||||
return ParquetUtils.fetchRecordKeyPartitionPathFromParquet(hoodieTable.getHadoopConf(), new Path(baseFile.getPath())).stream()
|
||||
.map(entry -> new Tuple2<>(entry,
|
||||
new HoodieRecordLocation(baseFile.getCommitTime(), baseFile.getFileId()))).iterator();
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,163 @@
|
||||
/*
|
||||
* 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.hudi.common.bloom.BloomFilter;
|
||||
import org.apache.hudi.common.model.HoodieBaseFile;
|
||||
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.collection.Pair;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
import org.apache.hudi.exception.HoodieIndexException;
|
||||
import org.apache.hudi.table.HoodieTable;
|
||||
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.log4j.LogManager;
|
||||
import org.apache.log4j.Logger;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.HashSet;
|
||||
import java.util.List;
|
||||
import java.util.Set;
|
||||
|
||||
/**
|
||||
* Takes a bunch of keys and returns ones that are present in the file group.
|
||||
*/
|
||||
public class HoodieKeyLookupHandle<T extends HoodieRecordPayload, I, K, O> extends HoodieReadHandle<T, I, K, O> {
|
||||
|
||||
private static final Logger LOG = 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, I, K, O> 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();
|
||||
|
||||
try {
|
||||
this.bloomFilter = createNewFileReader().readBloomFilter();
|
||||
} catch (IOException e) {
|
||||
throw new HoodieIndexException(String.format("Error reading bloom filter from %s: %s", partitionPathFilePair, e));
|
||||
}
|
||||
LOG.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 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 = createNewFileReader().filterRowKeys(new HashSet<>(candidateRecordKeys));
|
||||
foundRecordKeys.addAll(fileRowKeys);
|
||||
LOG.info(String.format("Checked keys against file %s, in %d ms. #candidates (%d) #found (%d)", filePath,
|
||||
timer.endTimer(), candidateRecordKeys.size(), foundRecordKeys.size()));
|
||||
if (LOG.isDebugEnabled()) {
|
||||
LOG.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 (LOG.isDebugEnabled()) {
|
||||
LOG.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 (LOG.isDebugEnabled()) {
|
||||
LOG.debug("#The candidate row keys for " + partitionPathFilePair + " => " + candidateRecordKeys);
|
||||
}
|
||||
|
||||
HoodieBaseFile dataFile = getLatestDataFile();
|
||||
List<String> matchingKeys =
|
||||
checkCandidatesAgainstFile(hoodieTable.getHadoopConf(), candidateRecordKeys, new Path(dataFile.getPath()));
|
||||
LOG.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,326 @@
|
||||
/*
|
||||
* 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.hudi.client.WriteStatus;
|
||||
import org.apache.hudi.client.common.TaskContextSupplier;
|
||||
import org.apache.hudi.common.fs.FSUtils;
|
||||
import org.apache.hudi.common.model.HoodieBaseFile;
|
||||
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.model.IOType;
|
||||
import org.apache.hudi.common.util.DefaultSizeEstimator;
|
||||
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.HoodieFileWriter;
|
||||
import org.apache.hudi.table.HoodieTable;
|
||||
|
||||
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.log4j.LogManager;
|
||||
import org.apache.log4j.Logger;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.HashSet;
|
||||
import java.util.Iterator;
|
||||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
|
||||
@SuppressWarnings("Duplicates")
|
||||
public class HoodieMergeHandle<T extends HoodieRecordPayload, I, K, O> extends HoodieWriteHandle<T, I, K, O> {
|
||||
|
||||
private static final Logger LOG = LogManager.getLogger(HoodieMergeHandle.class);
|
||||
|
||||
protected Map<String, HoodieRecord<T>> keyToNewRecords;
|
||||
protected Set<String> writtenRecordKeys;
|
||||
private HoodieFileWriter<IndexedRecord> fileWriter;
|
||||
|
||||
private Path newFilePath;
|
||||
private Path oldFilePath;
|
||||
private long recordsWritten = 0;
|
||||
private long recordsDeleted = 0;
|
||||
private long updatedRecordsWritten = 0;
|
||||
protected long insertRecordsWritten = 0;
|
||||
protected boolean useWriterSchema;
|
||||
private HoodieBaseFile baseFileToMerge;
|
||||
|
||||
public HoodieMergeHandle(HoodieWriteConfig config, String instantTime, HoodieTable<T, I, K, O> hoodieTable,
|
||||
Iterator<HoodieRecord<T>> recordItr, String partitionPath, String fileId,
|
||||
TaskContextSupplier taskContextSupplier) {
|
||||
super(config, instantTime, partitionPath, fileId, hoodieTable, taskContextSupplier);
|
||||
init(fileId, recordItr);
|
||||
init(fileId, partitionPath, hoodieTable.getBaseFileOnlyView().getLatestBaseFile(partitionPath, fileId).get());
|
||||
}
|
||||
|
||||
/**
|
||||
* Called by compactor code path.
|
||||
*/
|
||||
public HoodieMergeHandle(HoodieWriteConfig config, String instantTime, HoodieTable<T, I, K, O> hoodieTable,
|
||||
Map<String, HoodieRecord<T>> keyToNewRecords, String partitionPath, String fileId,
|
||||
HoodieBaseFile dataFileToBeMerged, TaskContextSupplier taskContextSupplier) {
|
||||
super(config, instantTime, partitionPath, fileId, hoodieTable, taskContextSupplier);
|
||||
this.keyToNewRecords = keyToNewRecords;
|
||||
this.useWriterSchema = true;
|
||||
init(fileId, this.partitionPath, dataFileToBeMerged);
|
||||
}
|
||||
|
||||
@Override
|
||||
public Schema getWriterSchemaWithMetafields() {
|
||||
return writerSchemaWithMetafields;
|
||||
}
|
||||
|
||||
public Schema getWriterSchema() {
|
||||
return writerSchema;
|
||||
}
|
||||
|
||||
/**
|
||||
* Extract old file path, initialize StorageWriter and WriteStatus.
|
||||
*/
|
||||
private void init(String fileId, String partitionPath, HoodieBaseFile baseFileToMerge) {
|
||||
LOG.info("partitionPath:" + partitionPath + ", fileId to be merged:" + fileId);
|
||||
this.baseFileToMerge = baseFileToMerge;
|
||||
this.writtenRecordKeys = new HashSet<>();
|
||||
writeStatus.setStat(new HoodieWriteStat());
|
||||
try {
|
||||
String latestValidFilePath = baseFileToMerge.getFileName();
|
||||
writeStatus.getStat().setPrevCommit(FSUtils.getCommitTime(latestValidFilePath));
|
||||
|
||||
HoodiePartitionMetadata partitionMetadata = new HoodiePartitionMetadata(fs, instantTime,
|
||||
new Path(config.getBasePath()), FSUtils.getPartitionPath(config.getBasePath(), partitionPath));
|
||||
partitionMetadata.trySave(getPartitionId());
|
||||
|
||||
oldFilePath = new Path(config.getBasePath() + "/" + partitionPath + "/" + latestValidFilePath);
|
||||
String newFileName = FSUtils.makeDataFileName(instantTime, writeToken, fileId, hoodieTable.getBaseFileExtension());
|
||||
String relativePath = new Path((partitionPath.isEmpty() ? "" : partitionPath + "/")
|
||||
+ newFileName).toString();
|
||||
newFilePath = new Path(config.getBasePath(), relativePath);
|
||||
|
||||
LOG.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, newFileName);
|
||||
|
||||
// Create the writer for writing the new version file
|
||||
fileWriter = createNewFileWriter(instantTime, newFilePath, hoodieTable, config, writerSchemaWithMetafields, taskContextSupplier);
|
||||
} catch (IOException io) {
|
||||
LOG.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 void init(String fileId, Iterator<HoodieRecord<T>> newRecordsItr) {
|
||||
try {
|
||||
// Load the new records in a map
|
||||
long memoryForMerge = IOUtils.getMaxMemoryPerPartitionMerge(taskContextSupplier, config.getProps());
|
||||
LOG.info("MaxMemoryPerPartitionMerge => " + memoryForMerge);
|
||||
this.keyToNewRecords = new ExternalSpillableMap<>(memoryForMerge, config.getSpillableMapBasePath(),
|
||||
new DefaultSizeEstimator(), new HoodieRecordSizeEstimator(writerSchema));
|
||||
} catch (IOException io) {
|
||||
throw new HoodieIOException("Cannot instantiate an ExternalSpillableMap", io);
|
||||
}
|
||||
while (newRecordsItr.hasNext()) {
|
||||
HoodieRecord<T> record = newRecordsItr.next();
|
||||
// update the new location of the record, so we know where to find it next
|
||||
record.unseal();
|
||||
record.setNewLocation(new HoodieRecordLocation(instantTime, fileId));
|
||||
record.seal();
|
||||
// NOTE: Once Records are added to map (spillable-map), DO NOT change it as they won't persist
|
||||
keyToNewRecords.put(record.getRecordKey(), record);
|
||||
}
|
||||
LOG.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());
|
||||
}
|
||||
|
||||
private boolean writeUpdateRecord(HoodieRecord<T> hoodieRecord, Option<IndexedRecord> indexedRecord) {
|
||||
if (indexedRecord.isPresent()) {
|
||||
updatedRecordsWritten++;
|
||||
}
|
||||
return writeRecord(hoodieRecord, indexedRecord);
|
||||
}
|
||||
|
||||
protected boolean writeRecord(HoodieRecord<T> hoodieRecord, Option<IndexedRecord> indexedRecord) {
|
||||
Option recordMetadata = hoodieRecord.getData().getMetadata();
|
||||
if (!partitionPath.equals(hoodieRecord.getPartitionPath())) {
|
||||
HoodieUpsertException failureEx = new HoodieUpsertException("mismatched partition path, record partition: "
|
||||
+ hoodieRecord.getPartitionPath() + " but trying to insert into partition: " + partitionPath);
|
||||
writeStatus.markFailure(hoodieRecord, failureEx, recordMetadata);
|
||||
return false;
|
||||
}
|
||||
try {
|
||||
if (indexedRecord.isPresent()) {
|
||||
// Convert GenericRecord to GenericRecord with hoodie commit metadata in schema
|
||||
IndexedRecord recordWithMetadataInSchema = rewriteRecord((GenericRecord) indexedRecord.get());
|
||||
fileWriter.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) {
|
||||
LOG.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 ? writerSchemaWithMetafields : writerSchema);
|
||||
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 {
|
||||
fileWriter.writeAvro(key, oldRecord);
|
||||
} catch (ClassCastException e) {
|
||||
LOG.error("Schema mismatch when rewriting old record " + oldRecord + " from file " + getOldFilePath()
|
||||
+ " to file " + newFilePath + " with writerSchema " + writerSchemaWithMetafields.toString(true));
|
||||
throw new HoodieUpsertException(errMsg, e);
|
||||
} catch (IOException e) {
|
||||
LOG.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)
|
||||
Iterator<HoodieRecord<T>> newRecordsItr = (keyToNewRecords instanceof ExternalSpillableMap)
|
||||
? ((ExternalSpillableMap)keyToNewRecords).iterator() : keyToNewRecords.values().iterator();
|
||||
while (newRecordsItr.hasNext()) {
|
||||
HoodieRecord<T> hoodieRecord = newRecordsItr.next();
|
||||
if (!writtenRecordKeys.contains(hoodieRecord.getRecordKey())) {
|
||||
if (useWriterSchema) {
|
||||
writeRecord(hoodieRecord, hoodieRecord.getData().getInsertValue(writerSchemaWithMetafields));
|
||||
} else {
|
||||
writeRecord(hoodieRecord, hoodieRecord.getData().getInsertValue(writerSchema));
|
||||
}
|
||||
insertRecordsWritten++;
|
||||
}
|
||||
}
|
||||
keyToNewRecords.clear();
|
||||
writtenRecordKeys.clear();
|
||||
|
||||
if (fileWriter != null) {
|
||||
fileWriter.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);
|
||||
|
||||
LOG.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;
|
||||
}
|
||||
|
||||
@Override
|
||||
public IOType getIOType() {
|
||||
return IOType.MERGE;
|
||||
}
|
||||
|
||||
public HoodieBaseFile baseFileForMerge() {
|
||||
return baseFileToMerge;
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,41 @@
|
||||
/*
|
||||
* 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.hudi.common.model.HoodieRecordPayload;
|
||||
import org.apache.hudi.common.util.collection.Pair;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
import org.apache.hudi.table.HoodieTable;
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
/**
|
||||
* Extract range information for a given file slice.
|
||||
*/
|
||||
public class HoodieRangeInfoHandle<T extends HoodieRecordPayload, I, K, O> extends HoodieReadHandle<T, I, K, O> {
|
||||
|
||||
public HoodieRangeInfoHandle(HoodieWriteConfig config, HoodieTable<T, I, K, O> hoodieTable,
|
||||
Pair<String, String> partitionPathFilePair) {
|
||||
super(config, null, hoodieTable, partitionPathFilePair);
|
||||
}
|
||||
|
||||
public String[] getMinMaxKeys() throws IOException {
|
||||
return createNewFileReader().readMinMaxRecordKeys();
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,69 @@
|
||||
/*
|
||||
* 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.hudi.common.model.HoodieBaseFile;
|
||||
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.io.storage.HoodieFileReader;
|
||||
import org.apache.hudi.io.storage.HoodieFileReaderFactory;
|
||||
import org.apache.hudi.table.HoodieTable;
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
|
||||
/**
|
||||
* Base class for read operations done logically on the file group.
|
||||
*/
|
||||
public abstract class HoodieReadHandle<T extends HoodieRecordPayload, I, K, O> extends HoodieIOHandle<T, I, K, O> {
|
||||
|
||||
protected final Pair<String, String> partitionPathFilePair;
|
||||
|
||||
public HoodieReadHandle(HoodieWriteConfig config, String instantTime, HoodieTable<T, I, K, O> 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 HoodieBaseFile getLatestDataFile() {
|
||||
return hoodieTable.getBaseFileOnlyView()
|
||||
.getLatestBaseFile(partitionPathFilePair.getLeft(), partitionPathFilePair.getRight()).get();
|
||||
}
|
||||
|
||||
protected HoodieFileReader createNewFileReader() throws IOException {
|
||||
return HoodieFileReaderFactory.getFileReader(hoodieTable.getHadoopConf(),
|
||||
new Path(getLatestDataFile().getPath()));
|
||||
}
|
||||
}
|
||||
@@ -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;
|
||||
|
||||
import org.apache.hudi.client.WriteStatus;
|
||||
import org.apache.hudi.client.common.TaskContextSupplier;
|
||||
import org.apache.hudi.common.model.HoodieBaseFile;
|
||||
import org.apache.hudi.common.model.HoodieRecord;
|
||||
import org.apache.hudi.common.model.HoodieRecordPayload;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
import org.apache.hudi.exception.HoodieUpsertException;
|
||||
import org.apache.hudi.table.HoodieTable;
|
||||
|
||||
import org.apache.avro.generic.GenericRecord;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.Iterator;
|
||||
import java.util.Map;
|
||||
import java.util.PriorityQueue;
|
||||
import java.util.Queue;
|
||||
|
||||
/**
|
||||
* Hoodie merge handle which writes records (new inserts or updates) sorted by their key.
|
||||
*
|
||||
* The implementation performs a merge-sort by comparing the key of the record being written to the list of
|
||||
* keys in newRecordKeys (sorted in-memory).
|
||||
*/
|
||||
public class HoodieSortedMergeHandle<T extends HoodieRecordPayload, I, K, O> extends HoodieMergeHandle<T, I, K, O> {
|
||||
|
||||
private Queue<String> newRecordKeysSorted = new PriorityQueue<>();
|
||||
|
||||
public HoodieSortedMergeHandle(HoodieWriteConfig config, String instantTime, HoodieTable<T, I, K, O> hoodieTable,
|
||||
Iterator<HoodieRecord<T>> recordItr, String partitionPath, String fileId, TaskContextSupplier taskContextSupplier) {
|
||||
super(config, instantTime, hoodieTable, recordItr, partitionPath, fileId, taskContextSupplier);
|
||||
newRecordKeysSorted.addAll(keyToNewRecords.keySet());
|
||||
}
|
||||
|
||||
/**
|
||||
* Called by compactor code path.
|
||||
*/
|
||||
public HoodieSortedMergeHandle(HoodieWriteConfig config, String instantTime, HoodieTable<T, I, K, O> hoodieTable,
|
||||
Map<String, HoodieRecord<T>> keyToNewRecordsOrig, String partitionPath, String fileId,
|
||||
HoodieBaseFile dataFileToBeMerged, TaskContextSupplier taskContextSupplier) {
|
||||
super(config, instantTime, hoodieTable, keyToNewRecordsOrig, partitionPath, fileId, dataFileToBeMerged,
|
||||
taskContextSupplier);
|
||||
|
||||
newRecordKeysSorted.addAll(keyToNewRecords.keySet());
|
||||
}
|
||||
|
||||
/**
|
||||
* Go through an old record. Here if we detect a newer version shows up, we write the new one to the file.
|
||||
*/
|
||||
@Override
|
||||
public void write(GenericRecord oldRecord) {
|
||||
String key = oldRecord.get(HoodieRecord.RECORD_KEY_METADATA_FIELD).toString();
|
||||
|
||||
// To maintain overall sorted order across updates and inserts, write any new inserts whose keys are less than
|
||||
// the oldRecord's key.
|
||||
while (!newRecordKeysSorted.isEmpty() && newRecordKeysSorted.peek().compareTo(key) <= 0) {
|
||||
String keyToPreWrite = newRecordKeysSorted.remove();
|
||||
if (keyToPreWrite.equals(key)) {
|
||||
// will be handled as an update later
|
||||
break;
|
||||
}
|
||||
|
||||
// This is a new insert
|
||||
HoodieRecord<T> hoodieRecord = new HoodieRecord<>(keyToNewRecords.get(keyToPreWrite));
|
||||
if (writtenRecordKeys.contains(keyToPreWrite)) {
|
||||
throw new HoodieUpsertException("Insert/Update not in sorted order");
|
||||
}
|
||||
try {
|
||||
if (useWriterSchema) {
|
||||
writeRecord(hoodieRecord, hoodieRecord.getData().getInsertValue(writerSchemaWithMetafields));
|
||||
} else {
|
||||
writeRecord(hoodieRecord, hoodieRecord.getData().getInsertValue(writerSchema));
|
||||
}
|
||||
insertRecordsWritten++;
|
||||
writtenRecordKeys.add(keyToPreWrite);
|
||||
} catch (IOException e) {
|
||||
throw new HoodieUpsertException("Failed to write records", e);
|
||||
}
|
||||
}
|
||||
|
||||
super.write(oldRecord);
|
||||
}
|
||||
|
||||
@Override
|
||||
public WriteStatus close() {
|
||||
// write out any pending records (this can happen when inserts are turned into updates)
|
||||
newRecordKeysSorted.stream().forEach(key -> {
|
||||
try {
|
||||
HoodieRecord<T> hoodieRecord = keyToNewRecords.get(key);
|
||||
if (!writtenRecordKeys.contains(hoodieRecord.getRecordKey())) {
|
||||
if (useWriterSchema) {
|
||||
writeRecord(hoodieRecord, hoodieRecord.getData().getInsertValue(writerSchemaWithMetafields));
|
||||
} else {
|
||||
writeRecord(hoodieRecord, hoodieRecord.getData().getInsertValue(writerSchema));
|
||||
}
|
||||
insertRecordsWritten++;
|
||||
}
|
||||
} catch (IOException e) {
|
||||
throw new HoodieUpsertException("Failed to close UpdateHandle", e);
|
||||
}
|
||||
});
|
||||
newRecordKeysSorted.clear();
|
||||
keyToNewRecords.clear();
|
||||
|
||||
return super.close();
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,197 @@
|
||||
/*
|
||||
* 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.hudi.avro.HoodieAvroUtils;
|
||||
import org.apache.hudi.client.WriteStatus;
|
||||
import org.apache.hudi.client.common.TaskContextSupplier;
|
||||
import org.apache.hudi.common.fs.FSUtils;
|
||||
import org.apache.hudi.common.model.HoodieRecord;
|
||||
import org.apache.hudi.common.model.HoodieRecordPayload;
|
||||
import org.apache.hudi.common.model.IOType;
|
||||
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.common.util.collection.Pair;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
import org.apache.hudi.exception.HoodieIOException;
|
||||
import org.apache.hudi.io.storage.HoodieFileWriter;
|
||||
import org.apache.hudi.io.storage.HoodieFileWriterFactory;
|
||||
import org.apache.hudi.table.HoodieTable;
|
||||
import org.apache.hudi.table.MarkerFiles;
|
||||
|
||||
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.log4j.LogManager;
|
||||
import org.apache.log4j.Logger;
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
/**
|
||||
* Base class for all write operations logically performed at the file group level.
|
||||
*/
|
||||
public abstract class HoodieWriteHandle<T extends HoodieRecordPayload, I, K, O> extends HoodieIOHandle<T, I, K, O> {
|
||||
|
||||
private static final Logger LOG = LogManager.getLogger(HoodieWriteHandle.class);
|
||||
|
||||
protected final Schema writerSchema;
|
||||
protected final Schema writerSchemaWithMetafields;
|
||||
protected HoodieTimer timer;
|
||||
protected final WriteStatus writeStatus;
|
||||
protected final String partitionPath;
|
||||
protected final String fileId;
|
||||
protected final String writeToken;
|
||||
protected final TaskContextSupplier taskContextSupplier;
|
||||
|
||||
public HoodieWriteHandle(HoodieWriteConfig config, String instantTime, String partitionPath,
|
||||
String fileId, HoodieTable<T, I, K, O> hoodieTable, TaskContextSupplier taskContextSupplier) {
|
||||
this(config, instantTime, partitionPath, fileId, hoodieTable,
|
||||
getWriterSchemaIncludingAndExcludingMetadataPair(config), taskContextSupplier);
|
||||
}
|
||||
|
||||
protected HoodieWriteHandle(HoodieWriteConfig config, String instantTime, String partitionPath, String fileId,
|
||||
HoodieTable<T, I, K, O> hoodieTable, Pair<Schema, Schema> writerSchemaIncludingAndExcludingMetadataPair,
|
||||
TaskContextSupplier taskContextSupplier) {
|
||||
super(config, instantTime, hoodieTable);
|
||||
this.partitionPath = partitionPath;
|
||||
this.fileId = fileId;
|
||||
this.writerSchema = writerSchemaIncludingAndExcludingMetadataPair.getKey();
|
||||
this.writerSchemaWithMetafields = writerSchemaIncludingAndExcludingMetadataPair.getValue();
|
||||
this.timer = new HoodieTimer().startTimer();
|
||||
this.writeStatus = (WriteStatus) ReflectionUtils.loadClass(config.getWriteStatusClassName(),
|
||||
!hoodieTable.getIndex().isImplicitWithStorage(), config.getWriteStatusFailureFraction());
|
||||
this.taskContextSupplier = taskContextSupplier;
|
||||
this.writeToken = makeWriteToken();
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns writer schema pairs containing
|
||||
* (a) Writer Schema from client
|
||||
* (b) (a) with hoodie metadata fields.
|
||||
* @param config Write Config
|
||||
* @return
|
||||
*/
|
||||
protected static Pair<Schema, Schema> getWriterSchemaIncludingAndExcludingMetadataPair(HoodieWriteConfig config) {
|
||||
Schema originalSchema = new Schema.Parser().parse(config.getSchema());
|
||||
Schema hoodieSchema = HoodieAvroUtils.addMetadataFields(originalSchema);
|
||||
return Pair.of(originalSchema, hoodieSchema);
|
||||
}
|
||||
|
||||
/**
|
||||
* Generate a write token based on the currently running spark task and its place in the spark dag.
|
||||
*/
|
||||
private String makeWriteToken() {
|
||||
return FSUtils.makeWriteToken(getPartitionId(), getStageId(), getAttemptId());
|
||||
}
|
||||
|
||||
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,
|
||||
hoodieTable.getMetaClient().getTableConfig().getBaseFileFormat().getFileExtension()));
|
||||
}
|
||||
|
||||
/**
|
||||
* Creates an empty marker file corresponding to storage writer path.
|
||||
*
|
||||
* @param partitionPath Partition path
|
||||
*/
|
||||
protected void createMarkerFile(String partitionPath, String dataFileName) {
|
||||
MarkerFiles markerFiles = new MarkerFiles(hoodieTable, instantTime);
|
||||
markerFiles.create(partitionPath, dataFileName, getIOType());
|
||||
}
|
||||
|
||||
public Schema getWriterSchemaWithMetafields() {
|
||||
return writerSchemaWithMetafields;
|
||||
}
|
||||
|
||||
/**
|
||||
* 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);
|
||||
LOG.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, writerSchemaWithMetafields);
|
||||
}
|
||||
|
||||
public abstract WriteStatus close();
|
||||
|
||||
public abstract WriteStatus getWriteStatus();
|
||||
|
||||
public abstract IOType getIOType();
|
||||
|
||||
@Override
|
||||
protected FileSystem getFileSystem() {
|
||||
return hoodieTable.getMetaClient().getFs();
|
||||
}
|
||||
|
||||
protected int getPartitionId() {
|
||||
return taskContextSupplier.getPartitionIdSupplier().get();
|
||||
}
|
||||
|
||||
protected int getStageId() {
|
||||
return taskContextSupplier.getStageIdSupplier().get();
|
||||
}
|
||||
|
||||
protected long getAttemptId() {
|
||||
return taskContextSupplier.getAttemptIdSupplier().get();
|
||||
}
|
||||
|
||||
protected HoodieFileWriter createNewFileWriter(String instantTime, Path path, HoodieTable<T, I, K, O> hoodieTable,
|
||||
HoodieWriteConfig config, Schema schema, TaskContextSupplier taskContextSupplier) throws IOException {
|
||||
return HoodieFileWriterFactory.getFileWriter(instantTime, path, hoodieTable, config, schema, taskContextSupplier);
|
||||
}
|
||||
}
|
||||
@@ -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;
|
||||
|
||||
import org.apache.hudi.client.common.EngineProperty;
|
||||
import org.apache.hudi.client.common.TaskContextSupplier;
|
||||
import org.apache.hudi.common.util.Option;
|
||||
|
||||
import java.util.Properties;
|
||||
|
||||
import static org.apache.hudi.config.HoodieMemoryConfig.DEFAULT_MAX_MEMORY_FOR_SPILLABLE_MAP_IN_BYTES;
|
||||
import static org.apache.hudi.config.HoodieMemoryConfig.DEFAULT_MAX_MEMORY_FRACTION_FOR_COMPACTION;
|
||||
import static org.apache.hudi.config.HoodieMemoryConfig.DEFAULT_MAX_MEMORY_FRACTION_FOR_MERGE;
|
||||
import static org.apache.hudi.config.HoodieMemoryConfig.DEFAULT_MIN_MEMORY_FOR_SPILLABLE_MAP_IN_BYTES;
|
||||
import static org.apache.hudi.config.HoodieMemoryConfig.MAX_MEMORY_FOR_COMPACTION_PROP;
|
||||
import static org.apache.hudi.config.HoodieMemoryConfig.MAX_MEMORY_FOR_MERGE_PROP;
|
||||
import static org.apache.hudi.config.HoodieMemoryConfig.MAX_MEMORY_FRACTION_FOR_COMPACTION_PROP;
|
||||
import static org.apache.hudi.config.HoodieMemoryConfig.MAX_MEMORY_FRACTION_FOR_MERGE_PROP;
|
||||
|
||||
public class IOUtils {
|
||||
/**
|
||||
* Dynamic calculation of max memory to use for for spillable map. user.available.memory = executor.memory *
|
||||
* (1 - memory.fraction) spillable.available.memory = user.available.memory * hoodie.memory.fraction. Anytime
|
||||
* the engine memory fractions/total memory is changed, the memory used for spillable map changes
|
||||
* accordingly
|
||||
*/
|
||||
public static long getMaxMemoryAllowedForMerge(TaskContextSupplier context, String maxMemoryFraction) {
|
||||
Option<String> totalMemoryOpt = context.getProperty(EngineProperty.TOTAL_MEMORY_AVAILABLE);
|
||||
Option<String> memoryFractionOpt = context.getProperty(EngineProperty.MEMORY_FRACTION_IN_USE);
|
||||
|
||||
if (totalMemoryOpt.isPresent() && memoryFractionOpt.isPresent()) {
|
||||
long executorMemoryInBytes = Long.parseLong(totalMemoryOpt.get());
|
||||
double memoryFraction = Double.parseDouble(memoryFractionOpt.get());
|
||||
double maxMemoryFractionForMerge = Double.parseDouble(maxMemoryFraction);
|
||||
double userAvailableMemory = executorMemoryInBytes * (1 - memoryFraction);
|
||||
long maxMemoryForMerge = (long) Math.floor(userAvailableMemory * maxMemoryFractionForMerge);
|
||||
return Math.max(DEFAULT_MIN_MEMORY_FOR_SPILLABLE_MAP_IN_BYTES, maxMemoryForMerge);
|
||||
} else {
|
||||
return DEFAULT_MAX_MEMORY_FOR_SPILLABLE_MAP_IN_BYTES;
|
||||
}
|
||||
}
|
||||
|
||||
public static long getMaxMemoryPerPartitionMerge(TaskContextSupplier context, Properties properties) {
|
||||
if (properties.containsKey(MAX_MEMORY_FOR_MERGE_PROP)) {
|
||||
return Long.parseLong(properties.getProperty(MAX_MEMORY_FOR_MERGE_PROP));
|
||||
}
|
||||
String fraction = properties.getProperty(MAX_MEMORY_FRACTION_FOR_MERGE_PROP, DEFAULT_MAX_MEMORY_FRACTION_FOR_MERGE);
|
||||
return getMaxMemoryAllowedForMerge(context, fraction);
|
||||
}
|
||||
|
||||
public static long getMaxMemoryPerCompaction(TaskContextSupplier context, Properties properties) {
|
||||
if (properties.containsKey(MAX_MEMORY_FOR_COMPACTION_PROP)) {
|
||||
return Long.parseLong(properties.getProperty(MAX_MEMORY_FOR_COMPACTION_PROP));
|
||||
}
|
||||
String fraction = properties.getProperty(MAX_MEMORY_FRACTION_FOR_COMPACTION_PROP, DEFAULT_MAX_MEMORY_FRACTION_FOR_COMPACTION);
|
||||
return getMaxMemoryAllowedForMerge(context, fraction);
|
||||
}
|
||||
}
|
||||
@@ -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.io;
|
||||
|
||||
import org.apache.hudi.client.common.TaskContextSupplier;
|
||||
import org.apache.hudi.common.model.HoodieRecordPayload;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
import org.apache.hudi.table.HoodieTable;
|
||||
|
||||
public abstract class WriteHandleFactory<T extends HoodieRecordPayload, I, K, O> {
|
||||
private int numFilesWritten = 0;
|
||||
|
||||
public abstract HoodieWriteHandle<T, I, K, O> create(HoodieWriteConfig config, String commitTime, HoodieTable<T, I, K, O> hoodieTable,
|
||||
String partitionPath, String fileIdPrefix, TaskContextSupplier taskContextSupplier);
|
||||
|
||||
protected String getNextFileId(String idPfx) {
|
||||
return String.format("%s-%d", idPfx, numFilesWritten++);
|
||||
}
|
||||
}
|
||||
@@ -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.io.storage;
|
||||
|
||||
import org.apache.hudi.avro.HoodieAvroWriteSupport;
|
||||
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.parquet.hadoop.metadata.CompressionCodecName;
|
||||
|
||||
/**
|
||||
* ParquetConfig for writing avro records in Parquet files.
|
||||
*/
|
||||
public class HoodieAvroParquetConfig extends HoodieBaseParquetConfig<HoodieAvroWriteSupport> {
|
||||
|
||||
public HoodieAvroParquetConfig(HoodieAvroWriteSupport writeSupport, CompressionCodecName compressionCodecName,
|
||||
int blockSize, int pageSize, long maxFileSize, Configuration hadoopConf,
|
||||
double compressionRatio) {
|
||||
super(writeSupport, compressionCodecName, blockSize, pageSize, maxFileSize, hadoopConf, compressionRatio);
|
||||
}
|
||||
}
|
||||
@@ -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.io.storage;
|
||||
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.parquet.hadoop.metadata.CompressionCodecName;
|
||||
|
||||
/**
|
||||
* Base ParquetConfig to hold config params for writing to Parquet.
|
||||
* @param <T>
|
||||
*/
|
||||
public class HoodieBaseParquetConfig<T> {
|
||||
private final T writeSupport;
|
||||
private CompressionCodecName compressionCodecName;
|
||||
private int blockSize;
|
||||
private int pageSize;
|
||||
private long maxFileSize;
|
||||
private Configuration hadoopConf;
|
||||
private double compressionRatio;
|
||||
|
||||
public HoodieBaseParquetConfig(T 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 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;
|
||||
}
|
||||
|
||||
public T getWriteSupport() {
|
||||
return writeSupport;
|
||||
}
|
||||
}
|
||||
@@ -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.io.storage;
|
||||
|
||||
import org.apache.hudi.common.model.HoodieRecord;
|
||||
|
||||
import org.apache.avro.generic.IndexedRecord;
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
public interface HoodieFileWriter<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,85 @@
|
||||
/*
|
||||
* 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.hudi.avro.HoodieAvroWriteSupport;
|
||||
import org.apache.hudi.client.common.TaskContextSupplier;
|
||||
import org.apache.hudi.common.bloom.BloomFilter;
|
||||
import org.apache.hudi.common.bloom.BloomFilterFactory;
|
||||
import org.apache.hudi.common.fs.FSUtils;
|
||||
import org.apache.hudi.common.model.HoodieRecordPayload;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
import org.apache.hudi.table.HoodieTable;
|
||||
|
||||
import org.apache.avro.Schema;
|
||||
import org.apache.avro.generic.IndexedRecord;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.parquet.avro.AvroSchemaConverter;
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
import static org.apache.hudi.common.model.HoodieFileFormat.PARQUET;
|
||||
import static org.apache.hudi.common.model.HoodieFileFormat.HFILE;
|
||||
|
||||
public class HoodieFileWriterFactory {
|
||||
|
||||
public static <T extends HoodieRecordPayload, R extends IndexedRecord, I, K, O> HoodieFileWriter<R> getFileWriter(
|
||||
String instantTime, Path path, HoodieTable<T, I, K, O> hoodieTable, HoodieWriteConfig config, Schema schema,
|
||||
TaskContextSupplier taskContextSupplier) throws IOException {
|
||||
final String extension = FSUtils.getFileExtension(path.getName());
|
||||
if (PARQUET.getFileExtension().equals(extension)) {
|
||||
return newParquetFileWriter(instantTime, path, config, schema, hoodieTable, taskContextSupplier);
|
||||
}
|
||||
if (HFILE.getFileExtension().equals(extension)) {
|
||||
return newHFileFileWriter(instantTime, path, config, schema, hoodieTable, taskContextSupplier);
|
||||
}
|
||||
throw new UnsupportedOperationException(extension + " format not supported yet.");
|
||||
}
|
||||
|
||||
private static <T extends HoodieRecordPayload, R extends IndexedRecord> HoodieFileWriter<R> newParquetFileWriter(
|
||||
String instantTime, Path path, HoodieWriteConfig config, Schema schema, HoodieTable hoodieTable,
|
||||
TaskContextSupplier taskContextSupplier) throws IOException {
|
||||
BloomFilter filter = createBloomFilter(config);
|
||||
HoodieAvroWriteSupport writeSupport =
|
||||
new HoodieAvroWriteSupport(new AvroSchemaConverter().convert(schema), schema, filter);
|
||||
|
||||
HoodieAvroParquetConfig parquetConfig = new HoodieAvroParquetConfig(writeSupport, config.getParquetCompressionCodec(),
|
||||
config.getParquetBlockSize(), config.getParquetPageSize(), config.getParquetMaxFileSize(),
|
||||
hoodieTable.getHadoopConf(), config.getParquetCompressionRatio());
|
||||
|
||||
return new HoodieParquetWriter<>(instantTime, path, parquetConfig, schema, taskContextSupplier);
|
||||
}
|
||||
|
||||
private static <T extends HoodieRecordPayload, R extends IndexedRecord> HoodieFileWriter<R> newHFileFileWriter(
|
||||
String instantTime, Path path, HoodieWriteConfig config, Schema schema, HoodieTable hoodieTable,
|
||||
TaskContextSupplier taskContextSupplier) throws IOException {
|
||||
|
||||
BloomFilter filter = createBloomFilter(config);
|
||||
HoodieHFileConfig hfileConfig = new HoodieHFileConfig(hoodieTable.getHadoopConf(),
|
||||
config.getHFileCompressionAlgorithm(), config.getHFileBlockSize(), config.getHFileMaxFileSize(), filter);
|
||||
|
||||
return new HoodieHFileWriter<>(instantTime, path, hfileConfig, schema, taskContextSupplier);
|
||||
}
|
||||
|
||||
private static BloomFilter createBloomFilter(HoodieWriteConfig config) {
|
||||
return BloomFilterFactory.createBloomFilter(config.getBloomFilterNumEntries(), config.getBloomFilterFPP(),
|
||||
config.getDynamicBloomFilterMaxNumEntries(),
|
||||
config.getBloomFilterType());
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,95 @@
|
||||
/*
|
||||
* 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.hadoop.hbase.HColumnDescriptor;
|
||||
import org.apache.hadoop.hbase.io.compress.Compression;
|
||||
import org.apache.hadoop.hbase.io.hfile.CacheConfig;
|
||||
import org.apache.hudi.common.bloom.BloomFilter;
|
||||
|
||||
public class HoodieHFileConfig {
|
||||
|
||||
private Compression.Algorithm compressionAlgorithm;
|
||||
private int blockSize;
|
||||
private long maxFileSize;
|
||||
private boolean prefetchBlocksOnOpen;
|
||||
private boolean cacheDataInL1;
|
||||
private boolean dropBehindCacheCompaction;
|
||||
private Configuration hadoopConf;
|
||||
private BloomFilter bloomFilter;
|
||||
|
||||
// This is private in CacheConfig so have been copied here.
|
||||
private static boolean DROP_BEHIND_CACHE_COMPACTION_DEFAULT = true;
|
||||
|
||||
public HoodieHFileConfig(Configuration hadoopConf, Compression.Algorithm compressionAlgorithm, int blockSize,
|
||||
long maxFileSize, BloomFilter bloomFilter) {
|
||||
this(hadoopConf, compressionAlgorithm, blockSize, maxFileSize, CacheConfig.DEFAULT_PREFETCH_ON_OPEN,
|
||||
HColumnDescriptor.DEFAULT_CACHE_DATA_IN_L1, DROP_BEHIND_CACHE_COMPACTION_DEFAULT, bloomFilter);
|
||||
}
|
||||
|
||||
public HoodieHFileConfig(Configuration hadoopConf, Compression.Algorithm compressionAlgorithm, int blockSize,
|
||||
long maxFileSize, boolean prefetchBlocksOnOpen, boolean cacheDataInL1,
|
||||
boolean dropBehindCacheCompaction, BloomFilter bloomFilter) {
|
||||
this.hadoopConf = hadoopConf;
|
||||
this.compressionAlgorithm = compressionAlgorithm;
|
||||
this.blockSize = blockSize;
|
||||
this.maxFileSize = maxFileSize;
|
||||
this.prefetchBlocksOnOpen = prefetchBlocksOnOpen;
|
||||
this.cacheDataInL1 = cacheDataInL1;
|
||||
this.dropBehindCacheCompaction = dropBehindCacheCompaction;
|
||||
this.bloomFilter = bloomFilter;
|
||||
}
|
||||
|
||||
public Configuration getHadoopConf() {
|
||||
return hadoopConf;
|
||||
}
|
||||
|
||||
public Compression.Algorithm getCompressionAlgorithm() {
|
||||
return compressionAlgorithm;
|
||||
}
|
||||
|
||||
public int getBlockSize() {
|
||||
return blockSize;
|
||||
}
|
||||
|
||||
public long getMaxFileSize() {
|
||||
return maxFileSize;
|
||||
}
|
||||
|
||||
public boolean shouldPrefetchBlocksOnOpen() {
|
||||
return prefetchBlocksOnOpen;
|
||||
}
|
||||
|
||||
public boolean shouldCacheDataInL1() {
|
||||
return cacheDataInL1;
|
||||
}
|
||||
|
||||
public boolean shouldDropBehindCacheCompaction() {
|
||||
return dropBehindCacheCompaction;
|
||||
}
|
||||
|
||||
public boolean useBloomFilter() {
|
||||
return bloomFilter != null;
|
||||
}
|
||||
|
||||
public BloomFilter getBloomFilter() {
|
||||
return bloomFilter;
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,166 @@
|
||||
/*
|
||||
* 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.hudi.avro.HoodieAvroUtils;
|
||||
import org.apache.hudi.client.common.TaskContextSupplier;
|
||||
import org.apache.hudi.common.bloom.BloomFilter;
|
||||
import org.apache.hudi.common.fs.FSUtils;
|
||||
import org.apache.hudi.common.fs.HoodieWrapperFileSystem;
|
||||
import org.apache.hudi.common.model.HoodieRecord;
|
||||
import org.apache.hudi.common.model.HoodieRecordPayload;
|
||||
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.hadoop.hbase.HColumnDescriptor;
|
||||
import org.apache.hadoop.hbase.KeyValue;
|
||||
import org.apache.hadoop.hbase.io.hfile.CacheConfig;
|
||||
import org.apache.hadoop.hbase.io.hfile.HFile;
|
||||
import org.apache.hadoop.hbase.io.hfile.HFileContext;
|
||||
import org.apache.hadoop.hbase.io.hfile.HFileContextBuilder;
|
||||
import org.apache.hadoop.io.Writable;
|
||||
|
||||
import java.io.DataInput;
|
||||
import java.io.DataOutput;
|
||||
import java.io.IOException;
|
||||
import java.util.concurrent.atomic.AtomicLong;
|
||||
|
||||
/**
|
||||
* HoodieHFileWriter writes IndexedRecords into an HFile. The record's key is used as the key and the
|
||||
* AVRO encoded record bytes are saved as the value.
|
||||
*
|
||||
* Limitations (compared to columnar formats like Parquet or ORC):
|
||||
* 1. Records should be added in order of keys
|
||||
* 2. There are no column stats
|
||||
*/
|
||||
public class HoodieHFileWriter<T extends HoodieRecordPayload, R extends IndexedRecord>
|
||||
implements HoodieFileWriter<R> {
|
||||
private static AtomicLong recordIndex = new AtomicLong(1);
|
||||
|
||||
private final Path file;
|
||||
private HoodieHFileConfig hfileConfig;
|
||||
private final HoodieWrapperFileSystem fs;
|
||||
private final long maxFileSize;
|
||||
private final String instantTime;
|
||||
private final TaskContextSupplier taskContextSupplier;
|
||||
private HFile.Writer writer;
|
||||
private String minRecordKey;
|
||||
private String maxRecordKey;
|
||||
|
||||
// This is private in CacheConfig so have been copied here.
|
||||
private static String DROP_BEHIND_CACHE_COMPACTION_KEY = "hbase.hfile.drop.behind.compaction";
|
||||
|
||||
public HoodieHFileWriter(String instantTime, Path file, HoodieHFileConfig hfileConfig, Schema schema,
|
||||
TaskContextSupplier taskContextSupplier) throws IOException {
|
||||
|
||||
Configuration conf = FSUtils.registerFileSystem(file, hfileConfig.getHadoopConf());
|
||||
this.file = HoodieWrapperFileSystem.convertToHoodiePath(file, conf);
|
||||
this.fs = (HoodieWrapperFileSystem) this.file.getFileSystem(conf);
|
||||
this.hfileConfig = hfileConfig;
|
||||
|
||||
// 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 = hfileConfig.getMaxFileSize()
|
||||
// + Math.round(hfileConfig.getMaxFileSize() * hfileConfig.getCompressionRatio());
|
||||
this.maxFileSize = hfileConfig.getMaxFileSize();
|
||||
this.instantTime = instantTime;
|
||||
this.taskContextSupplier = taskContextSupplier;
|
||||
|
||||
HFileContext context = new HFileContextBuilder().withBlockSize(hfileConfig.getBlockSize())
|
||||
.withCompression(hfileConfig.getCompressionAlgorithm())
|
||||
.build();
|
||||
|
||||
conf.set(CacheConfig.PREFETCH_BLOCKS_ON_OPEN_KEY, String.valueOf(hfileConfig.shouldPrefetchBlocksOnOpen()));
|
||||
conf.set(HColumnDescriptor.CACHE_DATA_IN_L1, String.valueOf(hfileConfig.shouldCacheDataInL1()));
|
||||
conf.set(DROP_BEHIND_CACHE_COMPACTION_KEY, String.valueOf(hfileConfig.shouldDropBehindCacheCompaction()));
|
||||
CacheConfig cacheConfig = new CacheConfig(conf);
|
||||
this.writer = HFile.getWriterFactory(conf, cacheConfig).withPath(this.fs, this.file).withFileContext(context).create();
|
||||
|
||||
writer.appendFileInfo(HoodieHFileReader.KEY_SCHEMA.getBytes(), schema.toString().getBytes());
|
||||
}
|
||||
|
||||
@Override
|
||||
public void writeAvroWithMetadata(R avroRecord, HoodieRecord record) throws IOException {
|
||||
String seqId =
|
||||
HoodieRecord.generateSequenceId(instantTime, taskContextSupplier.getPartitionIdSupplier().get(), recordIndex.getAndIncrement());
|
||||
HoodieAvroUtils.addHoodieKeyToRecord((GenericRecord) avroRecord, record.getRecordKey(), record.getPartitionPath(),
|
||||
file.getName());
|
||||
HoodieAvroUtils.addCommitMetadataToRecord((GenericRecord) avroRecord, instantTime, seqId);
|
||||
|
||||
writeAvro(record.getRecordKey(), (IndexedRecord)avroRecord);
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean canWrite() {
|
||||
return fs.getBytesWritten(file) < maxFileSize;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void writeAvro(String recordKey, IndexedRecord object) throws IOException {
|
||||
byte[] value = HoodieAvroUtils.avroToBytes((GenericRecord)object);
|
||||
KeyValue kv = new KeyValue(recordKey.getBytes(), null, null, value);
|
||||
writer.append(kv);
|
||||
|
||||
if (hfileConfig.useBloomFilter()) {
|
||||
hfileConfig.getBloomFilter().add(recordKey);
|
||||
if (minRecordKey != null) {
|
||||
minRecordKey = minRecordKey.compareTo(recordKey) <= 0 ? minRecordKey : recordKey;
|
||||
} else {
|
||||
minRecordKey = recordKey;
|
||||
}
|
||||
|
||||
if (maxRecordKey != null) {
|
||||
maxRecordKey = maxRecordKey.compareTo(recordKey) >= 0 ? maxRecordKey : recordKey;
|
||||
} else {
|
||||
maxRecordKey = recordKey;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void close() throws IOException {
|
||||
if (hfileConfig.useBloomFilter()) {
|
||||
final BloomFilter bloomFilter = hfileConfig.getBloomFilter();
|
||||
if (minRecordKey == null) {
|
||||
minRecordKey = "";
|
||||
}
|
||||
if (maxRecordKey == null) {
|
||||
maxRecordKey = "";
|
||||
}
|
||||
writer.appendFileInfo(HoodieHFileReader.KEY_MIN_RECORD.getBytes(), minRecordKey.getBytes());
|
||||
writer.appendFileInfo(HoodieHFileReader.KEY_MAX_RECORD.getBytes(), maxRecordKey.getBytes());
|
||||
writer.appendFileInfo(HoodieHFileReader.KEY_BLOOM_FILTER_TYPE_CODE.getBytes(),
|
||||
bloomFilter.getBloomFilterTypeCode().toString().getBytes());
|
||||
writer.appendMetaBlock(HoodieHFileReader.KEY_BLOOM_FILTER_META_BLOCK, new Writable() {
|
||||
@Override
|
||||
public void write(DataOutput out) throws IOException {
|
||||
out.write(bloomFilter.serializeToString().getBytes());
|
||||
}
|
||||
|
||||
@Override
|
||||
public void readFields(DataInput in) throws IOException { }
|
||||
});
|
||||
}
|
||||
|
||||
writer.close();
|
||||
writer = null;
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,97 @@
|
||||
/*
|
||||
* 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.hudi.avro.HoodieAvroUtils;
|
||||
import org.apache.hudi.avro.HoodieAvroWriteSupport;
|
||||
import org.apache.hudi.client.common.TaskContextSupplier;
|
||||
import org.apache.hudi.common.fs.FSUtils;
|
||||
import org.apache.hudi.common.fs.HoodieWrapperFileSystem;
|
||||
import org.apache.hudi.common.model.HoodieRecord;
|
||||
import org.apache.hudi.common.model.HoodieRecordPayload;
|
||||
|
||||
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.parquet.hadoop.ParquetFileWriter;
|
||||
import org.apache.parquet.hadoop.ParquetWriter;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.concurrent.atomic.AtomicLong;
|
||||
|
||||
/**
|
||||
* 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 HoodieFileWriter<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 instantTime;
|
||||
private final TaskContextSupplier taskContextSupplier;
|
||||
|
||||
public HoodieParquetWriter(String instantTime, Path file, HoodieAvroParquetConfig parquetConfig,
|
||||
Schema schema, TaskContextSupplier taskContextSupplier) throws IOException {
|
||||
super(HoodieWrapperFileSystem.convertToHoodiePath(file, parquetConfig.getHadoopConf()),
|
||||
ParquetFileWriter.Mode.CREATE, parquetConfig.getWriteSupport(), parquetConfig.getCompressionCodecName(),
|
||||
parquetConfig.getBlockSize(), parquetConfig.getPageSize(), parquetConfig.getPageSize(),
|
||||
DEFAULT_IS_DICTIONARY_ENABLED, DEFAULT_IS_VALIDATING_ENABLED,
|
||||
DEFAULT_WRITER_VERSION, FSUtils.registerFileSystem(file, parquetConfig.getHadoopConf()));
|
||||
this.file = HoodieWrapperFileSystem.convertToHoodiePath(file, parquetConfig.getHadoopConf());
|
||||
this.fs =
|
||||
(HoodieWrapperFileSystem) this.file.getFileSystem(FSUtils.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.instantTime = instantTime;
|
||||
this.taskContextSupplier = taskContextSupplier;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void writeAvroWithMetadata(R avroRecord, HoodieRecord record) throws IOException {
|
||||
String seqId =
|
||||
HoodieRecord.generateSequenceId(instantTime, taskContextSupplier.getPartitionIdSupplier().get(), recordIndex.getAndIncrement());
|
||||
HoodieAvroUtils.addHoodieKeyToRecord((GenericRecord) avroRecord, record.getRecordKey(), record.getPartitionPath(),
|
||||
file.getName());
|
||||
HoodieAvroUtils.addCommitMetadataToRecord((GenericRecord) avroRecord, instantTime, seqId);
|
||||
super.write(avroRecord);
|
||||
writeSupport.add(record.getRecordKey());
|
||||
}
|
||||
|
||||
@Override
|
||||
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,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.keygen;
|
||||
|
||||
import org.apache.avro.generic.GenericRecord;
|
||||
import org.apache.hudi.common.model.HoodieKey;
|
||||
|
||||
import java.io.Serializable;
|
||||
import java.util.List;
|
||||
|
||||
/**
|
||||
* Represents the interface key generators need to adhere to.
|
||||
*/
|
||||
public interface KeyGeneratorInterface extends Serializable {
|
||||
|
||||
HoodieKey getKey(GenericRecord record);
|
||||
|
||||
List<String> getRecordKeyFieldNames();
|
||||
|
||||
}
|
||||
@@ -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.metrics;
|
||||
|
||||
import java.io.Closeable;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
|
||||
import org.apache.log4j.LogManager;
|
||||
import org.apache.log4j.Logger;
|
||||
|
||||
import com.codahale.metrics.ConsoleReporter;
|
||||
import com.codahale.metrics.MetricFilter;
|
||||
import com.codahale.metrics.MetricRegistry;
|
||||
|
||||
/**
|
||||
* Hudi Console metrics reporter. Reports the metrics by printing them to the stdout on the console.
|
||||
*/
|
||||
public class ConsoleMetricsReporter extends MetricsReporter {
|
||||
|
||||
private static final Logger LOG = LogManager.getLogger(ConsoleMetricsReporter.class);
|
||||
private final ConsoleReporter consoleReporter;
|
||||
|
||||
public ConsoleMetricsReporter(MetricRegistry registry) {
|
||||
this.consoleReporter = ConsoleReporter.forRegistry(registry)
|
||||
.convertRatesTo(TimeUnit.SECONDS)
|
||||
.convertDurationsTo(TimeUnit.MILLISECONDS)
|
||||
.filter(MetricFilter.ALL).build();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void start() {
|
||||
if (consoleReporter != null) {
|
||||
consoleReporter.start(30, TimeUnit.SECONDS);
|
||||
} else {
|
||||
LOG.error("Cannot start as the consoleReporter is null.");
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void report() {
|
||||
if (consoleReporter != null) {
|
||||
consoleReporter.report();
|
||||
} else {
|
||||
LOG.error("Cannot report metrics as the consoleReporter is null.");
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public Closeable getReporter() {
|
||||
return consoleReporter;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void stop() {
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,52 @@
|
||||
/*
|
||||
* 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;
|
||||
|
||||
/**
|
||||
* Similar to {@link Gauge}, but metric value can be updated by {@link #setValue(T)}.
|
||||
*/
|
||||
public class HoodieGauge<T> implements Gauge<T> {
|
||||
private volatile T value;
|
||||
|
||||
/**
|
||||
* Create an instance with a default value.
|
||||
*/
|
||||
public HoodieGauge(T value) {
|
||||
this.value = value;
|
||||
}
|
||||
|
||||
/**
|
||||
* Set the metric to a new value.
|
||||
*/
|
||||
public void setValue(T value) {
|
||||
this.value = value;
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns the metric's current value.
|
||||
*
|
||||
* @return the metric's current value
|
||||
*/
|
||||
@Override
|
||||
public T getValue() {
|
||||
return value;
|
||||
}
|
||||
}
|
||||
Some files were not shown because too many files have changed in this diff Show More
Reference in New Issue
Block a user