1
0

HUDI-123 Rename code packages/constants to org.apache.hudi (#830)

- Rename com.uber.hoodie to org.apache.hudi
- Flag to pass com.uber.hoodie Input formats for hoodie-sync
- Works with HUDI demo. 
- Also tested for backwards compatibility with datasets built by com.uber.hoodie packages
- Migration guide : https://cwiki.apache.org/confluence/display/HUDI/Migration+Guide+From+com.uber.hoodie+to+org.apache.hudi
This commit is contained in:
Balaji Varadarajan
2019-08-11 17:48:17 -07:00
committed by vinoth chandar
parent 722b6be04a
commit a4f9d7575f
546 changed files with 3858 additions and 3562 deletions

View File

@@ -21,4 +21,4 @@ cp /var/hoodie/ws/docker/demo/config/spark-defaults.conf $SPARK_CONF_DIR/.
hadoop fs -mkdir -p /var/demo/ hadoop fs -mkdir -p /var/demo/
hadoop fs -mkdir -p /tmp/spark-events hadoop fs -mkdir -p /tmp/spark-events
hadoop fs -copyFromLocal -f /var/hoodie/ws/docker/demo/config /var/demo/. hadoop fs -copyFromLocal -f /var/hoodie/ws/docker/demo/config /var/demo/.
chmod +x /var/hoodie/ws/hoodie-hive/run_sync_tool.sh chmod +x /var/hoodie/ws/hudi-hive/run_sync_tool.sh

View File

@@ -17,13 +17,13 @@
--> -->
<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"> <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> <parent>
<artifactId>hoodie-hadoop-docker</artifactId> <artifactId>hudi-hadoop-docker</artifactId>
<groupId>com.uber.hoodie</groupId> <groupId>org.apache.hudi</groupId>
<version>0.4.8-SNAPSHOT</version> <version>0.5.0-SNAPSHOT</version>
</parent> </parent>
<modelVersion>4.0.0</modelVersion> <modelVersion>4.0.0</modelVersion>
<packaging>pom</packaging> <packaging>pom</packaging>
<artifactId>hoodie-hadoop-base-docker</artifactId> <artifactId>hudi-hadoop-base-docker</artifactId>
<description>Base Docker Image with Hoodie</description> <description>Base Docker Image with Hoodie</description>
@@ -34,8 +34,8 @@
<dependencies> <dependencies>
<dependency> <dependency>
<groupId>com.uber.hoodie</groupId> <groupId>org.apache.hudi</groupId>
<artifactId>hoodie-hadoop-docker</artifactId> <artifactId>hudi-hadoop-docker</artifactId>
<version>${project.version}</version> <version>${project.version}</version>
<type>pom</type> <type>pom</type>
<scope>import</scope> <scope>import</scope>
@@ -43,7 +43,7 @@
</dependencies> </dependencies>
<build> <build>
<finalName>hoodie</finalName> <finalName>hudi</finalName>
<plugins> <plugins>
<!-- Build Docker image --> <!-- Build Docker image -->
<plugin> <plugin>

View File

@@ -17,13 +17,13 @@
--> -->
<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"> <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> <parent>
<artifactId>hoodie-hadoop-docker</artifactId> <artifactId>hudi-hadoop-docker</artifactId>
<groupId>com.uber.hoodie</groupId> <groupId>org.apache.hudi</groupId>
<version>0.4.8-SNAPSHOT</version> <version>0.5.0-SNAPSHOT</version>
</parent> </parent>
<modelVersion>4.0.0</modelVersion> <modelVersion>4.0.0</modelVersion>
<packaging>pom</packaging> <packaging>pom</packaging>
<artifactId>hoodie-hadoop-datanode-docker</artifactId> <artifactId>hudi-hadoop-datanode-docker</artifactId>
<description>Base Docker Image with Hoodie</description> <description>Base Docker Image with Hoodie</description>
@@ -34,8 +34,8 @@
<dependencies> <dependencies>
<dependency> <dependency>
<groupId>com.uber.hoodie</groupId> <groupId>org.apache.hudi</groupId>
<artifactId>hoodie-hadoop-base-docker</artifactId> <artifactId>hudi-hadoop-base-docker</artifactId>
<version>${project.version}</version> <version>${project.version}</version>
<type>pom</type> <type>pom</type>
<scope>import</scope> <scope>import</scope>

View File

@@ -17,13 +17,13 @@
--> -->
<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"> <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> <parent>
<artifactId>hoodie-hadoop-docker</artifactId> <artifactId>hudi-hadoop-docker</artifactId>
<groupId>com.uber.hoodie</groupId> <groupId>org.apache.hudi</groupId>
<version>0.4.8-SNAPSHOT</version> <version>0.5.0-SNAPSHOT</version>
</parent> </parent>
<modelVersion>4.0.0</modelVersion> <modelVersion>4.0.0</modelVersion>
<packaging>pom</packaging> <packaging>pom</packaging>
<artifactId>hoodie-hadoop-history-docker</artifactId> <artifactId>hudi-hadoop-history-docker</artifactId>
<description>Base Docker Image with Hoodie</description> <description>Base Docker Image with Hoodie</description>
@@ -34,8 +34,8 @@
<dependencies> <dependencies>
<dependency> <dependency>
<groupId>com.uber.hoodie</groupId> <groupId>org.apache.hudi</groupId>
<artifactId>hoodie-hadoop-base-docker</artifactId> <artifactId>hudi-hadoop-base-docker</artifactId>
<version>${project.version}</version> <version>${project.version}</version>
<type>pom</type> <type>pom</type>
<scope>import</scope> <scope>import</scope>

View File

@@ -17,13 +17,13 @@
--> -->
<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"> <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> <parent>
<artifactId>hoodie-hadoop-docker</artifactId> <artifactId>hudi-hadoop-docker</artifactId>
<groupId>com.uber.hoodie</groupId> <groupId>org.apache.hudi</groupId>
<version>0.4.8-SNAPSHOT</version> <version>0.5.0-SNAPSHOT</version>
</parent> </parent>
<modelVersion>4.0.0</modelVersion> <modelVersion>4.0.0</modelVersion>
<packaging>pom</packaging> <packaging>pom</packaging>
<artifactId>hoodie-hadoop-hive-docker</artifactId> <artifactId>hudi-hadoop-hive-docker</artifactId>
<description>Base Docker Image with Hoodie</description> <description>Base Docker Image with Hoodie</description>
@@ -34,8 +34,8 @@
<dependencies> <dependencies>
<dependency> <dependency>
<groupId>com.uber.hoodie</groupId> <groupId>org.apache.hudi</groupId>
<artifactId>hoodie-hadoop-base-docker</artifactId> <artifactId>hudi-hadoop-base-docker</artifactId>
<version>${project.version}</version> <version>${project.version}</version>
<type>pom</type> <type>pom</type>
<scope>import</scope> <scope>import</scope>
@@ -52,11 +52,11 @@
<phase>package</phase> <phase>package</phase>
<configuration> <configuration>
<tasks> <tasks>
<copy file="${project.basedir}/../../../../packaging/hoodie-hadoop-mr-bundle/target/hoodie-hadoop-mr-bundle-${project.version}.jar" tofile="target/hoodie-hadoop-mr-bundle.jar" /> <copy file="${project.basedir}/../../../../packaging/hudi-hadoop-mr-bundle/target/hudi-hadoop-mr-bundle-${project.version}.jar" tofile="target/hoodie-hadoop-mr-bundle.jar" />
<copy file="${project.basedir}/../../../../packaging/hoodie-hive-bundle/target/hoodie-hive-bundle-${project.version}.jar" tofile="target/hoodie-hive-bundle.jar" /> <copy file="${project.basedir}/../../../../packaging/hudi-hive-bundle/target/hudi-hive-bundle-${project.version}.jar" tofile="target/hoodie-hive-bundle.jar" />
<copy file="${project.basedir}/../../../../packaging/hoodie-spark-bundle/target/hoodie-spark-bundle-${project.version}.jar" tofile="target/hoodie-spark-bundle.jar" /> <copy file="${project.basedir}/../../../../packaging/hudi-spark-bundle/target/hudi-spark-bundle-${project.version}.jar" tofile="target/hoodie-spark-bundle.jar" />
<copy <copy
file="${project.basedir}/../../../../packaging/hoodie-utilities-bundle/target/hoodie-utilities-bundle-${project.version}.jar" file="${project.basedir}/../../../../packaging/hudi-utilities-bundle/target/hudi-utilities-bundle-${project.version}.jar"
tofile="target/hoodie-utilities.jar"/> tofile="target/hoodie-utilities.jar"/>
</tasks> </tasks>
</configuration> </configuration>

View File

@@ -17,13 +17,13 @@
--> -->
<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"> <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> <parent>
<artifactId>hoodie-hadoop-docker</artifactId> <artifactId>hudi-hadoop-docker</artifactId>
<groupId>com.uber.hoodie</groupId> <groupId>org.apache.hudi</groupId>
<version>0.4.8-SNAPSHOT</version> <version>0.5.0-SNAPSHOT</version>
</parent> </parent>
<modelVersion>4.0.0</modelVersion> <modelVersion>4.0.0</modelVersion>
<packaging>pom</packaging> <packaging>pom</packaging>
<artifactId>hoodie-hadoop-namenode-docker</artifactId> <artifactId>hudi-hadoop-namenode-docker</artifactId>
<description>Base Docker Image with Hoodie</description> <description>Base Docker Image with Hoodie</description>
@@ -34,8 +34,8 @@
<dependencies> <dependencies>
<dependency> <dependency>
<groupId>com.uber.hoodie</groupId> <groupId>org.apache.hudi</groupId>
<artifactId>hoodie-hadoop-base-docker</artifactId> <artifactId>hudi-hadoop-base-docker</artifactId>
<version>${project.version}</version> <version>${project.version}</version>
<type>pom</type> <type>pom</type>
<scope>import</scope> <scope>import</scope>

View File

@@ -17,14 +17,14 @@
--> -->
<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"> <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> <parent>
<artifactId>hoodie</artifactId> <artifactId>hudi</artifactId>
<groupId>com.uber.hoodie</groupId> <groupId>org.apache.hudi</groupId>
<version>0.4.8-SNAPSHOT</version> <version>0.5.0-SNAPSHOT</version>
<relativePath>../../../pom.xml</relativePath> <relativePath>../../../pom.xml</relativePath>
</parent> </parent>
<modelVersion>4.0.0</modelVersion> <modelVersion>4.0.0</modelVersion>
<artifactId>hoodie-hadoop-docker</artifactId> <artifactId>hudi-hadoop-docker</artifactId>
<packaging>pom</packaging> <packaging>pom</packaging>
<modules> <modules>
<module>base</module> <module>base</module>
@@ -40,8 +40,8 @@
<dependencies> <dependencies>
<dependency> <dependency>
<groupId>com.uber.hoodie</groupId> <groupId>org.apache.hudi</groupId>
<artifactId>hoodie-spark-bundle</artifactId> <artifactId>hudi-spark-bundle</artifactId>
<version>${project.version}</version> <version>${project.version}</version>
</dependency> </dependency>
</dependencies> </dependencies>

View File

@@ -17,13 +17,13 @@
--> -->
<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"> <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> <parent>
<artifactId>hoodie-hadoop-docker</artifactId> <artifactId>hudi-hadoop-docker</artifactId>
<groupId>com.uber.hoodie</groupId> <groupId>org.apache.hudi</groupId>
<version>0.4.8-SNAPSHOT</version> <version>0.5.0-SNAPSHOT</version>
</parent> </parent>
<modelVersion>4.0.0</modelVersion> <modelVersion>4.0.0</modelVersion>
<packaging>pom</packaging> <packaging>pom</packaging>
<artifactId>hoodie-hadoop-sparkbase-docker</artifactId> <artifactId>hudi-hadoop-sparkbase-docker</artifactId>
<description>Base Docker Image with Hoodie</description> <description>Base Docker Image with Hoodie</description>
@@ -34,8 +34,8 @@
<dependencies> <dependencies>
<dependency> <dependency>
<groupId>com.uber.hoodie</groupId> <groupId>org.apache.hudi</groupId>
<artifactId>hoodie-hadoop-hive-docker</artifactId> <artifactId>hudi-hadoop-hive-docker</artifactId>
<version>${project.version}</version> <version>${project.version}</version>
<type>pom</type> <type>pom</type>
<scope>import</scope> <scope>import</scope>

View File

@@ -17,13 +17,13 @@
--> -->
<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"> <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> <parent>
<artifactId>hoodie-hadoop-docker</artifactId> <artifactId>hudi-hadoop-docker</artifactId>
<groupId>com.uber.hoodie</groupId> <groupId>org.apache.hudi</groupId>
<version>0.4.8-SNAPSHOT</version> <version>0.5.0-SNAPSHOT</version>
</parent> </parent>
<modelVersion>4.0.0</modelVersion> <modelVersion>4.0.0</modelVersion>
<packaging>pom</packaging> <packaging>pom</packaging>
<artifactId>hoodie-hadoop-sparkadhoc-docker</artifactId> <artifactId>hudi-hadoop-sparkadhoc-docker</artifactId>
<description>Base Docker Image with Hoodie</description> <description>Base Docker Image with Hoodie</description>
@@ -34,8 +34,8 @@
<dependencies> <dependencies>
<dependency> <dependency>
<groupId>com.uber.hoodie</groupId> <groupId>org.apache.hudi</groupId>
<artifactId>hoodie-hadoop-sparkbase-docker</artifactId> <artifactId>hudi-hadoop-sparkbase-docker</artifactId>
<version>${project.version}</version> <version>${project.version}</version>
<type>pom</type> <type>pom</type>
<scope>import</scope> <scope>import</scope>

View File

@@ -17,13 +17,13 @@
--> -->
<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"> <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> <parent>
<artifactId>hoodie-hadoop-docker</artifactId> <artifactId>hudi-hadoop-docker</artifactId>
<groupId>com.uber.hoodie</groupId> <groupId>org.apache.hudi</groupId>
<version>0.4.8-SNAPSHOT</version> <version>0.5.0-SNAPSHOT</version>
</parent> </parent>
<modelVersion>4.0.0</modelVersion> <modelVersion>4.0.0</modelVersion>
<packaging>pom</packaging> <packaging>pom</packaging>
<artifactId>hoodie-hadoop-sparkmaster-docker</artifactId> <artifactId>hudi-hadoop-sparkmaster-docker</artifactId>
<description>Base Docker Image with Hoodie</description> <description>Base Docker Image with Hoodie</description>
@@ -34,8 +34,8 @@
<dependencies> <dependencies>
<dependency> <dependency>
<groupId>com.uber.hoodie</groupId> <groupId>org.apache.hudi</groupId>
<artifactId>hoodie-hadoop-sparkbase-docker</artifactId> <artifactId>hudi-hadoop-sparkbase-docker</artifactId>
<version>${project.version}</version> <version>${project.version}</version>
<type>pom</type> <type>pom</type>
<scope>import</scope> <scope>import</scope>

View File

@@ -17,13 +17,13 @@
--> -->
<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"> <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> <parent>
<artifactId>hoodie-hadoop-docker</artifactId> <artifactId>hudi-hadoop-docker</artifactId>
<groupId>com.uber.hoodie</groupId> <groupId>org.apache.hudi</groupId>
<version>0.4.8-SNAPSHOT</version> <version>0.5.0-SNAPSHOT</version>
</parent> </parent>
<modelVersion>4.0.0</modelVersion> <modelVersion>4.0.0</modelVersion>
<packaging>pom</packaging> <packaging>pom</packaging>
<artifactId>hoodie-hadoop-sparkworker-docker</artifactId> <artifactId>hudi-hadoop-sparkworker-docker</artifactId>
<description>Base Docker Image with Hoodie</description> <description>Base Docker Image with Hoodie</description>
@@ -34,8 +34,8 @@
<dependencies> <dependencies>
<dependency> <dependency>
<groupId>com.uber.hoodie</groupId> <groupId>org.apache.hudi</groupId>
<artifactId>hoodie-hadoop-sparkbase-docker</artifactId> <artifactId>hudi-hadoop-sparkbase-docker</artifactId>
<version>${project.version}</version> <version>${project.version}</version>
<type>pom</type> <type>pom</type>
<scope>import</scope> <scope>import</scope>

View File

@@ -1,36 +0,0 @@
package com.uber.hoodie.utilities;
import static com.uber.hoodie.utilities.deltastreamer.SchedulerConfGenerator.SPARK_SCHEDULER_ALLOCATION_FILE_KEY;
import static com.uber.hoodie.utilities.deltastreamer.SchedulerConfGenerator.SPARK_SCHEDULER_MODE_KEY;
import static org.junit.Assert.assertNotNull;
import static org.junit.Assert.assertNull;
import com.uber.hoodie.common.model.HoodieTableType;
import com.uber.hoodie.utilities.deltastreamer.HoodieDeltaStreamer;
import com.uber.hoodie.utilities.deltastreamer.SchedulerConfGenerator;
import java.util.Map;
import org.junit.Test;
public class SchedulerConfGeneratorTest {
@Test
public void testGenerateSparkSchedulingConf() throws Exception {
HoodieDeltaStreamer.Config cfg = new HoodieDeltaStreamer.Config();
Map<String, String> configs = SchedulerConfGenerator.getSparkSchedulingConfigs(cfg);
assertNull("spark.scheduler.mode not set", configs.get(SPARK_SCHEDULER_ALLOCATION_FILE_KEY));
System.setProperty(SPARK_SCHEDULER_MODE_KEY, "FAIR");
cfg.continuousMode = false;
configs = SchedulerConfGenerator.getSparkSchedulingConfigs(cfg);
assertNull("continuousMode is false", configs.get(SPARK_SCHEDULER_ALLOCATION_FILE_KEY));
cfg.continuousMode = true;
cfg.storageType = HoodieTableType.COPY_ON_WRITE.name();
configs = SchedulerConfGenerator.getSparkSchedulingConfigs(cfg);
assertNull("storageType is not MERGE_ON_READ", configs.get(SPARK_SCHEDULER_ALLOCATION_FILE_KEY));
cfg.storageType = HoodieTableType.MERGE_ON_READ.name();
configs = SchedulerConfGenerator.getSparkSchedulingConfigs(cfg);
assertNotNull("all satisfies", configs.get(SPARK_SCHEDULER_ALLOCATION_FILE_KEY));
}
}

View File

@@ -1,7 +1,7 @@
#!/usr/bin/env bash #!/usr/bin/env bash
DIR="$( cd "$( dirname "${BASH_SOURCE[0]}" )" && pwd )" DIR="$( cd "$( dirname "${BASH_SOURCE[0]}" )" && pwd )"
HOODIE_JAR=`ls $DIR/target/hoodie-cli-*-SNAPSHOT.jar | grep -v source | grep -v javadoc` HUDI_JAR=`ls $DIR/target/hudi-cli-*-SNAPSHOT.jar | grep -v source | grep -v javadoc`
if [ -z "$HADOOP_CONF_DIR" ]; then if [ -z "$HADOOP_CONF_DIR" ]; then
echo "setting hadoop conf dir" echo "setting hadoop conf dir"
HADOOP_CONF_DIR="/etc/hadoop/conf" HADOOP_CONF_DIR="/etc/hadoop/conf"
@@ -13,4 +13,5 @@ fi
if [ -z "$CLIENT_JAR" ]; then if [ -z "$CLIENT_JAR" ]; then
echo "client jar location not set" echo "client jar location not set"
fi fi
java -cp ${HADOOP_CONF_DIR}:${SPARK_CONF_DIR}:$DIR/target/lib/*:$HOODIE_JAR:${CLIENT_JAR} -DSPARK_CONF_DIR=${SPARK_CONF_DIR} -DHADOOP_CONF_DIR=${HADOOP_CONF_DIR} org.springframework.shell.Bootstrap echo "java -cp ${HADOOP_CONF_DIR}:${SPARK_CONF_DIR}:$DIR/target/lib/*:$HUDI_JAR:${CLIENT_JAR} -DSPARK_CONF_DIR=${SPARK_CONF_DIR} -DHADOOP_CONF_DIR=${HADOOP_CONF_DIR} org.springframework.shell.Bootstrap"
java -cp ${HADOOP_CONF_DIR}:${SPARK_CONF_DIR}:$DIR/target/lib/*:$HUDI_JAR:${CLIENT_JAR} -DSPARK_CONF_DIR=${SPARK_CONF_DIR} -DHADOOP_CONF_DIR=${HADOOP_CONF_DIR} org.springframework.shell.Bootstrap

View File

@@ -17,13 +17,13 @@
--> -->
<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"> <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> <parent>
<artifactId>hoodie</artifactId> <artifactId>hudi</artifactId>
<groupId>com.uber.hoodie</groupId> <groupId>org.apache.hudi</groupId>
<version>0.4.8-SNAPSHOT</version> <version>0.5.0-SNAPSHOT</version>
</parent> </parent>
<modelVersion>4.0.0</modelVersion> <modelVersion>4.0.0</modelVersion>
<artifactId>hoodie-cli</artifactId> <artifactId>hudi-cli</artifactId>
<packaging>jar</packaging> <packaging>jar</packaging>
<properties> <properties>
@@ -172,14 +172,14 @@
</dependency> </dependency>
<dependency> <dependency>
<groupId>com.uber.hoodie</groupId> <groupId>org.apache.hudi</groupId>
<artifactId>hoodie-hive</artifactId> <artifactId>hudi-hive</artifactId>
<version>${project.version}</version> <version>${project.version}</version>
</dependency> </dependency>
<dependency> <dependency>
<groupId>com.uber.hoodie</groupId> <groupId>org.apache.hudi</groupId>
<artifactId>hoodie-client</artifactId> <artifactId>hudi-client</artifactId>
<version>${project.version}</version> <version>${project.version}</version>
</dependency> </dependency>
@@ -194,8 +194,8 @@
</dependency> </dependency>
<dependency> <dependency>
<groupId>com.uber.hoodie</groupId> <groupId>org.apache.hudi</groupId>
<artifactId>hoodie-common</artifactId> <artifactId>hudi-common</artifactId>
<version>${project.version}</version> <version>${project.version}</version>
</dependency> </dependency>
<dependency> <dependency>
@@ -215,8 +215,8 @@
<version>2.9.6</version> <version>2.9.6</version>
</dependency> </dependency>
<dependency> <dependency>
<groupId>com.uber.hoodie</groupId> <groupId>org.apache.hudi</groupId>
<artifactId>hoodie-utilities</artifactId> <artifactId>hudi-utilities</artifactId>
<version>${project.version}</version> <version>${project.version}</version>
</dependency> </dependency>

View File

@@ -16,14 +16,14 @@
* limitations under the License. * limitations under the License.
*/ */
package com.uber.hoodie.cli; package org.apache.hudi.cli;
import com.uber.hoodie.common.table.HoodieTableMetaClient;
import com.uber.hoodie.common.util.ConsistencyGuardConfig;
import com.uber.hoodie.common.util.FSUtils;
import java.io.IOException; import java.io.IOException;
import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.FileSystem;
import org.apache.hudi.common.table.HoodieTableMetaClient;
import org.apache.hudi.common.util.ConsistencyGuardConfig;
import org.apache.hudi.common.util.FSUtils;
public class HoodieCLI { public class HoodieCLI {

View File

@@ -16,7 +16,7 @@
* limitations under the License. * limitations under the License.
*/ */
package com.uber.hoodie.cli; package org.apache.hudi.cli;
import org.springframework.core.Ordered; import org.springframework.core.Ordered;
import org.springframework.core.annotation.Order; import org.springframework.core.annotation.Order;

View File

@@ -16,13 +16,13 @@
* limitations under the License. * limitations under the License.
*/ */
package com.uber.hoodie.cli; package org.apache.hudi.cli;
import com.jakewharton.fliptables.FlipTable; import com.jakewharton.fliptables.FlipTable;
import com.uber.hoodie.common.util.Option;
import java.util.List; import java.util.List;
import java.util.Map; import java.util.Map;
import java.util.function.Function; import java.util.function.Function;
import org.apache.hudi.common.util.Option;
/** /**
* Helper class to render table for hoodie-cli * Helper class to render table for hoodie-cli

View File

@@ -16,7 +16,7 @@
* limitations under the License. * limitations under the License.
*/ */
package com.uber.hoodie.cli; package org.apache.hudi.cli;
import org.springframework.core.Ordered; import org.springframework.core.Ordered;
import org.springframework.core.annotation.Order; import org.springframework.core.annotation.Order;
@@ -33,16 +33,16 @@ public class HoodiePrompt extends DefaultPromptProvider {
String tableName = HoodieCLI.tableMetadata.getTableConfig().getTableName(); String tableName = HoodieCLI.tableMetadata.getTableConfig().getTableName();
switch (HoodieCLI.state) { switch (HoodieCLI.state) {
case INIT: case INIT:
return "hoodie->"; return "hudi->";
case DATASET: case DATASET:
return "hoodie:" + tableName + "->"; return "hudi:" + tableName + "->";
case SYNC: case SYNC:
return "hoodie:" + tableName + " <==> " + HoodieCLI.syncTableMetadata.getTableConfig().getTableName() + "->"; return "hudi:" + tableName + " <==> " + HoodieCLI.syncTableMetadata.getTableConfig().getTableName() + "->";
default: default:
return "hoodie:" + tableName + "->"; return "hudi:" + tableName + "->";
} }
} }
return "hoodie->"; return "hudi->";
} }
@Override @Override

View File

@@ -16,7 +16,7 @@
* limitations under the License. * limitations under the License.
*/ */
package com.uber.hoodie.cli; package org.apache.hudi.cli;
import org.springframework.core.Ordered; import org.springframework.core.Ordered;
import org.springframework.core.annotation.Order; import org.springframework.core.annotation.Order;
@@ -28,14 +28,18 @@ import org.springframework.stereotype.Component;
@Order(Ordered.HIGHEST_PRECEDENCE) @Order(Ordered.HIGHEST_PRECEDENCE)
public class HoodieSplashScreen extends DefaultBannerProvider { public class HoodieSplashScreen extends DefaultBannerProvider {
static {
System.out.println("HoodieSplashScreen loaded");
}
private static String screen = "============================================" + OsUtils.LINE_SEPARATOR private static String screen = "============================================" + OsUtils.LINE_SEPARATOR
+ "* *" + OsUtils.LINE_SEPARATOR + "* *" + OsUtils.LINE_SEPARATOR
+ "* _ _ _ _ *" + OsUtils.LINE_SEPARATOR + "* _ _ _ _ *" + OsUtils.LINE_SEPARATOR
+ "* | | | | | (_) *" + OsUtils.LINE_SEPARATOR + "* | | | | | | (_) *" + OsUtils.LINE_SEPARATOR
+ "* | |__| | ___ ___ __| |_ ___ *" + OsUtils.LINE_SEPARATOR + "* | |__| | __| | - *" + OsUtils.LINE_SEPARATOR
+ "* | __ |/ _ \\ / _ \\ / _` | |/ _ \\ *" + OsUtils.LINE_SEPARATOR + "* | __ || | / _` | || *" + OsUtils.LINE_SEPARATOR
+ "* | | | | (_) | (_) | (_| | | __/ *" + OsUtils.LINE_SEPARATOR + "* | | | || || (_| | || *" + OsUtils.LINE_SEPARATOR
+ "* |_| |_|\\___/ \\___/ \\__,_|_|\\___| *" + OsUtils.LINE_SEPARATOR + "* |_| |_|\\___/ \\____/ || *" + OsUtils.LINE_SEPARATOR
+ "* *" + OsUtils.LINE_SEPARATOR + "* *" + OsUtils.LINE_SEPARATOR
+ "============================================" + OsUtils.LINE_SEPARATOR; + "============================================" + OsUtils.LINE_SEPARATOR;

View File

@@ -16,7 +16,7 @@
* limitations under the License. * limitations under the License.
*/ */
package com.uber.hoodie.cli; package org.apache.hudi.cli;
import java.io.IOException; import java.io.IOException;
import org.springframework.shell.Bootstrap; import org.springframework.shell.Bootstrap;
@@ -27,6 +27,8 @@ public class Main {
* Main class that delegates to Spring Shell's Bootstrap class in order to simplify debugging inside an IDE * Main class that delegates to Spring Shell's Bootstrap class in order to simplify debugging inside an IDE
*/ */
public static void main(String[] args) throws IOException { public static void main(String[] args) throws IOException {
System.out.println("Main called");
new HoodieSplashScreen();
Bootstrap.main(args); Bootstrap.main(args);
} }
} }

View File

@@ -16,9 +16,8 @@
* limitations under the License. * limitations under the License.
*/ */
package com.uber.hoodie.cli; package org.apache.hudi.cli;
import com.uber.hoodie.common.util.Option;
import java.util.ArrayList; import java.util.ArrayList;
import java.util.Arrays; import java.util.Arrays;
import java.util.Comparator; import java.util.Comparator;
@@ -29,6 +28,7 @@ import java.util.function.Consumer;
import java.util.function.Function; import java.util.function.Function;
import java.util.stream.Collectors; import java.util.stream.Collectors;
import java.util.stream.IntStream; import java.util.stream.IntStream;
import org.apache.hudi.common.util.Option;
/** /**
* Table to be rendered. This class takes care of ordering * Table to be rendered. This class takes care of ordering

View File

@@ -16,7 +16,7 @@
* limitations under the License. * limitations under the License.
*/ */
package com.uber.hoodie.cli; package org.apache.hudi.cli;
import java.util.ArrayList; import java.util.ArrayList;
import java.util.List; import java.util.List;

View File

@@ -16,18 +16,8 @@
* limitations under the License. * limitations under the License.
*/ */
package com.uber.hoodie.cli.commands; package org.apache.hudi.cli.commands;
import com.uber.hoodie.avro.model.HoodieArchivedMetaEntry;
import com.uber.hoodie.avro.model.HoodieCommitMetadata;
import com.uber.hoodie.cli.HoodieCLI;
import com.uber.hoodie.cli.HoodiePrintHelper;
import com.uber.hoodie.cli.TableHeader;
import com.uber.hoodie.common.model.HoodieLogFile;
import com.uber.hoodie.common.table.HoodieTimeline;
import com.uber.hoodie.common.table.log.HoodieLogFormat;
import com.uber.hoodie.common.table.log.block.HoodieAvroDataBlock;
import com.uber.hoodie.common.util.FSUtils;
import java.io.IOException; import java.io.IOException;
import java.util.ArrayList; import java.util.ArrayList;
import java.util.HashMap; import java.util.HashMap;
@@ -38,6 +28,17 @@ import org.apache.avro.generic.IndexedRecord;
import org.apache.avro.specific.SpecificData; import org.apache.avro.specific.SpecificData;
import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.Path;
import org.apache.hudi.avro.model.HoodieArchivedMetaEntry;
import org.apache.hudi.avro.model.HoodieCommitMetadata;
import org.apache.hudi.cli.HoodieCLI;
import org.apache.hudi.cli.HoodiePrintHelper;
import org.apache.hudi.cli.TableHeader;
import org.apache.hudi.common.model.HoodieLogFile;
import org.apache.hudi.common.table.HoodieTimeline;
import org.apache.hudi.common.table.log.HoodieLogFormat;
import org.apache.hudi.common.table.log.HoodieLogFormat.Reader;
import org.apache.hudi.common.table.log.block.HoodieAvroDataBlock;
import org.apache.hudi.common.util.FSUtils;
import org.springframework.shell.core.CommandMarker; import org.springframework.shell.core.CommandMarker;
import org.springframework.shell.core.annotation.CliAvailabilityIndicator; import org.springframework.shell.core.annotation.CliAvailabilityIndicator;
import org.springframework.shell.core.annotation.CliCommand; import org.springframework.shell.core.annotation.CliCommand;
@@ -71,7 +72,7 @@ public class ArchivedCommitsCommand implements CommandMarker {
List<Comparable[]> allStats = new ArrayList<>(); List<Comparable[]> allStats = new ArrayList<>();
for (FileStatus fs : fsStatuses) { for (FileStatus fs : fsStatuses) {
//read the archived file //read the archived file
HoodieLogFormat.Reader reader = HoodieLogFormat.newReader(FSUtils.getFs(basePath, HoodieCLI.conf), Reader reader = HoodieLogFormat.newReader(FSUtils.getFs(basePath, HoodieCLI.conf),
new HoodieLogFile(fs.getPath()), HoodieArchivedMetaEntry.getClassSchema()); new HoodieLogFile(fs.getPath()), HoodieArchivedMetaEntry.getClassSchema());
List<IndexedRecord> readRecords = new ArrayList<>(); List<IndexedRecord> readRecords = new ArrayList<>();

View File

@@ -16,17 +16,8 @@
* limitations under the License. * limitations under the License.
*/ */
package com.uber.hoodie.cli.commands; package org.apache.hudi.cli.commands;
import com.uber.hoodie.avro.model.HoodieCleanMetadata;
import com.uber.hoodie.avro.model.HoodieCleanPartitionMetadata;
import com.uber.hoodie.cli.HoodieCLI;
import com.uber.hoodie.cli.HoodiePrintHelper;
import com.uber.hoodie.cli.TableHeader;
import com.uber.hoodie.common.table.HoodieTimeline;
import com.uber.hoodie.common.table.timeline.HoodieActiveTimeline;
import com.uber.hoodie.common.table.timeline.HoodieInstant;
import com.uber.hoodie.common.util.AvroUtils;
import java.io.IOException; import java.io.IOException;
import java.util.ArrayList; import java.util.ArrayList;
import java.util.Collections; import java.util.Collections;
@@ -34,6 +25,15 @@ import java.util.HashMap;
import java.util.List; import java.util.List;
import java.util.Map; import java.util.Map;
import java.util.stream.Collectors; import java.util.stream.Collectors;
import org.apache.hudi.avro.model.HoodieCleanMetadata;
import org.apache.hudi.avro.model.HoodieCleanPartitionMetadata;
import org.apache.hudi.cli.HoodieCLI;
import org.apache.hudi.cli.HoodiePrintHelper;
import org.apache.hudi.cli.TableHeader;
import org.apache.hudi.common.table.HoodieTimeline;
import org.apache.hudi.common.table.timeline.HoodieActiveTimeline;
import org.apache.hudi.common.table.timeline.HoodieInstant;
import org.apache.hudi.common.util.AvroUtils;
import org.springframework.shell.core.CommandMarker; import org.springframework.shell.core.CommandMarker;
import org.springframework.shell.core.annotation.CliAvailabilityIndicator; import org.springframework.shell.core.annotation.CliAvailabilityIndicator;
import org.springframework.shell.core.annotation.CliCommand; import org.springframework.shell.core.annotation.CliCommand;

View File

@@ -16,20 +16,8 @@
* limitations under the License. * limitations under the License.
*/ */
package com.uber.hoodie.cli.commands; package org.apache.hudi.cli.commands;
import com.uber.hoodie.cli.HoodieCLI;
import com.uber.hoodie.cli.HoodiePrintHelper;
import com.uber.hoodie.cli.TableHeader;
import com.uber.hoodie.cli.utils.InputStreamConsumer;
import com.uber.hoodie.cli.utils.SparkUtil;
import com.uber.hoodie.common.model.HoodieCommitMetadata;
import com.uber.hoodie.common.model.HoodieWriteStat;
import com.uber.hoodie.common.table.HoodieTableMetaClient;
import com.uber.hoodie.common.table.HoodieTimeline;
import com.uber.hoodie.common.table.timeline.HoodieActiveTimeline;
import com.uber.hoodie.common.table.timeline.HoodieInstant;
import com.uber.hoodie.common.util.NumericUtils;
import java.io.IOException; import java.io.IOException;
import java.util.ArrayList; import java.util.ArrayList;
import java.util.Collections; import java.util.Collections;
@@ -38,6 +26,18 @@ import java.util.List;
import java.util.Map; import java.util.Map;
import java.util.function.Function; import java.util.function.Function;
import java.util.stream.Collectors; import java.util.stream.Collectors;
import org.apache.hudi.cli.HoodieCLI;
import org.apache.hudi.cli.HoodiePrintHelper;
import org.apache.hudi.cli.TableHeader;
import org.apache.hudi.cli.utils.InputStreamConsumer;
import org.apache.hudi.cli.utils.SparkUtil;
import org.apache.hudi.common.model.HoodieCommitMetadata;
import org.apache.hudi.common.model.HoodieWriteStat;
import org.apache.hudi.common.table.HoodieTableMetaClient;
import org.apache.hudi.common.table.HoodieTimeline;
import org.apache.hudi.common.table.timeline.HoodieActiveTimeline;
import org.apache.hudi.common.table.timeline.HoodieInstant;
import org.apache.hudi.common.util.NumericUtils;
import org.apache.spark.launcher.SparkLauncher; import org.apache.spark.launcher.SparkLauncher;
import org.springframework.shell.core.CommandMarker; import org.springframework.shell.core.CommandMarker;
import org.springframework.shell.core.annotation.CliAvailabilityIndicator; import org.springframework.shell.core.annotation.CliAvailabilityIndicator;

View File

@@ -16,28 +16,8 @@
* limitations under the License. * limitations under the License.
*/ */
package com.uber.hoodie.cli.commands; package org.apache.hudi.cli.commands;
import static com.uber.hoodie.common.table.HoodieTimeline.COMPACTION_ACTION;
import com.uber.hoodie.CompactionAdminClient.RenameOpResult;
import com.uber.hoodie.CompactionAdminClient.ValidationOpResult;
import com.uber.hoodie.avro.model.HoodieCompactionOperation;
import com.uber.hoodie.avro.model.HoodieCompactionPlan;
import com.uber.hoodie.cli.HoodieCLI;
import com.uber.hoodie.cli.HoodiePrintHelper;
import com.uber.hoodie.cli.TableHeader;
import com.uber.hoodie.cli.commands.SparkMain.SparkCommand;
import com.uber.hoodie.cli.utils.InputStreamConsumer;
import com.uber.hoodie.cli.utils.SparkUtil;
import com.uber.hoodie.common.model.HoodieTableType;
import com.uber.hoodie.common.table.HoodieTimeline;
import com.uber.hoodie.common.table.timeline.HoodieActiveTimeline;
import com.uber.hoodie.common.table.timeline.HoodieInstant;
import com.uber.hoodie.common.table.timeline.HoodieInstant.State;
import com.uber.hoodie.common.util.AvroUtils;
import com.uber.hoodie.common.util.Option;
import com.uber.hoodie.exception.HoodieIOException;
import java.io.IOException; import java.io.IOException;
import java.io.ObjectInputStream; import java.io.ObjectInputStream;
import java.util.ArrayList; import java.util.ArrayList;
@@ -52,6 +32,24 @@ import java.util.stream.Collectors;
import org.apache.hadoop.fs.FSDataInputStream; import org.apache.hadoop.fs.FSDataInputStream;
import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.Path;
import org.apache.hudi.CompactionAdminClient.RenameOpResult;
import org.apache.hudi.CompactionAdminClient.ValidationOpResult;
import org.apache.hudi.avro.model.HoodieCompactionOperation;
import org.apache.hudi.avro.model.HoodieCompactionPlan;
import org.apache.hudi.cli.HoodieCLI;
import org.apache.hudi.cli.HoodiePrintHelper;
import org.apache.hudi.cli.TableHeader;
import org.apache.hudi.cli.commands.SparkMain.SparkCommand;
import org.apache.hudi.cli.utils.InputStreamConsumer;
import org.apache.hudi.cli.utils.SparkUtil;
import org.apache.hudi.common.model.HoodieTableType;
import org.apache.hudi.common.table.HoodieTimeline;
import org.apache.hudi.common.table.timeline.HoodieActiveTimeline;
import org.apache.hudi.common.table.timeline.HoodieInstant;
import org.apache.hudi.common.table.timeline.HoodieInstant.State;
import org.apache.hudi.common.util.AvroUtils;
import org.apache.hudi.common.util.Option;
import org.apache.hudi.exception.HoodieIOException;
import org.apache.log4j.LogManager; import org.apache.log4j.LogManager;
import org.apache.log4j.Logger; import org.apache.log4j.Logger;
import org.apache.spark.launcher.SparkLauncher; import org.apache.spark.launcher.SparkLauncher;
@@ -98,7 +96,7 @@ public class CompactionCommand implements CommandMarker {
for (int i = 0; i < instants.size(); i++) { for (int i = 0; i < instants.size(); i++) {
HoodieInstant instant = instants.get(i); HoodieInstant instant = instants.get(i);
HoodieCompactionPlan workload = null; HoodieCompactionPlan workload = null;
if (!instant.getAction().equals(COMPACTION_ACTION)) { if (!instant.getAction().equals(HoodieTimeline.COMPACTION_ACTION)) {
try { try {
// This could be a completed compaction. Assume a compaction request file is present but skip if fails // This could be a completed compaction. Assume a compaction request file is present but skip if fails
workload = AvroUtils.deserializeCompactionPlan( workload = AvroUtils.deserializeCompactionPlan(

View File

@@ -16,19 +16,19 @@
* limitations under the License. * limitations under the License.
*/ */
package com.uber.hoodie.cli.commands; package org.apache.hudi.cli.commands;
import com.uber.hoodie.cli.HoodieCLI;
import com.uber.hoodie.cli.HoodiePrintHelper;
import com.uber.hoodie.cli.TableHeader;
import com.uber.hoodie.common.model.HoodieTableType;
import com.uber.hoodie.common.table.HoodieTableMetaClient;
import com.uber.hoodie.common.util.ConsistencyGuardConfig;
import com.uber.hoodie.exception.DatasetNotFoundException;
import java.io.IOException; import java.io.IOException;
import java.util.ArrayList; import java.util.ArrayList;
import java.util.HashMap; import java.util.HashMap;
import java.util.List; import java.util.List;
import org.apache.hudi.cli.HoodieCLI;
import org.apache.hudi.cli.HoodiePrintHelper;
import org.apache.hudi.cli.TableHeader;
import org.apache.hudi.common.model.HoodieTableType;
import org.apache.hudi.common.table.HoodieTableMetaClient;
import org.apache.hudi.common.util.ConsistencyGuardConfig;
import org.apache.hudi.exception.DatasetNotFoundException;
import org.springframework.shell.core.CommandMarker; import org.springframework.shell.core.CommandMarker;
import org.springframework.shell.core.annotation.CliAvailabilityIndicator; import org.springframework.shell.core.annotation.CliAvailabilityIndicator;
import org.springframework.shell.core.annotation.CliCommand; import org.springframework.shell.core.annotation.CliCommand;
@@ -38,6 +38,10 @@ import org.springframework.stereotype.Component;
@Component @Component
public class DatasetsCommand implements CommandMarker { public class DatasetsCommand implements CommandMarker {
static {
System.out.println("DatasetsCommand getting loaded");
}
@CliCommand(value = "connect", help = "Connect to a hoodie dataset") @CliCommand(value = "connect", help = "Connect to a hoodie dataset")
public String connect( public String connect(
@CliOption(key = {"path"}, mandatory = true, help = "Base Path of the dataset") final String path, @CliOption(key = {"path"}, mandatory = true, help = "Base Path of the dataset") final String path,
@@ -77,7 +81,7 @@ public class DatasetsCommand implements CommandMarker {
@CliOption(key = {"tableName"}, mandatory = true, help = "Hoodie Table Name") final String name, @CliOption(key = {"tableName"}, mandatory = true, help = "Hoodie Table Name") final String name,
@CliOption(key = {"tableType"}, unspecifiedDefaultValue = "COPY_ON_WRITE", @CliOption(key = {"tableType"}, unspecifiedDefaultValue = "COPY_ON_WRITE",
help = "Hoodie Table Type. Must be one of : COPY_ON_WRITE or MERGE_ON_READ") final String tableTypeStr, help = "Hoodie Table Type. Must be one of : COPY_ON_WRITE or MERGE_ON_READ") final String tableTypeStr,
@CliOption(key = {"payloadClass"}, unspecifiedDefaultValue = "com.uber.hoodie.common.model.HoodieAvroPayload", @CliOption(key = {"payloadClass"}, unspecifiedDefaultValue = "org.apache.hudi.common.model.HoodieAvroPayload",
help = "Payload Class") final String payloadClass) throws IOException { help = "Payload Class") final String payloadClass) throws IOException {
boolean initialized = HoodieCLI.initConf(); boolean initialized = HoodieCLI.initConf();

View File

@@ -16,19 +16,8 @@
* limitations under the License. * limitations under the License.
*/ */
package com.uber.hoodie.cli.commands; package org.apache.hudi.cli.commands;
import com.uber.hoodie.cli.HoodieCLI;
import com.uber.hoodie.cli.HoodiePrintHelper;
import com.uber.hoodie.cli.TableHeader;
import com.uber.hoodie.common.model.FileSlice;
import com.uber.hoodie.common.table.HoodieTableMetaClient;
import com.uber.hoodie.common.table.HoodieTimeline;
import com.uber.hoodie.common.table.timeline.HoodieDefaultTimeline;
import com.uber.hoodie.common.table.timeline.HoodieInstant;
import com.uber.hoodie.common.table.view.HoodieTableFileSystemView;
import com.uber.hoodie.common.util.NumericUtils;
import com.uber.hoodie.common.util.Option;
import java.io.IOException; import java.io.IOException;
import java.io.Serializable; import java.io.Serializable;
import java.util.ArrayList; import java.util.ArrayList;
@@ -42,6 +31,17 @@ import java.util.stream.Stream;
import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.Path;
import org.apache.hudi.cli.HoodieCLI;
import org.apache.hudi.cli.HoodiePrintHelper;
import org.apache.hudi.cli.TableHeader;
import org.apache.hudi.common.model.FileSlice;
import org.apache.hudi.common.table.HoodieTableMetaClient;
import org.apache.hudi.common.table.HoodieTimeline;
import org.apache.hudi.common.table.timeline.HoodieDefaultTimeline;
import org.apache.hudi.common.table.timeline.HoodieInstant;
import org.apache.hudi.common.table.view.HoodieTableFileSystemView;
import org.apache.hudi.common.util.NumericUtils;
import org.apache.hudi.common.util.Option;
import org.springframework.shell.core.CommandMarker; import org.springframework.shell.core.CommandMarker;
import org.springframework.shell.core.annotation.CliCommand; import org.springframework.shell.core.annotation.CliCommand;
import org.springframework.shell.core.annotation.CliOption; import org.springframework.shell.core.annotation.CliOption;

View File

@@ -16,13 +16,13 @@
* limitations under the License. * limitations under the License.
*/ */
package com.uber.hoodie.cli.commands; package org.apache.hudi.cli.commands;
import com.uber.hoodie.cli.HoodieCLI; import org.apache.hudi.cli.HoodieCLI;
import com.uber.hoodie.cli.commands.SparkMain.SparkCommand; import org.apache.hudi.cli.commands.SparkMain.SparkCommand;
import com.uber.hoodie.cli.utils.InputStreamConsumer; import org.apache.hudi.cli.utils.InputStreamConsumer;
import com.uber.hoodie.cli.utils.SparkUtil; import org.apache.hudi.cli.utils.SparkUtil;
import com.uber.hoodie.utilities.HDFSParquetImporter; import org.apache.hudi.utilities.HDFSParquetImporter.FormatValidator;
import org.apache.log4j.LogManager; import org.apache.log4j.LogManager;
import org.apache.log4j.Logger; import org.apache.log4j.Logger;
import org.apache.spark.launcher.SparkLauncher; import org.apache.spark.launcher.SparkLauncher;
@@ -58,7 +58,7 @@ public class HDFSParquetImportCommand implements CommandMarker {
@CliOption(key = "sparkMemory", mandatory = true, help = "Spark executor memory") final String sparkMemory, @CliOption(key = "sparkMemory", mandatory = true, help = "Spark executor memory") final String sparkMemory,
@CliOption(key = "retry", mandatory = true, help = "Number of retries") final String retry) throws Exception { @CliOption(key = "retry", mandatory = true, help = "Number of retries") final String retry) throws Exception {
(new HDFSParquetImporter.FormatValidator()).validate("format", format); (new FormatValidator()).validate("format", format);
boolean initialized = HoodieCLI.initConf(); boolean initialized = HoodieCLI.initConf();
HoodieCLI.initFS(initialized); HoodieCLI.initFS(initialized);

View File

@@ -16,27 +16,10 @@
* limitations under the License. * limitations under the License.
*/ */
package com.uber.hoodie.cli.commands; package org.apache.hudi.cli.commands;
import com.beust.jcommander.internal.Maps; import com.beust.jcommander.internal.Maps;
import com.fasterxml.jackson.databind.ObjectMapper; import com.fasterxml.jackson.databind.ObjectMapper;
import com.uber.hoodie.cli.HoodieCLI;
import com.uber.hoodie.cli.HoodiePrintHelper;
import com.uber.hoodie.cli.TableHeader;
import com.uber.hoodie.common.model.HoodieLogFile;
import com.uber.hoodie.common.model.HoodieRecord;
import com.uber.hoodie.common.model.HoodieRecordPayload;
import com.uber.hoodie.common.table.log.HoodieLogFormat;
import com.uber.hoodie.common.table.log.HoodieMergedLogRecordScanner;
import com.uber.hoodie.common.table.log.block.HoodieAvroDataBlock;
import com.uber.hoodie.common.table.log.block.HoodieCorruptBlock;
import com.uber.hoodie.common.table.log.block.HoodieLogBlock;
import com.uber.hoodie.common.table.log.block.HoodieLogBlock.HeaderMetadataType;
import com.uber.hoodie.common.table.log.block.HoodieLogBlock.HoodieLogBlockType;
import com.uber.hoodie.common.util.Option;
import com.uber.hoodie.config.HoodieCompactionConfig;
import com.uber.hoodie.config.HoodieMemoryConfig;
import com.uber.hoodie.hive.util.SchemaUtil;
import java.io.IOException; import java.io.IOException;
import java.util.ArrayList; import java.util.ArrayList;
import java.util.Arrays; import java.util.Arrays;
@@ -49,6 +32,24 @@ import org.apache.avro.generic.IndexedRecord;
import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.Path;
import org.apache.hudi.cli.HoodieCLI;
import org.apache.hudi.cli.HoodiePrintHelper;
import org.apache.hudi.cli.TableHeader;
import org.apache.hudi.common.model.HoodieLogFile;
import org.apache.hudi.common.model.HoodieRecord;
import org.apache.hudi.common.model.HoodieRecordPayload;
import org.apache.hudi.common.table.log.HoodieLogFormat;
import org.apache.hudi.common.table.log.HoodieLogFormat.Reader;
import org.apache.hudi.common.table.log.HoodieMergedLogRecordScanner;
import org.apache.hudi.common.table.log.block.HoodieAvroDataBlock;
import org.apache.hudi.common.table.log.block.HoodieCorruptBlock;
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.log.block.HoodieLogBlock.HoodieLogBlockType;
import org.apache.hudi.common.util.Option;
import org.apache.hudi.config.HoodieCompactionConfig;
import org.apache.hudi.config.HoodieMemoryConfig;
import org.apache.hudi.hive.util.SchemaUtil;
import org.springframework.shell.core.CommandMarker; import org.springframework.shell.core.CommandMarker;
import org.springframework.shell.core.annotation.CliAvailabilityIndicator; import org.springframework.shell.core.annotation.CliAvailabilityIndicator;
import org.springframework.shell.core.annotation.CliCommand; import org.springframework.shell.core.annotation.CliCommand;
@@ -90,7 +91,7 @@ public class HoodieLogFileCommand implements CommandMarker {
FileStatus[] fsStatus = fs.listStatus(new Path(logFilePath)); FileStatus[] fsStatus = fs.listStatus(new Path(logFilePath));
Schema writerSchema = new AvroSchemaConverter().convert( Schema writerSchema = new AvroSchemaConverter().convert(
SchemaUtil.readSchemaFromLogFile(HoodieCLI.tableMetadata.getFs(), new Path(logFilePath))); SchemaUtil.readSchemaFromLogFile(HoodieCLI.tableMetadata.getFs(), new Path(logFilePath)));
HoodieLogFormat.Reader reader = HoodieLogFormat Reader reader = HoodieLogFormat
.newReader(fs, new HoodieLogFile(fsStatus[0].getPath()), writerSchema); .newReader(fs, new HoodieLogFile(fsStatus[0].getPath()), writerSchema);
// read the avro blocks // read the avro blocks

View File

@@ -16,16 +16,16 @@
* limitations under the License. * limitations under the License.
*/ */
package com.uber.hoodie.cli.commands; package org.apache.hudi.cli.commands;
import com.uber.hoodie.cli.HoodieCLI;
import com.uber.hoodie.cli.utils.CommitUtil;
import com.uber.hoodie.cli.utils.HiveUtil;
import com.uber.hoodie.common.table.HoodieTableMetaClient;
import com.uber.hoodie.common.table.HoodieTimeline;
import com.uber.hoodie.common.table.timeline.HoodieInstant;
import java.util.List; import java.util.List;
import java.util.stream.Collectors; import java.util.stream.Collectors;
import org.apache.hudi.cli.HoodieCLI;
import org.apache.hudi.cli.utils.CommitUtil;
import org.apache.hudi.cli.utils.HiveUtil;
import org.apache.hudi.common.table.HoodieTableMetaClient;
import org.apache.hudi.common.table.HoodieTimeline;
import org.apache.hudi.common.table.timeline.HoodieInstant;
import org.springframework.shell.core.CommandMarker; import org.springframework.shell.core.CommandMarker;
import org.springframework.shell.core.annotation.CliAvailabilityIndicator; import org.springframework.shell.core.annotation.CliAvailabilityIndicator;
import org.springframework.shell.core.annotation.CliCommand; import org.springframework.shell.core.annotation.CliCommand;

View File

@@ -16,17 +16,17 @@
* limitations under the License. * limitations under the License.
*/ */
package com.uber.hoodie.cli.commands; package org.apache.hudi.cli.commands;
import com.uber.hoodie.cli.HoodieCLI;
import com.uber.hoodie.cli.HoodiePrintHelper;
import com.uber.hoodie.cli.utils.InputStreamConsumer;
import com.uber.hoodie.cli.utils.SparkUtil;
import com.uber.hoodie.common.model.HoodiePartitionMetadata;
import com.uber.hoodie.common.util.FSUtils;
import java.io.IOException; import java.io.IOException;
import java.util.List; import java.util.List;
import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.Path;
import org.apache.hudi.cli.HoodieCLI;
import org.apache.hudi.cli.HoodiePrintHelper;
import org.apache.hudi.cli.utils.InputStreamConsumer;
import org.apache.hudi.cli.utils.SparkUtil;
import org.apache.hudi.common.model.HoodiePartitionMetadata;
import org.apache.hudi.common.util.FSUtils;
import org.apache.spark.launcher.SparkLauncher; import org.apache.spark.launcher.SparkLauncher;
import org.springframework.shell.core.CommandMarker; import org.springframework.shell.core.CommandMarker;
import org.springframework.shell.core.annotation.CliAvailabilityIndicator; import org.springframework.shell.core.annotation.CliAvailabilityIndicator;

View File

@@ -16,27 +16,27 @@
* limitations under the License. * limitations under the License.
*/ */
package com.uber.hoodie.cli.commands; package org.apache.hudi.cli.commands;
import static com.uber.hoodie.common.table.HoodieTimeline.ROLLBACK_ACTION; import static org.apache.hudi.common.table.HoodieTimeline.ROLLBACK_ACTION;
import com.google.common.collect.ImmutableSet; import com.google.common.collect.ImmutableSet;
import com.uber.hoodie.avro.model.HoodieRollbackMetadata;
import com.uber.hoodie.cli.HoodieCLI;
import com.uber.hoodie.cli.HoodiePrintHelper;
import com.uber.hoodie.cli.TableHeader;
import com.uber.hoodie.common.table.HoodieTableMetaClient;
import com.uber.hoodie.common.table.HoodieTimeline;
import com.uber.hoodie.common.table.timeline.HoodieActiveTimeline;
import com.uber.hoodie.common.table.timeline.HoodieInstant;
import com.uber.hoodie.common.table.timeline.HoodieInstant.State;
import com.uber.hoodie.common.util.AvroUtils;
import java.io.IOException; import java.io.IOException;
import java.util.ArrayList; import java.util.ArrayList;
import java.util.HashMap; import java.util.HashMap;
import java.util.List; import java.util.List;
import java.util.stream.Stream; import java.util.stream.Stream;
import org.apache.commons.lang3.tuple.Pair; import org.apache.commons.lang3.tuple.Pair;
import org.apache.hudi.avro.model.HoodieRollbackMetadata;
import org.apache.hudi.cli.HoodieCLI;
import org.apache.hudi.cli.HoodiePrintHelper;
import org.apache.hudi.cli.TableHeader;
import org.apache.hudi.common.table.HoodieTableMetaClient;
import org.apache.hudi.common.table.HoodieTimeline;
import org.apache.hudi.common.table.timeline.HoodieActiveTimeline;
import org.apache.hudi.common.table.timeline.HoodieInstant;
import org.apache.hudi.common.table.timeline.HoodieInstant.State;
import org.apache.hudi.common.util.AvroUtils;
import org.springframework.shell.core.CommandMarker; import org.springframework.shell.core.CommandMarker;
import org.springframework.shell.core.annotation.CliCommand; import org.springframework.shell.core.annotation.CliCommand;
import org.springframework.shell.core.annotation.CliOption; import org.springframework.shell.core.annotation.CliOption;
@@ -126,7 +126,7 @@ public class RollbacksCommand implements CommandMarker {
class RollbackTimeline extends HoodieActiveTimeline { class RollbackTimeline extends HoodieActiveTimeline {
public RollbackTimeline(HoodieTableMetaClient metaClient) { public RollbackTimeline(HoodieTableMetaClient metaClient) {
super(metaClient, ImmutableSet.<String>builder().add(ROLLBACK_EXTENSION).build()); super(metaClient, ImmutableSet.<String>builder().add(HoodieTimeline.ROLLBACK_EXTENSION).build());
} }
} }
} }

View File

@@ -16,23 +16,23 @@
* limitations under the License. * limitations under the License.
*/ */
package com.uber.hoodie.cli.commands; package org.apache.hudi.cli.commands;
import com.uber.hoodie.HoodieWriteClient;
import com.uber.hoodie.cli.HoodieCLI;
import com.uber.hoodie.cli.HoodiePrintHelper;
import com.uber.hoodie.cli.utils.InputStreamConsumer;
import com.uber.hoodie.cli.utils.SparkUtil;
import com.uber.hoodie.common.table.HoodieTimeline;
import com.uber.hoodie.common.table.timeline.HoodieActiveTimeline;
import com.uber.hoodie.common.table.timeline.HoodieInstant;
import com.uber.hoodie.config.HoodieIndexConfig;
import com.uber.hoodie.config.HoodieWriteConfig;
import com.uber.hoodie.index.HoodieIndex;
import java.io.IOException; import java.io.IOException;
import java.util.Collections; import java.util.Collections;
import java.util.List; import java.util.List;
import java.util.stream.Collectors; import java.util.stream.Collectors;
import org.apache.hudi.HoodieWriteClient;
import org.apache.hudi.cli.HoodieCLI;
import org.apache.hudi.cli.HoodiePrintHelper;
import org.apache.hudi.cli.utils.InputStreamConsumer;
import org.apache.hudi.cli.utils.SparkUtil;
import org.apache.hudi.common.table.HoodieTimeline;
import org.apache.hudi.common.table.timeline.HoodieActiveTimeline;
import org.apache.hudi.common.table.timeline.HoodieInstant;
import org.apache.hudi.config.HoodieIndexConfig;
import org.apache.hudi.config.HoodieWriteConfig;
import org.apache.hudi.index.HoodieIndex;
import org.apache.spark.api.java.JavaSparkContext; import org.apache.spark.api.java.JavaSparkContext;
import org.apache.spark.launcher.SparkLauncher; import org.apache.spark.launcher.SparkLauncher;
import org.springframework.shell.core.CommandMarker; import org.springframework.shell.core.CommandMarker;

View File

@@ -16,20 +16,21 @@
* limitations under the License. * limitations under the License.
*/ */
package com.uber.hoodie.cli.commands; package org.apache.hudi.cli.commands;
import com.uber.hoodie.HoodieWriteClient; import org.apache.hudi.HoodieWriteClient;
import com.uber.hoodie.cli.DedupeSparkJob; import org.apache.hudi.cli.DedupeSparkJob;
import com.uber.hoodie.cli.utils.SparkUtil; import org.apache.hudi.cli.utils.SparkUtil;
import com.uber.hoodie.common.util.FSUtils; import org.apache.hudi.common.util.FSUtils;
import com.uber.hoodie.config.HoodieIndexConfig; import org.apache.hudi.config.HoodieIndexConfig;
import com.uber.hoodie.config.HoodieWriteConfig; import org.apache.hudi.config.HoodieWriteConfig;
import com.uber.hoodie.index.HoodieIndex; import org.apache.hudi.index.HoodieIndex;
import com.uber.hoodie.io.compact.strategy.UnBoundedCompactionStrategy; import org.apache.hudi.io.compact.strategy.UnBoundedCompactionStrategy;
import com.uber.hoodie.utilities.HDFSParquetImporter; import org.apache.hudi.utilities.HDFSParquetImporter;
import com.uber.hoodie.utilities.HoodieCompactionAdminTool; import org.apache.hudi.utilities.HDFSParquetImporter.Config;
import com.uber.hoodie.utilities.HoodieCompactionAdminTool.Operation; import org.apache.hudi.utilities.HoodieCompactionAdminTool;
import com.uber.hoodie.utilities.HoodieCompactor; import org.apache.hudi.utilities.HoodieCompactionAdminTool.Operation;
import org.apache.hudi.utilities.HoodieCompactor;
import org.apache.log4j.Logger; import org.apache.log4j.Logger;
import org.apache.spark.api.java.JavaSparkContext; import org.apache.spark.api.java.JavaSparkContext;
import org.apache.spark.sql.SQLContext; import org.apache.spark.sql.SQLContext;
@@ -116,7 +117,7 @@ public class SparkMain {
String srcPath, String targetPath, String tableName, String srcPath, String targetPath, String tableName,
String tableType, String rowKey, String partitionKey, int parallelism, String schemaFile, String sparkMaster, String tableType, String rowKey, String partitionKey, int parallelism, String schemaFile, String sparkMaster,
String sparkMemory, int retry) throws Exception { String sparkMemory, int retry) throws Exception {
HDFSParquetImporter.Config cfg = new HDFSParquetImporter.Config(); Config cfg = new Config();
cfg.command = command; cfg.command = command;
cfg.srcPath = srcPath; cfg.srcPath = srcPath;
cfg.targetPath = targetPath; cfg.targetPath = targetPath;

View File

@@ -16,20 +16,11 @@
* limitations under the License. * limitations under the License.
*/ */
package com.uber.hoodie.cli.commands; package org.apache.hudi.cli.commands;
import com.codahale.metrics.Histogram; import com.codahale.metrics.Histogram;
import com.codahale.metrics.Snapshot; import com.codahale.metrics.Snapshot;
import com.codahale.metrics.UniformReservoir; import com.codahale.metrics.UniformReservoir;
import com.uber.hoodie.cli.HoodieCLI;
import com.uber.hoodie.cli.HoodiePrintHelper;
import com.uber.hoodie.cli.TableHeader;
import com.uber.hoodie.common.model.HoodieCommitMetadata;
import com.uber.hoodie.common.table.HoodieTimeline;
import com.uber.hoodie.common.table.timeline.HoodieActiveTimeline;
import com.uber.hoodie.common.table.timeline.HoodieInstant;
import com.uber.hoodie.common.util.FSUtils;
import com.uber.hoodie.common.util.NumericUtils;
import java.io.IOException; import java.io.IOException;
import java.text.DecimalFormat; import java.text.DecimalFormat;
import java.util.ArrayList; import java.util.ArrayList;
@@ -41,6 +32,15 @@ import java.util.stream.Collectors;
import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.Path;
import org.apache.hudi.cli.HoodieCLI;
import org.apache.hudi.cli.HoodiePrintHelper;
import org.apache.hudi.cli.TableHeader;
import org.apache.hudi.common.model.HoodieCommitMetadata;
import org.apache.hudi.common.table.HoodieTimeline;
import org.apache.hudi.common.table.timeline.HoodieActiveTimeline;
import org.apache.hudi.common.table.timeline.HoodieInstant;
import org.apache.hudi.common.util.FSUtils;
import org.apache.hudi.common.util.NumericUtils;
import org.springframework.shell.core.CommandMarker; import org.springframework.shell.core.CommandMarker;
import org.springframework.shell.core.annotation.CliAvailabilityIndicator; import org.springframework.shell.core.annotation.CliAvailabilityIndicator;
import org.springframework.shell.core.annotation.CliCommand; import org.springframework.shell.core.annotation.CliCommand;

View File

@@ -16,7 +16,7 @@
* limitations under the License. * limitations under the License.
*/ */
package com.uber.hoodie.cli.commands; package org.apache.hudi.cli.commands;
import org.springframework.shell.core.CommandMarker; import org.springframework.shell.core.CommandMarker;
import org.springframework.shell.core.annotation.CliCommand; import org.springframework.shell.core.annotation.CliCommand;

View File

@@ -16,14 +16,14 @@
* limitations under the License. * limitations under the License.
*/ */
package com.uber.hoodie.cli.utils; package org.apache.hudi.cli.utils;
import com.uber.hoodie.common.model.HoodieCommitMetadata;
import com.uber.hoodie.common.table.HoodieTableMetaClient;
import com.uber.hoodie.common.table.HoodieTimeline;
import com.uber.hoodie.common.table.timeline.HoodieInstant;
import java.io.IOException; import java.io.IOException;
import java.util.List; import java.util.List;
import org.apache.hudi.common.model.HoodieCommitMetadata;
import org.apache.hudi.common.table.HoodieTableMetaClient;
import org.apache.hudi.common.table.HoodieTimeline;
import org.apache.hudi.common.table.timeline.HoodieInstant;
public class CommitUtil { public class CommitUtil {

View File

@@ -16,15 +16,15 @@
* limitations under the License. * limitations under the License.
*/ */
package com.uber.hoodie.cli.utils; package org.apache.hudi.cli.utils;
import com.uber.hoodie.common.table.HoodieTableMetaClient;
import java.sql.Connection; import java.sql.Connection;
import java.sql.ResultSet; import java.sql.ResultSet;
import java.sql.SQLException; import java.sql.SQLException;
import java.sql.Statement; import java.sql.Statement;
import javax.sql.DataSource; import javax.sql.DataSource;
import org.apache.commons.dbcp.BasicDataSource; import org.apache.commons.dbcp.BasicDataSource;
import org.apache.hudi.common.table.HoodieTableMetaClient;
import org.joda.time.DateTime; import org.joda.time.DateTime;
public class HiveUtil { public class HiveUtil {

View File

@@ -16,7 +16,7 @@
* limitations under the License. * limitations under the License.
*/ */
package com.uber.hoodie.cli.utils; package org.apache.hudi.cli.utils;
import java.io.BufferedReader; import java.io.BufferedReader;
import java.io.IOException; import java.io.IOException;

View File

@@ -16,15 +16,14 @@
* limitations under the License. * limitations under the License.
*/ */
package com.uber.hoodie.cli.utils; package org.apache.hudi.cli.utils;
import com.uber.hoodie.HoodieWriteClient;
import com.uber.hoodie.cli.commands.SparkMain;
import com.uber.hoodie.common.util.FSUtils;
import java.io.File; import java.io.File;
import java.net.URISyntaxException; import java.net.URISyntaxException;
import org.apache.commons.lang.StringUtils; import org.apache.commons.lang.StringUtils;
import org.apache.hudi.HoodieWriteClient;
import org.apache.hudi.cli.commands.SparkMain;
import org.apache.hudi.common.util.FSUtils;
import org.apache.log4j.Logger; import org.apache.log4j.Logger;
import org.apache.spark.SparkConf; import org.apache.spark.SparkConf;
import org.apache.spark.api.java.JavaSparkContext; import org.apache.spark.api.java.JavaSparkContext;

View File

@@ -23,6 +23,6 @@
xsi:schemaLocation="http://www.springframework.org/schema/beans http://www.springframework.org/schema/beans/spring-beans.xsd xsi:schemaLocation="http://www.springframework.org/schema/beans http://www.springframework.org/schema/beans/spring-beans.xsd
http://www.springframework.org/schema/context http://www.springframework.org/schema/context/spring-context.xsd"> http://www.springframework.org/schema/context http://www.springframework.org/schema/context/spring-context.xsd">
<context:component-scan base-package="com.uber.hoodie.cli"/> <context:component-scan base-package="org.apache.hudi.cli"/>
</beans> </beans>

View File

@@ -15,16 +15,16 @@
* limitations under the License. * limitations under the License.
*/ */
package com.uber.hoodie.cli package org.apache.hudi.cli
import java.util.stream.Collectors import java.util.stream.Collectors
import com.uber.hoodie.common.model.{HoodieDataFile, HoodieRecord}
import com.uber.hoodie.common.table.HoodieTableMetaClient
import com.uber.hoodie.common.table.view.HoodieTableFileSystemView
import com.uber.hoodie.common.util.FSUtils
import com.uber.hoodie.exception.HoodieException
import org.apache.hadoop.fs.{FileSystem, FileUtil, Path} import org.apache.hadoop.fs.{FileSystem, FileUtil, Path}
import org.apache.hudi.common.model.{HoodieDataFile, HoodieRecord}
import org.apache.hudi.common.table.HoodieTableMetaClient
import org.apache.hudi.common.table.view.HoodieTableFileSystemView
import org.apache.hudi.common.util.FSUtils
import org.apache.hudi.exception.HoodieException
import org.apache.log4j.Logger import org.apache.log4j.Logger
import org.apache.spark.sql.{DataFrame, SQLContext} import org.apache.spark.sql.{DataFrame, SQLContext}

View File

@@ -15,18 +15,18 @@
* limitations under the License. * limitations under the License.
*/ */
package com.uber.hoodie.cli package org.apache.hudi.cli
import com.uber.hoodie.avro.HoodieAvroWriteSupport
import com.uber.hoodie.common.model.HoodieRecord
import com.uber.hoodie.common.util.ParquetUtils
import com.uber.hoodie.common.{BloomFilter, HoodieJsonPayload}
import com.uber.hoodie.config.{HoodieIndexConfig, HoodieStorageConfig}
import com.uber.hoodie.io.storage.{HoodieParquetConfig, HoodieParquetWriter}
import org.apache.avro.Schema import org.apache.avro.Schema
import org.apache.avro.generic.IndexedRecord import org.apache.avro.generic.IndexedRecord
import org.apache.hadoop.conf.Configuration import org.apache.hadoop.conf.Configuration
import org.apache.hadoop.fs.{FileSystem, Path} import org.apache.hadoop.fs.{FileSystem, Path}
import org.apache.hudi.avro.HoodieAvroWriteSupport
import org.apache.hudi.common.model.HoodieRecord
import org.apache.hudi.common.util.ParquetUtils
import org.apache.hudi.common.{BloomFilter, HoodieJsonPayload}
import org.apache.hudi.config.{HoodieIndexConfig, HoodieStorageConfig}
import org.apache.hudi.io.storage.{HoodieParquetConfig, HoodieParquetWriter}
import org.apache.parquet.avro.AvroSchemaConverter import org.apache.parquet.avro.AvroSchemaConverter
import org.apache.parquet.hadoop.ParquetFileReader import org.apache.parquet.hadoop.ParquetFileReader
import org.apache.parquet.hadoop.metadata.CompressionCodecName import org.apache.parquet.hadoop.metadata.CompressionCodecName
@@ -123,7 +123,7 @@ class SparkHelper(sqlContext: SQLContext, fs: FileSystem) {
*/ */
def fileKeysAgainstBF(conf: Configuration, sqlContext: SQLContext, file: String): Boolean = { def fileKeysAgainstBF(conf: Configuration, sqlContext: SQLContext, file: String): Boolean = {
val bfStr = SparkHelpers.getBloomFilter(file, conf) val bfStr = SparkHelpers.getBloomFilter(file, conf)
val bf = new com.uber.hoodie.common.BloomFilter(bfStr) val bf = new BloomFilter(bfStr)
val foundCount = sqlContext.parquetFile(file) val foundCount = sqlContext.parquetFile(file)
.select(s"`${HoodieRecord.RECORD_KEY_METADATA_FIELD}`") .select(s"`${HoodieRecord.RECORD_KEY_METADATA_FIELD}`")
.collect().count(r => !bf.mightContain(r.getString(0))) .collect().count(r => !bf.mightContain(r.getString(0)))

View File

@@ -17,13 +17,13 @@
--> -->
<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"> <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> <parent>
<artifactId>hoodie</artifactId> <artifactId>hudi</artifactId>
<groupId>com.uber.hoodie</groupId> <groupId>org.apache.hudi</groupId>
<version>0.4.8-SNAPSHOT</version> <version>0.5.0-SNAPSHOT</version>
</parent> </parent>
<modelVersion>4.0.0</modelVersion> <modelVersion>4.0.0</modelVersion>
<artifactId>hoodie-client</artifactId> <artifactId>hudi-client</artifactId>
<packaging>jar</packaging> <packaging>jar</packaging>
<properties> <properties>
@@ -69,13 +69,13 @@
<dependencies> <dependencies>
<dependency> <dependency>
<groupId>com.uber.hoodie</groupId> <groupId>org.apache.hudi</groupId>
<artifactId>hoodie-common</artifactId> <artifactId>hudi-common</artifactId>
<version>${project.version}</version> <version>${project.version}</version>
</dependency> </dependency>
<dependency> <dependency>
<groupId>com.uber.hoodie</groupId> <groupId>org.apache.hudi</groupId>
<artifactId>hoodie-timeline-service</artifactId> <artifactId>hudi-timeline-service</artifactId>
<version>${project.version}</version> <version>${project.version}</version>
</dependency> </dependency>
<dependency> <dependency>
@@ -118,8 +118,8 @@
</exclusions> </exclusions>
</dependency> </dependency>
<dependency> <dependency>
<groupId>com.uber.hoodie</groupId> <groupId>org.apache.hudi</groupId>
<artifactId>hoodie-common</artifactId> <artifactId>hudi-common</artifactId>
<version>${project.version}</version> <version>${project.version}</version>
<classifier>tests</classifier> <classifier>tests</classifier>
<type>test-jar</type> <type>test-jar</type>
@@ -195,8 +195,8 @@
<scope>test</scope> <scope>test</scope>
</dependency> </dependency>
<dependency> <dependency>
<groupId>com.uber.hoodie</groupId> <groupId>org.apache.hudi</groupId>
<artifactId>hoodie-hadoop-mr</artifactId> <artifactId>hudi-hadoop-mr</artifactId>
<version>${project.version}</version> <version>${project.version}</version>
<scope>test</scope> <scope>test</scope>
</dependency> </dependency>

View File

@@ -16,17 +16,17 @@
* limitations under the License. * limitations under the License.
*/ */
package com.uber.hoodie; package org.apache.hudi;
import com.uber.hoodie.client.embedded.EmbeddedTimelineService;
import com.uber.hoodie.client.utils.ClientUtils;
import com.uber.hoodie.common.table.HoodieTableMetaClient;
import com.uber.hoodie.common.util.FSUtils;
import com.uber.hoodie.common.util.Option;
import com.uber.hoodie.config.HoodieWriteConfig;
import java.io.IOException; import java.io.IOException;
import java.io.Serializable; import java.io.Serializable;
import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.FileSystem;
import org.apache.hudi.client.embedded.EmbeddedTimelineService;
import org.apache.hudi.client.utils.ClientUtils;
import org.apache.hudi.common.table.HoodieTableMetaClient;
import org.apache.hudi.common.util.FSUtils;
import org.apache.hudi.common.util.Option;
import org.apache.hudi.config.HoodieWriteConfig;
import org.apache.log4j.LogManager; import org.apache.log4j.LogManager;
import org.apache.log4j.Logger; import org.apache.log4j.Logger;
import org.apache.spark.api.java.JavaSparkContext; import org.apache.spark.api.java.JavaSparkContext;

View File

@@ -16,34 +16,11 @@
* limitations under the License. * limitations under the License.
*/ */
package com.uber.hoodie; package org.apache.hudi;
import static com.uber.hoodie.common.table.HoodieTimeline.COMPACTION_ACTION; import static org.apache.hudi.common.table.HoodieTimeline.COMPACTION_ACTION;
import com.google.common.base.Preconditions; import com.google.common.base.Preconditions;
import com.uber.hoodie.avro.model.HoodieCompactionOperation;
import com.uber.hoodie.avro.model.HoodieCompactionPlan;
import com.uber.hoodie.client.embedded.EmbeddedTimelineService;
import com.uber.hoodie.common.model.CompactionOperation;
import com.uber.hoodie.common.model.FileSlice;
import com.uber.hoodie.common.model.HoodieDataFile;
import com.uber.hoodie.common.model.HoodieFileGroupId;
import com.uber.hoodie.common.model.HoodieLogFile;
import com.uber.hoodie.common.table.HoodieTableMetaClient;
import com.uber.hoodie.common.table.HoodieTimeline;
import com.uber.hoodie.common.table.log.HoodieLogFormat;
import com.uber.hoodie.common.table.timeline.HoodieInstant;
import com.uber.hoodie.common.table.timeline.HoodieInstant.State;
import com.uber.hoodie.common.table.view.HoodieTableFileSystemView;
import com.uber.hoodie.common.util.AvroUtils;
import com.uber.hoodie.common.util.CompactionUtils;
import com.uber.hoodie.common.util.FSUtils;
import com.uber.hoodie.common.util.Option;
import com.uber.hoodie.common.util.collection.Pair;
import com.uber.hoodie.config.HoodieWriteConfig;
import com.uber.hoodie.exception.HoodieException;
import com.uber.hoodie.exception.HoodieIOException;
import com.uber.hoodie.func.OperationResult;
import java.io.FileNotFoundException; import java.io.FileNotFoundException;
import java.io.IOException; import java.io.IOException;
import java.io.Serializable; import java.io.Serializable;
@@ -54,6 +31,29 @@ import java.util.Set;
import java.util.stream.Collectors; import java.util.stream.Collectors;
import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.Path;
import org.apache.hudi.avro.model.HoodieCompactionOperation;
import org.apache.hudi.avro.model.HoodieCompactionPlan;
import org.apache.hudi.client.embedded.EmbeddedTimelineService;
import org.apache.hudi.common.model.CompactionOperation;
import org.apache.hudi.common.model.FileSlice;
import org.apache.hudi.common.model.HoodieDataFile;
import org.apache.hudi.common.model.HoodieFileGroupId;
import org.apache.hudi.common.model.HoodieLogFile;
import org.apache.hudi.common.table.HoodieTableMetaClient;
import org.apache.hudi.common.table.HoodieTimeline;
import org.apache.hudi.common.table.log.HoodieLogFormat;
import org.apache.hudi.common.table.timeline.HoodieInstant;
import org.apache.hudi.common.table.timeline.HoodieInstant.State;
import org.apache.hudi.common.table.view.HoodieTableFileSystemView;
import org.apache.hudi.common.util.AvroUtils;
import org.apache.hudi.common.util.CompactionUtils;
import org.apache.hudi.common.util.FSUtils;
import org.apache.hudi.common.util.Option;
import org.apache.hudi.common.util.collection.Pair;
import org.apache.hudi.config.HoodieWriteConfig;
import org.apache.hudi.exception.HoodieException;
import org.apache.hudi.exception.HoodieIOException;
import org.apache.hudi.func.OperationResult;
import org.apache.log4j.LogManager; import org.apache.log4j.LogManager;
import org.apache.log4j.Logger; import org.apache.log4j.Logger;
import org.apache.spark.api.java.JavaSparkContext; import org.apache.spark.api.java.JavaSparkContext;

View File

@@ -16,28 +16,28 @@
* limitations under the License. * limitations under the License.
*/ */
package com.uber.hoodie; package org.apache.hudi;
import com.uber.hoodie.avro.model.HoodieCompactionPlan;
import com.uber.hoodie.client.embedded.EmbeddedTimelineService;
import com.uber.hoodie.common.model.HoodieDataFile;
import com.uber.hoodie.common.model.HoodieKey;
import com.uber.hoodie.common.model.HoodieRecord;
import com.uber.hoodie.common.model.HoodieRecordPayload;
import com.uber.hoodie.common.table.HoodieTableMetaClient;
import com.uber.hoodie.common.table.HoodieTimeline;
import com.uber.hoodie.common.util.CompactionUtils;
import com.uber.hoodie.common.util.Option;
import com.uber.hoodie.common.util.collection.Pair;
import com.uber.hoodie.config.HoodieIndexConfig;
import com.uber.hoodie.config.HoodieWriteConfig;
import com.uber.hoodie.exception.HoodieIndexException;
import com.uber.hoodie.index.HoodieIndex;
import com.uber.hoodie.table.HoodieTable;
import java.util.HashSet; import java.util.HashSet;
import java.util.List; import java.util.List;
import java.util.Set; import java.util.Set;
import java.util.stream.Collectors; import java.util.stream.Collectors;
import org.apache.hudi.avro.model.HoodieCompactionPlan;
import org.apache.hudi.client.embedded.EmbeddedTimelineService;
import org.apache.hudi.common.model.HoodieDataFile;
import org.apache.hudi.common.model.HoodieKey;
import org.apache.hudi.common.model.HoodieRecord;
import org.apache.hudi.common.model.HoodieRecordPayload;
import org.apache.hudi.common.table.HoodieTableMetaClient;
import org.apache.hudi.common.table.HoodieTimeline;
import org.apache.hudi.common.util.CompactionUtils;
import org.apache.hudi.common.util.Option;
import org.apache.hudi.common.util.collection.Pair;
import org.apache.hudi.config.HoodieIndexConfig;
import org.apache.hudi.config.HoodieWriteConfig;
import org.apache.hudi.exception.HoodieIndexException;
import org.apache.hudi.index.HoodieIndex;
import org.apache.hudi.table.HoodieTable;
import org.apache.log4j.LogManager; import org.apache.log4j.LogManager;
import org.apache.log4j.Logger; import org.apache.log4j.Logger;
import org.apache.spark.SparkConf; import org.apache.spark.SparkConf;

View File

@@ -16,55 +16,12 @@
* limitations under the License. * limitations under the License.
*/ */
package com.uber.hoodie; package org.apache.hudi;
import com.codahale.metrics.Timer; import com.codahale.metrics.Timer;
import com.google.common.annotations.VisibleForTesting; import com.google.common.annotations.VisibleForTesting;
import com.google.common.base.Preconditions; import com.google.common.base.Preconditions;
import com.google.common.collect.ImmutableMap; import com.google.common.collect.ImmutableMap;
import com.uber.hoodie.avro.model.HoodieCleanMetadata;
import com.uber.hoodie.avro.model.HoodieCompactionPlan;
import com.uber.hoodie.avro.model.HoodieRestoreMetadata;
import com.uber.hoodie.avro.model.HoodieRollbackMetadata;
import com.uber.hoodie.avro.model.HoodieSavepointMetadata;
import com.uber.hoodie.client.embedded.EmbeddedTimelineService;
import com.uber.hoodie.common.HoodieCleanStat;
import com.uber.hoodie.common.HoodieRollbackStat;
import com.uber.hoodie.common.model.HoodieCommitMetadata;
import com.uber.hoodie.common.model.HoodieDataFile;
import com.uber.hoodie.common.model.HoodieKey;
import com.uber.hoodie.common.model.HoodieRecord;
import com.uber.hoodie.common.model.HoodieRecordPayload;
import com.uber.hoodie.common.model.HoodieRollingStat;
import com.uber.hoodie.common.model.HoodieRollingStatMetadata;
import com.uber.hoodie.common.model.HoodieTableType;
import com.uber.hoodie.common.model.HoodieWriteStat;
import com.uber.hoodie.common.table.HoodieTableMetaClient;
import com.uber.hoodie.common.table.HoodieTimeline;
import com.uber.hoodie.common.table.TableFileSystemView;
import com.uber.hoodie.common.table.timeline.HoodieActiveTimeline;
import com.uber.hoodie.common.table.timeline.HoodieInstant;
import com.uber.hoodie.common.table.timeline.HoodieInstant.State;
import com.uber.hoodie.common.util.AvroUtils;
import com.uber.hoodie.common.util.FSUtils;
import com.uber.hoodie.common.util.Option;
import com.uber.hoodie.config.HoodieCompactionConfig;
import com.uber.hoodie.config.HoodieWriteConfig;
import com.uber.hoodie.exception.HoodieCommitException;
import com.uber.hoodie.exception.HoodieCompactionException;
import com.uber.hoodie.exception.HoodieIOException;
import com.uber.hoodie.exception.HoodieInsertException;
import com.uber.hoodie.exception.HoodieRollbackException;
import com.uber.hoodie.exception.HoodieSavepointException;
import com.uber.hoodie.exception.HoodieUpsertException;
import com.uber.hoodie.func.BulkInsertMapFunction;
import com.uber.hoodie.index.HoodieIndex;
import com.uber.hoodie.io.HoodieCommitArchiveLog;
import com.uber.hoodie.metrics.HoodieMetrics;
import com.uber.hoodie.table.HoodieTable;
import com.uber.hoodie.table.UserDefinedBulkInsertPartitioner;
import com.uber.hoodie.table.WorkloadProfile;
import com.uber.hoodie.table.WorkloadStat;
import java.io.IOException; import java.io.IOException;
import java.nio.charset.StandardCharsets; import java.nio.charset.StandardCharsets;
import java.text.ParseException; import java.text.ParseException;
@@ -76,6 +33,49 @@ import java.util.List;
import java.util.Map; import java.util.Map;
import java.util.stream.Collectors; import java.util.stream.Collectors;
import java.util.stream.IntStream; import java.util.stream.IntStream;
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.avro.model.HoodieSavepointMetadata;
import org.apache.hudi.client.embedded.EmbeddedTimelineService;
import org.apache.hudi.common.HoodieCleanStat;
import org.apache.hudi.common.HoodieRollbackStat;
import org.apache.hudi.common.model.HoodieCommitMetadata;
import org.apache.hudi.common.model.HoodieDataFile;
import org.apache.hudi.common.model.HoodieKey;
import org.apache.hudi.common.model.HoodieRecord;
import org.apache.hudi.common.model.HoodieRecordPayload;
import org.apache.hudi.common.model.HoodieRollingStat;
import org.apache.hudi.common.model.HoodieRollingStatMetadata;
import org.apache.hudi.common.model.HoodieTableType;
import org.apache.hudi.common.model.HoodieWriteStat;
import org.apache.hudi.common.table.HoodieTableMetaClient;
import org.apache.hudi.common.table.HoodieTimeline;
import org.apache.hudi.common.table.TableFileSystemView.ReadOptimizedView;
import org.apache.hudi.common.table.timeline.HoodieActiveTimeline;
import org.apache.hudi.common.table.timeline.HoodieInstant;
import org.apache.hudi.common.table.timeline.HoodieInstant.State;
import org.apache.hudi.common.util.AvroUtils;
import org.apache.hudi.common.util.FSUtils;
import org.apache.hudi.common.util.Option;
import org.apache.hudi.config.HoodieCompactionConfig;
import org.apache.hudi.config.HoodieWriteConfig;
import org.apache.hudi.exception.HoodieCommitException;
import org.apache.hudi.exception.HoodieCompactionException;
import org.apache.hudi.exception.HoodieIOException;
import org.apache.hudi.exception.HoodieInsertException;
import org.apache.hudi.exception.HoodieRollbackException;
import org.apache.hudi.exception.HoodieSavepointException;
import org.apache.hudi.exception.HoodieUpsertException;
import org.apache.hudi.func.BulkInsertMapFunction;
import org.apache.hudi.index.HoodieIndex;
import org.apache.hudi.io.HoodieCommitArchiveLog;
import org.apache.hudi.metrics.HoodieMetrics;
import org.apache.hudi.table.HoodieTable;
import org.apache.hudi.table.UserDefinedBulkInsertPartitioner;
import org.apache.hudi.table.WorkloadProfile;
import org.apache.hudi.table.WorkloadStat;
import org.apache.log4j.LogManager; import org.apache.log4j.LogManager;
import org.apache.log4j.Logger; import org.apache.log4j.Logger;
import org.apache.spark.Partitioner; import org.apache.spark.Partitioner;
@@ -644,7 +644,7 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> extends AbstractHo
.mapToPair((PairFunction<String, String, List<String>>) partitionPath -> { .mapToPair((PairFunction<String, String, List<String>>) partitionPath -> {
// Scan all partitions files with this commit time // Scan all partitions files with this commit time
logger.info("Collecting latest files in partition path " + partitionPath); logger.info("Collecting latest files in partition path " + partitionPath);
TableFileSystemView.ReadOptimizedView view = table.getROFileSystemView(); ReadOptimizedView view = table.getROFileSystemView();
List<String> latestFiles = view.getLatestDataFilesBeforeOrOn(partitionPath, commitTime) List<String> latestFiles = view.getLatestDataFilesBeforeOrOn(partitionPath, commitTime)
.map(HoodieDataFile::getFileName).collect(Collectors.toList()); .map(HoodieDataFile::getFileName).collect(Collectors.toList());
return new Tuple2<>(partitionPath, latestFiles); return new Tuple2<>(partitionPath, latestFiles);

View File

@@ -16,18 +16,18 @@
* limitations under the License. * limitations under the License.
*/ */
package com.uber.hoodie; package org.apache.hudi;
import com.uber.hoodie.common.model.HoodieKey;
import com.uber.hoodie.common.model.HoodieRecord;
import com.uber.hoodie.common.model.HoodieWriteStat;
import com.uber.hoodie.common.util.Option;
import java.io.Serializable; import java.io.Serializable;
import java.util.ArrayList; import java.util.ArrayList;
import java.util.HashMap; import java.util.HashMap;
import java.util.List; import java.util.List;
import java.util.Map; import java.util.Map;
import java.util.Random; import java.util.Random;
import org.apache.hudi.common.model.HoodieKey;
import org.apache.hudi.common.model.HoodieRecord;
import org.apache.hudi.common.model.HoodieWriteStat;
import org.apache.hudi.common.util.Option;
/** /**
* Status of a write operation. * Status of a write operation.

View File

@@ -16,16 +16,16 @@
* limitations under the License. * limitations under the License.
*/ */
package com.uber.hoodie.client.embedded; package org.apache.hudi.client.embedded;
import com.uber.hoodie.common.SerializableConfiguration;
import com.uber.hoodie.common.table.view.FileSystemViewManager;
import com.uber.hoodie.common.table.view.FileSystemViewStorageConfig;
import com.uber.hoodie.common.table.view.FileSystemViewStorageType;
import com.uber.hoodie.common.util.NetworkUtils;
import com.uber.hoodie.timeline.service.TimelineService;
import java.io.IOException; import java.io.IOException;
import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.conf.Configuration;
import org.apache.hudi.common.SerializableConfiguration;
import org.apache.hudi.common.table.view.FileSystemViewManager;
import org.apache.hudi.common.table.view.FileSystemViewStorageConfig;
import org.apache.hudi.common.table.view.FileSystemViewStorageType;
import org.apache.hudi.common.util.NetworkUtils;
import org.apache.hudi.timeline.service.TimelineService;
import org.apache.log4j.LogManager; import org.apache.log4j.LogManager;
import org.apache.log4j.Logger; import org.apache.log4j.Logger;
import org.apache.spark.SparkConf; import org.apache.spark.SparkConf;

View File

@@ -16,10 +16,10 @@
* limitations under the License. * limitations under the License.
*/ */
package com.uber.hoodie.client.utils; package org.apache.hudi.client.utils;
import com.uber.hoodie.common.table.HoodieTableMetaClient; import org.apache.hudi.common.table.HoodieTableMetaClient;
import com.uber.hoodie.config.HoodieWriteConfig; import org.apache.hudi.config.HoodieWriteConfig;
import org.apache.spark.api.java.JavaSparkContext; import org.apache.spark.api.java.JavaSparkContext;
public class ClientUtils { public class ClientUtils {

View File

@@ -16,18 +16,18 @@
* limitations under the License. * limitations under the License.
*/ */
package com.uber.hoodie.config; package org.apache.hudi.config;
import com.google.common.base.Preconditions; import com.google.common.base.Preconditions;
import com.uber.hoodie.common.model.HoodieAvroPayload;
import com.uber.hoodie.common.model.HoodieCleaningPolicy;
import com.uber.hoodie.io.compact.strategy.CompactionStrategy;
import com.uber.hoodie.io.compact.strategy.LogFileSizeBasedCompactionStrategy;
import java.io.File; import java.io.File;
import java.io.FileReader; import java.io.FileReader;
import java.io.IOException; import java.io.IOException;
import java.util.Properties; import java.util.Properties;
import javax.annotation.concurrent.Immutable; import javax.annotation.concurrent.Immutable;
import org.apache.hudi.common.model.HoodieAvroPayload;
import org.apache.hudi.common.model.HoodieCleaningPolicy;
import org.apache.hudi.io.compact.strategy.CompactionStrategy;
import org.apache.hudi.io.compact.strategy.LogFileSizeBasedCompactionStrategy;
/** /**
* Compaction related config * Compaction related config

View File

@@ -16,13 +16,13 @@
* limitations under the License. * limitations under the License.
*/ */
package com.uber.hoodie.config; package org.apache.hudi.config;
import com.uber.hoodie.index.hbase.DefaultHBaseQPSResourceAllocator;
import java.io.File; import java.io.File;
import java.io.FileReader; import java.io.FileReader;
import java.io.IOException; import java.io.IOException;
import java.util.Properties; import java.util.Properties;
import org.apache.hudi.index.hbase.DefaultHBaseQPSResourceAllocator;
public class HoodieHBaseIndexConfig extends DefaultHoodieConfig { public class HoodieHBaseIndexConfig extends DefaultHoodieConfig {

View File

@@ -16,14 +16,14 @@
* limitations under the License. * limitations under the License.
*/ */
package com.uber.hoodie.config; package org.apache.hudi.config;
import com.uber.hoodie.index.HoodieIndex;
import java.io.File; import java.io.File;
import java.io.FileReader; import java.io.FileReader;
import java.io.IOException; import java.io.IOException;
import java.util.Properties; import java.util.Properties;
import javax.annotation.concurrent.Immutable; import javax.annotation.concurrent.Immutable;
import org.apache.hudi.index.HoodieIndex;
/** /**
* Indexing related config * Indexing related config

View File

@@ -16,7 +16,7 @@
* limitations under the License. * limitations under the License.
*/ */
package com.uber.hoodie.config; package org.apache.hudi.config;
import java.io.File; import java.io.File;
import java.io.FileReader; import java.io.FileReader;

View File

@@ -16,14 +16,14 @@
* limitations under the License. * limitations under the License.
*/ */
package com.uber.hoodie.config; package org.apache.hudi.config;
import com.uber.hoodie.metrics.MetricsReporterType;
import java.io.File; import java.io.File;
import java.io.FileReader; import java.io.FileReader;
import java.io.IOException; import java.io.IOException;
import java.util.Properties; import java.util.Properties;
import javax.annotation.concurrent.Immutable; import javax.annotation.concurrent.Immutable;
import org.apache.hudi.metrics.MetricsReporterType;
/** /**
* Fetch the configurations used by the Metrics system. * Fetch the configurations used by the Metrics system.

View File

@@ -16,7 +16,7 @@
* limitations under the License. * limitations under the License.
*/ */
package com.uber.hoodie.config; package org.apache.hudi.config;
import java.io.File; import java.io.File;
import java.io.FileReader; import java.io.FileReader;

View File

@@ -16,17 +16,9 @@
* limitations under the License. * limitations under the License.
*/ */
package com.uber.hoodie.config; package org.apache.hudi.config;
import com.google.common.base.Preconditions; import com.google.common.base.Preconditions;
import com.uber.hoodie.WriteStatus;
import com.uber.hoodie.common.model.HoodieCleaningPolicy;
import com.uber.hoodie.common.table.view.FileSystemViewStorageConfig;
import com.uber.hoodie.common.util.ConsistencyGuardConfig;
import com.uber.hoodie.common.util.ReflectionUtils;
import com.uber.hoodie.index.HoodieIndex;
import com.uber.hoodie.io.compact.strategy.CompactionStrategy;
import com.uber.hoodie.metrics.MetricsReporterType;
import java.io.File; import java.io.File;
import java.io.FileReader; import java.io.FileReader;
import java.io.IOException; import java.io.IOException;
@@ -34,11 +26,20 @@ import java.io.InputStream;
import java.util.Map; import java.util.Map;
import java.util.Properties; import java.util.Properties;
import javax.annotation.concurrent.Immutable; import javax.annotation.concurrent.Immutable;
import org.apache.hudi.HoodieWriteClient;
import org.apache.hudi.WriteStatus;
import org.apache.hudi.common.model.HoodieCleaningPolicy;
import org.apache.hudi.common.table.view.FileSystemViewStorageConfig;
import org.apache.hudi.common.util.ConsistencyGuardConfig;
import org.apache.hudi.common.util.ReflectionUtils;
import org.apache.hudi.index.HoodieIndex;
import org.apache.hudi.io.compact.strategy.CompactionStrategy;
import org.apache.hudi.metrics.MetricsReporterType;
import org.apache.parquet.hadoop.metadata.CompressionCodecName; import org.apache.parquet.hadoop.metadata.CompressionCodecName;
import org.apache.spark.storage.StorageLevel; import org.apache.spark.storage.StorageLevel;
/** /**
* Class storing configs for the {@link com.uber.hoodie.HoodieWriteClient} * Class storing configs for the {@link HoodieWriteClient}
*/ */
@Immutable @Immutable
public class HoodieWriteConfig extends DefaultHoodieConfig { public class HoodieWriteConfig extends DefaultHoodieConfig {

View File

@@ -16,7 +16,7 @@
* limitations under the License. * limitations under the License.
*/ */
package com.uber.hoodie.exception; package org.apache.hudi.exception;
/** /**
* <p> Exception thrown for any higher level errors when <code>HoodieClient</code> is doing a delta * <p> Exception thrown for any higher level errors when <code>HoodieClient</code> is doing a delta

View File

@@ -16,7 +16,7 @@
* limitations under the License. * limitations under the License.
*/ */
package com.uber.hoodie.exception; package org.apache.hudi.exception;
/** /**
* <p> Exception thrown for any higher level errors when <code>HoodieClient</code> is doing a Commit * <p> Exception thrown for any higher level errors when <code>HoodieClient</code> is doing a Commit

View File

@@ -16,7 +16,7 @@
* limitations under the License. * limitations under the License.
*/ */
package com.uber.hoodie.exception; package org.apache.hudi.exception;
public class HoodieCompactionException extends HoodieException { public class HoodieCompactionException extends HoodieException {

View File

@@ -16,7 +16,7 @@
* limitations under the License. * limitations under the License.
*/ */
package com.uber.hoodie.exception; package org.apache.hudi.exception;
/** /**

View File

@@ -16,7 +16,7 @@
* limitations under the License. * limitations under the License.
*/ */
package com.uber.hoodie.exception; package org.apache.hudi.exception;
/** /**
* <p> Exception thrown for any higher level errors when <code>HoodieClient</code> is doing a bulk * <p> Exception thrown for any higher level errors when <code>HoodieClient</code> is doing a bulk

View File

@@ -16,7 +16,7 @@
* limitations under the License. * limitations under the License.
*/ */
package com.uber.hoodie.exception; package org.apache.hudi.exception;
public class HoodieRollbackException extends HoodieException { public class HoodieRollbackException extends HoodieException {

View File

@@ -16,7 +16,7 @@
* limitations under the License. * limitations under the License.
*/ */
package com.uber.hoodie.exception; package org.apache.hudi.exception;
public class HoodieSavepointException extends HoodieException { public class HoodieSavepointException extends HoodieException {

View File

@@ -16,7 +16,7 @@
* limitations under the License. * limitations under the License.
*/ */
package com.uber.hoodie.exception; package org.apache.hudi.exception;
/** /**
* <p> Exception thrown for any higher level errors when <code>HoodieClient</code> is doing a * <p> Exception thrown for any higher level errors when <code>HoodieClient</code> is doing a

View File

@@ -16,15 +16,15 @@
* limitations under the License. * limitations under the License.
*/ */
package com.uber.hoodie.func; package org.apache.hudi.func;
import com.uber.hoodie.WriteStatus;
import com.uber.hoodie.common.model.HoodieRecord;
import com.uber.hoodie.common.model.HoodieRecordPayload;
import com.uber.hoodie.config.HoodieWriteConfig;
import com.uber.hoodie.table.HoodieTable;
import java.util.Iterator; import java.util.Iterator;
import java.util.List; import java.util.List;
import org.apache.hudi.WriteStatus;
import org.apache.hudi.common.model.HoodieRecord;
import org.apache.hudi.common.model.HoodieRecordPayload;
import org.apache.hudi.config.HoodieWriteConfig;
import org.apache.hudi.table.HoodieTable;
import org.apache.spark.api.java.function.Function2; import org.apache.spark.api.java.function.Function2;

View File

@@ -16,25 +16,25 @@
* limitations under the License. * limitations under the License.
*/ */
package com.uber.hoodie.func; package org.apache.hudi.func;
import com.uber.hoodie.WriteStatus;
import com.uber.hoodie.common.model.HoodieRecord;
import com.uber.hoodie.common.model.HoodieRecordPayload;
import com.uber.hoodie.common.util.Option;
import com.uber.hoodie.common.util.queue.BoundedInMemoryExecutor;
import com.uber.hoodie.common.util.queue.BoundedInMemoryQueueConsumer;
import com.uber.hoodie.config.HoodieWriteConfig;
import com.uber.hoodie.exception.HoodieException;
import com.uber.hoodie.io.HoodieCreateHandle;
import com.uber.hoodie.io.HoodieWriteHandle;
import com.uber.hoodie.table.HoodieTable;
import java.util.ArrayList; import java.util.ArrayList;
import java.util.Iterator; import java.util.Iterator;
import java.util.List; import java.util.List;
import java.util.function.Function; import java.util.function.Function;
import org.apache.avro.Schema; import org.apache.avro.Schema;
import org.apache.avro.generic.IndexedRecord; import org.apache.avro.generic.IndexedRecord;
import org.apache.hudi.WriteStatus;
import org.apache.hudi.common.model.HoodieRecord;
import org.apache.hudi.common.model.HoodieRecordPayload;
import org.apache.hudi.common.util.Option;
import org.apache.hudi.common.util.queue.BoundedInMemoryExecutor;
import org.apache.hudi.common.util.queue.BoundedInMemoryQueueConsumer;
import org.apache.hudi.config.HoodieWriteConfig;
import org.apache.hudi.exception.HoodieException;
import org.apache.hudi.io.HoodieCreateHandle;
import org.apache.hudi.io.HoodieWriteHandle;
import org.apache.hudi.table.HoodieTable;
/** /**
* Lazy Iterable, that writes a stream of HoodieRecords sorted by the partitionPath, into new * Lazy Iterable, that writes a stream of HoodieRecords sorted by the partitionPath, into new

View File

@@ -16,7 +16,7 @@
* limitations under the License. * limitations under the License.
*/ */
package com.uber.hoodie.func; package org.apache.hudi.func;
import java.util.Iterator; import java.util.Iterator;

View File

@@ -16,17 +16,17 @@
* limitations under the License. * limitations under the License.
*/ */
package com.uber.hoodie.func; package org.apache.hudi.func;
import com.uber.hoodie.WriteStatus;
import com.uber.hoodie.common.model.HoodieRecord;
import com.uber.hoodie.common.model.HoodieRecordPayload;
import com.uber.hoodie.config.HoodieWriteConfig;
import com.uber.hoodie.io.HoodieAppendHandle;
import com.uber.hoodie.table.HoodieTable;
import java.util.ArrayList; import java.util.ArrayList;
import java.util.Iterator; import java.util.Iterator;
import java.util.List; import java.util.List;
import org.apache.hudi.WriteStatus;
import org.apache.hudi.common.model.HoodieRecord;
import org.apache.hudi.common.model.HoodieRecordPayload;
import org.apache.hudi.config.HoodieWriteConfig;
import org.apache.hudi.io.HoodieAppendHandle;
import org.apache.hudi.table.HoodieTable;
/** /**
* Lazy Iterable, that writes a stream of HoodieRecords sorted by the partitionPath, into new * Lazy Iterable, that writes a stream of HoodieRecords sorted by the partitionPath, into new

View File

@@ -16,10 +16,10 @@
* limitations under the License. * limitations under the License.
*/ */
package com.uber.hoodie.func; package org.apache.hudi.func;
import com.uber.hoodie.common.util.Option;
import java.io.Serializable; import java.io.Serializable;
import org.apache.hudi.common.util.Option;
/** /**
* Holds Operation result. Used as a result container for Compaction Admin Client (running as part of Spark-launcher * Holds Operation result. Used as a result container for Compaction Admin Client (running as part of Spark-launcher

View File

@@ -16,12 +16,12 @@
* limitations under the License. * limitations under the License.
*/ */
package com.uber.hoodie.func; package org.apache.hudi.func;
import com.uber.hoodie.common.util.queue.BoundedInMemoryQueue;
import com.uber.hoodie.exception.HoodieIOException;
import java.io.IOException; import java.io.IOException;
import java.util.Iterator; import java.util.Iterator;
import org.apache.hudi.common.util.queue.BoundedInMemoryQueue;
import org.apache.hudi.exception.HoodieIOException;
import org.apache.parquet.hadoop.ParquetReader; import org.apache.parquet.hadoop.ParquetReader;
/** /**

View File

@@ -16,16 +16,16 @@
* limitations under the License. * limitations under the License.
*/ */
package com.uber.hoodie.func; package org.apache.hudi.func;
import com.uber.hoodie.common.util.Option;
import com.uber.hoodie.common.util.queue.BoundedInMemoryExecutor;
import com.uber.hoodie.common.util.queue.BoundedInMemoryQueueConsumer;
import com.uber.hoodie.common.util.queue.BoundedInMemoryQueueProducer;
import com.uber.hoodie.common.util.queue.IteratorBasedQueueProducer;
import com.uber.hoodie.config.HoodieWriteConfig;
import java.util.Iterator; import java.util.Iterator;
import java.util.function.Function; import java.util.function.Function;
import org.apache.hudi.common.util.Option;
import org.apache.hudi.common.util.queue.BoundedInMemoryExecutor;
import org.apache.hudi.common.util.queue.BoundedInMemoryQueueConsumer;
import org.apache.hudi.common.util.queue.BoundedInMemoryQueueProducer;
import org.apache.hudi.common.util.queue.IteratorBasedQueueProducer;
import org.apache.hudi.config.HoodieWriteConfig;
import org.apache.spark.TaskContext; import org.apache.spark.TaskContext;
import org.apache.spark.TaskContext$; import org.apache.spark.TaskContext$;

View File

@@ -16,21 +16,22 @@
* limitations under the License. * limitations under the License.
*/ */
package com.uber.hoodie.index; package org.apache.hudi.index;
import com.uber.hoodie.WriteStatus;
import com.uber.hoodie.common.model.HoodieKey;
import com.uber.hoodie.common.model.HoodieRecord;
import com.uber.hoodie.common.model.HoodieRecordPayload;
import com.uber.hoodie.common.util.Option;
import com.uber.hoodie.common.util.collection.Pair;
import com.uber.hoodie.config.HoodieWriteConfig;
import com.uber.hoodie.exception.HoodieIndexException;
import com.uber.hoodie.index.bloom.HoodieBloomIndex;
import com.uber.hoodie.index.bloom.HoodieGlobalBloomIndex;
import com.uber.hoodie.index.hbase.HBaseIndex;
import com.uber.hoodie.table.HoodieTable;
import java.io.Serializable; import java.io.Serializable;
import org.apache.hudi.WriteStatus;
import org.apache.hudi.common.model.FileSlice;
import org.apache.hudi.common.model.HoodieKey;
import org.apache.hudi.common.model.HoodieRecord;
import org.apache.hudi.common.model.HoodieRecordPayload;
import org.apache.hudi.common.util.Option;
import org.apache.hudi.common.util.collection.Pair;
import org.apache.hudi.config.HoodieWriteConfig;
import org.apache.hudi.exception.HoodieIndexException;
import org.apache.hudi.index.bloom.HoodieBloomIndex;
import org.apache.hudi.index.bloom.HoodieGlobalBloomIndex;
import org.apache.hudi.index.hbase.HBaseIndex;
import org.apache.hudi.table.HoodieTable;
import org.apache.spark.api.java.JavaPairRDD; import org.apache.spark.api.java.JavaPairRDD;
import org.apache.spark.api.java.JavaRDD; import org.apache.spark.api.java.JavaRDD;
import org.apache.spark.api.java.JavaSparkContext; import org.apache.spark.api.java.JavaSparkContext;
@@ -102,7 +103,7 @@ public abstract class HoodieIndex<T extends HoodieRecordPayload> implements Seri
/** /**
* This is used by storage to determine, if its safe to send inserts, straight to the log, i.e * This is used by storage to determine, if its safe to send inserts, straight to the log, i.e
* having a {@link com.uber.hoodie.common.model.FileSlice}, with no data file. * having a {@link FileSlice}, with no data file.
* *
* @return Returns true/false depending on whether the impl has this capability * @return Returns true/false depending on whether the impl has this capability
*/ */

View File

@@ -16,22 +16,22 @@
* limitations under the License. * limitations under the License.
*/ */
package com.uber.hoodie.index; package org.apache.hudi.index;
import com.uber.hoodie.WriteStatus;
import com.uber.hoodie.common.model.HoodieKey;
import com.uber.hoodie.common.model.HoodieRecord;
import com.uber.hoodie.common.model.HoodieRecordLocation;
import com.uber.hoodie.common.model.HoodieRecordPayload;
import com.uber.hoodie.common.util.Option;
import com.uber.hoodie.common.util.collection.Pair;
import com.uber.hoodie.config.HoodieWriteConfig;
import com.uber.hoodie.table.HoodieTable;
import java.util.ArrayList; import java.util.ArrayList;
import java.util.Iterator; import java.util.Iterator;
import java.util.List; import java.util.List;
import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.ConcurrentMap; import java.util.concurrent.ConcurrentMap;
import org.apache.hudi.WriteStatus;
import org.apache.hudi.common.model.HoodieKey;
import org.apache.hudi.common.model.HoodieRecord;
import org.apache.hudi.common.model.HoodieRecordLocation;
import org.apache.hudi.common.model.HoodieRecordPayload;
import org.apache.hudi.common.util.Option;
import org.apache.hudi.common.util.collection.Pair;
import org.apache.hudi.config.HoodieWriteConfig;
import org.apache.hudi.table.HoodieTable;
import org.apache.spark.api.java.JavaPairRDD; import org.apache.spark.api.java.JavaPairRDD;
import org.apache.spark.api.java.JavaRDD; import org.apache.spark.api.java.JavaRDD;
import org.apache.spark.api.java.JavaSparkContext; import org.apache.spark.api.java.JavaSparkContext;

View File

@@ -16,7 +16,7 @@
* limitations under the License. * limitations under the License.
*/ */
package com.uber.hoodie.index.bloom; package org.apache.hudi.index.bloom;
import com.google.common.base.Objects; import com.google.common.base.Objects;
import java.io.Serializable; import java.io.Serializable;

View File

@@ -16,18 +16,17 @@
* limitations under the License. * limitations under the License.
*/ */
package com.uber.hoodie.index.bloom; package org.apache.hudi.index.bloom;
import com.google.common.annotations.VisibleForTesting; import com.google.common.annotations.VisibleForTesting;
import com.google.common.hash.Hashing; import com.google.common.hash.Hashing;
import com.uber.hoodie.common.util.collection.Pair;
import java.nio.charset.StandardCharsets; import java.nio.charset.StandardCharsets;
import java.util.ArrayList; import java.util.ArrayList;
import java.util.HashMap; import java.util.HashMap;
import java.util.List; import java.util.List;
import java.util.Map; import java.util.Map;
import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicInteger;
import org.apache.hudi.common.util.collection.Pair;
import org.apache.log4j.LogManager; import org.apache.log4j.LogManager;
import org.apache.log4j.Logger; import org.apache.log4j.Logger;
import org.apache.spark.Partitioner; import org.apache.spark.Partitioner;

View File

@@ -16,31 +16,32 @@
* limitations under the License. * limitations under the License.
*/ */
package com.uber.hoodie.index.bloom; package org.apache.hudi.index.bloom;
import static java.util.stream.Collectors.groupingBy; import static java.util.stream.Collectors.groupingBy;
import static java.util.stream.Collectors.mapping; import static java.util.stream.Collectors.mapping;
import static java.util.stream.Collectors.toList; import static java.util.stream.Collectors.toList;
import com.google.common.annotations.VisibleForTesting; import com.google.common.annotations.VisibleForTesting;
import com.uber.hoodie.WriteStatus;
import com.uber.hoodie.common.model.HoodieKey;
import com.uber.hoodie.common.model.HoodieRecord;
import com.uber.hoodie.common.model.HoodieRecordLocation;
import com.uber.hoodie.common.model.HoodieRecordPayload;
import com.uber.hoodie.common.table.timeline.HoodieInstant;
import com.uber.hoodie.common.util.Option;
import com.uber.hoodie.common.util.collection.Pair;
import com.uber.hoodie.config.HoodieWriteConfig;
import com.uber.hoodie.exception.MetadataNotFoundException;
import com.uber.hoodie.index.HoodieIndex;
import com.uber.hoodie.io.HoodieRangeInfoHandle;
import com.uber.hoodie.table.HoodieTable;
import java.util.ArrayList; import java.util.ArrayList;
import java.util.HashMap; import java.util.HashMap;
import java.util.List; import java.util.List;
import java.util.Map; import java.util.Map;
import java.util.stream.Collectors; import java.util.stream.Collectors;
import org.apache.hudi.WriteStatus;
import org.apache.hudi.common.model.HoodieKey;
import org.apache.hudi.common.model.HoodieRecord;
import org.apache.hudi.common.model.HoodieRecordLocation;
import org.apache.hudi.common.model.HoodieRecordPayload;
import org.apache.hudi.common.table.timeline.HoodieInstant;
import org.apache.hudi.common.util.Option;
import org.apache.hudi.common.util.collection.Pair;
import org.apache.hudi.config.HoodieIndexConfig;
import org.apache.hudi.config.HoodieWriteConfig;
import org.apache.hudi.exception.MetadataNotFoundException;
import org.apache.hudi.index.HoodieIndex;
import org.apache.hudi.io.HoodieRangeInfoHandle;
import org.apache.hudi.table.HoodieTable;
import org.apache.log4j.LogManager; import org.apache.log4j.LogManager;
import org.apache.log4j.Logger; import org.apache.log4j.Logger;
import org.apache.spark.Partitioner; import org.apache.spark.Partitioner;
@@ -192,7 +193,7 @@ public class HoodieBloomIndex<T extends HoodieRecordPayload> extends HoodieIndex
* Compute the minimum parallelism needed to play well with the spark 2GB limitation.. The index lookup can be skewed * Compute the minimum parallelism needed to play well with the spark 2GB limitation.. The index lookup can be skewed
* in three dimensions : #files, #partitions, #records <p> To be able to smoothly handle skews, we need to compute how * in three dimensions : #files, #partitions, #records <p> To be able to smoothly handle skews, we need to compute how
* to split each partitions into subpartitions. We do it here, in a way that keeps the amount of each Spark join * to split each partitions into subpartitions. We do it here, in a way that keeps the amount of each Spark join
* partition to < 2GB. <p> If {@link com.uber.hoodie.config.HoodieIndexConfig#BLOOM_INDEX_PARALLELISM_PROP} is * partition to < 2GB. <p> If {@link HoodieIndexConfig#BLOOM_INDEX_PARALLELISM_PROP} is
* specified as a NON-zero number, then that is used explicitly. * specified as a NON-zero number, then that is used explicitly.
*/ */
int computeSafeParallelism(Map<String, Long> recordsPerPartition, Map<String, Long> comparisonsPerFileGroup) { int computeSafeParallelism(Map<String, Long> recordsPerPartition, Map<String, Long> comparisonsPerFileGroup) {

View File

@@ -16,20 +16,20 @@
* limitations under the License. * limitations under the License.
*/ */
package com.uber.hoodie.index.bloom; package org.apache.hudi.index.bloom;
import com.uber.hoodie.common.model.HoodieKey;
import com.uber.hoodie.common.util.collection.Pair;
import com.uber.hoodie.config.HoodieWriteConfig;
import com.uber.hoodie.exception.HoodieException;
import com.uber.hoodie.exception.HoodieIndexException;
import com.uber.hoodie.func.LazyIterableIterator;
import com.uber.hoodie.io.HoodieKeyLookupHandle;
import com.uber.hoodie.io.HoodieKeyLookupHandle.KeyLookupResult;
import com.uber.hoodie.table.HoodieTable;
import java.util.ArrayList; import java.util.ArrayList;
import java.util.Iterator; import java.util.Iterator;
import java.util.List; import java.util.List;
import org.apache.hudi.common.model.HoodieKey;
import org.apache.hudi.common.util.collection.Pair;
import org.apache.hudi.config.HoodieWriteConfig;
import org.apache.hudi.exception.HoodieException;
import org.apache.hudi.exception.HoodieIndexException;
import org.apache.hudi.func.LazyIterableIterator;
import org.apache.hudi.io.HoodieKeyLookupHandle;
import org.apache.hudi.io.HoodieKeyLookupHandle.KeyLookupResult;
import org.apache.hudi.table.HoodieTable;
import org.apache.spark.api.java.function.Function2; import org.apache.spark.api.java.function.Function2;
import scala.Tuple2; import scala.Tuple2;

View File

@@ -16,25 +16,25 @@
* limitations under the License. * limitations under the License.
*/ */
package com.uber.hoodie.index.bloom; package org.apache.hudi.index.bloom;
import com.google.common.annotations.VisibleForTesting; import com.google.common.annotations.VisibleForTesting;
import com.uber.hoodie.common.model.HoodieKey;
import com.uber.hoodie.common.model.HoodieRecord;
import com.uber.hoodie.common.model.HoodieRecordLocation;
import com.uber.hoodie.common.model.HoodieRecordPayload;
import com.uber.hoodie.common.table.HoodieTableMetaClient;
import com.uber.hoodie.common.util.FSUtils;
import com.uber.hoodie.common.util.Option;
import com.uber.hoodie.config.HoodieWriteConfig;
import com.uber.hoodie.exception.HoodieIOException;
import com.uber.hoodie.table.HoodieTable;
import java.io.IOException; import java.io.IOException;
import java.util.HashMap; import java.util.HashMap;
import java.util.List; import java.util.List;
import java.util.Map; import java.util.Map;
import java.util.Map.Entry; import java.util.Map.Entry;
import java.util.stream.Collectors; import java.util.stream.Collectors;
import org.apache.hudi.common.model.HoodieKey;
import org.apache.hudi.common.model.HoodieRecord;
import org.apache.hudi.common.model.HoodieRecordLocation;
import org.apache.hudi.common.model.HoodieRecordPayload;
import org.apache.hudi.common.table.HoodieTableMetaClient;
import org.apache.hudi.common.util.FSUtils;
import org.apache.hudi.common.util.Option;
import org.apache.hudi.config.HoodieWriteConfig;
import org.apache.hudi.exception.HoodieIOException;
import org.apache.hudi.table.HoodieTable;
import org.apache.spark.api.java.JavaPairRDD; import org.apache.spark.api.java.JavaPairRDD;
import org.apache.spark.api.java.JavaRDD; import org.apache.spark.api.java.JavaRDD;
import org.apache.spark.api.java.JavaSparkContext; import org.apache.spark.api.java.JavaSparkContext;

View File

@@ -16,7 +16,7 @@
* limitations under the License. * limitations under the License.
*/ */
package com.uber.hoodie.index.bloom; package org.apache.hudi.index.bloom;
import java.io.Serializable; import java.io.Serializable;
import java.util.Set; import java.util.Set;

View File

@@ -16,7 +16,7 @@
* limitations under the License. * limitations under the License.
*/ */
package com.uber.hoodie.index.bloom; package org.apache.hudi.index.bloom;
import java.util.Collection; import java.util.Collection;
import java.util.Collections; import java.util.Collections;

View File

@@ -16,7 +16,7 @@
* limitations under the License. * limitations under the License.
*/ */
package com.uber.hoodie.index.bloom; package org.apache.hudi.index.bloom;
import java.util.Collections; import java.util.Collections;
import java.util.HashMap; import java.util.HashMap;

View File

@@ -16,7 +16,7 @@
* limitations under the License. * limitations under the License.
*/ */
package com.uber.hoodie.index.bloom; package org.apache.hudi.index.bloom;
import java.util.List; import java.util.List;

View File

@@ -16,7 +16,7 @@
* limitations under the License. * limitations under the License.
*/ */
package com.uber.hoodie.index.bloom; package org.apache.hudi.index.bloom;
import java.io.Serializable; import java.io.Serializable;
import java.util.HashSet; import java.util.HashSet;

View File

@@ -16,7 +16,7 @@
* limitations under the License. * limitations under the License.
*/ */
package com.uber.hoodie.index.bloom; package org.apache.hudi.index.bloom;
import java.io.Serializable; import java.io.Serializable;
import java.util.ArrayList; import java.util.ArrayList;

View File

@@ -16,7 +16,7 @@
* limitations under the License. * limitations under the License.
*/ */
package com.uber.hoodie.index.bloom; package org.apache.hudi.index.bloom;
import java.util.HashSet; import java.util.HashSet;
import java.util.List; import java.util.List;

View File

@@ -16,7 +16,7 @@
* limitations under the License. * limitations under the License.
*/ */
package com.uber.hoodie.index.bloom; package org.apache.hudi.index.bloom;
import java.util.HashSet; import java.util.HashSet;
import java.util.List; import java.util.List;

View File

@@ -16,10 +16,9 @@
* limitations under the License. * limitations under the License.
*/ */
package com.uber.hoodie.index.hbase; package org.apache.hudi.index.hbase;
import com.uber.hoodie.config.HoodieWriteConfig;
import org.apache.hudi.config.HoodieWriteConfig;
import org.apache.log4j.LogManager; import org.apache.log4j.LogManager;
import org.apache.log4j.Logger; import org.apache.log4j.Logger;

View File

@@ -16,26 +16,9 @@
* limitations under the License. * limitations under the License.
*/ */
package com.uber.hoodie.index.hbase; package org.apache.hudi.index.hbase;
import com.google.common.annotations.VisibleForTesting; import com.google.common.annotations.VisibleForTesting;
import com.uber.hoodie.WriteStatus;
import com.uber.hoodie.common.model.HoodieKey;
import com.uber.hoodie.common.model.HoodieRecord;
import com.uber.hoodie.common.model.HoodieRecordLocation;
import com.uber.hoodie.common.model.HoodieRecordPayload;
import com.uber.hoodie.common.table.HoodieTableMetaClient;
import com.uber.hoodie.common.table.HoodieTimeline;
import com.uber.hoodie.common.table.timeline.HoodieInstant;
import com.uber.hoodie.common.util.Option;
import com.uber.hoodie.common.util.ReflectionUtils;
import com.uber.hoodie.common.util.collection.Pair;
import com.uber.hoodie.config.HoodieIndexConfig;
import com.uber.hoodie.config.HoodieWriteConfig;
import com.uber.hoodie.exception.HoodieDependentSystemUnavailableException;
import com.uber.hoodie.exception.HoodieIndexException;
import com.uber.hoodie.index.HoodieIndex;
import com.uber.hoodie.table.HoodieTable;
import java.io.IOException; import java.io.IOException;
import java.io.Serializable; import java.io.Serializable;
import java.util.ArrayList; import java.util.ArrayList;
@@ -54,6 +37,23 @@ import org.apache.hadoop.hbase.client.Put;
import org.apache.hadoop.hbase.client.RegionLocator; import org.apache.hadoop.hbase.client.RegionLocator;
import org.apache.hadoop.hbase.client.Result; import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hudi.WriteStatus;
import org.apache.hudi.common.model.HoodieKey;
import org.apache.hudi.common.model.HoodieRecord;
import org.apache.hudi.common.model.HoodieRecordLocation;
import org.apache.hudi.common.model.HoodieRecordPayload;
import org.apache.hudi.common.table.HoodieTableMetaClient;
import org.apache.hudi.common.table.HoodieTimeline;
import org.apache.hudi.common.table.timeline.HoodieInstant;
import org.apache.hudi.common.util.Option;
import org.apache.hudi.common.util.ReflectionUtils;
import org.apache.hudi.common.util.collection.Pair;
import org.apache.hudi.config.HoodieIndexConfig;
import org.apache.hudi.config.HoodieWriteConfig;
import org.apache.hudi.exception.HoodieDependentSystemUnavailableException;
import org.apache.hudi.exception.HoodieIndexException;
import org.apache.hudi.index.HoodieIndex;
import org.apache.hudi.table.HoodieTable;
import org.apache.log4j.LogManager; import org.apache.log4j.LogManager;
import org.apache.log4j.Logger; import org.apache.log4j.Logger;
import org.apache.spark.SparkConf; import org.apache.spark.SparkConf;

View File

@@ -16,7 +16,7 @@
* limitations under the License. * limitations under the License.
*/ */
package com.uber.hoodie.index.hbase; package org.apache.hudi.index.hbase;
import java.io.Serializable; import java.io.Serializable;

View File

@@ -16,32 +16,9 @@
* limitations under the License. * limitations under the License.
*/ */
package com.uber.hoodie.io; package org.apache.hudi.io;
import com.beust.jcommander.internal.Maps; import com.beust.jcommander.internal.Maps;
import com.uber.hoodie.WriteStatus;
import com.uber.hoodie.common.model.FileSlice;
import com.uber.hoodie.common.model.HoodieDeltaWriteStat;
import com.uber.hoodie.common.model.HoodieKey;
import com.uber.hoodie.common.model.HoodieLogFile;
import com.uber.hoodie.common.model.HoodieRecord;
import com.uber.hoodie.common.model.HoodieRecordLocation;
import com.uber.hoodie.common.model.HoodieRecordPayload;
import com.uber.hoodie.common.model.HoodieWriteStat;
import com.uber.hoodie.common.model.HoodieWriteStat.RuntimeStats;
import com.uber.hoodie.common.table.TableFileSystemView.RealtimeView;
import com.uber.hoodie.common.table.log.HoodieLogFormat;
import com.uber.hoodie.common.table.log.HoodieLogFormat.Writer;
import com.uber.hoodie.common.table.log.block.HoodieAvroDataBlock;
import com.uber.hoodie.common.table.log.block.HoodieDeleteBlock;
import com.uber.hoodie.common.table.log.block.HoodieLogBlock;
import com.uber.hoodie.common.util.FSUtils;
import com.uber.hoodie.common.util.HoodieAvroUtils;
import com.uber.hoodie.common.util.Option;
import com.uber.hoodie.config.HoodieWriteConfig;
import com.uber.hoodie.exception.HoodieAppendException;
import com.uber.hoodie.exception.HoodieUpsertException;
import com.uber.hoodie.table.HoodieTable;
import java.io.IOException; import java.io.IOException;
import java.util.ArrayList; import java.util.ArrayList;
import java.util.Iterator; import java.util.Iterator;
@@ -51,6 +28,30 @@ import java.util.concurrent.atomic.AtomicLong;
import org.apache.avro.generic.GenericRecord; import org.apache.avro.generic.GenericRecord;
import org.apache.avro.generic.IndexedRecord; import org.apache.avro.generic.IndexedRecord;
import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.Path;
import org.apache.hudi.WriteStatus;
import org.apache.hudi.common.model.FileSlice;
import org.apache.hudi.common.model.HoodieDeltaWriteStat;
import org.apache.hudi.common.model.HoodieKey;
import org.apache.hudi.common.model.HoodieLogFile;
import org.apache.hudi.common.model.HoodieRecord;
import org.apache.hudi.common.model.HoodieRecordLocation;
import org.apache.hudi.common.model.HoodieRecordPayload;
import org.apache.hudi.common.model.HoodieWriteStat;
import org.apache.hudi.common.model.HoodieWriteStat.RuntimeStats;
import org.apache.hudi.common.table.TableFileSystemView.RealtimeView;
import org.apache.hudi.common.table.log.HoodieLogFormat;
import org.apache.hudi.common.table.log.HoodieLogFormat.Writer;
import org.apache.hudi.common.table.log.block.HoodieAvroDataBlock;
import org.apache.hudi.common.table.log.block.HoodieDeleteBlock;
import org.apache.hudi.common.table.log.block.HoodieLogBlock;
import org.apache.hudi.common.table.log.block.HoodieLogBlock.HeaderMetadataType;
import org.apache.hudi.common.util.FSUtils;
import org.apache.hudi.common.util.HoodieAvroUtils;
import org.apache.hudi.common.util.Option;
import org.apache.hudi.config.HoodieWriteConfig;
import org.apache.hudi.exception.HoodieAppendException;
import org.apache.hudi.exception.HoodieUpsertException;
import org.apache.hudi.table.HoodieTable;
import org.apache.log4j.LogManager; import org.apache.log4j.LogManager;
import org.apache.log4j.Logger; import org.apache.log4j.Logger;
import org.apache.spark.TaskContext; import org.apache.spark.TaskContext;
@@ -91,7 +92,7 @@ public class HoodieAppendHandle<T extends HoodieRecordPayload> extends HoodieWri
// Max block size to limit to for a log block // Max block size to limit to for a log block
private int maxBlockSize = config.getLogFileDataBlockMaxSize(); private int maxBlockSize = config.getLogFileDataBlockMaxSize();
// Header metadata for a log block // Header metadata for a log block
private Map<HoodieLogBlock.HeaderMetadataType, String> header = Maps.newHashMap(); private Map<HeaderMetadataType, String> header = Maps.newHashMap();
// Total number of new records inserted into the delta file // Total number of new records inserted into the delta file
private long insertRecordsWritten = 0; private long insertRecordsWritten = 0;
@@ -198,7 +199,7 @@ public class HoodieAppendHandle<T extends HoodieRecordPayload> extends HoodieWri
estimatedNumberOfBytesWritten += averageRecordSize * numberOfRecords; estimatedNumberOfBytesWritten += averageRecordSize * numberOfRecords;
} }
private void doAppend(Map<HoodieLogBlock.HeaderMetadataType, String> header) { private void doAppend(Map<HeaderMetadataType, String> header) {
try { try {
header.put(HoodieLogBlock.HeaderMetadataType.INSTANT_TIME, instantTime); header.put(HoodieLogBlock.HeaderMetadataType.INSTANT_TIME, instantTime);
header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, writerSchema.toString()); header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, writerSchema.toString());

View File

@@ -16,29 +16,30 @@
* limitations under the License. * limitations under the License.
*/ */
package com.uber.hoodie.io; package org.apache.hudi.io;
import com.uber.hoodie.common.model.CompactionOperation;
import com.uber.hoodie.common.model.FileSlice;
import com.uber.hoodie.common.model.HoodieCleaningPolicy;
import com.uber.hoodie.common.model.HoodieDataFile;
import com.uber.hoodie.common.model.HoodieFileGroup;
import com.uber.hoodie.common.model.HoodieFileGroupId;
import com.uber.hoodie.common.model.HoodieRecordPayload;
import com.uber.hoodie.common.model.HoodieTableType;
import com.uber.hoodie.common.table.HoodieTimeline;
import com.uber.hoodie.common.table.SyncableFileSystemView;
import com.uber.hoodie.common.table.timeline.HoodieInstant;
import com.uber.hoodie.common.util.Option;
import com.uber.hoodie.common.util.collection.Pair;
import com.uber.hoodie.config.HoodieWriteConfig;
import com.uber.hoodie.table.HoodieTable;
import java.io.IOException; import java.io.IOException;
import java.util.ArrayList; import java.util.ArrayList;
import java.util.Iterator; import java.util.Iterator;
import java.util.List; import java.util.List;
import java.util.Map; import java.util.Map;
import java.util.stream.Collectors; import java.util.stream.Collectors;
import org.apache.hudi.common.model.CompactionOperation;
import org.apache.hudi.common.model.FileSlice;
import org.apache.hudi.common.model.HoodieCleaningPolicy;
import org.apache.hudi.common.model.HoodieCommitMetadata;
import org.apache.hudi.common.model.HoodieDataFile;
import org.apache.hudi.common.model.HoodieFileGroup;
import org.apache.hudi.common.model.HoodieFileGroupId;
import org.apache.hudi.common.model.HoodieRecordPayload;
import org.apache.hudi.common.model.HoodieTableType;
import org.apache.hudi.common.table.HoodieTimeline;
import org.apache.hudi.common.table.SyncableFileSystemView;
import org.apache.hudi.common.table.timeline.HoodieInstant;
import org.apache.hudi.common.util.Option;
import org.apache.hudi.common.util.collection.Pair;
import org.apache.hudi.config.HoodieWriteConfig;
import org.apache.hudi.table.HoodieTable;
import org.apache.log4j.LogManager; import org.apache.log4j.LogManager;
import org.apache.log4j.Logger; import org.apache.log4j.Logger;
@@ -46,7 +47,7 @@ import org.apache.log4j.Logger;
* Cleaner is responsible for garbage collecting older files in a given partition path, such that * Cleaner is responsible for garbage collecting older files in a given partition path, such that
* <p> 1) It provides sufficient time for existing queries running on older versions, to close <p> * <p> 1) It provides sufficient time for existing queries running on older versions, to close <p>
* 2) It bounds the growth of the files in the file system <p> TODO: Should all cleaning be done * 2) It bounds the growth of the files in the file system <p> TODO: Should all cleaning be done
* based on {@link com.uber.hoodie.common.model.HoodieCommitMetadata} * based on {@link HoodieCommitMetadata}
*/ */
public class HoodieCleanHelper<T extends HoodieRecordPayload<T>> { public class HoodieCleanHelper<T extends HoodieRecordPayload<T>> {

View File

@@ -16,39 +16,12 @@
* limitations under the License. * limitations under the License.
*/ */
package com.uber.hoodie.io; package org.apache.hudi.io;
import static com.uber.hoodie.common.table.HoodieTimeline.COMMIT_ACTION;
import static com.uber.hoodie.common.table.HoodieTimeline.DELTA_COMMIT_ACTION;
import static com.uber.hoodie.common.table.HoodieTimeline.LESSER_OR_EQUAL;
import com.fasterxml.jackson.databind.DeserializationFeature; import com.fasterxml.jackson.databind.DeserializationFeature;
import com.fasterxml.jackson.databind.ObjectMapper; import com.fasterxml.jackson.databind.ObjectMapper;
import com.google.common.collect.Maps; import com.google.common.collect.Maps;
import com.google.common.collect.Sets; import com.google.common.collect.Sets;
import com.uber.hoodie.avro.model.HoodieArchivedMetaEntry;
import com.uber.hoodie.avro.model.HoodieCleanMetadata;
import com.uber.hoodie.avro.model.HoodieRollbackMetadata;
import com.uber.hoodie.avro.model.HoodieSavepointMetadata;
import com.uber.hoodie.common.model.ActionType;
import com.uber.hoodie.common.model.HoodieArchivedLogFile;
import com.uber.hoodie.common.model.HoodieCommitMetadata;
import com.uber.hoodie.common.model.HoodieRollingStatMetadata;
import com.uber.hoodie.common.table.HoodieTableMetaClient;
import com.uber.hoodie.common.table.HoodieTimeline;
import com.uber.hoodie.common.table.log.HoodieLogFormat;
import com.uber.hoodie.common.table.log.block.HoodieAvroDataBlock;
import com.uber.hoodie.common.table.log.block.HoodieLogBlock;
import com.uber.hoodie.common.table.timeline.HoodieActiveTimeline;
import com.uber.hoodie.common.table.timeline.HoodieArchivedTimeline;
import com.uber.hoodie.common.table.timeline.HoodieInstant;
import com.uber.hoodie.common.util.AvroUtils;
import com.uber.hoodie.common.util.Option;
import com.uber.hoodie.config.HoodieWriteConfig;
import com.uber.hoodie.exception.HoodieCommitException;
import com.uber.hoodie.exception.HoodieException;
import com.uber.hoodie.exception.HoodieIOException;
import com.uber.hoodie.table.HoodieTable;
import java.io.IOException; import java.io.IOException;
import java.util.ArrayList; import java.util.ArrayList;
import java.util.Comparator; import java.util.Comparator;
@@ -59,6 +32,31 @@ import java.util.stream.Stream;
import org.apache.avro.Schema; import org.apache.avro.Schema;
import org.apache.avro.generic.IndexedRecord; import org.apache.avro.generic.IndexedRecord;
import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.Path;
import org.apache.hudi.avro.model.HoodieArchivedMetaEntry;
import org.apache.hudi.avro.model.HoodieCleanMetadata;
import org.apache.hudi.avro.model.HoodieRollbackMetadata;
import org.apache.hudi.avro.model.HoodieSavepointMetadata;
import org.apache.hudi.common.model.ActionType;
import org.apache.hudi.common.model.HoodieArchivedLogFile;
import org.apache.hudi.common.model.HoodieCommitMetadata;
import org.apache.hudi.common.model.HoodieRollingStatMetadata;
import org.apache.hudi.common.table.HoodieTableMetaClient;
import org.apache.hudi.common.table.HoodieTimeline;
import org.apache.hudi.common.table.log.HoodieLogFormat;
import org.apache.hudi.common.table.log.HoodieLogFormat.Writer;
import org.apache.hudi.common.table.log.block.HoodieAvroDataBlock;
import org.apache.hudi.common.table.log.block.HoodieLogBlock;
import org.apache.hudi.common.table.log.block.HoodieLogBlock.HeaderMetadataType;
import org.apache.hudi.common.table.timeline.HoodieActiveTimeline;
import org.apache.hudi.common.table.timeline.HoodieArchivedTimeline;
import org.apache.hudi.common.table.timeline.HoodieInstant;
import org.apache.hudi.common.util.AvroUtils;
import org.apache.hudi.common.util.Option;
import org.apache.hudi.config.HoodieWriteConfig;
import org.apache.hudi.exception.HoodieCommitException;
import org.apache.hudi.exception.HoodieException;
import org.apache.hudi.exception.HoodieIOException;
import org.apache.hudi.table.HoodieTable;
import org.apache.log4j.LogManager; import org.apache.log4j.LogManager;
import org.apache.log4j.Logger; import org.apache.log4j.Logger;
import org.apache.spark.api.java.JavaSparkContext; import org.apache.spark.api.java.JavaSparkContext;
@@ -73,7 +71,7 @@ public class HoodieCommitArchiveLog {
private final Path archiveFilePath; private final Path archiveFilePath;
private final HoodieTableMetaClient metaClient; private final HoodieTableMetaClient metaClient;
private final HoodieWriteConfig config; private final HoodieWriteConfig config;
private HoodieLogFormat.Writer writer; private Writer writer;
public HoodieCommitArchiveLog(HoodieWriteConfig config, HoodieTableMetaClient metaClient) { public HoodieCommitArchiveLog(HoodieWriteConfig config, HoodieTableMetaClient metaClient) {
this.config = config; this.config = config;
@@ -81,7 +79,7 @@ public class HoodieCommitArchiveLog {
this.archiveFilePath = HoodieArchivedTimeline.getArchiveLogPath(metaClient.getArchivePath()); this.archiveFilePath = HoodieArchivedTimeline.getArchiveLogPath(metaClient.getArchivePath());
} }
private HoodieLogFormat.Writer openWriter() { private Writer openWriter() {
try { try {
if (this.writer == null) { if (this.writer == null) {
return HoodieLogFormat.newWriterBuilder().onParentPath(archiveFilePath.getParent()) return HoodieLogFormat.newWriterBuilder().onParentPath(archiveFilePath.getParent())
@@ -200,7 +198,8 @@ public class HoodieCommitArchiveLog {
Option.fromJavaOptional(archivedInstants.stream() Option.fromJavaOptional(archivedInstants.stream()
.filter(i -> { .filter(i -> {
return i.isCompleted() return i.isCompleted()
&& (i.getAction().equals(COMMIT_ACTION) || (i.getAction().equals(DELTA_COMMIT_ACTION))); && (i.getAction().equals(HoodieTimeline.COMMIT_ACTION) || (i.getAction().equals(
HoodieTimeline.DELTA_COMMIT_ACTION)));
}).max(Comparator.comparing(HoodieInstant::getTimestamp))); }).max(Comparator.comparing(HoodieInstant::getTimestamp)));
if (latestCommitted.isPresent()) { if (latestCommitted.isPresent()) {
success &= deleteAllInstantsOlderorEqualsInAuxMetaFolder(latestCommitted.get()); success &= deleteAllInstantsOlderorEqualsInAuxMetaFolder(latestCommitted.get());
@@ -224,7 +223,7 @@ public class HoodieCommitArchiveLog {
List<HoodieInstant> instantsToBeDeleted = List<HoodieInstant> instantsToBeDeleted =
instants.stream().filter(instant1 -> HoodieTimeline.compareTimestamps(instant1.getTimestamp(), instants.stream().filter(instant1 -> HoodieTimeline.compareTimestamps(instant1.getTimestamp(),
thresholdInstant.getTimestamp(), LESSER_OR_EQUAL)).collect(Collectors.toList()); thresholdInstant.getTimestamp(), HoodieTimeline.LESSER_OR_EQUAL)).collect(Collectors.toList());
boolean success = true; boolean success = true;
for (HoodieInstant deleteInstant : instantsToBeDeleted) { for (HoodieInstant deleteInstant : instantsToBeDeleted) {
@@ -270,7 +269,7 @@ public class HoodieCommitArchiveLog {
private void writeToFile(Schema wrapperSchema, List<IndexedRecord> records) throws Exception { private void writeToFile(Schema wrapperSchema, List<IndexedRecord> records) throws Exception {
if (records.size() > 0) { if (records.size() > 0) {
Map<HoodieLogBlock.HeaderMetadataType, String> header = Maps.newHashMap(); Map<HeaderMetadataType, String> header = Maps.newHashMap();
header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, wrapperSchema.toString()); header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, wrapperSchema.toString());
HoodieAvroDataBlock block = new HoodieAvroDataBlock(records, header); HoodieAvroDataBlock block = new HoodieAvroDataBlock(records, header);
this.writer = writer.appendBlock(block); this.writer = writer.appendBlock(block);
@@ -290,7 +289,7 @@ public class HoodieCommitArchiveLog {
archivedMetaWrapper.setActionType(ActionType.clean.name()); archivedMetaWrapper.setActionType(ActionType.clean.name());
break; break;
} }
case COMMIT_ACTION: { case HoodieTimeline.COMMIT_ACTION: {
HoodieCommitMetadata commitMetadata = HoodieCommitMetadata HoodieCommitMetadata commitMetadata = HoodieCommitMetadata
.fromBytes(commitTimeline.getInstantDetails(hoodieInstant).get(), HoodieCommitMetadata.class); .fromBytes(commitTimeline.getInstantDetails(hoodieInstant).get(), HoodieCommitMetadata.class);
archivedMetaWrapper.setHoodieCommitMetadata(commitMetadataConverter(commitMetadata)); archivedMetaWrapper.setHoodieCommitMetadata(commitMetadataConverter(commitMetadata));
@@ -324,13 +323,13 @@ public class HoodieCommitArchiveLog {
return archivedMetaWrapper; return archivedMetaWrapper;
} }
private com.uber.hoodie.avro.model.HoodieCommitMetadata commitMetadataConverter( private org.apache.hudi.avro.model.HoodieCommitMetadata commitMetadataConverter(
HoodieCommitMetadata hoodieCommitMetadata) { HoodieCommitMetadata hoodieCommitMetadata) {
ObjectMapper mapper = new ObjectMapper(); ObjectMapper mapper = new ObjectMapper();
//Need this to ignore other public get() methods //Need this to ignore other public get() methods
mapper.configure(DeserializationFeature.FAIL_ON_UNKNOWN_PROPERTIES, false); mapper.configure(DeserializationFeature.FAIL_ON_UNKNOWN_PROPERTIES, false);
com.uber.hoodie.avro.model.HoodieCommitMetadata avroMetaData = mapper org.apache.hudi.avro.model.HoodieCommitMetadata avroMetaData = mapper
.convertValue(hoodieCommitMetadata, com.uber.hoodie.avro.model.HoodieCommitMetadata.class); .convertValue(hoodieCommitMetadata, org.apache.hudi.avro.model.HoodieCommitMetadata.class);
// Do not archive Rolling Stats, cannot set to null since AVRO will throw null pointer // Do not archive Rolling Stats, cannot set to null since AVRO will throw null pointer
avroMetaData.getExtraMetadata().put(HoodieRollingStatMetadata.ROLLING_STAT_METADATA_KEY, ""); avroMetaData.getExtraMetadata().put(HoodieRollingStatMetadata.ROLLING_STAT_METADATA_KEY, "");
return avroMetaData; return avroMetaData;

View File

@@ -16,27 +16,27 @@
* limitations under the License. * limitations under the License.
*/ */
package com.uber.hoodie.io; package org.apache.hudi.io;
import com.uber.hoodie.WriteStatus;
import com.uber.hoodie.common.model.HoodiePartitionMetadata;
import com.uber.hoodie.common.model.HoodieRecord;
import com.uber.hoodie.common.model.HoodieRecordLocation;
import com.uber.hoodie.common.model.HoodieRecordPayload;
import com.uber.hoodie.common.model.HoodieWriteStat;
import com.uber.hoodie.common.model.HoodieWriteStat.RuntimeStats;
import com.uber.hoodie.common.util.FSUtils;
import com.uber.hoodie.common.util.Option;
import com.uber.hoodie.config.HoodieWriteConfig;
import com.uber.hoodie.exception.HoodieInsertException;
import com.uber.hoodie.io.storage.HoodieStorageWriter;
import com.uber.hoodie.io.storage.HoodieStorageWriterFactory;
import com.uber.hoodie.table.HoodieTable;
import java.io.IOException; import java.io.IOException;
import java.util.Iterator; import java.util.Iterator;
import org.apache.avro.generic.GenericRecord; import org.apache.avro.generic.GenericRecord;
import org.apache.avro.generic.IndexedRecord; import org.apache.avro.generic.IndexedRecord;
import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.Path;
import org.apache.hudi.WriteStatus;
import org.apache.hudi.common.model.HoodiePartitionMetadata;
import org.apache.hudi.common.model.HoodieRecord;
import org.apache.hudi.common.model.HoodieRecordLocation;
import org.apache.hudi.common.model.HoodieRecordPayload;
import org.apache.hudi.common.model.HoodieWriteStat;
import org.apache.hudi.common.model.HoodieWriteStat.RuntimeStats;
import org.apache.hudi.common.util.FSUtils;
import org.apache.hudi.common.util.Option;
import org.apache.hudi.config.HoodieWriteConfig;
import org.apache.hudi.exception.HoodieInsertException;
import org.apache.hudi.io.storage.HoodieStorageWriter;
import org.apache.hudi.io.storage.HoodieStorageWriterFactory;
import org.apache.hudi.table.HoodieTable;
import org.apache.log4j.LogManager; import org.apache.log4j.LogManager;
import org.apache.log4j.Logger; import org.apache.log4j.Logger;
import org.apache.spark.TaskContext; import org.apache.spark.TaskContext;

View File

@@ -16,12 +16,12 @@
* limitations under the License. * limitations under the License.
*/ */
package com.uber.hoodie.io; package org.apache.hudi.io;
import com.uber.hoodie.common.model.HoodieRecordPayload;
import com.uber.hoodie.config.HoodieWriteConfig;
import com.uber.hoodie.table.HoodieTable;
import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.FileSystem;
import org.apache.hudi.common.model.HoodieRecordPayload;
import org.apache.hudi.config.HoodieWriteConfig;
import org.apache.hudi.table.HoodieTable;
public abstract class HoodieIOHandle<T extends HoodieRecordPayload> { public abstract class HoodieIOHandle<T extends HoodieRecordPayload> {

Some files were not shown because too many files have changed in this diff Show More