Import from Hoodie private repo: Part 1
This commit is contained in:
4
hoodie-cli/hoodie-cli.sh
Executable file
4
hoodie-cli/hoodie-cli.sh
Executable file
@@ -0,0 +1,4 @@
|
||||
#!/usr/bin/env bash
|
||||
DIR="$( cd "$( dirname "${BASH_SOURCE[0]}" )" && pwd )"
|
||||
HOODIE_JAR=`ls $DIR/target/hoodie-cli-*-SNAPSHOT.jar`
|
||||
java -cp /etc/hadoop/conf:/etc/spark/conf:$DIR/target/lib/*:$HOODIE_JAR org.springframework.shell.Bootstrap
|
||||
BIN
hoodie-cli/lib/dnl/utils/textutils/0.3.3/textutils-0.3.3.jar
Normal file
BIN
hoodie-cli/lib/dnl/utils/textutils/0.3.3/textutils-0.3.3.jar
Normal file
Binary file not shown.
208
hoodie-cli/pom.xml
Normal file
208
hoodie-cli/pom.xml
Normal file
@@ -0,0 +1,208 @@
|
||||
<?xml version="1.0" encoding="UTF-8"?>
|
||||
<!--
|
||||
~ Copyright (c) 2016 Uber Technologies, Inc. (hoodie-dev-group@uber.com)
|
||||
~
|
||||
~ Licensed under the Apache License, Version 2.0 (the "License");
|
||||
~ you may not use this file except in compliance with the License.
|
||||
~ You may obtain a copy of the License at
|
||||
~
|
||||
~ http://www.apache.org/licenses/LICENSE-2.0
|
||||
~
|
||||
~ Unless required by applicable law or agreed to in writing, software
|
||||
~ distributed under the License is distributed on an "AS IS" BASIS,
|
||||
~ WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
~ See the License for the specific language governing permissions and
|
||||
~ limitations under the License.
|
||||
-->
|
||||
|
||||
<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
|
||||
<parent>
|
||||
<artifactId>hoodie</artifactId>
|
||||
<groupId>com.uber.hoodie</groupId>
|
||||
<version>0.2.5-SNAPSHOT</version>
|
||||
</parent>
|
||||
<modelVersion>4.0.0</modelVersion>
|
||||
|
||||
<artifactId>hoodie-cli</artifactId>
|
||||
<packaging>jar</packaging>
|
||||
|
||||
<properties>
|
||||
<spring.shell.version>1.2.0.RELEASE</spring.shell.version>
|
||||
<jar.mainclass>org.springframework.shell.Bootstrap</jar.mainclass>
|
||||
<log4j.version>1.2.17</log4j.version>
|
||||
<junit.version>4.10</junit.version>
|
||||
</properties>
|
||||
|
||||
<repositories>
|
||||
<repository>
|
||||
<id>libs-milestone</id>
|
||||
<url>http://repo.spring.io/libs-milestone/</url>
|
||||
</repository>
|
||||
<repository>
|
||||
<id>libs-release</id>
|
||||
<url>http://repo.spring.io/libs-release/</url>
|
||||
</repository>
|
||||
<repository>
|
||||
<id>scala-tools.org</id>
|
||||
<name>Scala-tools Maven2 Repository</name>
|
||||
<url>http://scala-tools.org/repo-releases</url>
|
||||
</repository>
|
||||
</repositories>
|
||||
|
||||
<build>
|
||||
<pluginManagement>
|
||||
<plugins>
|
||||
<plugin>
|
||||
<groupId>net.alchim31.maven</groupId>
|
||||
<artifactId>scala-maven-plugin</artifactId>
|
||||
<version>3.2.1</version>
|
||||
</plugin>
|
||||
</plugins>
|
||||
</pluginManagement>
|
||||
|
||||
<plugins>
|
||||
<plugin>
|
||||
<groupId>org.apache.maven.plugins</groupId>
|
||||
<artifactId>maven-compiler-plugin</artifactId>
|
||||
<configuration>
|
||||
<source>1.5</source>
|
||||
<target>1.5</target>
|
||||
</configuration>
|
||||
</plugin>
|
||||
<plugin>
|
||||
<groupId>org.apache.maven.plugins</groupId>
|
||||
<artifactId>maven-dependency-plugin</artifactId>
|
||||
<executions>
|
||||
<execution>
|
||||
<id>copy-dependencies</id>
|
||||
<phase>prepare-package</phase>
|
||||
<goals>
|
||||
<goal>copy-dependencies</goal>
|
||||
</goals>
|
||||
<configuration>
|
||||
<outputDirectory>${project.build.directory}/lib</outputDirectory>
|
||||
<overWriteReleases>true</overWriteReleases>
|
||||
<overWriteSnapshots>true</overWriteSnapshots>
|
||||
<overWriteIfNewer>true</overWriteIfNewer>
|
||||
</configuration>
|
||||
</execution>
|
||||
</executions>
|
||||
</plugin>
|
||||
<plugin>
|
||||
<groupId>org.apache.maven.plugins</groupId>
|
||||
<artifactId>maven-jar-plugin</artifactId>
|
||||
<configuration>
|
||||
<archive>
|
||||
<manifest>
|
||||
<addClasspath>true</addClasspath>
|
||||
<useUniqueVersions>false</useUniqueVersions>
|
||||
<classpathPrefix>lib/</classpathPrefix>
|
||||
<mainClass>${jar.mainclass}</mainClass>
|
||||
</manifest>
|
||||
<manifestEntries>
|
||||
<version>${project.version}</version>
|
||||
</manifestEntries>
|
||||
</archive>
|
||||
</configuration>
|
||||
</plugin>
|
||||
<plugin>
|
||||
<groupId>net.alchim31.maven</groupId>
|
||||
<artifactId>scala-maven-plugin</artifactId>
|
||||
<executions>
|
||||
<execution>
|
||||
<id>scala-compile-first</id>
|
||||
<phase>process-resources</phase>
|
||||
<goals>
|
||||
<goal>add-source</goal>
|
||||
<goal>compile</goal>
|
||||
</goals>
|
||||
</execution>
|
||||
<execution>
|
||||
<id>scala-test-compile</id>
|
||||
<phase>process-test-resources</phase>
|
||||
<goals>
|
||||
<goal>testCompile</goal>
|
||||
</goals>
|
||||
</execution>
|
||||
</executions>
|
||||
</plugin>
|
||||
<plugin>
|
||||
<groupId>org.apache.rat</groupId>
|
||||
<artifactId>apache-rat-plugin</artifactId>
|
||||
</plugin>
|
||||
</plugins>
|
||||
</build>
|
||||
|
||||
|
||||
<dependencies>
|
||||
|
||||
<dependency>
|
||||
<groupId>org.scala-lang</groupId>
|
||||
<artifactId>scala-library</artifactId>
|
||||
<version>2.10.5</version>
|
||||
</dependency>
|
||||
|
||||
<dependency>
|
||||
<groupId>org.springframework.shell</groupId>
|
||||
<artifactId>spring-shell</artifactId>
|
||||
<version>${spring.shell.version}</version>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>de.vandermeer</groupId>
|
||||
<artifactId>asciitable</artifactId>
|
||||
<version>0.2.5</version>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.apache.spark</groupId>
|
||||
<artifactId>spark-core_2.10</artifactId>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.apache.spark</groupId>
|
||||
<artifactId>spark-sql_2.10</artifactId>
|
||||
<version>${spark.version}</version>
|
||||
<scope>provided</scope>
|
||||
</dependency>
|
||||
|
||||
|
||||
<dependency>
|
||||
<groupId>dnl.utils</groupId>
|
||||
<artifactId>textutils</artifactId>
|
||||
<version>0.3.3</version>
|
||||
<scope>system</scope>
|
||||
<systemPath>${basedir}/lib/dnl/utils/textutils/0.3.3/textutils-0.3.3.jar</systemPath>
|
||||
</dependency>
|
||||
|
||||
<dependency>
|
||||
<groupId>log4j</groupId>
|
||||
<artifactId>log4j</artifactId>
|
||||
<version>${log4j.version}</version>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>com.uber.hoodie</groupId>
|
||||
<artifactId>hoodie-client</artifactId>
|
||||
<version>${project.version}</version>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>com.uber.hoodie</groupId>
|
||||
<artifactId>hoodie-common</artifactId>
|
||||
<version>${project.version}</version>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>com.uber.hoodie</groupId>
|
||||
<artifactId>hoodie-mr</artifactId>
|
||||
<version>${project.version}</version>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>com.uber.hoodie</groupId>
|
||||
<artifactId>hoodie-tools</artifactId>
|
||||
<version>${project.version}</version>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>junit</groupId>
|
||||
<artifactId>junit-dep</artifactId>
|
||||
<version>${junit.version}</version>
|
||||
<scope>test</scope>
|
||||
</dependency>
|
||||
</dependencies>
|
||||
|
||||
</project>
|
||||
54
hoodie-cli/src/main/java/com/uber/hoodie/cli/HoodieCLI.java
Normal file
54
hoodie-cli/src/main/java/com/uber/hoodie/cli/HoodieCLI.java
Normal file
@@ -0,0 +1,54 @@
|
||||
/*
|
||||
* Copyright (c) 2016 Uber Technologies, Inc. (hoodie-dev-group@uber.com)
|
||||
*
|
||||
* Licensed under the Apache License, Version 2.0 (the "License");
|
||||
* you may not use this file except in compliance with the License.
|
||||
* You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package com.uber.hoodie.cli;
|
||||
|
||||
import com.uber.hoodie.common.model.HoodieTableMetadata;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
public class HoodieCLI {
|
||||
public static Configuration conf;
|
||||
public static FileSystem fs;
|
||||
public static CLIState state = CLIState.INIT;
|
||||
public static HoodieTableMetadata tableMetadata;
|
||||
public static HoodieTableMetadata syncTableMetadata;
|
||||
|
||||
|
||||
public enum CLIState {
|
||||
INIT, DATASET, SYNC
|
||||
}
|
||||
|
||||
public static boolean initConf() {
|
||||
if (HoodieCLI.conf == null) {
|
||||
HoodieCLI.conf = new Configuration();
|
||||
return true;
|
||||
}
|
||||
return false;
|
||||
}
|
||||
|
||||
public static void initFS(boolean force) throws IOException {
|
||||
if(fs == null || force) {
|
||||
fs = FileSystem.get(conf);
|
||||
}
|
||||
}
|
||||
|
||||
public static void setTableMetadata(HoodieTableMetadata tableMetadata) {
|
||||
HoodieCLI.tableMetadata = tableMetadata;
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,37 @@
|
||||
/*
|
||||
* Copyright (c) 2016 Uber Technologies, Inc. (hoodie-dev-group@uber.com)
|
||||
*
|
||||
* Licensed under the Apache License, Version 2.0 (the "License");
|
||||
* you may not use this file except in compliance with the License.
|
||||
* You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package com.uber.hoodie.cli;
|
||||
|
||||
import org.springframework.core.Ordered;
|
||||
import org.springframework.core.annotation.Order;
|
||||
import org.springframework.shell.plugin.support.DefaultHistoryFileNameProvider;
|
||||
import org.springframework.stereotype.Component;
|
||||
|
||||
@Component
|
||||
@Order(Ordered.HIGHEST_PRECEDENCE)
|
||||
public class HoodieHistoryFileNameProvider extends DefaultHistoryFileNameProvider {
|
||||
|
||||
public String getHistoryFileName() {
|
||||
return "hoodie-cmd.log";
|
||||
}
|
||||
|
||||
@Override
|
||||
public String getProviderName() {
|
||||
return "Hoodie file name provider";
|
||||
}
|
||||
|
||||
}
|
||||
@@ -0,0 +1,34 @@
|
||||
/*
|
||||
* Copyright (c) 2016 Uber Technologies, Inc. (hoodie-dev-group@uber.com)
|
||||
*
|
||||
* Licensed under the Apache License, Version 2.0 (the "License");
|
||||
* you may not use this file except in compliance with the License.
|
||||
* You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package com.uber.hoodie.cli;
|
||||
|
||||
import dnl.utils.text.table.TextTable;
|
||||
|
||||
import java.io.ByteArrayOutputStream;
|
||||
import java.io.PrintStream;
|
||||
import java.nio.charset.Charset;
|
||||
|
||||
public class HoodiePrintHelper {
|
||||
|
||||
public static String print(String[] header, String[][] rows) {
|
||||
TextTable textTable = new TextTable(header, rows);
|
||||
ByteArrayOutputStream baos = new ByteArrayOutputStream();
|
||||
PrintStream ps = new PrintStream(baos);
|
||||
textTable.printTable(ps, 4);
|
||||
return new String(baos.toByteArray(), Charset.forName("utf-8"));
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,49 @@
|
||||
/*
|
||||
* Copyright (c) 2016 Uber Technologies, Inc. (hoodie-dev-group@uber.com)
|
||||
*
|
||||
* Licensed under the Apache License, Version 2.0 (the "License");
|
||||
* you may not use this file except in compliance with the License.
|
||||
* You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package com.uber.hoodie.cli;
|
||||
|
||||
import org.springframework.core.Ordered;
|
||||
import org.springframework.core.annotation.Order;
|
||||
import org.springframework.shell.plugin.support.DefaultPromptProvider;
|
||||
import org.springframework.stereotype.Component;
|
||||
|
||||
@Component
|
||||
@Order(Ordered.HIGHEST_PRECEDENCE)
|
||||
public class HoodiePrompt extends DefaultPromptProvider {
|
||||
|
||||
@Override
|
||||
public String getPrompt() {
|
||||
switch (HoodieCLI.state) {
|
||||
case INIT:
|
||||
return "hoodie->";
|
||||
case DATASET:
|
||||
return "hoodie:" + HoodieCLI.tableMetadata.getTableName() + "->";
|
||||
case SYNC:
|
||||
return "hoodie:" + HoodieCLI.tableMetadata.getTableName() + " <==> "
|
||||
+ HoodieCLI.syncTableMetadata.getTableName() + "->";
|
||||
}
|
||||
if (HoodieCLI.tableMetadata != null)
|
||||
return "hoodie:" + HoodieCLI.tableMetadata.getTableName() + "->";
|
||||
return "hoodie->";
|
||||
}
|
||||
|
||||
@Override
|
||||
public String getProviderName() {
|
||||
return "Hoodie provider";
|
||||
}
|
||||
|
||||
}
|
||||
@@ -0,0 +1,55 @@
|
||||
/*
|
||||
* Copyright (c) 2016 Uber Technologies, Inc. (hoodie-dev-group@uber.com)
|
||||
*
|
||||
* Licensed under the Apache License, Version 2.0 (the "License");
|
||||
* you may not use this file except in compliance with the License.
|
||||
* You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package com.uber.hoodie.cli;
|
||||
|
||||
import org.springframework.core.Ordered;
|
||||
import org.springframework.core.annotation.Order;
|
||||
import org.springframework.shell.plugin.support.DefaultBannerProvider;
|
||||
import org.springframework.shell.support.util.OsUtils;
|
||||
import org.springframework.stereotype.Component;
|
||||
|
||||
@Component @Order(Ordered.HIGHEST_PRECEDENCE) public class HoodieSplashScreen
|
||||
extends DefaultBannerProvider {
|
||||
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;
|
||||
|
||||
public String getBanner() {
|
||||
return screen;
|
||||
}
|
||||
|
||||
public String getVersion() {
|
||||
return "1.0";
|
||||
}
|
||||
|
||||
public String getWelcomeMessage() {
|
||||
return "Welcome to Hoodie CLI. Please type help if you are looking for help. ";
|
||||
}
|
||||
|
||||
@Override public String getProviderName() {
|
||||
return "Hoodie Banner";
|
||||
}
|
||||
}
|
||||
33
hoodie-cli/src/main/java/com/uber/hoodie/cli/Main.java
Normal file
33
hoodie-cli/src/main/java/com/uber/hoodie/cli/Main.java
Normal file
@@ -0,0 +1,33 @@
|
||||
/*
|
||||
* Copyright (c) 2016 Uber Technologies, Inc. (hoodie-dev-group@uber.com)
|
||||
*
|
||||
* Licensed under the Apache License, Version 2.0 (the "License");
|
||||
* you may not use this file except in compliance with the License.
|
||||
* You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package com.uber.hoodie.cli;
|
||||
|
||||
import org.springframework.shell.Bootstrap;
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
public class Main {
|
||||
/**
|
||||
* Main class that delegates to Spring Shell's Bootstrap class in order to simplify debugging inside an IDE
|
||||
*
|
||||
* @param args
|
||||
* @throws IOException
|
||||
*/
|
||||
public static void main(String[] args) throws IOException {
|
||||
Bootstrap.main(args);
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,244 @@
|
||||
/*
|
||||
* Copyright (c) 2016 Uber Technologies, Inc. (hoodie-dev-group@uber.com)
|
||||
*
|
||||
* Licensed under the Apache License, Version 2.0 (the "License");
|
||||
* you may not use this file except in compliance with the License.
|
||||
* You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package com.uber.hoodie.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.HoodieCommitMetadata;
|
||||
import com.uber.hoodie.common.model.HoodieCommits;
|
||||
import com.uber.hoodie.common.model.HoodieTableMetadata;
|
||||
import com.uber.hoodie.common.model.HoodieWriteStat;
|
||||
import com.uber.hoodie.common.util.NumericUtils;
|
||||
|
||||
import org.apache.spark.launcher.SparkLauncher;
|
||||
import org.springframework.shell.core.CommandMarker;
|
||||
import org.springframework.shell.core.annotation.CliAvailabilityIndicator;
|
||||
import org.springframework.shell.core.annotation.CliCommand;
|
||||
import org.springframework.shell.core.annotation.CliOption;
|
||||
import org.springframework.stereotype.Component;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Collections;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.SortedMap;
|
||||
|
||||
@Component
|
||||
public class CommitsCommand implements CommandMarker {
|
||||
@CliAvailabilityIndicator({"commits show"})
|
||||
public boolean isShowAvailable() {
|
||||
return HoodieCLI.tableMetadata != null;
|
||||
}
|
||||
|
||||
@CliAvailabilityIndicator({"commits refresh"})
|
||||
public boolean isRefreshAvailable() {
|
||||
return HoodieCLI.tableMetadata != null;
|
||||
}
|
||||
|
||||
@CliAvailabilityIndicator({"commit rollback"})
|
||||
public boolean isRollbackAvailable() {
|
||||
return HoodieCLI.tableMetadata != null;
|
||||
}
|
||||
|
||||
@CliAvailabilityIndicator({"commit show"})
|
||||
public boolean isCommitShowAvailable() {
|
||||
return HoodieCLI.tableMetadata != null;
|
||||
}
|
||||
|
||||
@CliCommand(value = "commits show", help = "Show the commits")
|
||||
public String showCommits(
|
||||
@CliOption(key = {
|
||||
"limit"}, mandatory = false, help = "Limit commits", unspecifiedDefaultValue = "10")
|
||||
final Integer limit) throws IOException {
|
||||
SortedMap<String, HoodieCommitMetadata> map =
|
||||
HoodieCLI.tableMetadata.getAllCommitMetadata();
|
||||
int arraySize =
|
||||
Math.min(limit, HoodieCLI.tableMetadata.getAllCommits().getCommitList().size());
|
||||
String[][] rows = new String[arraySize][];
|
||||
ArrayList<String> commitList =
|
||||
new ArrayList<String>(HoodieCLI.tableMetadata.getAllCommits().getCommitList());
|
||||
Collections.reverse(commitList);
|
||||
for (int i = 0; i < arraySize; i++) {
|
||||
String commit = commitList.get(i);
|
||||
HoodieCommitMetadata commitMetadata = map.get(commit);
|
||||
rows[i] = new String[] {commit,
|
||||
NumericUtils.humanReadableByteCount(commitMetadata.fetchTotalBytesWritten()),
|
||||
String.valueOf(commitMetadata.fetchTotalFilesInsert()),
|
||||
String.valueOf(commitMetadata.fetchTotalFilesUpdated()),
|
||||
String.valueOf(commitMetadata.fetchTotalPartitionsWritten()),
|
||||
String.valueOf(commitMetadata.fetchTotalRecordsWritten()),
|
||||
String.valueOf(commitMetadata.fetchTotalUpdateRecordsWritten()),
|
||||
String.valueOf(commitMetadata.fetchTotalWriteErrors())};
|
||||
}
|
||||
return HoodiePrintHelper.print(
|
||||
new String[] {"CommitTime", "Total Written (B)", "Total Files Added",
|
||||
"Total Files Updated", "Total Partitions Written", "Total Records Written",
|
||||
"Total Update Records Written", "Total Errors"}, rows);
|
||||
}
|
||||
|
||||
@CliCommand(value = "commits refresh", help = "Refresh the commits")
|
||||
public String refreshCommits() throws IOException {
|
||||
HoodieTableMetadata metadata =
|
||||
new HoodieTableMetadata(HoodieCLI.fs, HoodieCLI.tableMetadata.getBasePath());
|
||||
HoodieCLI.setTableMetadata(metadata);
|
||||
return "Metadata for table " + metadata.getTableName() + " refreshed.";
|
||||
}
|
||||
|
||||
@CliCommand(value = "commit rollback", help = "Rollback a commit")
|
||||
public String rollbackCommit(
|
||||
@CliOption(key = {"commit"}, help = "Commit to rollback")
|
||||
final String commitTime,
|
||||
@CliOption(key = {"sparkProperties"}, help = "Spark Properites File Path")
|
||||
final String sparkPropertiesPath) throws Exception {
|
||||
if (!HoodieCLI.tableMetadata.getAllCommits().contains(commitTime)) {
|
||||
return "Commit " + commitTime + " not found in Commits " + HoodieCLI.tableMetadata
|
||||
.getAllCommits();
|
||||
}
|
||||
SparkLauncher sparkLauncher = SparkUtil.initLauncher(sparkPropertiesPath);
|
||||
sparkLauncher.addAppArgs(SparkMain.SparkCommand.ROLLBACK.toString(),
|
||||
commitTime,
|
||||
HoodieCLI.tableMetadata.getBasePath());
|
||||
Process process = sparkLauncher.launch();
|
||||
InputStreamConsumer.captureOutput(process);
|
||||
int exitCode = process.waitFor();
|
||||
// Refresh the current
|
||||
refreshCommits();
|
||||
if (exitCode != 0) {
|
||||
return "Commit " + commitTime + " failed to roll back";
|
||||
}
|
||||
return "Commit " + commitTime + " rolled back";
|
||||
}
|
||||
|
||||
@CliCommand(value = "commit showpartitions", help = "Show partition level details of a commit")
|
||||
public String showCommitPartitions(
|
||||
@CliOption(key = {"commit"}, help = "Commit to show")
|
||||
final String commitTime) throws Exception {
|
||||
if (!HoodieCLI.tableMetadata.getAllCommits().contains(commitTime)) {
|
||||
return "Commit " + commitTime + " not found in Commits " + HoodieCLI.tableMetadata
|
||||
.getAllCommits();
|
||||
}
|
||||
HoodieCommitMetadata meta = HoodieCLI.tableMetadata.getAllCommitMetadata().get(commitTime);
|
||||
List<String[]> rows = new ArrayList<String[]>();
|
||||
for (Map.Entry<String, List<HoodieWriteStat>> entry : meta.getPartitionToWriteStats()
|
||||
.entrySet()) {
|
||||
String path = entry.getKey();
|
||||
List<HoodieWriteStat> stats = entry.getValue();
|
||||
long totalFilesAdded = 0;
|
||||
long totalFilesUpdated = 0;
|
||||
long totalRecordsUpdated = 0;
|
||||
long totalRecordsInserted = 0;
|
||||
long totalBytesWritten = 0;
|
||||
long totalWriteErrors = 0;
|
||||
for (HoodieWriteStat stat : stats) {
|
||||
if (stat.getPrevCommit().equals(HoodieWriteStat.NULL_COMMIT)) {
|
||||
totalFilesAdded += 1;
|
||||
totalRecordsInserted += stat.getNumWrites();
|
||||
} else {
|
||||
totalFilesUpdated += 1;
|
||||
totalRecordsUpdated += stat.getNumUpdateWrites();
|
||||
}
|
||||
totalBytesWritten += stat.getTotalWriteBytes();
|
||||
totalWriteErrors += stat.getTotalWriteErrors();
|
||||
}
|
||||
rows.add(new String[] {path, String.valueOf(totalFilesAdded),
|
||||
String.valueOf(totalFilesUpdated), String.valueOf(totalRecordsInserted),
|
||||
String.valueOf(totalRecordsUpdated),
|
||||
NumericUtils.humanReadableByteCount(totalBytesWritten),
|
||||
String.valueOf(totalWriteErrors)});
|
||||
|
||||
}
|
||||
return HoodiePrintHelper.print(
|
||||
new String[] {"Partition Path", "Total Files Added", "Total Files Updated",
|
||||
"Total Records Inserted", "Total Records Updated", "Total Bytes Written",
|
||||
"Total Errors"}, rows.toArray(new String[rows.size()][]));
|
||||
}
|
||||
|
||||
@CliCommand(value = "commit showfiles", help = "Show file level details of a commit")
|
||||
public String showCommitFiles(
|
||||
@CliOption(key = {"commit"}, help = "Commit to show")
|
||||
final String commitTime) throws Exception {
|
||||
if (!HoodieCLI.tableMetadata.getAllCommits().contains(commitTime)) {
|
||||
return "Commit " + commitTime + " not found in Commits " + HoodieCLI.tableMetadata
|
||||
.getAllCommits();
|
||||
}
|
||||
HoodieCommitMetadata meta = HoodieCLI.tableMetadata.getAllCommitMetadata().get(commitTime);
|
||||
List<String[]> rows = new ArrayList<String[]>();
|
||||
for (Map.Entry<String, List<HoodieWriteStat>> entry : meta.getPartitionToWriteStats()
|
||||
.entrySet()) {
|
||||
String path = entry.getKey();
|
||||
List<HoodieWriteStat> stats = entry.getValue();
|
||||
for (HoodieWriteStat stat : stats) {
|
||||
rows.add(new String[] {path, stat.getFileId(), stat.getPrevCommit(),
|
||||
String.valueOf(stat.getNumUpdateWrites()), String.valueOf(stat.getNumWrites()),
|
||||
String.valueOf(stat.getTotalWriteBytes()),
|
||||
String.valueOf(stat.getTotalWriteErrors())});
|
||||
}
|
||||
}
|
||||
return HoodiePrintHelper.print(
|
||||
new String[] {"Partition Path", "File ID", "Previous Commit", "Total Records Updated",
|
||||
"Total Records Written", "Total Bytes Written", "Total Errors"},
|
||||
rows.toArray(new String[rows.size()][]));
|
||||
}
|
||||
|
||||
@CliAvailabilityIndicator({"commits compare"})
|
||||
public boolean isCompareCommitsAvailable() {
|
||||
return HoodieCLI.tableMetadata != null;
|
||||
}
|
||||
|
||||
@CliCommand(value = "commits compare", help = "Compare commits with another Hoodie dataset")
|
||||
public String compareCommits(
|
||||
@CliOption(key = {"path"}, help = "Path of the dataset to compare to")
|
||||
final String path) throws Exception {
|
||||
HoodieTableMetadata target = new HoodieTableMetadata(HoodieCLI.fs, path);
|
||||
HoodieTableMetadata source = HoodieCLI.tableMetadata;
|
||||
String targetLatestCommit =
|
||||
target.isCommitsEmpty() ? "0" : target.getAllCommits().lastCommit();
|
||||
String sourceLatestCommit =
|
||||
source.isCommitsEmpty() ? "0" : source.getAllCommits().lastCommit();
|
||||
|
||||
if (sourceLatestCommit != null && HoodieCommits
|
||||
.isCommit1After(targetLatestCommit, sourceLatestCommit)) {
|
||||
// source is behind the target
|
||||
List<String> commitsToCatchup = target.findCommitsSinceTs(sourceLatestCommit);
|
||||
return "Source " + source.getTableName() + " is behind by " + commitsToCatchup.size()
|
||||
+ " commits. Commits to catch up - " + commitsToCatchup;
|
||||
} else {
|
||||
List<String> commitsToCatchup = source.findCommitsSinceTs(targetLatestCommit);
|
||||
return "Source " + source.getTableName() + " is ahead by " + commitsToCatchup.size()
|
||||
+ " commits. Commits to catch up - " + commitsToCatchup;
|
||||
}
|
||||
}
|
||||
|
||||
@CliAvailabilityIndicator({"commits sync"})
|
||||
public boolean isSyncCommitsAvailable() {
|
||||
return HoodieCLI.tableMetadata != null;
|
||||
}
|
||||
|
||||
@CliCommand(value = "commits sync", help = "Compare commits with another Hoodie dataset")
|
||||
public String syncCommits(
|
||||
@CliOption(key = {"path"}, help = "Path of the dataset to compare to")
|
||||
final String path) throws Exception {
|
||||
HoodieCLI.syncTableMetadata = new HoodieTableMetadata(HoodieCLI.fs, path);
|
||||
HoodieCLI.state = HoodieCLI.CLIState.SYNC;
|
||||
return "Load sync state between " + HoodieCLI.tableMetadata.getTableName() + " and "
|
||||
+ HoodieCLI.syncTableMetadata.getTableName();
|
||||
}
|
||||
|
||||
}
|
||||
@@ -0,0 +1,42 @@
|
||||
/*
|
||||
* Copyright (c) 2016 Uber Technologies, Inc. (hoodie-dev-group@uber.com)
|
||||
*
|
||||
* Licensed under the Apache License, Version 2.0 (the "License");
|
||||
* you may not use this file except in compliance with the License.
|
||||
* You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package com.uber.hoodie.cli.commands;
|
||||
|
||||
import com.uber.hoodie.cli.HoodieCLI;
|
||||
import com.uber.hoodie.common.model.HoodieTableMetadata;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.springframework.shell.core.CommandMarker;
|
||||
import org.springframework.shell.core.annotation.CliCommand;
|
||||
import org.springframework.shell.core.annotation.CliOption;
|
||||
import org.springframework.stereotype.Component;
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
@Component
|
||||
public class DatasetsCommand implements CommandMarker {
|
||||
@CliCommand(value = "connect", help = "Connect to a hoodie dataset")
|
||||
public String connect(
|
||||
@CliOption(key = {"path"}, mandatory = true, help = "Base Path of the dataset")
|
||||
final String path) throws IOException {
|
||||
boolean initialized = HoodieCLI.initConf();
|
||||
HoodieCLI.initFS(initialized);
|
||||
HoodieCLI.setTableMetadata(new HoodieTableMetadata(HoodieCLI.fs, path));
|
||||
HoodieCLI.state = HoodieCLI.CLIState.DATASET;
|
||||
return "Metadata for table " + HoodieCLI.tableMetadata.getTableName() + " loaded";
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,106 @@
|
||||
/*
|
||||
* Copyright (c) 2016 Uber Technologies, Inc. (hoodie-dev-group@uber.com)
|
||||
*
|
||||
* Licensed under the Apache License, Version 2.0 (the "License");
|
||||
* you may not use this file except in compliance with the License.
|
||||
* You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package com.uber.hoodie.cli.commands;
|
||||
|
||||
import com.uber.hoodie.cli.utils.CommitUtil;
|
||||
import com.uber.hoodie.cli.utils.HiveUtil;
|
||||
import com.uber.hoodie.cli.HoodieCLI;
|
||||
import com.uber.hoodie.common.model.HoodieCommits;
|
||||
import com.uber.hoodie.common.model.HoodieTableMetadata;
|
||||
import org.springframework.shell.core.CommandMarker;
|
||||
import org.springframework.shell.core.annotation.CliAvailabilityIndicator;
|
||||
import org.springframework.shell.core.annotation.CliCommand;
|
||||
import org.springframework.shell.core.annotation.CliOption;
|
||||
import org.springframework.stereotype.Component;
|
||||
|
||||
import java.util.List;
|
||||
|
||||
@Component
|
||||
public class HoodieSyncCommand implements CommandMarker {
|
||||
@CliAvailabilityIndicator({"sync validate"})
|
||||
public boolean isSyncVerificationAvailable() {
|
||||
return HoodieCLI.tableMetadata != null && HoodieCLI.syncTableMetadata != null;
|
||||
}
|
||||
|
||||
@CliCommand(value = "sync validate", help = "Validate the sync by counting the number of records")
|
||||
public String validateSync(
|
||||
@CliOption(key = {"mode"}, unspecifiedDefaultValue = "complete", help = "Check mode")
|
||||
final String mode,
|
||||
@CliOption(key = {
|
||||
"sourceDb"}, unspecifiedDefaultValue = "rawdata", help = "source database")
|
||||
final String srcDb,
|
||||
@CliOption(key = {
|
||||
"targetDb"}, unspecifiedDefaultValue = "dwh_hoodie", help = "target database")
|
||||
final String tgtDb,
|
||||
@CliOption(key = {
|
||||
"partitionCount"}, unspecifiedDefaultValue = "5", help = "total number of recent partitions to validate")
|
||||
final int partitionCount,
|
||||
@CliOption(key = {
|
||||
"hiveServerUrl"}, mandatory = true, help = "hiveServerURL to connect to")
|
||||
final String hiveServerUrl,
|
||||
@CliOption(key = {
|
||||
"hiveUser"}, mandatory = false, unspecifiedDefaultValue = "", help = "hive username to connect to")
|
||||
final String hiveUser,
|
||||
@CliOption(key = {
|
||||
"hivePass"}, mandatory = true, unspecifiedDefaultValue = "", help = "hive password to connect to")
|
||||
final String hivePass) throws Exception {
|
||||
HoodieTableMetadata target = HoodieCLI.syncTableMetadata;
|
||||
HoodieTableMetadata source = HoodieCLI.tableMetadata;
|
||||
long sourceCount = 0;
|
||||
long targetCount = 0;
|
||||
if ("complete".equals(mode)) {
|
||||
sourceCount = HiveUtil.countRecords(hiveServerUrl, source, srcDb, hiveUser, hivePass);
|
||||
targetCount = HiveUtil.countRecords(hiveServerUrl, target, tgtDb, hiveUser, hivePass);
|
||||
} else if ("latestPartitions".equals(mode)) {
|
||||
sourceCount = HiveUtil.countRecords(hiveServerUrl, source, srcDb, partitionCount, hiveUser, hivePass);
|
||||
targetCount = HiveUtil.countRecords(hiveServerUrl, target, tgtDb, partitionCount, hiveUser, hivePass);
|
||||
}
|
||||
|
||||
String targetLatestCommit =
|
||||
target.isCommitsEmpty() ? "0" : target.getAllCommits().lastCommit();
|
||||
String sourceLatestCommit =
|
||||
source.isCommitsEmpty() ? "0" : source.getAllCommits().lastCommit();
|
||||
|
||||
if (sourceLatestCommit != null && HoodieCommits
|
||||
.isCommit1After(targetLatestCommit, sourceLatestCommit)) {
|
||||
// source is behind the target
|
||||
List<String> commitsToCatchup = target.findCommitsSinceTs(sourceLatestCommit);
|
||||
if (commitsToCatchup.isEmpty()) {
|
||||
return "Count difference now is (count(" + target.getTableName() + ") - count("
|
||||
+ source.getTableName() + ") == " + (targetCount - sourceCount);
|
||||
} else {
|
||||
long newInserts = CommitUtil.countNewRecords(target, commitsToCatchup);
|
||||
return "Count difference now is (count(" + target.getTableName() + ") - count("
|
||||
+ source.getTableName() + ") == " + (targetCount - sourceCount)
|
||||
+ ". Catch up count is " + newInserts;
|
||||
}
|
||||
} else {
|
||||
List<String> commitsToCatchup = source.findCommitsSinceTs(targetLatestCommit);
|
||||
if (commitsToCatchup.isEmpty()) {
|
||||
return "Count difference now is (count(" + source.getTableName() + ") - count("
|
||||
+ target.getTableName() + ") == " + (sourceCount - targetCount);
|
||||
} else {
|
||||
long newInserts = CommitUtil.countNewRecords(source, commitsToCatchup);
|
||||
return "Count difference now is (count(" + source.getTableName() + ") - count("
|
||||
+ target.getTableName() + ") == " + (sourceCount - targetCount)
|
||||
+ ". Catch up count is " + newInserts;
|
||||
}
|
||||
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
@@ -0,0 +1,78 @@
|
||||
/*
|
||||
* Copyright (c) 2016 Uber Technologies, Inc. (hoodie-dev-group@uber.com)
|
||||
*
|
||||
* Licensed under the Apache License, Version 2.0 (the "License");
|
||||
* you may not use this file except in compliance with the License.
|
||||
* You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package com.uber.hoodie.cli.commands;
|
||||
|
||||
import com.uber.hoodie.cli.HoodieCLI;
|
||||
import com.uber.hoodie.cli.utils.InputStreamConsumer;
|
||||
import com.uber.hoodie.cli.utils.SparkUtil;
|
||||
import org.apache.spark.launcher.SparkLauncher;
|
||||
import org.springframework.shell.core.CommandMarker;
|
||||
import org.springframework.shell.core.annotation.CliAvailabilityIndicator;
|
||||
import org.springframework.shell.core.annotation.CliCommand;
|
||||
import org.springframework.shell.core.annotation.CliOption;
|
||||
import org.springframework.stereotype.Component;
|
||||
|
||||
@Component
|
||||
public class RecordsCommand implements CommandMarker {
|
||||
|
||||
@CliAvailabilityIndicator({"records deduplicate"})
|
||||
public boolean isRecordsDeduplicateAvailable() {
|
||||
return HoodieCLI.tableMetadata != null;
|
||||
}
|
||||
|
||||
@CliCommand(value = "records deduplicate", help = "De-duplicate a partition path contains duplicates & produce repaired files to replace with")
|
||||
public String deduplicate(
|
||||
@CliOption(key = {
|
||||
"duplicatedPartitionPath"}, help = "Partition Path containing the duplicates")
|
||||
final String duplicatedPartitionPath,
|
||||
@CliOption(key = {"repairedOutputPath"}, help = "Location to place the repaired files")
|
||||
final String repairedOutputPath,
|
||||
@CliOption(key = {"sparkProperties"}, help = "Spark Properites File Path")
|
||||
final String sparkPropertiesPath) throws Exception {
|
||||
SparkLauncher sparkLauncher = SparkUtil.initLauncher(sparkPropertiesPath);
|
||||
sparkLauncher
|
||||
.addAppArgs(SparkMain.SparkCommand.DEDUPLICATE.toString(), duplicatedPartitionPath,
|
||||
repairedOutputPath, HoodieCLI.tableMetadata.getBasePath());
|
||||
Process process = sparkLauncher.launch();
|
||||
InputStreamConsumer.captureOutput(process);
|
||||
int exitCode = process.waitFor();
|
||||
|
||||
if (exitCode != 0) {
|
||||
return "Deduplicated files placed in: " + repairedOutputPath;
|
||||
}
|
||||
return "Deduplication failed ";
|
||||
}
|
||||
|
||||
// @CliCommand(value = "records find", help = "Find Records in a hoodie dataset")
|
||||
// public String findRecords(
|
||||
// @CliOption(key = {"keys"}, help = "Keys To Find (Comma seperated)")
|
||||
// final String hoodieKeys,
|
||||
// @CliOption(key = {"sparkProperties"}, help = "Spark Properites File Path")
|
||||
// final String sparkPropertiesPath) throws Exception {
|
||||
// SparkLauncher sparkLauncher = SparkUtil.initLauncher(sparkPropertiesPath);
|
||||
// sparkLauncher
|
||||
// .addAppArgs(SparkMain.RECORD_FIND, hoodieKeys, HoodieCLI.tableMetadata.getBasePath());
|
||||
// Process process = sparkLauncher.launch();
|
||||
// InputStreamConsumer.captureOutput(process);
|
||||
// int exitCode = process.waitFor();
|
||||
//
|
||||
// if (exitCode != 0) {
|
||||
// return "Deduplicated files placed in: " + repairedOutputPath;
|
||||
// }
|
||||
// return "Deduplication failed ";
|
||||
// }
|
||||
}
|
||||
@@ -0,0 +1,94 @@
|
||||
/*
|
||||
* Copyright (c) 2016 Uber Technologies, Inc. (hoodie-dev-group@uber.com)
|
||||
*
|
||||
* Licensed under the Apache License, Version 2.0 (the "License");
|
||||
* you may not use this file except in compliance with the License.
|
||||
* You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package com.uber.hoodie.cli.commands;
|
||||
|
||||
import com.uber.hoodie.HoodieWriteClient;
|
||||
import com.uber.hoodie.cli.DedupeSparkJob;
|
||||
import com.uber.hoodie.cli.utils.SparkUtil;
|
||||
import com.uber.hoodie.common.util.FSUtils;
|
||||
import com.uber.hoodie.config.HoodieIndexConfig;
|
||||
import com.uber.hoodie.config.HoodieWriteConfig;
|
||||
import com.uber.hoodie.index.HoodieIndex;
|
||||
|
||||
import org.apache.log4j.Logger;
|
||||
import org.apache.spark.api.java.JavaSparkContext;
|
||||
import org.apache.spark.sql.SQLContext;
|
||||
|
||||
public class SparkMain {
|
||||
|
||||
protected final static Logger LOG = Logger.getLogger(SparkMain.class);
|
||||
|
||||
|
||||
/**
|
||||
* Commands
|
||||
*/
|
||||
enum SparkCommand {
|
||||
ROLLBACK,
|
||||
DEDUPLICATE
|
||||
}
|
||||
|
||||
public static void main(String[] args) throws Exception {
|
||||
String command = args[0];
|
||||
LOG.info("Invoking SparkMain:" + command);
|
||||
|
||||
SparkCommand cmd = SparkCommand.valueOf(command);
|
||||
|
||||
JavaSparkContext jsc = SparkUtil.initJavaSparkConf("hoodie-cli-" + command);
|
||||
int returnCode = 0;
|
||||
if (SparkCommand.ROLLBACK.equals(cmd)) {
|
||||
assert (args.length == 3);
|
||||
returnCode = rollback(jsc, args[1], args[2]);
|
||||
} else if(SparkCommand.DEDUPLICATE.equals(cmd)) {
|
||||
assert (args.length == 4);
|
||||
returnCode = deduplicatePartitionPath(jsc, args[1], args[2], args[3]);
|
||||
}
|
||||
|
||||
System.exit(returnCode);
|
||||
}
|
||||
|
||||
private static int deduplicatePartitionPath(JavaSparkContext jsc,
|
||||
String duplicatedPartitionPath,
|
||||
String repairedOutputPath,
|
||||
String basePath)
|
||||
throws Exception {
|
||||
DedupeSparkJob job = new DedupeSparkJob(basePath,
|
||||
duplicatedPartitionPath,repairedOutputPath,new SQLContext(jsc), FSUtils.getFs());
|
||||
job.fixDuplicates(true);
|
||||
return 0;
|
||||
}
|
||||
|
||||
private static int rollback(JavaSparkContext jsc, String commitTime, String basePath)
|
||||
throws Exception {
|
||||
HoodieWriteClient client = createHoodieClient(jsc, basePath);
|
||||
if (client.rollback(commitTime)) {
|
||||
LOG.info(String.format("The commit \"%s\" rolled back.", commitTime));
|
||||
return -1;
|
||||
} else {
|
||||
LOG.info(String.format("The commit \"%s\" failed to roll back.", commitTime));
|
||||
}
|
||||
return 0;
|
||||
}
|
||||
|
||||
private static HoodieWriteClient createHoodieClient(JavaSparkContext jsc, String basePath)
|
||||
throws Exception {
|
||||
HoodieWriteConfig config = HoodieWriteConfig.newBuilder().withPath(basePath)
|
||||
.withIndexConfig(
|
||||
HoodieIndexConfig.newBuilder().withIndexType(HoodieIndex.IndexType.BLOOM).build())
|
||||
.build();
|
||||
return new HoodieWriteClient(jsc, config);
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,136 @@
|
||||
/*
|
||||
* Copyright (c) 2016 Uber Technologies, Inc. (hoodie-dev-group@uber.com)
|
||||
*
|
||||
* Licensed under the Apache License, Version 2.0 (the "License");
|
||||
* you may not use this file except in compliance with the License.
|
||||
* You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package com.uber.hoodie.cli.commands;
|
||||
|
||||
|
||||
import com.codahale.metrics.Histogram;
|
||||
import com.codahale.metrics.Snapshot;
|
||||
import com.codahale.metrics.UniformReservoir;
|
||||
import com.uber.hoodie.cli.HoodieCLI;
|
||||
import com.uber.hoodie.cli.HoodiePrintHelper;
|
||||
import com.uber.hoodie.common.model.HoodieCommitMetadata;
|
||||
import com.uber.hoodie.common.util.FSUtils;
|
||||
import com.uber.hoodie.common.util.NumericUtils;
|
||||
|
||||
import org.apache.hadoop.fs.FileStatus;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.springframework.shell.core.CommandMarker;
|
||||
import org.springframework.shell.core.annotation.CliAvailabilityIndicator;
|
||||
import org.springframework.shell.core.annotation.CliCommand;
|
||||
import org.springframework.shell.core.annotation.CliOption;
|
||||
import org.springframework.stereotype.Component;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.text.DecimalFormat;
|
||||
import java.util.HashMap;
|
||||
import java.util.Map;
|
||||
|
||||
@Component
|
||||
public class StatsCommand implements CommandMarker {
|
||||
@CliAvailabilityIndicator({"stats wa"})
|
||||
public boolean isWriteAmpAvailable() {
|
||||
return HoodieCLI.tableMetadata != null;
|
||||
}
|
||||
|
||||
@CliCommand(value = "stats wa", help = "Write Amplification. Ratio of how many records were upserted to how many records were actually written")
|
||||
public String writeAmplificationStats() throws IOException {
|
||||
long totalRecordsUpserted = 0;
|
||||
long totalRecordsWritten = 0;
|
||||
|
||||
String[][] rows = new String[HoodieCLI.tableMetadata.getAllCommitMetadata().size() + 1][];
|
||||
int i = 0;
|
||||
DecimalFormat df = new DecimalFormat("#.00");
|
||||
for (Map.Entry<String, HoodieCommitMetadata> commit : HoodieCLI.tableMetadata
|
||||
.getAllCommitMetadata().entrySet()) {
|
||||
String waf = "0";
|
||||
if (commit.getValue().fetchTotalUpdateRecordsWritten() > 0) {
|
||||
waf = df.format(
|
||||
(float) commit.getValue().fetchTotalRecordsWritten() / commit.getValue()
|
||||
.fetchTotalUpdateRecordsWritten());
|
||||
}
|
||||
rows[i++] = new String[] {commit.getKey(),
|
||||
String.valueOf(commit.getValue().fetchTotalUpdateRecordsWritten()),
|
||||
String.valueOf(commit.getValue().fetchTotalRecordsWritten()), waf};
|
||||
totalRecordsUpserted += commit.getValue().fetchTotalUpdateRecordsWritten();
|
||||
totalRecordsWritten += commit.getValue().fetchTotalRecordsWritten();
|
||||
}
|
||||
String waf = "0";
|
||||
if (totalRecordsUpserted > 0) {
|
||||
waf = df.format((float) totalRecordsWritten / totalRecordsUpserted);
|
||||
}
|
||||
rows[i] = new String[] {"Total", String.valueOf(totalRecordsUpserted),
|
||||
String.valueOf(totalRecordsWritten), waf};
|
||||
return HoodiePrintHelper.print(
|
||||
new String[] {"CommitTime", "Total Upserted", "Total Written",
|
||||
"Write Amplifiation Factor"}, rows);
|
||||
|
||||
}
|
||||
|
||||
|
||||
private String[] printFileSizeHistogram(String commitTime, Snapshot s) {
|
||||
return new String[]{
|
||||
commitTime,
|
||||
NumericUtils.humanReadableByteCount(s.getMin()),
|
||||
NumericUtils.humanReadableByteCount(s.getValue(0.1)),
|
||||
NumericUtils.humanReadableByteCount(s.getMedian()),
|
||||
NumericUtils.humanReadableByteCount(s.getMean()),
|
||||
NumericUtils.humanReadableByteCount(s.get95thPercentile()),
|
||||
NumericUtils.humanReadableByteCount(s.getMax()),
|
||||
String.valueOf(s.size()),
|
||||
NumericUtils.humanReadableByteCount(s.getStdDev())
|
||||
};
|
||||
}
|
||||
|
||||
@CliCommand(value = "stats filesizes", help = "File Sizes. Display summary stats on sizes of files")
|
||||
public String fileSizeStats(
|
||||
@CliOption(key = {"partitionPath"}, help = "regex to select files, eg: 2016/08/02", unspecifiedDefaultValue = "*/*/*")
|
||||
final String globRegex) throws IOException {
|
||||
|
||||
FileSystem fs = HoodieCLI.fs;
|
||||
String globPath = String.format("%s/%s/*",
|
||||
HoodieCLI.tableMetadata.getBasePath(),
|
||||
globRegex);
|
||||
FileStatus[] statuses = fs.globStatus(new Path(globPath));
|
||||
|
||||
// max, min, #small files < 10MB, 50th, avg, 95th
|
||||
final int MAX_FILES = 1000000;
|
||||
Histogram globalHistogram = new Histogram(new UniformReservoir(MAX_FILES));
|
||||
HashMap<String, Histogram> commitHistoMap = new HashMap<String, Histogram>();
|
||||
for (FileStatus fileStatus: statuses) {
|
||||
String commitTime = FSUtils.getCommitTime(fileStatus.getPath().getName());
|
||||
long sz = fileStatus.getLen();
|
||||
if (!commitHistoMap.containsKey(commitTime)) {
|
||||
commitHistoMap.put(commitTime, new Histogram(new UniformReservoir(MAX_FILES)));
|
||||
}
|
||||
commitHistoMap.get(commitTime).update(sz);
|
||||
globalHistogram.update(sz);
|
||||
}
|
||||
|
||||
String[][] rows = new String[commitHistoMap.size() + 1][];
|
||||
int ind = 0;
|
||||
for (String commitTime: commitHistoMap.keySet()) {
|
||||
Snapshot s = commitHistoMap.get(commitTime).getSnapshot();
|
||||
rows[ind++] = printFileSizeHistogram(commitTime, s);
|
||||
}
|
||||
Snapshot s = globalHistogram.getSnapshot();
|
||||
rows[ind++] = printFileSizeHistogram("ALL", s);
|
||||
|
||||
return HoodiePrintHelper.print(
|
||||
new String[] {"CommitTime", "Min", "10th", "50th", "avg", "95th", "Max", "NumFiles", "StdDev"}, rows);
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,34 @@
|
||||
/*
|
||||
* Copyright (c) 2016 Uber Technologies, Inc. (hoodie-dev-group@uber.com)
|
||||
*
|
||||
* Licensed under the Apache License, Version 2.0 (the "License");
|
||||
* you may not use this file except in compliance with the License.
|
||||
* You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package com.uber.hoodie.cli.commands;
|
||||
|
||||
import org.springframework.shell.core.CommandMarker;
|
||||
import org.springframework.shell.core.annotation.CliCommand;
|
||||
import org.springframework.shell.core.annotation.CliOption;
|
||||
import org.springframework.stereotype.Component;
|
||||
|
||||
@Component
|
||||
public class UtilsCommand implements CommandMarker {
|
||||
@CliCommand(value = "utils loadClass", help = "Load a class" )
|
||||
public String loadClass(
|
||||
@CliOption(key = {"class"}, help = "Check mode" ) final String clazz
|
||||
) throws Exception {
|
||||
Class klass = Class.forName(clazz);
|
||||
return klass.getProtectionDomain().getCodeSource().getLocation().toExternalForm();
|
||||
}
|
||||
|
||||
}
|
||||
@@ -0,0 +1,38 @@
|
||||
/*
|
||||
* Copyright (c) 2016 Uber Technologies, Inc. (hoodie-dev-group@uber.com)
|
||||
*
|
||||
* Licensed under the Apache License, Version 2.0 (the "License");
|
||||
* you may not use this file except in compliance with the License.
|
||||
* You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package com.uber.hoodie.cli.utils;
|
||||
|
||||
import com.uber.hoodie.common.model.HoodieCommitMetadata;
|
||||
import com.uber.hoodie.common.model.HoodieTableMetadata;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.List;
|
||||
import java.util.SortedMap;
|
||||
import java.util.TreeMap;
|
||||
|
||||
public class CommitUtil {
|
||||
public static long countNewRecords(HoodieTableMetadata target, List<String> commitsToCatchup)
|
||||
throws IOException {
|
||||
long totalNew = 0;
|
||||
SortedMap<String, HoodieCommitMetadata> meta = target.getAllCommitMetadata();
|
||||
for(String commit:commitsToCatchup) {
|
||||
HoodieCommitMetadata c = meta.get(commit);
|
||||
totalNew += c.fetchTotalRecordsWritten() - c.fetchTotalUpdateRecordsWritten();
|
||||
}
|
||||
return totalNew;
|
||||
}
|
||||
}
|
||||
125
hoodie-cli/src/main/java/com/uber/hoodie/cli/utils/HiveUtil.java
Normal file
125
hoodie-cli/src/main/java/com/uber/hoodie/cli/utils/HiveUtil.java
Normal file
@@ -0,0 +1,125 @@
|
||||
/*
|
||||
* Copyright (c) 2016 Uber Technologies, Inc. (hoodie-dev-group@uber.com)
|
||||
*
|
||||
* Licensed under the Apache License, Version 2.0 (the "License");
|
||||
* you may not use this file except in compliance with the License.
|
||||
* You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package com.uber.hoodie.cli.utils;
|
||||
|
||||
import com.uber.hoodie.common.model.HoodieTableMetadata;
|
||||
import com.uber.hoodie.hadoop.HoodieInputFormat;
|
||||
import org.apache.commons.dbcp.BasicDataSource;
|
||||
import org.joda.time.DateTime;
|
||||
|
||||
import javax.sql.DataSource;
|
||||
import java.sql.Connection;
|
||||
import java.sql.ResultSet;
|
||||
import java.sql.SQLException;
|
||||
import java.sql.Statement;
|
||||
|
||||
public class HiveUtil {
|
||||
private static String driverName = "org.apache.hive.jdbc.HiveDriver";
|
||||
|
||||
static {
|
||||
try {
|
||||
Class.forName(driverName);
|
||||
} catch (ClassNotFoundException e) {
|
||||
throw new IllegalStateException("Could not find " + driverName + " in classpath. ", e);
|
||||
}
|
||||
}
|
||||
|
||||
private static Connection connection;
|
||||
|
||||
private static Connection getConnection(String jdbcUrl, String user, String pass) throws SQLException {
|
||||
DataSource ds = getDatasource(jdbcUrl, user, pass);
|
||||
return ds.getConnection();
|
||||
}
|
||||
|
||||
private static DataSource getDatasource(String jdbcUrl, String user, String pass) {
|
||||
BasicDataSource ds = new BasicDataSource();
|
||||
ds.setDriverClassName(driverName);
|
||||
ds.setUrl(jdbcUrl);
|
||||
ds.setUsername(user);
|
||||
ds.setPassword(pass);
|
||||
return ds;
|
||||
}
|
||||
|
||||
public static long countRecords(String jdbcUrl, HoodieTableMetadata source, String dbName, String user, String pass) throws SQLException {
|
||||
Connection conn = HiveUtil.getConnection(jdbcUrl, user, pass);
|
||||
ResultSet rs = null;
|
||||
Statement stmt = conn.createStatement();
|
||||
try {
|
||||
//stmt.execute("set mapred.job.queue.name=<queue_name>");
|
||||
stmt.execute("set hive.input.format=org.apache.hadoop.hive.ql.io.HiveInputFormat" );
|
||||
stmt.execute("set hive.stats.autogather=false" );
|
||||
System.out.println("Class " + HoodieInputFormat.class.getName());
|
||||
rs = stmt.executeQuery(
|
||||
"select count(`_hoodie_commit_time`) as cnt from " + dbName + "." + source
|
||||
.getTableName());
|
||||
long count = -1;
|
||||
if(rs.next()) {
|
||||
count = rs.getLong("cnt");
|
||||
}
|
||||
System.out.println("Total records in " + source.getTableName() + " is " + count);
|
||||
return count;
|
||||
} finally {
|
||||
if (rs != null) {
|
||||
rs.close();
|
||||
}
|
||||
if (stmt != null) {
|
||||
stmt.close();
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
public static long countRecords(String jdbcUrl, HoodieTableMetadata source, String srcDb,
|
||||
int partitions, String user, String pass) throws SQLException {
|
||||
DateTime dateTime = DateTime.now();
|
||||
String endDateStr =
|
||||
dateTime.getYear() + "-" + String.format("%02d", dateTime.getMonthOfYear()) + "-" +
|
||||
String.format("%02d", dateTime.getDayOfMonth());
|
||||
dateTime = dateTime.minusDays(partitions);
|
||||
String startDateStr =
|
||||
dateTime.getYear() + "-" + String.format("%02d", dateTime.getMonthOfYear()) + "-" +
|
||||
String.format("%02d", dateTime.getDayOfMonth());
|
||||
System.out.println("Start date " + startDateStr + " and end date " + endDateStr);
|
||||
return countRecords(jdbcUrl, source, srcDb, startDateStr, endDateStr, user, pass);
|
||||
}
|
||||
|
||||
private static long countRecords(String jdbcUrl, HoodieTableMetadata source, String srcDb, String startDateStr,
|
||||
String endDateStr, String user, String pass) throws SQLException {
|
||||
Connection conn = HiveUtil.getConnection(jdbcUrl, user, pass);
|
||||
ResultSet rs = null;
|
||||
Statement stmt = conn.createStatement();
|
||||
try {
|
||||
//stmt.execute("set mapred.job.queue.name=<queue_name>");
|
||||
stmt.execute("set hive.input.format=org.apache.hadoop.hive.ql.io.HiveInputFormat");
|
||||
stmt.execute("set hive.stats.autogather=false");
|
||||
rs = stmt.executeQuery(
|
||||
"select count(`_hoodie_commit_time`) as cnt from " + srcDb + "." + source
|
||||
.getTableName() + " where datestr>'" + startDateStr + "' and datestr<='"
|
||||
+ endDateStr + "'");
|
||||
if(rs.next()) {
|
||||
return rs.getLong("cnt");
|
||||
}
|
||||
return -1;
|
||||
} finally {
|
||||
if (rs != null) {
|
||||
rs.close();
|
||||
}
|
||||
if (stmt != null) {
|
||||
stmt.close();
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,56 @@
|
||||
/*
|
||||
* Copyright (c) 2016 Uber Technologies, Inc. (hoodie-dev-group@uber.com)
|
||||
*
|
||||
* Licensed under the Apache License, Version 2.0 (the "License");
|
||||
* you may not use this file except in compliance with the License.
|
||||
* You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package com.uber.hoodie.cli.utils;
|
||||
|
||||
import java.io.BufferedReader;
|
||||
import java.io.IOException;
|
||||
import java.io.InputStream;
|
||||
import java.io.InputStreamReader;
|
||||
import java.util.logging.Logger;
|
||||
|
||||
public class InputStreamConsumer extends Thread {
|
||||
protected final static Logger LOG = Logger.getLogger(InputStreamConsumer.class.getName());
|
||||
private InputStream is;
|
||||
public InputStreamConsumer(InputStream is) {
|
||||
this.is = is;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void run() {
|
||||
try {
|
||||
InputStreamReader isr = new InputStreamReader(is);
|
||||
BufferedReader br = new BufferedReader(isr);
|
||||
String line;
|
||||
while ( (line = br.readLine()) != null)
|
||||
LOG.info(line);
|
||||
} catch (IOException ioe) {
|
||||
LOG.severe(ioe.toString());
|
||||
ioe.printStackTrace();
|
||||
}
|
||||
}
|
||||
|
||||
public static void captureOutput(Process p) {
|
||||
InputStreamConsumer stdout;
|
||||
InputStreamConsumer errout;
|
||||
errout = new InputStreamConsumer(p.getErrorStream());
|
||||
stdout = new InputStreamConsumer(p.getInputStream());
|
||||
errout.start();
|
||||
stdout.start();
|
||||
}
|
||||
|
||||
|
||||
}
|
||||
@@ -0,0 +1,75 @@
|
||||
/*
|
||||
* Copyright (c) 2016 Uber Technologies, Inc. (hoodie-dev-group@uber.com)
|
||||
*
|
||||
* Licensed under the Apache License, Version 2.0 (the "License");
|
||||
* you may not use this file except in compliance with the License.
|
||||
* You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package com.uber.hoodie.cli.utils;
|
||||
|
||||
import com.uber.hoodie.HoodieWriteClient;
|
||||
import com.uber.hoodie.cli.commands.SparkMain;
|
||||
|
||||
import org.apache.log4j.Logger;
|
||||
import org.apache.spark.SparkConf;
|
||||
import org.apache.spark.api.java.JavaSparkContext;
|
||||
import org.apache.spark.launcher.SparkLauncher;
|
||||
|
||||
import java.io.File;
|
||||
import java.net.URISyntaxException;
|
||||
|
||||
public class SparkUtil {
|
||||
|
||||
public static Logger logger = Logger.getLogger(SparkUtil.class);
|
||||
|
||||
/**
|
||||
*
|
||||
* TODO: Need to fix a bunch of hardcoded stuff here eg: history server, spark distro
|
||||
*
|
||||
* @return
|
||||
* @throws URISyntaxException
|
||||
*/
|
||||
public static SparkLauncher initLauncher(String propertiesFile) throws URISyntaxException {
|
||||
String currentJar = new File(
|
||||
SparkUtil.class.getProtectionDomain().getCodeSource().getLocation().toURI().getPath())
|
||||
.getAbsolutePath();
|
||||
SparkLauncher sparkLauncher =
|
||||
new SparkLauncher().setAppResource(currentJar)
|
||||
.setMainClass(SparkMain.class.getName())
|
||||
.setPropertiesFile(propertiesFile);
|
||||
File libDirectory = new File(new File(currentJar).getParent(), "lib");
|
||||
for (String library : libDirectory.list()) {
|
||||
sparkLauncher.addJar(new File(libDirectory, library).getAbsolutePath());
|
||||
}
|
||||
return sparkLauncher;
|
||||
}
|
||||
|
||||
public static JavaSparkContext initJavaSparkConf(String name) {
|
||||
SparkConf sparkConf = new SparkConf().setAppName(name);
|
||||
sparkConf.setMaster("yarn-client");
|
||||
sparkConf.set("spark.serializer", "org.apache.spark.serializer.KryoSerializer");
|
||||
sparkConf.set("spark.driver.maxResultSize", "2g");
|
||||
sparkConf.set("spark.eventLog.overwrite", "true");
|
||||
sparkConf.set("spark.eventLog.enabled", "true");
|
||||
|
||||
// Configure hadoop conf
|
||||
sparkConf.set("spark.hadoop.mapred.output.compress", "true");
|
||||
sparkConf.set("spark.hadoop.mapred.output.compression.codec", "true");
|
||||
sparkConf.set("spark.hadoop.mapred.output.compression.codec", "org.apache.hadoop.io.compress.GzipCodec");
|
||||
sparkConf.set("spark.hadoop.mapred.output.compression.type", "BLOCK");
|
||||
|
||||
sparkConf = HoodieWriteClient.registerClasses(sparkConf);
|
||||
JavaSparkContext jsc = new JavaSparkContext(sparkConf);
|
||||
jsc.hadoopConfiguration().setBoolean("parquet.enable.summary-metadata", false);
|
||||
return jsc;
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,26 @@
|
||||
<?xml version="1.0" encoding="UTF-8"?>
|
||||
<!--
|
||||
~ Copyright (c) 2016 Uber Technologies, Inc. (hoodie-dev-group@uber.com)
|
||||
~
|
||||
~ Licensed under the Apache License, Version 2.0 (the "License");
|
||||
~ you may not use this file except in compliance with the License.
|
||||
~ You may obtain a copy of the License at
|
||||
~
|
||||
~ http://www.apache.org/licenses/LICENSE-2.0
|
||||
~
|
||||
~ Unless required by applicable law or agreed to in writing, software
|
||||
~ distributed under the License is distributed on an "AS IS" BASIS,
|
||||
~ WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
~ See the License for the specific language governing permissions and
|
||||
~ limitations under the License.
|
||||
-->
|
||||
|
||||
<beans xmlns="http://www.springframework.org/schema/beans"
|
||||
xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
|
||||
xmlns:context="http://www.springframework.org/schema/context"
|
||||
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">
|
||||
|
||||
<context:component-scan base-package="com.uber.hoodie.cli" />
|
||||
|
||||
</beans>
|
||||
@@ -0,0 +1,179 @@
|
||||
/*
|
||||
* Copyright (c) 2016 Uber Technologies, Inc. (hoodie-dev-group@uber.com)
|
||||
*
|
||||
* Licensed under the Apache License, Version 2.0 (the "License");
|
||||
* you may not use this file except in compliance with the License.
|
||||
* You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package com.uber.hoodie.cli
|
||||
|
||||
import com.uber.hoodie.common.model.{HoodieRecord, HoodieTableMetadata}
|
||||
import com.uber.hoodie.common.util.FSUtils
|
||||
import com.uber.hoodie.exception.HoodieException
|
||||
import org.apache.hadoop.fs.{FileSystem, FileUtil, Path}
|
||||
import org.apache.log4j.Logger
|
||||
import org.apache.spark.sql.{DataFrame, SQLContext}
|
||||
|
||||
import scala.collection.JavaConversions._
|
||||
import scala.collection.mutable._
|
||||
|
||||
|
||||
/**
|
||||
* Spark job to de-duplicate data present in a partition path
|
||||
*/
|
||||
class DedupeSparkJob (basePath: String,
|
||||
duplicatedPartitionPath: String,
|
||||
repairOutputPath: String,
|
||||
sqlContext: SQLContext,
|
||||
fs: FileSystem) {
|
||||
|
||||
|
||||
val sparkHelper = new SparkHelper(sqlContext, fs)
|
||||
val LOG = Logger.getLogger(this.getClass)
|
||||
|
||||
|
||||
/**
|
||||
*
|
||||
* @param tblName
|
||||
* @return
|
||||
*/
|
||||
def getDupeKeyDF(tblName: String) : DataFrame = {
|
||||
val dupeSql = s"""
|
||||
select `${HoodieRecord.RECORD_KEY_METADATA_FIELD}` as dupe_key,
|
||||
count(*) as dupe_cnt
|
||||
from ${tblName}
|
||||
group by `${HoodieRecord.RECORD_KEY_METADATA_FIELD}`
|
||||
having dupe_cnt > 1
|
||||
"""
|
||||
return sqlContext.sql(dupeSql)
|
||||
}
|
||||
|
||||
|
||||
/**
|
||||
*
|
||||
* Check a given partition for duplicates and suggest the deletions that need to be done in each file,
|
||||
* in order to set things right.
|
||||
*
|
||||
* @return
|
||||
*/
|
||||
private def planDuplicateFix() : HashMap[String, HashSet[String]] = {
|
||||
|
||||
val tmpTableName = s"htbl_${System.currentTimeMillis()}"
|
||||
val dedupeTblName = s"${tmpTableName}_dupeKeys"
|
||||
|
||||
val metadata = new HoodieTableMetadata(fs, basePath)
|
||||
val allFiles = fs.listStatus(new org.apache.hadoop.fs.Path(s"${basePath}/${duplicatedPartitionPath}"))
|
||||
val filteredStatuses = metadata.getLatestVersions(allFiles).map(f => f.getPath.toString);
|
||||
LOG.info(s" List of files under partition: ${} => ${filteredStatuses.mkString(" ")}")
|
||||
|
||||
val df = sqlContext.parquetFile(filteredStatuses:_*)
|
||||
df.registerTempTable(tmpTableName)
|
||||
val dupeKeyDF = getDupeKeyDF(tmpTableName)
|
||||
dupeKeyDF.registerTempTable(dedupeTblName)
|
||||
|
||||
// Obtain necessary satellite information for duplicate rows
|
||||
val dupeDataSql = s"""
|
||||
SELECT `_hoodie_record_key`, `_hoodie_partition_path`, `_hoodie_file_name`, `_hoodie_commit_time`
|
||||
FROM ${tmpTableName} h
|
||||
JOIN ${dedupeTblName} d
|
||||
ON h.`_hoodie_record_key` = d.dupe_key
|
||||
"""
|
||||
val dupeMap = sqlContext.sql(dupeDataSql).collectAsList().groupBy(r => r.getString(0))
|
||||
val fileToDeleteKeyMap = new HashMap[String, HashSet[String]]()
|
||||
|
||||
// Mark all files except the one with latest commits for deletion
|
||||
dupeMap.foreach(rt => {
|
||||
val key = rt._1
|
||||
val rows = rt._2
|
||||
var maxCommit = -1L
|
||||
|
||||
rows.foreach(r => {
|
||||
val c = r(3).asInstanceOf[String].toLong
|
||||
if (c > maxCommit)
|
||||
maxCommit = c
|
||||
})
|
||||
|
||||
rows.foreach(r => {
|
||||
val c = r(3).asInstanceOf[String].toLong
|
||||
if (c != maxCommit){
|
||||
val f = r(2).asInstanceOf[String].split("_")(0)
|
||||
if (!fileToDeleteKeyMap.contains(f)){
|
||||
fileToDeleteKeyMap(f) = HashSet[String]()
|
||||
}
|
||||
fileToDeleteKeyMap(f).add(key)
|
||||
}
|
||||
})
|
||||
})
|
||||
return fileToDeleteKeyMap
|
||||
}
|
||||
|
||||
|
||||
def fixDuplicates(dryRun: Boolean = true) = {
|
||||
val metadata = new HoodieTableMetadata(fs, basePath)
|
||||
val allFiles = fs.listStatus(new Path(s"${basePath}/${duplicatedPartitionPath}"))
|
||||
val fileNameToPathMap = metadata.getLatestVersions(allFiles).map(f => (FSUtils.getFileId(f.getPath.getName), f.getPath)).toMap;
|
||||
val dupeFixPlan = planDuplicateFix()
|
||||
|
||||
// 1. Copy all latest files into the temp fix path
|
||||
fileNameToPathMap.foreach{ case(fileName, filePath) => {
|
||||
val badSuffix = if (dupeFixPlan.contains(fileName)) ".bad" else ""
|
||||
val dstPath = new Path(s"${repairOutputPath}/${filePath.getName}${badSuffix}")
|
||||
LOG.info(s"Copying from ${filePath} to ${dstPath}")
|
||||
FileUtil.copy(fs, filePath, fs, dstPath, false, true, fs.getConf)
|
||||
}}
|
||||
|
||||
// 2. Remove duplicates from the bad files
|
||||
dupeFixPlan.foreach{case(fileName, keysToSkip) => {
|
||||
val commitTime = FSUtils.getCommitTime(fileNameToPathMap(fileName).getName)
|
||||
val badFilePath = new Path(s"${repairOutputPath}/${fileNameToPathMap(fileName).getName}.bad")
|
||||
val newFilePath = new Path(s"${repairOutputPath}/${fileNameToPathMap(fileName).getName}")
|
||||
LOG.info(" Skipping and writing new file for : " + fileName)
|
||||
SparkHelpers.skipKeysAndWriteNewFile(commitTime, fs, badFilePath, newFilePath, dupeFixPlan(fileName))
|
||||
fs.delete(badFilePath, false)
|
||||
}}
|
||||
|
||||
// 3. Check that there are no duplicates anymore.
|
||||
val df = sqlContext.read.parquet(s"${repairOutputPath}/*.parquet")
|
||||
df.registerTempTable("fixedTbl")
|
||||
val dupeKeyDF = getDupeKeyDF("fixedTbl")
|
||||
val dupeCnt = dupeKeyDF.count();
|
||||
if (dupeCnt != 0) {
|
||||
dupeKeyDF.show()
|
||||
throw new HoodieException("Still found some duplicates!!.. Inspect output")
|
||||
}
|
||||
|
||||
// 4. Additionally ensure no record keys are left behind.
|
||||
val sourceDF = sparkHelper.getDistinctKeyDF(fileNameToPathMap.map(t => t._2.toString).toList)
|
||||
val fixedDF = sparkHelper.getDistinctKeyDF(fileNameToPathMap.map(t => s"${repairOutputPath}/${t._2.getName}").toList)
|
||||
val missedRecordKeysDF = sourceDF.except(fixedDF)
|
||||
val missedCnt = missedRecordKeysDF.count()
|
||||
if (missedCnt != 0) {
|
||||
missedRecordKeysDF.show()
|
||||
throw new HoodieException("Some records in source are not found in fixed files. Inspect output!!")
|
||||
}
|
||||
|
||||
|
||||
println("No duplicates found & counts are in check!!!! ")
|
||||
// 4. Prepare to copy the fixed files back.
|
||||
fileNameToPathMap.foreach { case (fileName, filePath) => {
|
||||
val srcPath = new Path(s"${repairOutputPath}/${filePath.getName}")
|
||||
val dstPath = new Path(s"${basePath}/${duplicatedPartitionPath}/${filePath.getName}")
|
||||
if (dryRun) {
|
||||
LOG.info(s"[JUST KIDDING!!!] Copying from ${srcPath} to ${dstPath}")
|
||||
} else {
|
||||
// for real
|
||||
LOG.info(s"[FOR REAL!!!] Copying from ${srcPath} to ${dstPath}")
|
||||
FileUtil.copy(fs, srcPath, fs, dstPath, false, true, fs.getConf)
|
||||
}
|
||||
}}
|
||||
}
|
||||
}
|
||||
141
hoodie-cli/src/main/scala/com/uber/hoodie/cli/SparkHelpers.scala
Normal file
141
hoodie-cli/src/main/scala/com/uber/hoodie/cli/SparkHelpers.scala
Normal file
@@ -0,0 +1,141 @@
|
||||
/*
|
||||
* Copyright (c) 2016 Uber Technologies, Inc. (hoodie-dev-group@uber.com)
|
||||
*
|
||||
* Licensed under the Apache License, Version 2.0 (the "License");
|
||||
* you may not use this file except in compliance with the License.
|
||||
* You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package com.uber.hoodie.cli
|
||||
|
||||
import com.uber.hoodie.avro.HoodieAvroWriteSupport
|
||||
import com.uber.hoodie.common.BloomFilter
|
||||
import com.uber.hoodie.common.model.HoodieRecord
|
||||
import com.uber.hoodie.common.util.ParquetUtils
|
||||
import com.uber.hoodie.config.{HoodieIndexConfig, HoodieStorageConfig}
|
||||
import com.uber.hoodie.io.storage.{HoodieParquetConfig, HoodieParquetWriter}
|
||||
import com.uber.hoodie.stream.GenericHoodiePayload
|
||||
import org.apache.avro.Schema
|
||||
import org.apache.avro.generic.IndexedRecord
|
||||
import org.apache.hadoop.conf.Configuration
|
||||
import org.apache.hadoop.fs.{FileSystem, Path}
|
||||
import org.apache.parquet.avro.AvroSchemaConverter
|
||||
import org.apache.parquet.hadoop.ParquetFileReader
|
||||
import org.apache.parquet.hadoop.metadata.CompressionCodecName
|
||||
import org.apache.spark.sql.{DataFrame, SQLContext}
|
||||
|
||||
import scala.collection.JavaConversions._
|
||||
import scala.collection.mutable._
|
||||
|
||||
|
||||
object SparkHelpers {
|
||||
@throws[Exception]
|
||||
def skipKeysAndWriteNewFile(commitTime: String, fs: FileSystem, sourceFile: Path, destinationFile: Path, keysToSkip: Set[String]) {
|
||||
val sourceRecords = ParquetUtils.readAvroRecords(sourceFile)
|
||||
val schema: Schema = sourceRecords.get(0).getSchema
|
||||
val filter: BloomFilter = new BloomFilter(HoodieIndexConfig.DEFAULT_BLOOM_FILTER_NUM_ENTRIES.toInt, HoodieIndexConfig.DEFAULT_BLOOM_FILTER_FPP.toDouble)
|
||||
val writeSupport: HoodieAvroWriteSupport = new HoodieAvroWriteSupport(new AvroSchemaConverter().convert(schema), schema, filter)
|
||||
val parquetConfig: HoodieParquetConfig = new HoodieParquetConfig(writeSupport, CompressionCodecName.GZIP, HoodieStorageConfig.DEFAULT_PARQUET_BLOCK_SIZE_BYTES.toInt, HoodieStorageConfig.DEFAULT_PARQUET_PAGE_SIZE_BYTES.toInt, HoodieStorageConfig.DEFAULT_PARQUET_FILE_MAX_BYTES.toInt, fs.getConf)
|
||||
val writer = new HoodieParquetWriter[GenericHoodiePayload, IndexedRecord](commitTime, destinationFile, parquetConfig, schema)
|
||||
for (rec <- sourceRecords) {
|
||||
val key: String = rec.get(HoodieRecord.RECORD_KEY_METADATA_FIELD).toString
|
||||
if (!keysToSkip.contains(key)) {
|
||||
writer.writeAvro(key, rec)
|
||||
}
|
||||
}
|
||||
writer.close
|
||||
}
|
||||
|
||||
def getBloomFilter(file: String, conf: Configuration): String = {
|
||||
val footer = ParquetFileReader.readFooter(conf, new Path(file));
|
||||
return footer.getFileMetaData().getKeyValueMetaData().get(HoodieAvroWriteSupport.HOODIE_AVRO_BLOOM_FILTER_METADATA_KEY)
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
/**
|
||||
* Bunch of Spark Shell/Scala stuff useful for debugging
|
||||
*/
|
||||
class SparkHelper(sqlContext: SQLContext, fs: FileSystem) {
|
||||
|
||||
|
||||
/**
|
||||
* Print keys from a file
|
||||
*
|
||||
* @param file
|
||||
*/
|
||||
def printKeysFromFile(file: String) = {
|
||||
getRowKeyDF(file).collect().foreach(println(_))
|
||||
}
|
||||
|
||||
/**
|
||||
*
|
||||
* @param file
|
||||
* @return
|
||||
*/
|
||||
def getRowKeyDF(file: String): DataFrame = {
|
||||
sqlContext.read.parquet(file).select(s"`${HoodieRecord.RECORD_KEY_METADATA_FIELD}`")
|
||||
}
|
||||
|
||||
|
||||
/**
|
||||
* Does the rowKey actually exist in the file.
|
||||
*
|
||||
* @param rowKey
|
||||
* @param file
|
||||
* @return
|
||||
*/
|
||||
def isFileContainsKey(rowKey: String, file: String): Boolean = {
|
||||
println(s"Checking ${file} for key ${rowKey}")
|
||||
val ff = getRowKeyDF(file).filter(s"`${HoodieRecord.RECORD_KEY_METADATA_FIELD}` = '${rowKey}'")
|
||||
if (ff.count() > 0)
|
||||
return true
|
||||
else
|
||||
return false
|
||||
}
|
||||
|
||||
/**
|
||||
* Number of keys in a given file
|
||||
*
|
||||
* @param file
|
||||
* @param sqlContext
|
||||
*/
|
||||
def getKeyCount(file: String, sqlContext: org.apache.spark.sql.SQLContext) ={
|
||||
println(getRowKeyDF(file).collect().size)
|
||||
}
|
||||
|
||||
|
||||
/**
|
||||
*
|
||||
* Checks that all the keys in the file, have been added to the bloom filter
|
||||
* in the footer
|
||||
*
|
||||
* @param conf
|
||||
* @param sqlContext
|
||||
* @param file
|
||||
* @return
|
||||
*/
|
||||
def fileKeysAgainstBF(conf: Configuration, sqlContext: SQLContext, file: String) : Boolean = {
|
||||
val bfStr = SparkHelpers.getBloomFilter(file, conf)
|
||||
val bf = new com.uber.hoodie.common.BloomFilter(bfStr)
|
||||
val foundCount = sqlContext.parquetFile(file)
|
||||
.select(s"`${HoodieRecord.RECORD_KEY_METADATA_FIELD}`")
|
||||
.collect().
|
||||
filter(r => !bf.mightContain(r.getString(0))).size
|
||||
val totalCount = getKeyCount(file, sqlContext)
|
||||
s"totalCount: ${totalCount}, foundCount: ${foundCount}"
|
||||
totalCount == foundCount
|
||||
}
|
||||
|
||||
def getDistinctKeyDF(paths: List[String]) : DataFrame = {
|
||||
sqlContext.read.parquet(paths:_*).select(s"`${HoodieRecord.RECORD_KEY_METADATA_FIELD}`").distinct()
|
||||
}
|
||||
}
|
||||
94
hoodie-common/pom.xml
Normal file
94
hoodie-common/pom.xml
Normal file
@@ -0,0 +1,94 @@
|
||||
<?xml version="1.0" encoding="UTF-8"?>
|
||||
<!--
|
||||
~ Copyright (c) 2016 Uber Technologies, Inc. (hoodie-dev-group@uber.com)
|
||||
~
|
||||
~ Licensed under the Apache License, Version 2.0 (the "License");
|
||||
~ you may not use this file except in compliance with the License.
|
||||
~ You may obtain a copy of the License at
|
||||
~
|
||||
~ http://www.apache.org/licenses/LICENSE-2.0
|
||||
~
|
||||
~ Unless required by applicable law or agreed to in writing, software
|
||||
~ distributed under the License is distributed on an "AS IS" BASIS,
|
||||
~ WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
~ See the License for the specific language governing permissions and
|
||||
~ limitations under the License.
|
||||
-->
|
||||
|
||||
<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
|
||||
<parent>
|
||||
<artifactId>hoodie</artifactId>
|
||||
<groupId>com.uber.hoodie</groupId>
|
||||
<version>0.2.5-SNAPSHOT</version>
|
||||
</parent>
|
||||
<modelVersion>4.0.0</modelVersion>
|
||||
|
||||
<artifactId>hoodie-common</artifactId>
|
||||
|
||||
<build>
|
||||
<plugins>
|
||||
<plugin>
|
||||
<groupId>org.codehaus.mojo</groupId>
|
||||
<artifactId>cobertura-maven-plugin</artifactId>
|
||||
</plugin>
|
||||
<plugin>
|
||||
<groupId>org.apache.maven.plugins</groupId>
|
||||
<artifactId>maven-jar-plugin</artifactId>
|
||||
<version>2.5</version>
|
||||
<executions>
|
||||
<execution>
|
||||
<goals>
|
||||
<goal>test-jar</goal>
|
||||
</goals>
|
||||
</execution>
|
||||
</executions>
|
||||
</plugin>
|
||||
<plugin>
|
||||
<groupId>org.apache.rat</groupId>
|
||||
<artifactId>apache-rat-plugin</artifactId>
|
||||
</plugin>
|
||||
</plugins>
|
||||
</build>
|
||||
|
||||
<dependencies>
|
||||
<dependency>
|
||||
<groupId>org.apache.avro</groupId>
|
||||
<artifactId>avro</artifactId>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.apache.hadoop</groupId>
|
||||
<artifactId>hadoop-client</artifactId>
|
||||
<exclusions>
|
||||
<exclusion>
|
||||
<groupId>javax.servlet</groupId>
|
||||
<artifactId>*</artifactId>
|
||||
</exclusion>
|
||||
</exclusions>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>junit</groupId>
|
||||
<artifactId>junit</artifactId>
|
||||
<version>${junit.version}</version>
|
||||
<scope>test</scope>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>com.fasterxml.jackson.core</groupId>
|
||||
<artifactId>jackson-annotations</artifactId>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.codehaus.jackson</groupId>
|
||||
<artifactId>jackson-mapper-asl</artifactId>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.apache.parquet</groupId>
|
||||
<artifactId>parquet-avro</artifactId>
|
||||
<version>${parquet.version}</version>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.mockito</groupId>
|
||||
<artifactId>mockito-all</artifactId>
|
||||
<version>1.10.19</version>
|
||||
<scope>test</scope>
|
||||
</dependency>
|
||||
</dependencies>
|
||||
</project>
|
||||
@@ -0,0 +1,54 @@
|
||||
/*
|
||||
* Copyright (c) 2016 Uber Technologies, Inc. (hoodie-dev-group@uber.com)
|
||||
*
|
||||
* Licensed under the Apache License, Version 2.0 (the "License");
|
||||
* you may not use this file except in compliance with the License.
|
||||
* You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package com.uber.hoodie.avro;
|
||||
|
||||
import com.uber.hoodie.common.BloomFilter;
|
||||
|
||||
import org.apache.avro.Schema;
|
||||
import org.apache.parquet.avro.AvroWriteSupport;
|
||||
import org.apache.parquet.hadoop.api.WriteSupport;
|
||||
import org.apache.parquet.schema.MessageType;
|
||||
|
||||
import java.io.*;
|
||||
import java.util.HashMap;
|
||||
|
||||
/**
|
||||
* Wrap AvroWriterSupport for plugging in the bloom filter.
|
||||
*/
|
||||
public class HoodieAvroWriteSupport extends AvroWriteSupport {
|
||||
private BloomFilter bloomFilter;
|
||||
public final static String HOODIE_AVRO_BLOOM_FILTER_METADATA_KEY =
|
||||
"com.uber.hoodie.bloomfilter";
|
||||
|
||||
public HoodieAvroWriteSupport(MessageType schema, Schema avroSchema, BloomFilter bloomFilter) {
|
||||
super(schema, avroSchema);
|
||||
this.bloomFilter = bloomFilter;
|
||||
}
|
||||
|
||||
@Override public WriteSupport.FinalizedWriteContext finalizeWrite() {
|
||||
HashMap<String, String> extraMetaData = new HashMap<>();
|
||||
if (bloomFilter != null) {
|
||||
extraMetaData
|
||||
.put(HOODIE_AVRO_BLOOM_FILTER_METADATA_KEY, bloomFilter.serializeToString());
|
||||
}
|
||||
return new WriteSupport.FinalizedWriteContext(extraMetaData);
|
||||
}
|
||||
|
||||
public void add(String recordKey) {
|
||||
this.bloomFilter.add(recordKey);
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,166 @@
|
||||
/*
|
||||
* Copyright (c) 2016 Uber Technologies, Inc. (hoodie-dev-group@uber.com)
|
||||
*
|
||||
* Licensed under the Apache License, Version 2.0 (the "License");
|
||||
* you may not use this file except in compliance with the License.
|
||||
* You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package com.uber.hoodie.avro;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.HashMap;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
|
||||
import org.apache.avro.Schema;
|
||||
import org.apache.avro.generic.GenericData;
|
||||
import org.apache.avro.generic.GenericRecord;
|
||||
import org.apache.avro.generic.GenericRecordBuilder;
|
||||
import org.codehaus.jackson.map.ObjectMapper;
|
||||
|
||||
/**
|
||||
* Marjority of this is copied from
|
||||
* https://github.com/jwills/avro-json/blob/master/src/main/java/com/cloudera/science/avro/common/JsonConverter.java
|
||||
* Adjusted for expected behavior of our use cases
|
||||
*/
|
||||
public class MercifulJsonConverter {
|
||||
private final ObjectMapper mapper = new ObjectMapper();
|
||||
private final Schema baseSchema;
|
||||
|
||||
public MercifulJsonConverter(Schema schema) {
|
||||
this.baseSchema = schema;
|
||||
}
|
||||
|
||||
|
||||
public GenericRecord convert(String json) throws IOException {
|
||||
try {
|
||||
return convert(mapper.readValue(json, Map.class), baseSchema);
|
||||
} catch (IOException e) {
|
||||
throw new IOException("Failed to parse as Json: " + json + "\n\n" + e.getMessage());
|
||||
}
|
||||
}
|
||||
|
||||
private GenericRecord convert(Map<String, Object> raw, Schema schema)
|
||||
throws IOException {
|
||||
GenericRecord result = new GenericData.Record(schema);
|
||||
for (Schema.Field f : schema.getFields()) {
|
||||
String name = f.name();
|
||||
Object rawValue = raw.get(name);
|
||||
if (rawValue != null) {
|
||||
result.put(f.pos(), typeConvert(rawValue, name, f.schema()));
|
||||
}
|
||||
}
|
||||
|
||||
return result;
|
||||
}
|
||||
|
||||
private Object typeConvert(Object value, String name, Schema schema) throws IOException {
|
||||
if (isOptional(schema)) {
|
||||
if (value == null) {
|
||||
return null;
|
||||
} else {
|
||||
schema = getNonNull(schema);
|
||||
}
|
||||
} else if (value == null) {
|
||||
// Always fail on null for non-nullable schemas
|
||||
throw new JsonConversionException(null, name, schema);
|
||||
}
|
||||
|
||||
switch (schema.getType()) {
|
||||
case BOOLEAN:
|
||||
if (value instanceof Boolean) {
|
||||
return (Boolean) value;
|
||||
}
|
||||
break;
|
||||
case DOUBLE:
|
||||
if (value instanceof Number) {
|
||||
return ((Number) value).doubleValue();
|
||||
}
|
||||
break;
|
||||
case FLOAT:
|
||||
if (value instanceof Number) {
|
||||
return ((Number) value).floatValue();
|
||||
}
|
||||
break;
|
||||
case INT:
|
||||
if (value instanceof Number) {
|
||||
return ((Number) value).intValue();
|
||||
}
|
||||
break;
|
||||
case LONG:
|
||||
if (value instanceof Number) {
|
||||
return ((Number) value).longValue();
|
||||
}
|
||||
break;
|
||||
case STRING:
|
||||
return value.toString();
|
||||
case ENUM:
|
||||
if (schema.getEnumSymbols().contains(value.toString())) {
|
||||
return new GenericData.EnumSymbol(schema, value.toString());
|
||||
}
|
||||
throw new JsonConversionException(String.format("Symbol %s not in enum", value.toString()),
|
||||
schema.getFullName(), schema);
|
||||
case RECORD:
|
||||
return convert((Map<String, Object>) value, schema);
|
||||
case ARRAY:
|
||||
Schema elementSchema = schema.getElementType();
|
||||
List listRes = new ArrayList();
|
||||
for (Object v : (List) value) {
|
||||
listRes.add(typeConvert(v, name, elementSchema));
|
||||
}
|
||||
return listRes;
|
||||
case MAP:
|
||||
Schema valueSchema = schema.getValueType();
|
||||
Map<String, Object> mapRes = new HashMap<String, Object>();
|
||||
for (Map.Entry<String, Object> v : ((Map<String, Object>) value).entrySet()) {
|
||||
mapRes.put(v.getKey(), typeConvert(v.getValue(), name, valueSchema));
|
||||
}
|
||||
return mapRes;
|
||||
default:
|
||||
throw new IllegalArgumentException(
|
||||
"JsonConverter cannot handle type: " + schema.getType());
|
||||
}
|
||||
throw new JsonConversionException(value, name, schema);
|
||||
}
|
||||
|
||||
private boolean isOptional(Schema schema) {
|
||||
return schema.getType().equals(Schema.Type.UNION) &&
|
||||
schema.getTypes().size() == 2 &&
|
||||
(schema.getTypes().get(0).getType().equals(Schema.Type.NULL) ||
|
||||
schema.getTypes().get(1).getType().equals(Schema.Type.NULL));
|
||||
}
|
||||
|
||||
private Schema getNonNull(Schema schema) {
|
||||
List<Schema> types = schema.getTypes();
|
||||
return types.get(0).getType().equals(Schema.Type.NULL) ? types.get(1) : types.get(0);
|
||||
}
|
||||
|
||||
public static class JsonConversionException extends RuntimeException {
|
||||
|
||||
private Object value;
|
||||
private String fieldName;
|
||||
private Schema schema;
|
||||
|
||||
public JsonConversionException(Object value, String fieldName, Schema schema) {
|
||||
this.value = value;
|
||||
this.fieldName = fieldName;
|
||||
this.schema = schema;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
return String.format("Type conversion error for field %s, %s for %s",
|
||||
fieldName, value, schema);
|
||||
}
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,100 @@
|
||||
/*
|
||||
* Copyright (c) 2016 Uber Technologies, Inc. (hoodie-dev-group@uber.com)
|
||||
*
|
||||
* Licensed under the Apache License, Version 2.0 (the "License");
|
||||
* you may not use this file except in compliance with the License.
|
||||
* You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package com.uber.hoodie.common;
|
||||
|
||||
import com.uber.hoodie.exception.HoodieIndexException;
|
||||
import org.apache.commons.io.output.ByteArrayOutputStream;
|
||||
import org.apache.hadoop.util.bloom.Key;
|
||||
import org.apache.hadoop.util.hash.Hash;
|
||||
|
||||
import javax.xml.bind.DatatypeConverter;
|
||||
|
||||
import java.io.*;
|
||||
import java.nio.charset.StandardCharsets;
|
||||
|
||||
/**
|
||||
* A Bloom filter implementation built on top of {@link org.apache.hadoop.util.bloom.BloomFilter}.
|
||||
*/
|
||||
public class BloomFilter {
|
||||
/**
|
||||
* Used in computing the optimal Bloom filter size. This approximately equals 0.480453.
|
||||
*/
|
||||
public static final double LOG2_SQUARED = Math.log(2) * Math.log(2);
|
||||
|
||||
private org.apache.hadoop.util.bloom.BloomFilter filter = null;
|
||||
|
||||
public BloomFilter(int numEntries, double errorRate) {
|
||||
this(numEntries, errorRate, Hash.MURMUR_HASH);
|
||||
}
|
||||
|
||||
/**
|
||||
* Create a new Bloom filter with the given configurations.
|
||||
*/
|
||||
public BloomFilter(int numEntries, double errorRate, int hashType) {
|
||||
// Bit size
|
||||
int bitSize = (int) Math.ceil(numEntries * (-Math.log(errorRate) / LOG2_SQUARED));
|
||||
// Number of the hash functions
|
||||
int numHashs = (int) Math.ceil(Math.log(2) * bitSize / numEntries);
|
||||
// The filter
|
||||
this.filter = new org.apache.hadoop.util.bloom.BloomFilter(bitSize, numHashs, hashType);
|
||||
}
|
||||
|
||||
/**
|
||||
* Create the bloom filter from serialized string.
|
||||
*/
|
||||
public BloomFilter(String filterStr) {
|
||||
this.filter = new org.apache.hadoop.util.bloom.BloomFilter();
|
||||
byte[] bytes = DatatypeConverter.parseBase64Binary(filterStr);
|
||||
DataInputStream dis = new DataInputStream(new ByteArrayInputStream(bytes));
|
||||
try {
|
||||
this.filter.readFields(dis);
|
||||
dis.close();
|
||||
} catch (IOException e) {
|
||||
throw new HoodieIndexException("Could not deserialize BloomFilter instance", e);
|
||||
}
|
||||
}
|
||||
|
||||
public void add(String key) {
|
||||
if (key == null) {
|
||||
throw new NullPointerException("Key cannot by null");
|
||||
}
|
||||
filter.add(new Key(key.getBytes(StandardCharsets.UTF_8)));
|
||||
}
|
||||
|
||||
public boolean mightContain(String key) {
|
||||
if (key == null) {
|
||||
throw new NullPointerException("Key cannot by null");
|
||||
}
|
||||
return filter.membershipTest(new Key(key.getBytes(StandardCharsets.UTF_8)));
|
||||
}
|
||||
|
||||
/**
|
||||
* Serialize the bloom filter as a string.
|
||||
*/
|
||||
public String serializeToString() {
|
||||
ByteArrayOutputStream baos = new ByteArrayOutputStream();
|
||||
DataOutputStream dos = new DataOutputStream(baos);
|
||||
try {
|
||||
filter.write(dos);
|
||||
byte[] bytes = baos.toByteArray();
|
||||
dos.close();
|
||||
return DatatypeConverter.printBase64Binary(bytes);
|
||||
} catch (IOException e) {
|
||||
throw new HoodieIndexException("Could not serialize BloomFilter instance", e);
|
||||
}
|
||||
}
|
||||
}
|
||||
File diff suppressed because it is too large
Load Diff
@@ -0,0 +1,190 @@
|
||||
/*
|
||||
* Copyright (c) 2016 Uber Technologies, Inc. (hoodie-dev-group@uber.com)
|
||||
*
|
||||
* Licensed under the Apache License, Version 2.0 (the "License");
|
||||
* you may not use this file except in compliance with the License.
|
||||
* You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package com.uber.hoodie.common.model;
|
||||
|
||||
import org.apache.log4j.LogManager;
|
||||
import org.apache.log4j.Logger;
|
||||
import org.codehaus.jackson.annotate.JsonAutoDetect;
|
||||
import org.codehaus.jackson.annotate.JsonMethod;
|
||||
import org.codehaus.jackson.map.ObjectMapper;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.io.Serializable;
|
||||
import java.util.ArrayList;
|
||||
import java.util.HashMap;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
|
||||
/**
|
||||
* All the metadata that gets stored along with a commit.
|
||||
*/
|
||||
public class HoodieCommitMetadata implements Serializable {
|
||||
private static volatile Logger log = LogManager.getLogger(HoodieCommitMetadata.class);
|
||||
private HashMap<String, List<HoodieWriteStat>> partitionToWriteStats;
|
||||
|
||||
public HoodieCommitMetadata() {
|
||||
partitionToWriteStats = new HashMap<>();
|
||||
}
|
||||
|
||||
public void addWriteStat(String partitionPath, HoodieWriteStat stat) {
|
||||
if (!partitionToWriteStats.containsKey(partitionPath)) {
|
||||
partitionToWriteStats.put(partitionPath, new ArrayList<HoodieWriteStat>());
|
||||
}
|
||||
partitionToWriteStats.get(partitionPath).add(stat);
|
||||
}
|
||||
|
||||
public List<HoodieWriteStat> getWriteStats(String partitionPath) {
|
||||
return partitionToWriteStats.get(partitionPath);
|
||||
}
|
||||
|
||||
public HashMap<String, List<HoodieWriteStat>> getPartitionToWriteStats() {
|
||||
return partitionToWriteStats;
|
||||
}
|
||||
|
||||
public HashMap<String, String> getFileIdAndFullPaths() {
|
||||
HashMap<String, String> filePaths = new HashMap<>();
|
||||
// list all partitions paths
|
||||
for (Map.Entry<String, List<HoodieWriteStat>> entry: getPartitionToWriteStats().entrySet()) {
|
||||
for (HoodieWriteStat stat: entry.getValue()) {
|
||||
filePaths.put(stat.getFileId(), stat.getFullPath());
|
||||
}
|
||||
}
|
||||
return filePaths;
|
||||
}
|
||||
|
||||
|
||||
public String toJsonString() throws IOException {
|
||||
if(partitionToWriteStats.containsKey(null)) {
|
||||
log.info("partition path is null for " + partitionToWriteStats.get(null));
|
||||
partitionToWriteStats.remove(null);
|
||||
}
|
||||
ObjectMapper mapper = new ObjectMapper();
|
||||
mapper.setVisibility(JsonMethod.FIELD, JsonAutoDetect.Visibility.ANY);
|
||||
return mapper.defaultPrettyPrintingWriter().writeValueAsString(this);
|
||||
}
|
||||
|
||||
public static HoodieCommitMetadata fromJsonString(String jsonStr) throws IOException {
|
||||
if (jsonStr == null || jsonStr.isEmpty()) {
|
||||
// For empty commit file (no data or somethings bad happen).
|
||||
return new HoodieCommitMetadata();
|
||||
}
|
||||
ObjectMapper mapper = new ObjectMapper();
|
||||
mapper.setVisibility(JsonMethod.FIELD, JsonAutoDetect.Visibility.ANY);
|
||||
return mapper.readValue(jsonStr, HoodieCommitMetadata.class);
|
||||
}
|
||||
|
||||
// Here the functions are named "fetch" instead of "get", to get avoid of the json conversion.
|
||||
public long fetchTotalPartitionsWritten() {
|
||||
return partitionToWriteStats.size();
|
||||
}
|
||||
|
||||
public long fetchTotalFilesInsert() {
|
||||
long totalFilesInsert = 0;
|
||||
for (List<HoodieWriteStat> stats : partitionToWriteStats.values()) {
|
||||
for (HoodieWriteStat stat : stats) {
|
||||
if (stat.getPrevCommit() != null && stat.getPrevCommit().equals("null")) {
|
||||
totalFilesInsert ++;
|
||||
}
|
||||
}
|
||||
}
|
||||
return totalFilesInsert;
|
||||
}
|
||||
|
||||
public long fetchTotalFilesUpdated() {
|
||||
long totalFilesUpdated = 0;
|
||||
for (List<HoodieWriteStat> stats : partitionToWriteStats.values()) {
|
||||
for (HoodieWriteStat stat : stats) {
|
||||
if (stat.getPrevCommit() != null && !stat.getPrevCommit().equals("null")) {
|
||||
totalFilesUpdated ++;
|
||||
}
|
||||
}
|
||||
}
|
||||
return totalFilesUpdated;
|
||||
}
|
||||
|
||||
public long fetchTotalUpdateRecordsWritten() {
|
||||
long totalUpdateRecordsWritten = 0;
|
||||
for (List<HoodieWriteStat> stats : partitionToWriteStats.values()) {
|
||||
for (HoodieWriteStat stat : stats) {
|
||||
totalUpdateRecordsWritten += stat.getNumUpdateWrites();
|
||||
}
|
||||
}
|
||||
return totalUpdateRecordsWritten;
|
||||
}
|
||||
|
||||
public long fetchTotalInsertRecordsWritten() {
|
||||
long totalInsertRecordsWritten = 0;
|
||||
for (List<HoodieWriteStat> stats : partitionToWriteStats.values()) {
|
||||
for (HoodieWriteStat stat : stats) {
|
||||
if (stat.getPrevCommit() != null && stat.getPrevCommit().equals("null")) {
|
||||
totalInsertRecordsWritten += stat.getNumWrites();
|
||||
}
|
||||
}
|
||||
}
|
||||
return totalInsertRecordsWritten;
|
||||
}
|
||||
|
||||
public long fetchTotalRecordsWritten() {
|
||||
long totalRecordsWritten = 0;
|
||||
for (List<HoodieWriteStat> stats : partitionToWriteStats.values()) {
|
||||
for (HoodieWriteStat stat : stats) {
|
||||
totalRecordsWritten += stat.getNumWrites();
|
||||
}
|
||||
}
|
||||
return totalRecordsWritten;
|
||||
}
|
||||
|
||||
public long fetchTotalBytesWritten() {
|
||||
long totalBytesWritten = 0;
|
||||
for (List<HoodieWriteStat> stats : partitionToWriteStats.values()) {
|
||||
for (HoodieWriteStat stat : stats) {
|
||||
totalBytesWritten += stat.getTotalWriteBytes();
|
||||
}
|
||||
}
|
||||
return totalBytesWritten;
|
||||
}
|
||||
|
||||
public long fetchTotalWriteErrors() {
|
||||
long totalWriteErrors = 0;
|
||||
for (List<HoodieWriteStat> stats : partitionToWriteStats.values()) {
|
||||
for (HoodieWriteStat stat : stats) {
|
||||
totalWriteErrors += stat.getTotalWriteErrors();
|
||||
}
|
||||
}
|
||||
return totalWriteErrors;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean equals(Object o) {
|
||||
if (this == o)
|
||||
return true;
|
||||
if (o == null || getClass() != o.getClass())
|
||||
return false;
|
||||
|
||||
HoodieCommitMetadata that = (HoodieCommitMetadata) o;
|
||||
|
||||
return partitionToWriteStats != null ?
|
||||
partitionToWriteStats.equals(that.partitionToWriteStats) :
|
||||
that.partitionToWriteStats == null;
|
||||
|
||||
}
|
||||
|
||||
@Override
|
||||
public int hashCode() {
|
||||
return partitionToWriteStats != null ? partitionToWriteStats.hashCode() : 0;
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,191 @@
|
||||
/*
|
||||
* Copyright (c) 2016 Uber Technologies, Inc. (hoodie-dev-group@uber.com)
|
||||
*
|
||||
* Licensed under the Apache License, Version 2.0 (the "License");
|
||||
* you may not use this file except in compliance with the License.
|
||||
* You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package com.uber.hoodie.common.model;
|
||||
|
||||
|
||||
import java.io.Serializable;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Collections;
|
||||
import java.util.List;
|
||||
|
||||
/**
|
||||
* Manages the commit meta and provides operations on the commit timeline
|
||||
*/
|
||||
public class HoodieCommits implements Serializable {
|
||||
|
||||
private List<String> commitList;
|
||||
|
||||
public HoodieCommits(List<String> commitList) {
|
||||
this.commitList = new ArrayList<>(commitList);
|
||||
Collections.sort(this.commitList);
|
||||
this.commitList = Collections.unmodifiableList(this.commitList);
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns the commits which are in the range (startsTs, endTs].
|
||||
*
|
||||
* @param startTs - exclusive start commit ts
|
||||
* @param endTs - inclusive end commit ts
|
||||
*/
|
||||
public List<String> findCommitsInRange(String startTs, String endTs) {
|
||||
if (commitList.isEmpty()) {
|
||||
return Collections.EMPTY_LIST;
|
||||
}
|
||||
int startIndex = 0;
|
||||
if (startTs != null) {
|
||||
startIndex = Collections.binarySearch(commitList, startTs);
|
||||
// If startIndex is negative
|
||||
if (startIndex < 0) {
|
||||
startIndex = -(startIndex + 1);
|
||||
}
|
||||
}
|
||||
|
||||
int endIndex = Collections.binarySearch(commitList, endTs);
|
||||
// If endIndex is negative
|
||||
if (endIndex < 0) {
|
||||
endIndex = -(endIndex + 1);
|
||||
}
|
||||
|
||||
if (endIndex < startIndex) {
|
||||
throw new IllegalArgumentException(
|
||||
"Start Commit Ts " + startTs + " cannot be less than end commit ts" + endTs);
|
||||
}
|
||||
List<String> returns = new ArrayList<>(commitList.subList(startIndex, endIndex));
|
||||
if(endIndex < commitList.size()) {
|
||||
// Be inclusive of the endIndex
|
||||
returns.add(commitList.get(endIndex));
|
||||
}
|
||||
return Collections.unmodifiableList(returns);
|
||||
}
|
||||
|
||||
/**
|
||||
* Finds the list of commits on or before asOfTs
|
||||
*/
|
||||
public List<String> findCommitsAfter(String commitTimeStamp, int numCommits) {
|
||||
if (commitList.isEmpty()) {
|
||||
return null;
|
||||
}
|
||||
|
||||
int startIndex = Collections.binarySearch(commitList, commitTimeStamp);
|
||||
if (startIndex < 0) {
|
||||
startIndex = -(startIndex + 1);
|
||||
} else {
|
||||
// we found asOfTs at startIndex. We want to exclude it.
|
||||
startIndex++;
|
||||
}
|
||||
|
||||
|
||||
List<String> commits = new ArrayList<>();
|
||||
while (numCommits > 0 && startIndex < commitList.size()) {
|
||||
commits.add(commitList.get(startIndex));
|
||||
startIndex++;
|
||||
numCommits--;
|
||||
}
|
||||
|
||||
return Collections.unmodifiableList(commits);
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
final StringBuilder sb = new StringBuilder("HoodieCommits{");
|
||||
sb.append("commitList=").append(commitList);
|
||||
sb.append('}');
|
||||
return sb.toString();
|
||||
}
|
||||
|
||||
public boolean isEmpty() {
|
||||
return commitList.isEmpty();
|
||||
}
|
||||
|
||||
public int getNumCommits() {
|
||||
return commitList.size();
|
||||
}
|
||||
|
||||
public String firstCommit() {
|
||||
return commitList.isEmpty() ? null : commitList.get(0);
|
||||
}
|
||||
|
||||
public String nthCommit(int n) {
|
||||
return commitList.isEmpty() || n >= commitList.size() ? null : commitList.get(n);
|
||||
}
|
||||
|
||||
public String lastCommit() {
|
||||
return commitList.isEmpty() ? null : commitList.get(commitList.size() - 1);
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns the nth commit from the latest commit such that lastCommit(0) => lastCommit()
|
||||
*/
|
||||
public String lastCommit(int n) {
|
||||
if (commitList.size() < n + 1) {
|
||||
return null;
|
||||
}
|
||||
return commitList.get(commitList.size() - 1 - n);
|
||||
}
|
||||
|
||||
public boolean contains(String commitTs) {
|
||||
return commitList.contains(commitTs);
|
||||
}
|
||||
|
||||
public String max(String commit1, String commit2) {
|
||||
if (commit1 == null && commit2 == null) {
|
||||
return null;
|
||||
}
|
||||
if (commit1 == null) {
|
||||
return commit2;
|
||||
}
|
||||
if (commit2 == null) {
|
||||
return commit1;
|
||||
}
|
||||
return (isCommit1BeforeOrOn(commit1, commit2) ? commit2 : commit1);
|
||||
}
|
||||
|
||||
public static boolean isCommit1BeforeOrOn(String commit1, String commit2) {
|
||||
return commit1.compareTo(commit2) <= 0;
|
||||
}
|
||||
|
||||
public static boolean isCommit1After(String commit1, String commit2) {
|
||||
return commit1.compareTo(commit2) > 0;
|
||||
}
|
||||
|
||||
public List<String> getCommitList() {
|
||||
return commitList;
|
||||
}
|
||||
|
||||
public boolean isCommitBeforeEarliestCommit(String commitTs) {
|
||||
return isCommit1BeforeOrOn(commitTs, firstCommit());
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean equals(Object o) {
|
||||
if (this == o)
|
||||
return true;
|
||||
if (o == null || getClass() != o.getClass())
|
||||
return false;
|
||||
|
||||
HoodieCommits that = (HoodieCommits) o;
|
||||
|
||||
return commitList != null ? commitList.equals(that.commitList) : that.commitList == null;
|
||||
|
||||
}
|
||||
|
||||
@Override
|
||||
public int hashCode() {
|
||||
return commitList != null ? commitList.hashCode() : 0;
|
||||
}
|
||||
|
||||
}
|
||||
@@ -0,0 +1,57 @@
|
||||
/*
|
||||
* Copyright (c) 2016 Uber Technologies, Inc. (hoodie-dev-group@uber.com)
|
||||
*
|
||||
* Licensed under the Apache License, Version 2.0 (the "License");
|
||||
* you may not use this file except in compliance with the License.
|
||||
* You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package com.uber.hoodie.common.model;
|
||||
|
||||
import com.uber.hoodie.common.util.FSUtils;
|
||||
|
||||
import org.apache.hadoop.fs.FileStatus;
|
||||
|
||||
public class HoodieFile {
|
||||
|
||||
private final FileStatus fileStatus;
|
||||
private String fileNameWithoutCommitTs;
|
||||
private String commitTs;
|
||||
|
||||
public HoodieFile(FileStatus fileStatus) {
|
||||
this.fileStatus = fileStatus;
|
||||
String fileName = fileStatus.getPath().getName();
|
||||
this.fileNameWithoutCommitTs = FSUtils.getFileId(fileName);
|
||||
this.commitTs = FSUtils.getCommitTime(fileName);
|
||||
}
|
||||
|
||||
public String getFileNameWithoutCommitTs() {
|
||||
return fileNameWithoutCommitTs;
|
||||
}
|
||||
|
||||
public String getCommitTs() {
|
||||
return commitTs;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
final StringBuilder sb = new StringBuilder("HoodieFile{");
|
||||
sb.append("fileStatus=").append(fileStatus);
|
||||
sb.append(", fileNameWithoutCommitTs='").append(fileNameWithoutCommitTs).append('\'');
|
||||
sb.append(", commitTs='").append(commitTs).append('\'');
|
||||
sb.append('}');
|
||||
return sb.toString();
|
||||
}
|
||||
|
||||
public FileStatus getFileStatus() {
|
||||
return fileStatus;
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,73 @@
|
||||
/*
|
||||
* Copyright (c) 2016 Uber Technologies, Inc. (hoodie-dev-group@uber.com)
|
||||
*
|
||||
* Licensed under the Apache License, Version 2.0 (the "License");
|
||||
* you may not use this file except in compliance with the License.
|
||||
* You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package com.uber.hoodie.common.model;
|
||||
|
||||
import com.google.common.base.Objects;
|
||||
|
||||
import java.io.Serializable;
|
||||
|
||||
/**
|
||||
* HoodieKey consists of
|
||||
*
|
||||
* - recordKey : a recordKey that acts as primary key for a record - partitionPath : path to the
|
||||
* partition that contains the record
|
||||
*/
|
||||
public class HoodieKey implements Serializable {
|
||||
|
||||
|
||||
private final String recordKey;
|
||||
|
||||
private final String partitionPath;
|
||||
|
||||
public HoodieKey(String recordKey, String partitionPath) {
|
||||
this.recordKey = recordKey;
|
||||
this.partitionPath = partitionPath;
|
||||
}
|
||||
|
||||
public String getRecordKey() {
|
||||
return recordKey;
|
||||
}
|
||||
|
||||
public String getPartitionPath() {
|
||||
return partitionPath;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean equals(Object o) {
|
||||
if (this == o)
|
||||
return true;
|
||||
if (o == null || getClass() != o.getClass())
|
||||
return false;
|
||||
HoodieKey otherKey = (HoodieKey) o;
|
||||
return Objects.equal(recordKey, otherKey.recordKey) &&
|
||||
Objects.equal(partitionPath, otherKey.partitionPath);
|
||||
}
|
||||
|
||||
@Override
|
||||
public int hashCode() {
|
||||
return Objects.hashCode(recordKey, partitionPath);
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
final StringBuilder sb = new StringBuilder("HoodieKey {");
|
||||
sb.append(" recordKey=").append(recordKey);
|
||||
sb.append(" partitionPath=").append(partitionPath);
|
||||
sb.append('}');
|
||||
return sb.toString();
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,153 @@
|
||||
/*
|
||||
* Copyright (c) 2016 Uber Technologies, Inc. (hoodie-dev-group@uber.com)
|
||||
*
|
||||
* Licensed under the Apache License, Version 2.0 (the "License");
|
||||
* you may not use this file except in compliance with the License.
|
||||
* You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package com.uber.hoodie.common.model;
|
||||
|
||||
import com.google.common.base.Objects;
|
||||
|
||||
import java.io.Serializable;
|
||||
|
||||
/**
|
||||
* A Single Record managed by Hoodie TODO - Make this generic
|
||||
*/
|
||||
public class HoodieRecord<T extends HoodieRecordPayload> implements Serializable {
|
||||
|
||||
public static String COMMIT_TIME_METADATA_FIELD = "_hoodie_commit_time";
|
||||
public static String COMMIT_SEQNO_METADATA_FIELD = "_hoodie_commit_seqno";
|
||||
public static String RECORD_KEY_METADATA_FIELD = "_hoodie_record_key";
|
||||
public static String PARTITION_PATH_METADATA_FIELD = "_hoodie_partition_path";
|
||||
public static String FILENAME_METADATA_FIELD = "_hoodie_file_name";
|
||||
|
||||
/**
|
||||
* Identifies the record across the table
|
||||
*/
|
||||
private HoodieKey key;
|
||||
|
||||
/**
|
||||
* Actual payload of the record
|
||||
*/
|
||||
private T data;
|
||||
|
||||
/**
|
||||
* Current location of record on storage. Filled in by looking up index
|
||||
*/
|
||||
private HoodieRecordLocation currentLocation;
|
||||
|
||||
/**
|
||||
* New location of record on storage, after written
|
||||
*/
|
||||
private HoodieRecordLocation newLocation;
|
||||
|
||||
public HoodieRecord(HoodieKey key, T data) {
|
||||
this.key = key;
|
||||
this.data = data;
|
||||
this.currentLocation = null;
|
||||
this.newLocation = null;
|
||||
}
|
||||
|
||||
public HoodieKey getKey() {
|
||||
return key;
|
||||
}
|
||||
|
||||
public T getData() {
|
||||
if (data == null) {
|
||||
throw new IllegalStateException("Payload already deflated for record.");
|
||||
}
|
||||
return data;
|
||||
}
|
||||
|
||||
/**
|
||||
* Release the actual payload, to ease memory pressure. To be called after the record
|
||||
* has been written to storage. Once deflated, cannot be inflated.
|
||||
*/
|
||||
public void deflate() {
|
||||
this.data = null;
|
||||
}
|
||||
|
||||
|
||||
/**
|
||||
* Sets the current currentLocation of the record. This should happen exactly-once
|
||||
*/
|
||||
public HoodieRecord setCurrentLocation(HoodieRecordLocation location) {
|
||||
assert currentLocation == null;
|
||||
this.currentLocation = location;
|
||||
return this;
|
||||
}
|
||||
|
||||
public HoodieRecordLocation getCurrentLocation() {
|
||||
return currentLocation;
|
||||
}
|
||||
|
||||
/**
|
||||
* Sets the new currentLocation of the record, after being written. This again should happen
|
||||
* exactly-once.
|
||||
*/
|
||||
public HoodieRecord setNewLocation(HoodieRecordLocation location) {
|
||||
assert newLocation == null;
|
||||
this.newLocation = location;
|
||||
return this;
|
||||
}
|
||||
|
||||
public HoodieRecordLocation getNewLocation() {
|
||||
return this.newLocation;
|
||||
}
|
||||
|
||||
public boolean isCurrentLocationKnown() {
|
||||
return this.currentLocation != null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean equals(Object o) {
|
||||
if (this == o)
|
||||
return true;
|
||||
if (o == null || getClass() != o.getClass())
|
||||
return false;
|
||||
HoodieRecord that = (HoodieRecord) o;
|
||||
return Objects.equal(key, that.key) &&
|
||||
Objects.equal(data, that.data) &&
|
||||
Objects.equal(currentLocation, that.currentLocation) &&
|
||||
Objects.equal(newLocation, that.newLocation);
|
||||
}
|
||||
|
||||
@Override
|
||||
public int hashCode() {
|
||||
return Objects.hashCode(key, data, currentLocation, newLocation);
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
final StringBuilder sb = new StringBuilder("HoodieRecord{");
|
||||
sb.append("key=").append(key);
|
||||
sb.append(", currentLocation='").append(currentLocation).append('\'');
|
||||
sb.append(", newLocation='").append(newLocation).append('\'');
|
||||
sb.append('}');
|
||||
return sb.toString();
|
||||
}
|
||||
|
||||
public static String generateSequenceId(String commitTime, int partitionId, long recordIndex) {
|
||||
return commitTime + "_" + partitionId + "_" + recordIndex;
|
||||
}
|
||||
|
||||
public String getPartitionPath() {
|
||||
assert key != null;
|
||||
return key.getPartitionPath();
|
||||
}
|
||||
|
||||
public String getRecordKey() {
|
||||
assert key != null;
|
||||
return key.getRecordKey();
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,69 @@
|
||||
/*
|
||||
* Copyright (c) 2016 Uber Technologies, Inc. (hoodie-dev-group@uber.com)
|
||||
*
|
||||
* Licensed under the Apache License, Version 2.0 (the "License");
|
||||
* you may not use this file except in compliance with the License.
|
||||
* You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package com.uber.hoodie.common.model;
|
||||
|
||||
import com.google.common.base.Objects;
|
||||
|
||||
import java.io.Serializable;
|
||||
|
||||
/**
|
||||
* Location of a HoodieRecord within the parition it belongs to. Ultimately, this points to an
|
||||
* actual file on disk
|
||||
*/
|
||||
public class HoodieRecordLocation implements Serializable {
|
||||
|
||||
private final String commitTime;
|
||||
private final String fileId;
|
||||
|
||||
public HoodieRecordLocation(String commitTime, String fileId) {
|
||||
this.commitTime = commitTime;
|
||||
this.fileId = fileId;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean equals(Object o) {
|
||||
if (this == o)
|
||||
return true;
|
||||
if (o == null || getClass() != o.getClass())
|
||||
return false;
|
||||
HoodieRecordLocation otherLoc = (HoodieRecordLocation) o;
|
||||
return Objects.equal(commitTime, otherLoc.commitTime) &&
|
||||
Objects.equal(fileId, otherLoc.fileId);
|
||||
}
|
||||
|
||||
@Override
|
||||
public int hashCode() {
|
||||
return Objects.hashCode(commitTime, fileId);
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
final StringBuilder sb = new StringBuilder("HoodieRecordLocation {");
|
||||
sb.append("commitTime=").append(commitTime).append(", ");
|
||||
sb.append("fileId=").append(fileId);
|
||||
sb.append('}');
|
||||
return sb.toString();
|
||||
}
|
||||
|
||||
public String getCommitTime() {
|
||||
return commitTime;
|
||||
}
|
||||
|
||||
public String getFileId() {
|
||||
return fileId;
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,57 @@
|
||||
/*
|
||||
* Copyright (c) 2016 Uber Technologies, Inc. (hoodie-dev-group@uber.com)
|
||||
*
|
||||
* Licensed under the Apache License, Version 2.0 (the "License");
|
||||
* you may not use this file except in compliance with the License.
|
||||
* You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package com.uber.hoodie.common.model;
|
||||
|
||||
import org.apache.avro.Schema;
|
||||
import org.apache.avro.generic.IndexedRecord;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.io.Serializable;
|
||||
|
||||
/**
|
||||
* Every Hoodie dataset has an implementation of the <code>HoodieRecordPayload</code>
|
||||
* This abstracts out callbacks which depend on record specific logic
|
||||
*/
|
||||
public interface HoodieRecordPayload<T extends HoodieRecordPayload> extends Serializable {
|
||||
/**
|
||||
* When more than one HoodieRecord have the same HoodieKey, this function combines them
|
||||
* before attempting to insert/upsert (if combining turned on in HoodieClientConfig)
|
||||
*/
|
||||
T preCombine(T another);
|
||||
|
||||
/**
|
||||
*
|
||||
* This methods lets you write custom merging/combining logic to produce new values
|
||||
* as a function of current value on storage and whats contained in this object.
|
||||
*
|
||||
* eg:
|
||||
* 1) You are updating counters, you may want to add counts to currentValue and write back updated counts
|
||||
* 2) You may be reading DB redo logs, and merge them with current image for a database row on storage
|
||||
*
|
||||
* @param currentValue Current value in storage, to merge/combine this payload with
|
||||
* @param schema Schema used for record
|
||||
* @return new combined/merged value to be written back to storage
|
||||
*/
|
||||
IndexedRecord combineAndGetUpdateValue(IndexedRecord currentValue, Schema schema) throws IOException;
|
||||
|
||||
/**
|
||||
* Generates an avro record out of the given HoodieRecordPayload, to be written out to storage.
|
||||
* Called when writing a new value for the given HoodieKey, wherein there is no existing record in
|
||||
* storage to be combined against. (i.e insert)
|
||||
*/
|
||||
IndexedRecord getInsertValue(Schema schema) throws IOException;
|
||||
}
|
||||
@@ -0,0 +1,480 @@
|
||||
/*
|
||||
* Copyright (c) 2016 Uber Technologies, Inc. (hoodie-dev-group@uber.com)
|
||||
*
|
||||
* Licensed under the Apache License, Version 2.0 (the "License");
|
||||
* you may not use this file except in compliance with the License.
|
||||
* You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package com.uber.hoodie.common.model;
|
||||
|
||||
import com.uber.hoodie.common.util.FSUtils;
|
||||
|
||||
import com.uber.hoodie.exception.DatasetNotFoundException;
|
||||
import com.uber.hoodie.exception.HoodieIOException;
|
||||
import com.uber.hoodie.exception.InvalidDatasetException;
|
||||
import org.apache.commons.io.IOUtils;
|
||||
import org.apache.hadoop.fs.FSDataInputStream;
|
||||
import org.apache.hadoop.fs.FSDataOutputStream;
|
||||
import org.apache.hadoop.fs.FileStatus;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.fs.PathFilter;
|
||||
import org.apache.log4j.LogManager;
|
||||
import org.apache.log4j.Logger;
|
||||
|
||||
import java.io.FileNotFoundException;
|
||||
import java.io.IOException;
|
||||
import java.io.Serializable;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Collections;
|
||||
import java.util.Comparator;
|
||||
import java.util.Date;
|
||||
import java.util.HashMap;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Properties;
|
||||
import java.util.SortedMap;
|
||||
import java.util.TreeMap;
|
||||
|
||||
/**
|
||||
* Manages all file system level interactions for the Hoodie tables.
|
||||
*/
|
||||
public class HoodieTableMetadata implements Serializable {
|
||||
public static final String MAX_COMMIT_TS = String.valueOf(Long.MAX_VALUE);
|
||||
public static final String HOODIE_TABLE_NAME_PROP_NAME = "hoodie.table.name";
|
||||
public static final String HOODIE_TABLE_TYPE_PROP_NAME = "hoodie.table.type";
|
||||
public static final HoodieTableType DEFAULT_TABLE_TYPE = HoodieTableType.COPY_ON_WRITE;
|
||||
|
||||
public static final String HOODIE_PROPERTIES_FILE = "hoodie.properties";
|
||||
private static final String HOODIE_HDRONE_PROFILE_DEFAULT_VALUE = "HOODIE";
|
||||
private static final java.lang.String HOODIE_HDRONE_PROFILE_PROP_NAME =
|
||||
"hoodie.hdrone.dataset.profile";
|
||||
|
||||
private static Logger log = LogManager.getLogger(HoodieTableMetadata.class);
|
||||
private transient final FileSystem fs;
|
||||
private transient final Path metadataFolder;
|
||||
private final Properties properties;
|
||||
private HoodieCommits commits;
|
||||
private List<String> inflightCommits;
|
||||
private String basePath;
|
||||
|
||||
public static final String METAFOLDER_NAME = ".hoodie";
|
||||
public static final String COMMIT_FILE_SUFFIX = ".commit";
|
||||
public static final String INFLIGHT_FILE_SUFFIX = ".inflight";
|
||||
|
||||
/**
|
||||
* Constructor which initializes the hoodie table metadata. It will initialize the meta-data if not already present.
|
||||
*
|
||||
* @param fs
|
||||
* @param basePath
|
||||
* @param tableName
|
||||
* @throws IOException
|
||||
*/
|
||||
public HoodieTableMetadata(FileSystem fs, String basePath, String tableName) {
|
||||
this(fs, basePath, tableName, true);
|
||||
}
|
||||
|
||||
/**
|
||||
* Constructor which loads the hoodie table metadata, It requires the meta-data to be present already
|
||||
* @param fs
|
||||
* @param basePath
|
||||
* @throws IOException
|
||||
*/
|
||||
public HoodieTableMetadata(FileSystem fs, String basePath) {
|
||||
this(fs, basePath, null, false);
|
||||
}
|
||||
|
||||
private HoodieTableMetadata(FileSystem fs, String basePath, String tableName,
|
||||
boolean initOnMissing) {
|
||||
this.fs = fs;
|
||||
this.basePath = basePath;
|
||||
|
||||
try {
|
||||
Path basePathDir = new Path(this.basePath);
|
||||
if (!fs.exists(basePathDir)) {
|
||||
if (initOnMissing) {
|
||||
fs.mkdirs(basePathDir);
|
||||
} else {
|
||||
throw new DatasetNotFoundException(this.basePath);
|
||||
}
|
||||
}
|
||||
|
||||
if (!fs.isDirectory(new Path(basePath))) {
|
||||
throw new DatasetNotFoundException(this.basePath);
|
||||
}
|
||||
|
||||
// create .hoodie folder if it does not exist.
|
||||
this.metadataFolder = new Path(this.basePath, METAFOLDER_NAME);
|
||||
Path propertyPath = new Path(metadataFolder, HOODIE_PROPERTIES_FILE);
|
||||
if (!fs.exists(propertyPath)) {
|
||||
if (initOnMissing) {
|
||||
createHoodieProperties(metadataFolder, tableName);
|
||||
} else {
|
||||
throw new InvalidDatasetException(this.basePath);
|
||||
}
|
||||
}
|
||||
|
||||
// Load meta data
|
||||
this.commits = new HoodieCommits(scanCommits(COMMIT_FILE_SUFFIX));
|
||||
this.inflightCommits = scanCommits(INFLIGHT_FILE_SUFFIX);
|
||||
this.properties = readHoodieProperties();
|
||||
log.info("All commits :" + commits);
|
||||
} catch (IOException e) {
|
||||
throw new HoodieIOException("Could not load HoodieMetadata from path " + basePath, e);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns all the commit metadata for this table. Reads all the commit files from HDFS.
|
||||
* Expensive operation, use with caution.
|
||||
*
|
||||
* @return SortedMap of CommitTime,<class>HoodieCommitMetadata</class>
|
||||
* @throws IOException
|
||||
*/
|
||||
public SortedMap<String, HoodieCommitMetadata> getAllCommitMetadata() {
|
||||
try {
|
||||
TreeMap<String, HoodieCommitMetadata> metadataMap = new TreeMap<>();
|
||||
for (String commitTs : commits.getCommitList()) {
|
||||
metadataMap.put(commitTs, getCommitMetadata(commitTs));
|
||||
}
|
||||
return Collections.unmodifiableSortedMap(metadataMap);
|
||||
} catch (IOException e) {
|
||||
throw new HoodieIOException("Could not load all commits for table " + getTableName(),
|
||||
e);
|
||||
}
|
||||
}
|
||||
|
||||
public HoodieCommitMetadata getCommitMetadata(String commitTime) throws IOException {
|
||||
FSDataInputStream is = fs.open(new Path(metadataFolder, FSUtils.makeCommitFileName(commitTime)));
|
||||
try {
|
||||
String jsonStr = IOUtils.toString(is);
|
||||
return HoodieCommitMetadata.fromJsonString(jsonStr);
|
||||
} finally {
|
||||
is.close();
|
||||
}
|
||||
}
|
||||
|
||||
public HoodieTableType getTableType() {
|
||||
return HoodieTableType.valueOf(properties.getProperty(HOODIE_TABLE_TYPE_PROP_NAME));
|
||||
}
|
||||
|
||||
/**
|
||||
* Lookup the file name for specified <code>HoodieRecord</code>
|
||||
* <p/>
|
||||
* TODO(vc): This metadata needs to be cached in each executor, statically, and used across, if
|
||||
* we need to be nicer to the NameNode
|
||||
*/
|
||||
public String getFilenameForRecord(FileSystem fs, final HoodieRecord record) {
|
||||
String fileId = record.getCurrentLocation().getFileId();
|
||||
return getFilenameForRecord(fs, record, fileId);
|
||||
}
|
||||
|
||||
|
||||
public String getFilenameForRecord(FileSystem fs, final HoodieRecord record, String fileId) {
|
||||
try {
|
||||
FileStatus[] files = fs.listStatus(new Path(basePath, record.getPartitionPath()));
|
||||
Map<String, List<FileStatus>> fileIdToVersions =
|
||||
groupFilesByFileId(files, commits.lastCommit());
|
||||
// If the record is not found
|
||||
if(!fileIdToVersions.containsKey(fileId)) {
|
||||
throw new FileNotFoundException("Cannot find valid versions for fileId " + fileId);
|
||||
}
|
||||
|
||||
List<FileStatus> statuses = fileIdToVersions.get(fileId);
|
||||
return statuses.get(0).getPath().getName();
|
||||
} catch (IOException e) {
|
||||
throw new HoodieIOException(
|
||||
"Could not get Filename for record " + record, e);
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
|
||||
/**
|
||||
* Get only the latest file in the partition with precondition commitTime(file) < maxCommitTime
|
||||
*
|
||||
* @param fs
|
||||
* @param partitionPathStr
|
||||
* @param maxCommitTime
|
||||
* @return
|
||||
*/
|
||||
public FileStatus[] getLatestVersionInPartition(FileSystem fs, String partitionPathStr,
|
||||
String maxCommitTime) {
|
||||
try {
|
||||
Path partitionPath = new Path(basePath, partitionPathStr);
|
||||
if(!fs.exists(partitionPath)) {
|
||||
return new FileStatus[0];
|
||||
}
|
||||
FileStatus[] files = fs.listStatus(partitionPath);
|
||||
Map<String, List<FileStatus>> fileIdToVersions =
|
||||
groupFilesByFileId(files, commits.lastCommit());
|
||||
HashMap<String, FileStatus> validFiles = new HashMap<>();
|
||||
for (String fileId : fileIdToVersions.keySet()) {
|
||||
List<FileStatus> versions = fileIdToVersions.get(fileId);
|
||||
for (FileStatus file : versions) {
|
||||
String filename = file.getPath().getName();
|
||||
String commitTime = FSUtils.getCommitTime(filename);
|
||||
if (HoodieCommits.isCommit1BeforeOrOn(commitTime, maxCommitTime)) {
|
||||
validFiles.put(fileId, file);
|
||||
break;
|
||||
}
|
||||
}
|
||||
}
|
||||
return validFiles.values().toArray(new FileStatus[validFiles.size()]);
|
||||
} catch (IOException e) {
|
||||
throw new HoodieIOException(
|
||||
"Could not get latest versions in Partition " + partitionPathStr, e);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Get ALL the data files in partition grouped by fileId and sorted by the commitTime
|
||||
* Given a partition path, provide all the files with a list of their commits, sorted by commit time.
|
||||
*/
|
||||
public Map<String, List<FileStatus>> getAllVersionsInPartition(FileSystem fs, String partitionPath) {
|
||||
try {
|
||||
FileStatus[] files = fs.listStatus(new Path(basePath, partitionPath));
|
||||
return groupFilesByFileId(files, commits.lastCommit());
|
||||
} catch (IOException e) {
|
||||
throw new HoodieIOException(
|
||||
"Could not load all file versions in partition " + partitionPath, e);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Get all the versions of files, within the commit range provided.
|
||||
*
|
||||
* @param commitsToReturn - commits to include
|
||||
*/
|
||||
public FileStatus[] getLatestVersionInRange(FileStatus[] fileStatuses, List<String> commitsToReturn) {
|
||||
if (commitsToReturn.isEmpty()) {
|
||||
return new FileStatus[0];
|
||||
}
|
||||
try {
|
||||
Map<String, List<FileStatus>> fileIdToVersions =
|
||||
groupFilesByFileId(fileStatuses, commits.lastCommit());
|
||||
|
||||
List<FileStatus> statuses = new ArrayList<>();
|
||||
for (List<FileStatus> entry : fileIdToVersions.values()) {
|
||||
for (FileStatus status : entry) {
|
||||
String commitTime = FSUtils.getCommitTime(status.getPath().getName());
|
||||
if (commitsToReturn.contains(commitTime)) {
|
||||
statuses.add(status);
|
||||
break;
|
||||
}
|
||||
}
|
||||
}
|
||||
return statuses.toArray(new FileStatus[statuses.size()]);
|
||||
} catch (IOException e) {
|
||||
throw new HoodieIOException("Could not filter files from commits " + commitsToReturn, e);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
*
|
||||
* Get the latest versions of all the files.
|
||||
*
|
||||
* @param fileStatuses
|
||||
* @return
|
||||
*/
|
||||
public FileStatus[] getLatestVersions(FileStatus[] fileStatuses) {
|
||||
try {
|
||||
Map<String, List<FileStatus>> fileIdToVersions =
|
||||
groupFilesByFileId(fileStatuses, commits.lastCommit());
|
||||
|
||||
List<FileStatus> statuses = new ArrayList<>();
|
||||
for(List<FileStatus> entry:fileIdToVersions.values()) {
|
||||
// first file is the latest one
|
||||
statuses.add(entry.get(0));
|
||||
}
|
||||
return statuses.toArray(new FileStatus[statuses.size()]);
|
||||
} catch (IOException e) {
|
||||
throw new HoodieIOException("Could not filter files for latest version ", e);
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
/**
|
||||
* Get the base path for the Hoodie Table
|
||||
*
|
||||
* @return
|
||||
*/
|
||||
public String getBasePath() {
|
||||
return basePath;
|
||||
}
|
||||
|
||||
|
||||
public boolean isCommitsEmpty() {
|
||||
return commits.isEmpty();
|
||||
}
|
||||
|
||||
public boolean isCommitTsSafe(String commitTs) {
|
||||
return !isCommitsEmpty() && (commits.isCommitBeforeEarliestCommit(commitTs) || commits
|
||||
.contains(commitTs));
|
||||
}
|
||||
|
||||
public List<String> findCommitsSinceTs(String startTs) {
|
||||
return commits.findCommitsInRange(startTs, MAX_COMMIT_TS);
|
||||
}
|
||||
|
||||
public List<String> findCommitsInRange(String startTs, String endTs) {
|
||||
return commits.findCommitsInRange(startTs, endTs);
|
||||
}
|
||||
|
||||
public List<String> findCommitsAfter(String startTs, Integer maxCommits) {
|
||||
return commits.findCommitsAfter(startTs, maxCommits);
|
||||
}
|
||||
|
||||
public HoodieCommits getAllCommits() {
|
||||
return commits;
|
||||
}
|
||||
|
||||
public List<String> getAllInflightCommits() {
|
||||
return inflightCommits;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
final StringBuilder sb = new StringBuilder("HoodieTableMetadata{");
|
||||
sb.append("commits=").append(commits);
|
||||
sb.append('}');
|
||||
return sb.toString();
|
||||
}
|
||||
|
||||
public String getTableName() {
|
||||
return properties.getProperty(HOODIE_TABLE_NAME_PROP_NAME);
|
||||
}
|
||||
|
||||
public String getHDroneDatasetProfile() {
|
||||
return properties.getProperty(HOODIE_HDRONE_PROFILE_PROP_NAME, HOODIE_HDRONE_PROFILE_DEFAULT_VALUE);
|
||||
}
|
||||
|
||||
/**
|
||||
* Initialize the hoodie meta directory and any necessary files inside the meta (including the hoodie.properties)
|
||||
*
|
||||
* @param metadataFolder
|
||||
* @param tableName
|
||||
* @throws IOException
|
||||
*/
|
||||
private void createHoodieProperties(Path metadataFolder, String tableName) throws IOException {
|
||||
if (!fs.exists(metadataFolder)) {
|
||||
fs.mkdirs(metadataFolder);
|
||||
}
|
||||
Path propertyPath = new Path(metadataFolder, HOODIE_PROPERTIES_FILE);
|
||||
FSDataOutputStream outputStream = fs.create(propertyPath);
|
||||
try {
|
||||
Properties props = new Properties();
|
||||
props.setProperty(HOODIE_TABLE_NAME_PROP_NAME, tableName);
|
||||
props.setProperty(HOODIE_TABLE_TYPE_PROP_NAME, DEFAULT_TABLE_TYPE.name());
|
||||
props
|
||||
.store(outputStream, "Properties saved on " + new Date(System.currentTimeMillis()));
|
||||
} finally {
|
||||
outputStream.close();
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Loads the hoodie table properties from the hoodie.properties file under the .hoodie path
|
||||
*/
|
||||
private Properties readHoodieProperties() throws IOException {
|
||||
Properties props = new Properties();
|
||||
Path propertyPath = new Path(metadataFolder, HOODIE_PROPERTIES_FILE);
|
||||
FSDataInputStream inputStream = fs.open(propertyPath);
|
||||
try {
|
||||
props.load(inputStream);
|
||||
} finally {
|
||||
inputStream.close();
|
||||
}
|
||||
return props;
|
||||
}
|
||||
|
||||
/**
|
||||
* Scan the commit times (only choosing commit file with the given suffix)
|
||||
*/
|
||||
private List<String> scanCommits(final String commitFileSuffix) throws IOException {
|
||||
log.info("Attempting to load the commits under " + metadataFolder + " with suffix " + commitFileSuffix);
|
||||
final List<String> commitFiles = new ArrayList<>();
|
||||
fs.listStatus(metadataFolder, new PathFilter() {
|
||||
@Override
|
||||
public boolean accept(Path path) {
|
||||
if (path.getName().endsWith(commitFileSuffix)) {
|
||||
commitFiles.add(path.getName().split("\\.")[0]);
|
||||
return true;
|
||||
}
|
||||
return false;
|
||||
}
|
||||
});
|
||||
return commitFiles;
|
||||
}
|
||||
|
||||
/**
|
||||
* Takes a bunch of file versions, and returns a map keyed by fileId, with the necessary
|
||||
* version safety checking. Returns a map of commitTime and Sorted list of FileStats
|
||||
* ( by reverse commit time )
|
||||
*
|
||||
* @param maxCommitTime maximum permissible commit time
|
||||
*
|
||||
* @return
|
||||
*/
|
||||
private Map<String, List<FileStatus>> groupFilesByFileId(FileStatus[] files,
|
||||
String maxCommitTime) throws IOException {
|
||||
HashMap<String, List<FileStatus>> fileIdtoVersions = new HashMap<>();
|
||||
for (FileStatus file : files) {
|
||||
String filename = file.getPath().getName();
|
||||
String fileId = FSUtils.getFileId(filename);
|
||||
String commitTime = FSUtils.getCommitTime(filename);
|
||||
if (isCommitTsSafe(commitTime) && HoodieCommits
|
||||
.isCommit1BeforeOrOn(commitTime, maxCommitTime)) {
|
||||
if (!fileIdtoVersions.containsKey(fileId)) {
|
||||
fileIdtoVersions.put(fileId, new ArrayList<FileStatus>());
|
||||
}
|
||||
fileIdtoVersions.get(fileId).add(file);
|
||||
}
|
||||
}
|
||||
for (Map.Entry<String, List<FileStatus>> entry : fileIdtoVersions.entrySet()) {
|
||||
Collections.sort(fileIdtoVersions.get(entry.getKey()), new Comparator<FileStatus>() {
|
||||
@Override
|
||||
public int compare(FileStatus o1, FileStatus o2) {
|
||||
String o1CommitTime = FSUtils.getCommitTime(o1.getPath().getName());
|
||||
String o2CommitTime = FSUtils.getCommitTime(o2.getPath().getName());
|
||||
// Reverse the order
|
||||
return o2CommitTime.compareTo(o1CommitTime);
|
||||
}
|
||||
});
|
||||
}
|
||||
return fileIdtoVersions;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean equals(Object o) {
|
||||
if (this == o)
|
||||
return true;
|
||||
if (o == null || getClass() != o.getClass())
|
||||
return false;
|
||||
|
||||
HoodieTableMetadata metadata = (HoodieTableMetadata) o;
|
||||
|
||||
if (commits != null ? !commits.equals(metadata.commits) : metadata.commits != null)
|
||||
return false;
|
||||
return basePath != null ? basePath.equals(metadata.basePath) : metadata.basePath == null;
|
||||
|
||||
}
|
||||
|
||||
@Override
|
||||
public int hashCode() {
|
||||
int result = commits != null ? commits.hashCode() : 0;
|
||||
result = 31 * result + (basePath != null ? basePath.hashCode() : 0);
|
||||
return result;
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
@@ -0,0 +1,35 @@
|
||||
/*
|
||||
* Copyright (c) 2016 Uber Technologies, Inc. (hoodie-dev-group@uber.com)
|
||||
*
|
||||
* Licensed under the Apache License, Version 2.0 (the "License");
|
||||
* you may not use this file except in compliance with the License.
|
||||
* You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package com.uber.hoodie.common.model;
|
||||
|
||||
/**
|
||||
* Type of the Hoodie Table.
|
||||
*
|
||||
* Currently, 1 type is supported
|
||||
*
|
||||
* COPY_ON_WRITE - Performs upserts by versioning entire files, with later versions containing newer
|
||||
* value of a record.
|
||||
*
|
||||
* In the future, following might be added.
|
||||
*
|
||||
* MERGE_ON_READ - Speeds up upserts, by delaying merge until enough work piles up.
|
||||
*
|
||||
* SIMPLE_LSM - A simple 2 level LSM tree.
|
||||
*/
|
||||
public enum HoodieTableType {
|
||||
COPY_ON_WRITE
|
||||
}
|
||||
@@ -0,0 +1,158 @@
|
||||
/*
|
||||
* Copyright (c) 2016 Uber Technologies, Inc. (hoodie-dev-group@uber.com)
|
||||
*
|
||||
* Licensed under the Apache License, Version 2.0 (the "License");
|
||||
* you may not use this file except in compliance with the License.
|
||||
* You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package com.uber.hoodie.common.model;
|
||||
|
||||
import com.fasterxml.jackson.annotation.JsonIgnoreProperties;
|
||||
import com.uber.hoodie.common.util.FSUtils;
|
||||
|
||||
import java.io.Serializable;
|
||||
|
||||
/**
|
||||
* Statistics about a single Hoodie write operation.
|
||||
*/
|
||||
@JsonIgnoreProperties(ignoreUnknown = true)
|
||||
public class HoodieWriteStat implements Serializable {
|
||||
|
||||
public static final String NULL_COMMIT = "null";
|
||||
|
||||
/**
|
||||
* Id of the file being written
|
||||
*/
|
||||
private String fileId;
|
||||
|
||||
/**
|
||||
* Full path to the file on underlying file system
|
||||
*/
|
||||
private String fullPath;
|
||||
|
||||
/**
|
||||
* The previous version of the file. (null if this is the first version. i.e insert)
|
||||
*/
|
||||
private String prevCommit;
|
||||
|
||||
/**
|
||||
* Total number of records written for this file.
|
||||
* - for updates, its the entire number of records in the file
|
||||
* - for inserts, its the actual number of records inserted.
|
||||
*/
|
||||
private long numWrites;
|
||||
|
||||
/**
|
||||
* Total number of records actually changed. (0 for inserts)
|
||||
*/
|
||||
private long numUpdateWrites;
|
||||
|
||||
/**
|
||||
* Total size of file written
|
||||
*/
|
||||
private long totalWriteBytes;
|
||||
|
||||
/**
|
||||
* Total number of records, that were n't able to be written due to errors.
|
||||
*/
|
||||
private long totalWriteErrors;
|
||||
|
||||
public HoodieWriteStat() {
|
||||
// called by jackson json lib
|
||||
}
|
||||
|
||||
public void setFileId(String fileId) {
|
||||
this.fileId = fileId;
|
||||
}
|
||||
|
||||
public void setFullPath(String fullFilePath) {
|
||||
this.fullPath = fullFilePath;
|
||||
}
|
||||
|
||||
public void setPrevCommit(String prevCommit) {
|
||||
this.prevCommit = prevCommit;
|
||||
}
|
||||
|
||||
public void setNumWrites(long numWrites) {
|
||||
this.numWrites = numWrites;
|
||||
}
|
||||
|
||||
public void setNumUpdateWrites(long numUpdateWrites) {
|
||||
this.numUpdateWrites = numUpdateWrites;
|
||||
}
|
||||
|
||||
public long getTotalWriteBytes() {
|
||||
return totalWriteBytes;
|
||||
}
|
||||
|
||||
public void setTotalWriteBytes(long totalWriteBytes) {
|
||||
this.totalWriteBytes = totalWriteBytes;
|
||||
}
|
||||
|
||||
public long getTotalWriteErrors() { return totalWriteErrors; }
|
||||
|
||||
public void setTotalWriteErrors(long totalWriteErrors) { this.totalWriteErrors = totalWriteErrors; }
|
||||
|
||||
public String getPrevCommit() {
|
||||
return prevCommit;
|
||||
}
|
||||
|
||||
public long getNumWrites() {
|
||||
return numWrites;
|
||||
}
|
||||
|
||||
public long getNumUpdateWrites() {
|
||||
return numUpdateWrites;
|
||||
}
|
||||
|
||||
public String getFileId() {
|
||||
return fileId;
|
||||
}
|
||||
|
||||
public String getFullPath() {
|
||||
return fullPath;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
return new StringBuilder()
|
||||
.append("HoodieWriteStat {")
|
||||
.append("fullPath='" + fullPath + '\'')
|
||||
.append(", prevCommit='" + prevCommit + '\'')
|
||||
.append(", numWrites=" + numWrites)
|
||||
.append(", numUpdateWrites=" + numUpdateWrites)
|
||||
.append(", numWriteBytes=" + totalWriteBytes)
|
||||
.append('}')
|
||||
.toString();
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean equals(Object o) {
|
||||
if (this == o)
|
||||
return true;
|
||||
if (o == null || getClass() != o.getClass())
|
||||
return false;
|
||||
|
||||
HoodieWriteStat that = (HoodieWriteStat) o;
|
||||
if (!fullPath.equals(that.fullPath))
|
||||
return false;
|
||||
return prevCommit.equals(that.prevCommit);
|
||||
|
||||
}
|
||||
|
||||
@Override
|
||||
public int hashCode() {
|
||||
int result = fullPath.hashCode();
|
||||
result = 31 * result + prevCommit.hashCode();
|
||||
return result;
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,117 @@
|
||||
/*
|
||||
* Copyright (c) 2016 Uber Technologies, Inc. (hoodie-dev-group@uber.com)
|
||||
*
|
||||
* Licensed under the Apache License, Version 2.0 (the "License");
|
||||
* you may not use this file except in compliance with the License.
|
||||
* You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package com.uber.hoodie.common.util;
|
||||
|
||||
import com.uber.hoodie.common.model.HoodieTableMetadata;
|
||||
import com.uber.hoodie.exception.HoodieIOException;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.FileStatus;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.log4j.LogManager;
|
||||
import org.apache.log4j.Logger;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.List;
|
||||
|
||||
/**
|
||||
* Utility functions related to accessing the file storage
|
||||
*/
|
||||
public class FSUtils {
|
||||
|
||||
private static final Logger LOG = LogManager.getLogger(FSUtils.class);
|
||||
|
||||
public static FileSystem getFs() {
|
||||
Configuration conf = new Configuration();
|
||||
conf.set("fs.hdfs.impl", org.apache.hadoop.hdfs.DistributedFileSystem.class.getName());
|
||||
conf.set("fs.file.impl", org.apache.hadoop.fs.LocalFileSystem.class.getName());
|
||||
FileSystem fs;
|
||||
try {
|
||||
fs = FileSystem.get(conf);
|
||||
} catch (IOException e) {
|
||||
throw new HoodieIOException("Failed to get instance of " + FileSystem.class.getName(),
|
||||
e);
|
||||
}
|
||||
LOG.info(String.format("Hadoop Configuration: fs.defaultFS: [%s], Config:[%s], FileSystem: [%s]",
|
||||
conf.getRaw("fs.defaultFS"), conf.toString(), fs.toString()));
|
||||
return fs;
|
||||
}
|
||||
|
||||
public static String makeDataFileName(String commitTime, int taskPartitionId, String fileId) {
|
||||
return String.format("%s_%d_%s.parquet", fileId, taskPartitionId, commitTime);
|
||||
}
|
||||
|
||||
public static String maskWithoutFileId(String commitTime, int taskPartitionId) {
|
||||
return String.format("*_%s_%s.parquet", taskPartitionId, commitTime);
|
||||
}
|
||||
|
||||
public static String maskWithoutTaskPartitionId(String commitTime, String fileId) {
|
||||
return String.format("%s_*_%s.parquet", fileId, commitTime);
|
||||
}
|
||||
|
||||
public static String maskWithOnlyCommitTime(String commitTime) {
|
||||
return String.format("*_*_%s.parquet", commitTime);
|
||||
}
|
||||
|
||||
public static String makeInflightCommitFileName(String commitTime) {
|
||||
return commitTime + HoodieTableMetadata.INFLIGHT_FILE_SUFFIX;
|
||||
}
|
||||
|
||||
public static String makeCommitFileName(String commitTime) {
|
||||
return commitTime + HoodieTableMetadata.COMMIT_FILE_SUFFIX;
|
||||
}
|
||||
|
||||
public static String getCommitFromCommitFile(String commitFileName) {
|
||||
return commitFileName.split("\\.")[0];
|
||||
}
|
||||
|
||||
public static String getCommitTime(String fullFileName) {
|
||||
return fullFileName.split("_")[2].split("\\.")[0];
|
||||
}
|
||||
|
||||
public static long getFileSize(FileSystem fs, Path path) throws IOException {
|
||||
return fs.listStatus(path)[0].getLen();
|
||||
}
|
||||
|
||||
public static String globAllFiles(String basePath) {
|
||||
return String.format("%s/*/*/*/*", basePath);
|
||||
}
|
||||
|
||||
// TODO (weiy): rename the function for better readability
|
||||
public static String getFileId(String fullFileName) {
|
||||
return fullFileName.split("_")[0];
|
||||
}
|
||||
|
||||
/**
|
||||
* Obtain all the partition paths, that are present in this table.
|
||||
*/
|
||||
public static List<String> getAllPartitionPaths(FileSystem fs, String basePath) throws IOException {
|
||||
List<String> partitionsToClean = new ArrayList<>();
|
||||
// TODO(vc): For now, assume partitions are two levels down from base path.
|
||||
FileStatus[] folders = fs.globStatus(new Path(basePath + "/*/*/*"));
|
||||
for (FileStatus status : folders) {
|
||||
Path path = status.getPath();
|
||||
partitionsToClean.add(String.format("%s/%s/%s",
|
||||
path.getParent().getParent().getName(),
|
||||
path.getParent().getName(),
|
||||
path.getName()));
|
||||
}
|
||||
return partitionsToClean;
|
||||
}
|
||||
|
||||
}
|
||||
@@ -0,0 +1,140 @@
|
||||
/*
|
||||
* Copyright (c) 2016 Uber Technologies, Inc. (hoodie-dev-group@uber.com)
|
||||
*
|
||||
* Licensed under the Apache License, Version 2.0 (the "License");
|
||||
* you may not use this file except in compliance with the License.
|
||||
* You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package com.uber.hoodie.common.util;
|
||||
|
||||
import com.uber.hoodie.common.model.HoodieRecord;
|
||||
|
||||
import org.apache.avro.Schema;
|
||||
import org.apache.avro.generic.*;
|
||||
import org.apache.avro.io.BinaryEncoder;
|
||||
import org.apache.avro.io.Decoder;
|
||||
import org.apache.avro.io.DecoderFactory;
|
||||
import org.apache.avro.io.EncoderFactory;
|
||||
|
||||
import java.io.ByteArrayOutputStream;
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Arrays;
|
||||
import java.util.List;
|
||||
|
||||
/**
|
||||
* Helper class to do common stuff across Avro.
|
||||
*/
|
||||
public class HoodieAvroUtils {
|
||||
|
||||
// All metadata fields are optional strings.
|
||||
private final static Schema METADATA_FIELD_SCHEMA = Schema.createUnion(Arrays.asList(
|
||||
Schema.create(Schema.Type.NULL),
|
||||
Schema.create(Schema.Type.STRING)));
|
||||
|
||||
private final static Schema RECORD_KEY_SCHEMA = initRecordKeySchema();
|
||||
|
||||
/**
|
||||
* Convert a given avro record to bytes
|
||||
*/
|
||||
public static byte[] avroToBytes(GenericRecord record) throws IOException {
|
||||
GenericDatumWriter<GenericRecord> writer =
|
||||
new GenericDatumWriter<>(record.getSchema());
|
||||
ByteArrayOutputStream out = new ByteArrayOutputStream();
|
||||
BinaryEncoder encoder = EncoderFactory.get().binaryEncoder(out, null);
|
||||
writer.write(record, encoder);
|
||||
encoder.flush();
|
||||
out.close();
|
||||
return out.toByteArray();
|
||||
}
|
||||
|
||||
/**
|
||||
* Convert serialized bytes back into avro record
|
||||
*/
|
||||
public static GenericRecord bytesToAvro(byte[] bytes, Schema schema) throws IOException {
|
||||
Decoder decoder = DecoderFactory.get().binaryDecoder(bytes, null);
|
||||
GenericDatumReader<GenericRecord> reader = new GenericDatumReader<GenericRecord>(schema);
|
||||
return reader.read(null, decoder);
|
||||
}
|
||||
|
||||
|
||||
/**
|
||||
* Adds the Hoodie metadata fields to the given schema
|
||||
*/
|
||||
public static Schema addMetadataFields(Schema schema) {
|
||||
List<Schema.Field> parentFields = new ArrayList<>();
|
||||
|
||||
Schema.Field commitTimeField = new Schema.Field(HoodieRecord.COMMIT_TIME_METADATA_FIELD, METADATA_FIELD_SCHEMA, "", null);
|
||||
Schema.Field commitSeqnoField = new Schema.Field(HoodieRecord.COMMIT_SEQNO_METADATA_FIELD, METADATA_FIELD_SCHEMA, "", null);
|
||||
Schema.Field recordKeyField = new Schema.Field(HoodieRecord.RECORD_KEY_METADATA_FIELD, METADATA_FIELD_SCHEMA, "", null);
|
||||
Schema.Field partitionPathField = new Schema.Field(HoodieRecord.PARTITION_PATH_METADATA_FIELD, METADATA_FIELD_SCHEMA, "", null);
|
||||
Schema.Field fileNameField = new Schema.Field(HoodieRecord.FILENAME_METADATA_FIELD, METADATA_FIELD_SCHEMA, "", null);
|
||||
|
||||
parentFields.add(commitTimeField);
|
||||
parentFields.add(commitSeqnoField);
|
||||
parentFields.add(recordKeyField);
|
||||
parentFields.add(partitionPathField);
|
||||
parentFields.add(fileNameField);
|
||||
for (Schema.Field field : schema.getFields()) {
|
||||
parentFields.add(new Schema.Field(field.name(), field.schema(), field.doc(), null));
|
||||
}
|
||||
|
||||
Schema mergedSchema = Schema.createRecord(schema.getName(), schema.getDoc(), schema.getNamespace(), false);
|
||||
mergedSchema.setFields(parentFields);
|
||||
return mergedSchema;
|
||||
}
|
||||
|
||||
private static Schema initRecordKeySchema() {
|
||||
Schema.Field recordKeyField = new Schema.Field(HoodieRecord.RECORD_KEY_METADATA_FIELD, METADATA_FIELD_SCHEMA, "", null);
|
||||
Schema recordKeySchema = Schema.createRecord("HoodieRecordKey", "", "", false);
|
||||
recordKeySchema.setFields(Arrays.asList(recordKeyField));
|
||||
return recordKeySchema;
|
||||
}
|
||||
|
||||
public static Schema getRecordKeySchema() {
|
||||
return RECORD_KEY_SCHEMA;
|
||||
}
|
||||
|
||||
public static GenericRecord addHoodieKeyToRecord(GenericRecord record, String recordKey, String partitionPath, String fileName) {
|
||||
record.put(HoodieRecord.FILENAME_METADATA_FIELD, fileName);
|
||||
record.put(HoodieRecord.PARTITION_PATH_METADATA_FIELD, partitionPath);
|
||||
record.put(HoodieRecord.RECORD_KEY_METADATA_FIELD, recordKey);
|
||||
return record;
|
||||
}
|
||||
|
||||
/**
|
||||
* Adds the Hoodie commit metadata into the provided Generic Record.
|
||||
*/
|
||||
public static GenericRecord addCommitMetadataToRecord(GenericRecord record, String commitTime, String commitSeqno) {
|
||||
record.put(HoodieRecord.COMMIT_TIME_METADATA_FIELD, commitTime);
|
||||
record.put(HoodieRecord.COMMIT_SEQNO_METADATA_FIELD, commitSeqno);
|
||||
return record;
|
||||
}
|
||||
|
||||
|
||||
/**
|
||||
* Given a avro record with a given schema, rewrites it into the new schema
|
||||
*/
|
||||
public static GenericRecord rewriteRecord(GenericRecord record, Schema newSchema)
|
||||
throws Exception {
|
||||
GenericRecord newRecord = new GenericData.Record(newSchema);
|
||||
for (Schema.Field f : record.getSchema().getFields()) {
|
||||
newRecord.put(f.name(), record.get(f.name()));
|
||||
}
|
||||
if (!new GenericData().validate(newSchema, newRecord)) {
|
||||
throw new Exception(
|
||||
"Unable to validate the rewritten record " + record + " against schema "
|
||||
+ newSchema);
|
||||
}
|
||||
return newRecord;
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,26 @@
|
||||
/*
|
||||
* Copyright (c) 2016 Uber Technologies, Inc. (hoodie-dev-group@uber.com)
|
||||
*
|
||||
* Licensed under the Apache License, Version 2.0 (the "License");
|
||||
* you may not use this file except in compliance with the License.
|
||||
* You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package com.uber.hoodie.common.util;
|
||||
|
||||
public class NumericUtils {
|
||||
public static String humanReadableByteCount(double bytes) {
|
||||
if (bytes < 1024) return String.format("%.1f B", bytes);
|
||||
int exp = (int) (Math.log(bytes) / Math.log(1024));
|
||||
String pre = "KMGTPE".charAt(exp-1) + "";
|
||||
return String.format("%.1f %sB", bytes / Math.pow(1024, exp), pre);
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,138 @@
|
||||
/*
|
||||
* Copyright (c) 2016 Uber Technologies, Inc. (hoodie-dev-group@uber.com)
|
||||
*
|
||||
* Licensed under the Apache License, Version 2.0 (the "License");
|
||||
* you may not use this file except in compliance with the License.
|
||||
* You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package com.uber.hoodie.common.util;
|
||||
|
||||
import com.uber.hoodie.avro.HoodieAvroWriteSupport;
|
||||
import com.uber.hoodie.common.BloomFilter;
|
||||
import com.uber.hoodie.common.model.HoodieRecord;
|
||||
|
||||
import com.uber.hoodie.exception.HoodieIOException;
|
||||
import com.uber.hoodie.exception.HoodieIndexException;
|
||||
import org.apache.avro.Schema;
|
||||
import org.apache.avro.generic.GenericRecord;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.parquet.avro.AvroParquetReader;
|
||||
import org.apache.parquet.avro.AvroReadSupport;
|
||||
import org.apache.parquet.hadoop.ParquetFileReader;
|
||||
import org.apache.parquet.hadoop.ParquetReader;
|
||||
import org.apache.parquet.hadoop.metadata.ParquetMetadata;
|
||||
|
||||
import java.io.*;
|
||||
import java.util.ArrayList;
|
||||
import java.util.HashSet;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
|
||||
/**
|
||||
* Utility functions involving with parquet.
|
||||
*/
|
||||
public class ParquetUtils {
|
||||
|
||||
/**
|
||||
* Read the rowKey list from the given parquet file.
|
||||
*
|
||||
* @param filePath The parquet file path.
|
||||
*/
|
||||
public static Set<String> readRowKeysFromParquet(Path filePath) {
|
||||
Configuration conf = new Configuration();
|
||||
Schema readSchema = HoodieAvroUtils.getRecordKeySchema();
|
||||
AvroReadSupport.setAvroReadSchema(conf, readSchema);
|
||||
AvroReadSupport.setRequestedProjection(conf, readSchema);
|
||||
ParquetReader reader = null;
|
||||
Set<String> rowKeys = new HashSet<>();
|
||||
try {
|
||||
reader = AvroParquetReader.builder(filePath).withConf(conf).build();
|
||||
Object obj = reader.read();
|
||||
while (obj != null) {
|
||||
if (obj instanceof GenericRecord) {
|
||||
rowKeys.add(((GenericRecord) obj).get(HoodieRecord.RECORD_KEY_METADATA_FIELD).toString());
|
||||
}
|
||||
obj = reader.read();
|
||||
}
|
||||
} catch (IOException e) {
|
||||
throw new HoodieIOException("Failed to read row keys from Parquet " + filePath, e);
|
||||
|
||||
} finally {
|
||||
if (reader != null) {
|
||||
try {
|
||||
reader.close();
|
||||
} catch (IOException e) {
|
||||
// ignore
|
||||
}
|
||||
}
|
||||
}
|
||||
return rowKeys;
|
||||
}
|
||||
|
||||
/**
|
||||
* Read out the bloom filter from the parquet file meta data.
|
||||
*/
|
||||
public static BloomFilter readBloomFilterFromParquetMetadata(Path parquetFilePath) {
|
||||
ParquetMetadata footer;
|
||||
try {
|
||||
footer = ParquetFileReader.readFooter(new Configuration(), parquetFilePath);
|
||||
} catch (IOException e) {
|
||||
throw new HoodieIndexException("Failed to read footer for parquet " + parquetFilePath,
|
||||
e);
|
||||
}
|
||||
Map<String, String> metadata = footer.getFileMetaData().getKeyValueMetaData();
|
||||
if (metadata.containsKey(HoodieAvroWriteSupport.HOODIE_AVRO_BLOOM_FILTER_METADATA_KEY)) {
|
||||
return new BloomFilter(metadata.get(HoodieAvroWriteSupport.HOODIE_AVRO_BLOOM_FILTER_METADATA_KEY));
|
||||
} else {
|
||||
throw new HoodieIndexException("Could not find index in Parquet footer. Looked for key "
|
||||
+ HoodieAvroWriteSupport.HOODIE_AVRO_BLOOM_FILTER_METADATA_KEY + " in "
|
||||
+ parquetFilePath);
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
/**
|
||||
*
|
||||
* NOTE: This literally reads the entire file contents, thus should be used with caution.
|
||||
*
|
||||
* @param filePath
|
||||
* @return
|
||||
*/
|
||||
public static List<GenericRecord> readAvroRecords(Path filePath) {
|
||||
ParquetReader reader = null;
|
||||
List<GenericRecord> records = new ArrayList<>();
|
||||
try {
|
||||
reader = AvroParquetReader.builder(filePath).build();
|
||||
Object obj = reader.read();
|
||||
while (obj != null) {
|
||||
if (obj instanceof GenericRecord) {
|
||||
records.add(((GenericRecord) obj));
|
||||
}
|
||||
obj = reader.read();
|
||||
}
|
||||
} catch (IOException e) {
|
||||
throw new HoodieIOException("Failed to read avro records from Parquet " + filePath, e);
|
||||
|
||||
} finally {
|
||||
if (reader != null) {
|
||||
try {
|
||||
reader.close();
|
||||
} catch (IOException e) {
|
||||
// ignore
|
||||
}
|
||||
}
|
||||
}
|
||||
return records;
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,43 @@
|
||||
/*
|
||||
* Copyright (c) 2016 Uber Technologies, Inc. (hoodie-dev-group@uber.com)
|
||||
*
|
||||
* Licensed under the Apache License, Version 2.0 (the "License");
|
||||
* you may not use this file except in compliance with the License.
|
||||
* You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package com.uber.hoodie.common.util;
|
||||
|
||||
import com.uber.hoodie.common.model.HoodieRecordPayload;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.HashMap;
|
||||
import java.util.Map;
|
||||
|
||||
public class ReflectionUtils {
|
||||
private static Map<String, Class<?>> clazzCache = new HashMap<>();
|
||||
|
||||
public static <T extends HoodieRecordPayload> T loadPayload(String recordPayloadClass) throws IOException {
|
||||
try {
|
||||
if(clazzCache.get(recordPayloadClass) == null) {
|
||||
Class<?> clazz = Class.<HoodieRecordPayload>forName(recordPayloadClass);
|
||||
clazzCache.put(recordPayloadClass, clazz);
|
||||
}
|
||||
return (T) clazzCache.get(recordPayloadClass).newInstance();
|
||||
} catch (ClassNotFoundException e) {
|
||||
throw new IOException("Could not load payload class " + recordPayloadClass, e);
|
||||
} catch (InstantiationException e) {
|
||||
throw new IOException("Could not load payload class " + recordPayloadClass, e);
|
||||
} catch (IllegalAccessException e) {
|
||||
throw new IOException("Could not load payload class " + recordPayloadClass, e);
|
||||
}
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,32 @@
|
||||
/*
|
||||
* Copyright (c) 2016 Uber Technologies, Inc. (hoodie-dev-group@uber.com)
|
||||
*
|
||||
* Licensed under the Apache License, Version 2.0 (the "License");
|
||||
* you may not use this file except in compliance with the License.
|
||||
* You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package com.uber.hoodie.exception;
|
||||
|
||||
/**
|
||||
* <p>
|
||||
* Exception thrown to indicate that a hoodie dataset was not found on the path provided
|
||||
* <p>
|
||||
*/
|
||||
public class DatasetNotFoundException extends HoodieException {
|
||||
public DatasetNotFoundException(String basePath) {
|
||||
super(getErrorMessage(basePath));
|
||||
}
|
||||
|
||||
private static String getErrorMessage(String basePath) {
|
||||
return "Hoodie dataset not found in path " + basePath;
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,55 @@
|
||||
/*
|
||||
* Copyright (c) 2016 Uber Technologies, Inc. (hoodie-dev-group@uber.com)
|
||||
*
|
||||
* Licensed under the Apache License, Version 2.0 (the "License");
|
||||
* you may not use this file except in compliance with the License.
|
||||
* You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package com.uber.hoodie.exception;
|
||||
|
||||
/**
|
||||
* <p>
|
||||
* Exception thrown for Hoodie failures. The root of
|
||||
* the exception hierarchy.
|
||||
* </p>
|
||||
* <p>
|
||||
* Hoodie Write/Read clients will throw this exception if
|
||||
* any of its operations fail. This is a runtime (unchecked) exception.
|
||||
* </p>
|
||||
*
|
||||
*/
|
||||
public class HoodieException extends RuntimeException {
|
||||
public HoodieException() {
|
||||
super();
|
||||
}
|
||||
|
||||
public HoodieException(String message) {
|
||||
super(message);
|
||||
}
|
||||
|
||||
public HoodieException(String message, Throwable t) {
|
||||
super(message, t);
|
||||
}
|
||||
|
||||
public HoodieException(Throwable t) {
|
||||
super(t);
|
||||
}
|
||||
|
||||
protected static String format(String message, Object... args) {
|
||||
String[] argStrings = new String[args.length];
|
||||
for (int i = 0; i < args.length; i += 1) {
|
||||
argStrings[i] = String.valueOf(args[i]);
|
||||
}
|
||||
return String.format(String.valueOf(message), (Object[]) argStrings);
|
||||
}
|
||||
|
||||
}
|
||||
@@ -0,0 +1,37 @@
|
||||
/*
|
||||
* Copyright (c) 2016 Uber Technologies, Inc. (hoodie-dev-group@uber.com)
|
||||
*
|
||||
* Licensed under the Apache License, Version 2.0 (the "License");
|
||||
* you may not use this file except in compliance with the License.
|
||||
* You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package com.uber.hoodie.exception;
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
/**
|
||||
* <p>
|
||||
* Exception thrown for dataset IO-related failures.
|
||||
* </p>
|
||||
*/
|
||||
public class HoodieIOException extends HoodieException {
|
||||
private final IOException ioException;
|
||||
|
||||
public HoodieIOException(String msg, IOException t) {
|
||||
super(msg, t);
|
||||
this.ioException = t;
|
||||
}
|
||||
|
||||
public IOException getIOException() {
|
||||
return ioException;
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,34 @@
|
||||
/*
|
||||
* Copyright (c) 2016 Uber Technologies, Inc. (hoodie-dev-group@uber.com)
|
||||
*
|
||||
* Licensed under the Apache License, Version 2.0 (the "License");
|
||||
* you may not use this file except in compliance with the License.
|
||||
* You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package com.uber.hoodie.exception;
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
/**
|
||||
* <p>
|
||||
* Exception thrown for HoodieIndex related errors.
|
||||
* </p>
|
||||
*/
|
||||
public class HoodieIndexException extends HoodieException {
|
||||
public HoodieIndexException(String msg) {
|
||||
super(msg);
|
||||
}
|
||||
|
||||
public HoodieIndexException(String msg, Throwable e) {
|
||||
super(msg, e);
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,35 @@
|
||||
/*
|
||||
* Copyright (c) 2016 Uber Technologies, Inc. (hoodie-dev-group@uber.com)
|
||||
*
|
||||
* Licensed under the Apache License, Version 2.0 (the "License");
|
||||
* you may not use this file except in compliance with the License.
|
||||
* You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package com.uber.hoodie.exception;
|
||||
|
||||
import com.uber.hoodie.common.model.HoodieRecord;
|
||||
|
||||
/**
|
||||
* <p>
|
||||
* Exception throws when indexing fails to locate the hoodie record.
|
||||
* HoodieRecord current location and partition path does not match.
|
||||
* This is an unrecoverable error
|
||||
* </p>
|
||||
*/
|
||||
public class HoodieRecordMissingException extends HoodieException {
|
||||
public HoodieRecordMissingException(HoodieRecord record) {
|
||||
super(
|
||||
"Record " + record.getRecordKey() + " with partition path " + record.getPartitionPath()
|
||||
+ " in current location " + record.getCurrentLocation()
|
||||
+ " is not found in the partition");
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,32 @@
|
||||
/*
|
||||
* Copyright (c) 2016 Uber Technologies, Inc. (hoodie-dev-group@uber.com)
|
||||
*
|
||||
* Licensed under the Apache License, Version 2.0 (the "License");
|
||||
* you may not use this file except in compliance with the License.
|
||||
* You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package com.uber.hoodie.exception;
|
||||
|
||||
/**
|
||||
* <p>
|
||||
* Exception thrown to indicate that a hoodie dataset is invalid
|
||||
* <p>
|
||||
*/
|
||||
public class InvalidDatasetException extends HoodieException {
|
||||
public InvalidDatasetException(String basePath) {
|
||||
super(getErrorMessage(basePath));
|
||||
}
|
||||
|
||||
private static String getErrorMessage(String basePath) {
|
||||
return "Invalid Hoodie Dataset. " + basePath;
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,45 @@
|
||||
/*
|
||||
* Copyright (c) 2016 Uber Technologies, Inc. (hoodie-dev-group@uber.com)
|
||||
*
|
||||
* Licensed under the Apache License, Version 2.0 (the "License");
|
||||
* you may not use this file except in compliance with the License.
|
||||
* You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package com.uber.hoodie.common;
|
||||
|
||||
import org.junit.Test;
|
||||
|
||||
import java.io.*;
|
||||
|
||||
public class
|
||||
|
||||
TestBloomFilter {
|
||||
@Test
|
||||
public void testAddKey() {
|
||||
BloomFilter filter = new BloomFilter(100, 0.0000001);
|
||||
filter.add("key1");
|
||||
assert (filter.mightContain("key1"));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testSerialize() throws IOException, ClassNotFoundException {
|
||||
BloomFilter filter = new BloomFilter(1000, 0.0000001);
|
||||
filter.add("key1");
|
||||
filter.add("key2");
|
||||
String filterStr = filter.serializeToString();
|
||||
|
||||
// Rebuild
|
||||
BloomFilter newFilter = new BloomFilter(filterStr);
|
||||
assert (newFilter.mightContain("key1"));
|
||||
assert (newFilter.mightContain("key2"));
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,96 @@
|
||||
/*
|
||||
* Copyright (c) 2016 Uber Technologies, Inc. (hoodie-dev-group@uber.com)
|
||||
*
|
||||
* Licensed under the Apache License, Version 2.0 (the "License");
|
||||
* you may not use this file except in compliance with the License.
|
||||
* You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package com.uber.hoodie.common.model;
|
||||
|
||||
import com.uber.hoodie.common.util.FSUtils;
|
||||
|
||||
import org.junit.rules.TemporaryFolder;
|
||||
|
||||
import java.io.File;
|
||||
import java.io.FileWriter;
|
||||
import java.io.IOException;
|
||||
import java.text.SimpleDateFormat;
|
||||
import java.util.Date;
|
||||
import java.util.Properties;
|
||||
import java.util.UUID;
|
||||
|
||||
public class HoodieTestUtils {
|
||||
|
||||
public static final String RAW_TRIPS_TEST_NAME = "raw_trips";
|
||||
|
||||
public static final void initializeHoodieDirectory(String basePath) throws IOException {
|
||||
new File(basePath + "/" + HoodieTableMetadata.METAFOLDER_NAME).mkdirs();
|
||||
Properties properties = new Properties();
|
||||
properties.setProperty(HoodieTableMetadata.HOODIE_TABLE_NAME_PROP_NAME, RAW_TRIPS_TEST_NAME);
|
||||
properties.setProperty(HoodieTableMetadata.HOODIE_TABLE_TYPE_PROP_NAME, HoodieTableMetadata.DEFAULT_TABLE_TYPE.name());
|
||||
FileWriter fileWriter = new FileWriter(new File(basePath + "/.hoodie/hoodie.properties"));
|
||||
try {
|
||||
properties.store(fileWriter, "");
|
||||
} finally {
|
||||
fileWriter.close();
|
||||
}
|
||||
}
|
||||
|
||||
public static final String initializeTempHoodieBasePath() throws IOException {
|
||||
// Create a temp folder as the base path
|
||||
TemporaryFolder folder = new TemporaryFolder();
|
||||
folder.create();
|
||||
String basePath = folder.getRoot().getAbsolutePath();
|
||||
HoodieTestUtils.initializeHoodieDirectory(basePath);
|
||||
return basePath;
|
||||
}
|
||||
|
||||
public static final String getNewCommitTime() {
|
||||
return new SimpleDateFormat("yyyyMMddHHmmss").format(new Date());
|
||||
}
|
||||
|
||||
public static final void createCommitFiles(String basePath, String... commitTimes) throws IOException {
|
||||
for (String commitTime: commitTimes) {
|
||||
new File(basePath + "/" + HoodieTableMetadata.METAFOLDER_NAME+ "/" + FSUtils.makeCommitFileName(commitTime)).createNewFile();
|
||||
}
|
||||
}
|
||||
|
||||
public static final void createInflightCommitFiles(String basePath, String... commitTimes) throws IOException {
|
||||
for (String commitTime: commitTimes) {
|
||||
new File(basePath + "/" + HoodieTableMetadata.METAFOLDER_NAME+ "/" + FSUtils.makeInflightCommitFileName(commitTime)).createNewFile();
|
||||
}
|
||||
}
|
||||
|
||||
public static final String createNewDataFile(String basePath, String partitionPath, String commitTime) throws IOException {
|
||||
String fileID = UUID.randomUUID().toString();
|
||||
return createDataFile(basePath, partitionPath, commitTime, fileID);
|
||||
}
|
||||
|
||||
public static final String createDataFile(String basePath, String partitionPath, String commitTime, String fileID) throws IOException {
|
||||
String folderPath = basePath + "/" + partitionPath + "/";
|
||||
new File(folderPath).mkdirs();
|
||||
new File(folderPath + FSUtils.makeDataFileName(commitTime, 1, fileID)).createNewFile();
|
||||
return fileID;
|
||||
}
|
||||
|
||||
public static final boolean doesDataFileExist(String basePath, String partitionPath, String commitTime, String fileID) throws IOException {
|
||||
return new File(basePath + "/" + partitionPath + "/" + FSUtils.makeDataFileName(commitTime, 1, fileID)).exists();
|
||||
}
|
||||
|
||||
public static final boolean doesCommitExist(String basePath, String commitTime) {
|
||||
return new File(basePath + "/" + HoodieTableMetadata.METAFOLDER_NAME+ "/" + commitTime + HoodieTableMetadata.COMMIT_FILE_SUFFIX).exists();
|
||||
}
|
||||
|
||||
public static final boolean doesInflightExist(String basePath, String commitTime) {
|
||||
return new File(basePath + "/" + HoodieTableMetadata.METAFOLDER_NAME+ "/" + commitTime + HoodieTableMetadata.INFLIGHT_FILE_SUFFIX).exists();
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,48 @@
|
||||
/*
|
||||
* Copyright (c) 2016 Uber Technologies, Inc. (hoodie-dev-group@uber.com)
|
||||
*
|
||||
* Licensed under the Apache License, Version 2.0 (the "License");
|
||||
* you may not use this file except in compliance with the License.
|
||||
* You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package com.uber.hoodie.common.model;
|
||||
|
||||
|
||||
import org.junit.Test;
|
||||
|
||||
import java.util.Arrays;
|
||||
|
||||
import static org.junit.Assert.*;
|
||||
|
||||
/**
|
||||
*
|
||||
*/
|
||||
public class TestHoodieCommits {
|
||||
|
||||
@Test
|
||||
public void testHoodieCommits() throws Exception {
|
||||
HoodieCommits commits = new HoodieCommits(Arrays.asList("001", "005", "004", "002"));
|
||||
assertFalse(commits.contains("003"));
|
||||
assertTrue(commits.contains("002"));
|
||||
assertEquals(Arrays.asList("004", "005"), commits.findCommitsAfter("003", 2));
|
||||
assertEquals(Arrays.asList("001", "002", "004"), commits.findCommitsInRange("000", "004"));
|
||||
assertEquals(commits.lastCommit(), commits.lastCommit(0));
|
||||
assertEquals("001", commits.lastCommit(3));
|
||||
assertEquals(null, commits.lastCommit(4));
|
||||
|
||||
assertEquals(commits.max("001", "000"), "001");
|
||||
assertFalse(HoodieCommits.isCommit1After("001", "002"));
|
||||
assertFalse(HoodieCommits.isCommit1After("001", "001"));
|
||||
assertTrue(HoodieCommits.isCommit1After("003", "002"));
|
||||
assertTrue(HoodieCommits.isCommit1BeforeOrOn("003", "003"));
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,242 @@
|
||||
/*
|
||||
* Copyright (c) 2016 Uber Technologies, Inc. (hoodie-dev-group@uber.com)
|
||||
*
|
||||
* Licensed under the Apache License, Version 2.0 (the "License");
|
||||
* you may not use this file except in compliance with the License.
|
||||
* You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package com.uber.hoodie.common.model;
|
||||
|
||||
import com.google.common.collect.Sets;
|
||||
|
||||
import com.uber.hoodie.common.util.FSUtils;
|
||||
|
||||
import com.uber.hoodie.exception.HoodieIOException;
|
||||
import com.uber.hoodie.exception.HoodieRecordMissingException;
|
||||
import org.apache.hadoop.fs.FileStatus;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.junit.After;
|
||||
import org.junit.Before;
|
||||
import org.junit.Rule;
|
||||
import org.junit.Test;
|
||||
import org.junit.rules.ExpectedException;
|
||||
|
||||
import java.io.File;
|
||||
import java.io.FileNotFoundException;
|
||||
import java.io.IOException;
|
||||
import java.util.*;
|
||||
|
||||
import static org.junit.Assert.*;
|
||||
import static org.mockito.Mockito.mock;
|
||||
import static org.mockito.Mockito.when;
|
||||
|
||||
public class TestHoodieTableMetadata {
|
||||
private String basePath = null;
|
||||
private HoodieTableMetadata metadata = null;
|
||||
@Rule
|
||||
public final ExpectedException exception = ExpectedException.none();
|
||||
|
||||
@Before
|
||||
public void init() throws Exception {
|
||||
basePath = HoodieTestUtils.initializeTempHoodieBasePath();
|
||||
metadata = new HoodieTableMetadata(FSUtils.getFs(), basePath, "testTable");
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testScanCommitTs() throws Exception {
|
||||
// Empty commit dir
|
||||
assertTrue(metadata.getAllCommits().isEmpty());
|
||||
|
||||
// Create some commit files
|
||||
new File(basePath + "/.hoodie/20160504123032.commit").createNewFile();
|
||||
new File(basePath + "/.hoodie/20160503122032.commit").createNewFile();
|
||||
metadata = new HoodieTableMetadata(FSUtils.getFs(), basePath, "testTable");
|
||||
List<String> list = metadata.getAllCommits().getCommitList();
|
||||
assertEquals(list.size(), 2);
|
||||
assertTrue(list.contains("20160504123032"));
|
||||
assertTrue(list.contains("20160503122032"));
|
||||
|
||||
// Check the .inflight files
|
||||
assertTrue(metadata.getAllInflightCommits().isEmpty());
|
||||
new File(basePath + "/.hoodie/20160505123032.inflight").createNewFile();
|
||||
new File(basePath + "/.hoodie/20160506122032.inflight").createNewFile();
|
||||
metadata = new HoodieTableMetadata(FSUtils.getFs(), basePath, "testTable");
|
||||
list = metadata.getAllInflightCommits();
|
||||
assertEquals(list.size(), 2);
|
||||
assertTrue(list.contains("20160505123032"));
|
||||
assertTrue(list.contains("20160506122032"));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testGetLastValidFileNameForRecord() throws Exception {
|
||||
FileSystem fs = FSUtils.getFs();
|
||||
String partitionPath = "2016/05/01";
|
||||
new File(basePath + "/" + partitionPath).mkdirs();
|
||||
String fileId = UUID.randomUUID().toString();
|
||||
HoodieRecord record = mock(HoodieRecord.class);
|
||||
when(record.getPartitionPath()).thenReturn(partitionPath);
|
||||
when(record.getCurrentLocation()).thenReturn(new HoodieRecordLocation("001", fileId));
|
||||
|
||||
// First, no commit for this record
|
||||
exception.expect(HoodieIOException.class);
|
||||
metadata.getFilenameForRecord(fs, record);
|
||||
|
||||
// Only one commit, but is not safe
|
||||
String commitTime1 = "20160501123212";
|
||||
String fileName1 = FSUtils.makeDataFileName(commitTime1, 1, fileId);
|
||||
new File(basePath + "/" + partitionPath + "/" + fileName1).createNewFile();
|
||||
assertNull(metadata.getFilenameForRecord(fs, record));
|
||||
|
||||
// Make this commit safe
|
||||
new File(basePath + "/.hoodie/" + commitTime1 + ".commit").createNewFile();
|
||||
metadata = new HoodieTableMetadata(fs, basePath, "testTable");
|
||||
assertTrue(metadata.getFilenameForRecord(fs, record).equals(fileName1));
|
||||
|
||||
// Do another commit, but not safe
|
||||
String commitTime2 = "20160502123012";
|
||||
String fileName2 = FSUtils.makeDataFileName(commitTime2, 1, fileId);
|
||||
new File(basePath + "/" + partitionPath + "/" + fileName2).createNewFile();
|
||||
assertTrue(metadata.getFilenameForRecord(fs, record).equals(fileName1));
|
||||
|
||||
// Make it safe
|
||||
new File(basePath + "/.hoodie/" + commitTime2 + ".commit").createNewFile();
|
||||
metadata = new HoodieTableMetadata(fs, basePath, "testTable");
|
||||
assertTrue(metadata.getFilenameForRecord(fs, record).equals(fileName2));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testGetAllPartitionPaths() throws IOException {
|
||||
FileSystem fs = FSUtils.getFs();
|
||||
|
||||
// Empty
|
||||
List<String> partitions = FSUtils.getAllPartitionPaths(fs, basePath);
|
||||
assertEquals(partitions.size(), 0);
|
||||
|
||||
// Add some dirs
|
||||
new File(basePath + "/2016/04/01").mkdirs();
|
||||
new File(basePath + "/2015/04/01").mkdirs();
|
||||
partitions = FSUtils.getAllPartitionPaths(fs, basePath);
|
||||
assertEquals(partitions.size(), 2);
|
||||
assertTrue(partitions.contains("2016/04/01"));
|
||||
assertTrue(partitions.contains("2015/04/01"));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testGetFileVersionsInPartition() throws IOException {
|
||||
// Put some files in the partition
|
||||
String fullPartitionPath = basePath + "/2016/05/01/";
|
||||
new File(fullPartitionPath).mkdirs();
|
||||
|
||||
String commitTime1 = "20160501123032";
|
||||
String commitTime2 = "20160502123032";
|
||||
String commitTime3 = "20160503123032";
|
||||
String commitTime4 = "20160504123032";
|
||||
|
||||
HoodieTestUtils.createCommitFiles(basePath, commitTime1, commitTime2, commitTime3, commitTime4);
|
||||
|
||||
String fileId1 = UUID.randomUUID().toString();
|
||||
String fileId2 = UUID.randomUUID().toString();
|
||||
String fileId3 = UUID.randomUUID().toString();
|
||||
|
||||
new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime1, 1, fileId1)).createNewFile();
|
||||
new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime4, 1, fileId1)).createNewFile();
|
||||
new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime1, 1, fileId2)).createNewFile();
|
||||
new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime2, 1, fileId2)).createNewFile();
|
||||
new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime3, 1, fileId2)).createNewFile();
|
||||
new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime3, 1, fileId3)).createNewFile();
|
||||
new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime4, 1, fileId3)).createNewFile();
|
||||
|
||||
metadata = new HoodieTableMetadata(FSUtils.getFs(), basePath, "testTable");
|
||||
|
||||
Map<String, List<FileStatus>> fileVersions = metadata.getAllVersionsInPartition(FSUtils.getFs(), "2016/05/01");
|
||||
assertEquals(fileVersions.get(fileId1).size(), 2);
|
||||
assertEquals(fileVersions.get(fileId2).size(), 3);
|
||||
assertEquals(fileVersions.get(fileId3).size(), 2);
|
||||
String commitTs = FSUtils.getCommitTime(fileVersions.get(fileId1).get(fileVersions.get(fileId1).size() - 1).getPath().getName());
|
||||
assertTrue(commitTs.equals(commitTime1));
|
||||
commitTs = FSUtils.getCommitTime(fileVersions.get(fileId1).get(fileVersions.get(fileId1).size() - 2).getPath().getName());
|
||||
assertTrue(commitTs.equals(commitTime4));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testGetOnlyLatestVersionFiles() throws Exception {
|
||||
// Put some files in the partition
|
||||
String fullPartitionPath = basePath + "/2016/05/01/";
|
||||
new File(fullPartitionPath).mkdirs();
|
||||
String commitTime1 = "20160501123032";
|
||||
String commitTime2 = "20160502123032";
|
||||
String commitTime3 = "20160503123032";
|
||||
String commitTime4 = "20160504123032";
|
||||
String fileId1 = UUID.randomUUID().toString();
|
||||
String fileId2 = UUID.randomUUID().toString();
|
||||
String fileId3 = UUID.randomUUID().toString();
|
||||
|
||||
new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime1, 1, fileId1)).createNewFile();
|
||||
new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime4, 1, fileId1)).createNewFile();
|
||||
new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime1, 1, fileId2)).createNewFile();
|
||||
new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime2, 1, fileId2)).createNewFile();
|
||||
new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime3, 1, fileId2)).createNewFile();
|
||||
new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime3, 1, fileId3)).createNewFile();
|
||||
new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime4, 1, fileId3)).createNewFile();
|
||||
|
||||
new File(basePath + "/.hoodie/" + commitTime1 + ".commit").createNewFile();
|
||||
new File(basePath + "/.hoodie/" + commitTime2 + ".commit").createNewFile();
|
||||
new File(basePath + "/.hoodie/" + commitTime3 + ".commit").createNewFile();
|
||||
new File(basePath + "/.hoodie/" + commitTime4 + ".commit").createNewFile();
|
||||
|
||||
// Now we list the entire partition
|
||||
FileSystem fs = FSUtils.getFs();
|
||||
FileStatus[] statuses = fs.listStatus(new Path(fullPartitionPath));
|
||||
assertEquals(statuses.length, 7);
|
||||
|
||||
metadata = new HoodieTableMetadata(fs, basePath, "testTable");
|
||||
FileStatus[] statuses1 = metadata
|
||||
.getLatestVersionInPartition(fs, "2016/05/01", commitTime4);
|
||||
assertEquals(statuses1.length, 3);
|
||||
Set<String> filenames = Sets.newHashSet();
|
||||
for (FileStatus status : statuses1) {
|
||||
filenames.add(status.getPath().getName());
|
||||
}
|
||||
assertTrue(filenames.contains(FSUtils.makeDataFileName(commitTime4, 1, fileId1)));
|
||||
assertTrue(filenames.contains(FSUtils.makeDataFileName(commitTime3, 1, fileId2)));
|
||||
assertTrue(filenames.contains(FSUtils.makeDataFileName(commitTime4, 1, fileId3)));
|
||||
|
||||
// Reset the max commit time
|
||||
FileStatus[] statuses2 = metadata
|
||||
.getLatestVersionInPartition(fs, "2016/05/01", commitTime3);
|
||||
assertEquals(statuses2.length, 3);
|
||||
filenames = Sets.newHashSet();
|
||||
for (FileStatus status : statuses2) {
|
||||
filenames.add(status.getPath().getName());
|
||||
}
|
||||
assertTrue(filenames.contains(FSUtils.makeDataFileName(commitTime1, 1, fileId1)));
|
||||
assertTrue(filenames.contains(FSUtils.makeDataFileName(commitTime3, 1, fileId2)));
|
||||
assertTrue(filenames.contains(FSUtils.makeDataFileName(commitTime3, 1, fileId3)));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testCommitTimeComparison() {
|
||||
String commitTime1 = "20160504123032";
|
||||
String commitTime2 = "20151231203159";
|
||||
assertTrue(HoodieCommits.isCommit1After(commitTime1, commitTime2));
|
||||
assertTrue(HoodieCommits.isCommit1BeforeOrOn(commitTime1, commitTime1));
|
||||
assertTrue(HoodieCommits.isCommit1BeforeOrOn(commitTime2, commitTime1));
|
||||
}
|
||||
|
||||
@After
|
||||
public void cleanup() {
|
||||
if (basePath != null) {
|
||||
new File(basePath).delete();
|
||||
}
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,70 @@
|
||||
/*
|
||||
* Copyright (c) 2016 Uber Technologies, Inc. (hoodie-dev-group@uber.com)
|
||||
*
|
||||
* Licensed under the Apache License, Version 2.0 (the "License");
|
||||
* you may not use this file except in compliance with the License.
|
||||
* You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package com.uber.hoodie.common.util;
|
||||
|
||||
import org.junit.Test;
|
||||
|
||||
import java.text.SimpleDateFormat;
|
||||
import java.util.Date;
|
||||
import java.util.UUID;
|
||||
|
||||
import static org.junit.Assert.assertTrue;
|
||||
|
||||
public class TestFSUtils {
|
||||
|
||||
@Test
|
||||
public void testMakeDataFileName() {
|
||||
String commitTime = new SimpleDateFormat("yyyyMMddHHmmss").format(new Date());
|
||||
int taskPartitionId = 2;
|
||||
String fileName = UUID.randomUUID().toString();
|
||||
assertTrue(FSUtils.makeDataFileName(commitTime, taskPartitionId, fileName)
|
||||
.equals(fileName + "_" + taskPartitionId + "_" + commitTime + ".parquet"));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testMaskFileName() {
|
||||
String commitTime = new SimpleDateFormat("yyyyMMddHHmmss").format(new Date());
|
||||
int taskPartitionId = 2;
|
||||
assertTrue(FSUtils.maskWithoutFileId(commitTime, taskPartitionId)
|
||||
.equals("*_" + taskPartitionId + "_" + commitTime + ".parquet"));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testGetCommitTime() {
|
||||
String commitTime = new SimpleDateFormat("yyyyMMddHHmmss").format(new Date());
|
||||
int taskPartitionId = 2;
|
||||
String fileName = UUID.randomUUID().toString();
|
||||
String fullFileName = FSUtils.makeDataFileName(commitTime, taskPartitionId, fileName);
|
||||
assertTrue(FSUtils.getCommitTime(fullFileName).equals(commitTime));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testGetCommitFromCommitFile() {
|
||||
String commitTime = new SimpleDateFormat("yyyyMMddHHmmss").format(new Date());
|
||||
String commitFileName = FSUtils.makeCommitFileName(commitTime);
|
||||
assertTrue(FSUtils.getCommitFromCommitFile(commitFileName).equals(commitTime));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testGetFileNameWithoutMeta() {
|
||||
String commitTime = new SimpleDateFormat("yyyyMMddHHmmss").format(new Date());
|
||||
int taskPartitionId = 2;
|
||||
String fileName = UUID.randomUUID().toString();
|
||||
String fullFileName = FSUtils.makeDataFileName(commitTime, taskPartitionId, fileName);
|
||||
assertTrue(FSUtils.getFileId(fullFileName).equals(fileName));
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,35 @@
|
||||
/*
|
||||
* Copyright (c) 2016 Uber Technologies, Inc. (hoodie-dev-group@uber.com)
|
||||
*
|
||||
* Licensed under the Apache License, Version 2.0 (the "License");
|
||||
* you may not use this file except in compliance with the License.
|
||||
* You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package com.uber.hoodie.common.util;
|
||||
|
||||
import org.junit.Test;
|
||||
import static org.junit.Assert.*;
|
||||
|
||||
public class TestNumericUtils {
|
||||
@Test
|
||||
public void testHumanReadableByteCount() {
|
||||
assertTrue(NumericUtils.humanReadableByteCount(0).equals("0.0 B"));
|
||||
assertTrue(NumericUtils.humanReadableByteCount(27).equals("27.0 B"));
|
||||
assertTrue(NumericUtils.humanReadableByteCount(1023).equals("1023.0 B"));
|
||||
assertTrue(NumericUtils.humanReadableByteCount(1024).equals("1.0 KB"));
|
||||
assertTrue(NumericUtils.humanReadableByteCount(110592).equals("108.0 KB"));
|
||||
assertTrue(NumericUtils.humanReadableByteCount(28991029248L).equals("27.0 GB"));
|
||||
assertTrue(NumericUtils.humanReadableByteCount(1855425871872L).equals("1.7 TB"));
|
||||
assertTrue(NumericUtils.humanReadableByteCount(9223372036854775807L).equals("8.0 EB"));
|
||||
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,92 @@
|
||||
/*
|
||||
* Copyright (c) 2016 Uber Technologies, Inc. (hoodie-dev-group@uber.com)
|
||||
*
|
||||
* Licensed under the Apache License, Version 2.0 (the "License");
|
||||
* you may not use this file except in compliance with the License.
|
||||
* You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package com.uber.hoodie.common.util;
|
||||
|
||||
import com.uber.hoodie.avro.HoodieAvroWriteSupport;
|
||||
import com.uber.hoodie.common.BloomFilter;
|
||||
import com.uber.hoodie.common.model.HoodieRecord;
|
||||
|
||||
import org.apache.avro.Schema;
|
||||
import org.apache.avro.generic.GenericData;
|
||||
import org.apache.avro.generic.GenericRecord;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.parquet.avro.AvroSchemaConverter;
|
||||
import org.apache.parquet.hadoop.ParquetWriter;
|
||||
import org.apache.parquet.hadoop.metadata.CompressionCodecName;
|
||||
import org.junit.Before;
|
||||
import org.junit.Test;
|
||||
import org.junit.rules.TemporaryFolder;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Collections;
|
||||
import java.util.List;
|
||||
import java.util.UUID;
|
||||
|
||||
import static org.junit.Assert.*;
|
||||
|
||||
public class TestParquetUtils {
|
||||
|
||||
|
||||
private String basePath;
|
||||
|
||||
@Before
|
||||
public void setup() throws IOException {
|
||||
// Create a temp folder as the base path
|
||||
TemporaryFolder folder = new TemporaryFolder();
|
||||
folder.create();
|
||||
basePath = folder.getRoot().getAbsolutePath();
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testHoodieWriteSupport() throws Exception {
|
||||
|
||||
List<String> rowKeys = new ArrayList<>();
|
||||
for (int i = 0; i < 1000; i++) {
|
||||
rowKeys.add(UUID.randomUUID().toString());
|
||||
}
|
||||
|
||||
// Write out a parquet file
|
||||
Schema schema = HoodieAvroUtils.getRecordKeySchema();
|
||||
BloomFilter filter = new BloomFilter(1000, 0.0001);
|
||||
HoodieAvroWriteSupport writeSupport = new HoodieAvroWriteSupport(new AvroSchemaConverter().convert(schema), schema, filter);
|
||||
|
||||
|
||||
String filePath = basePath + "/test.parquet";
|
||||
ParquetWriter writer = new ParquetWriter(new Path(filePath),
|
||||
writeSupport, CompressionCodecName.GZIP, 120 * 1024 * 1024, ParquetWriter.DEFAULT_PAGE_SIZE);
|
||||
for (String rowKey : rowKeys) {
|
||||
GenericRecord rec = new GenericData.Record(schema);
|
||||
rec.put(HoodieRecord.RECORD_KEY_METADATA_FIELD, rowKey);
|
||||
writer.write(rec);
|
||||
filter.add(rowKey);
|
||||
}
|
||||
writer.close();
|
||||
|
||||
|
||||
// Read and verify
|
||||
List<String> rowKeysInFile = new ArrayList<>(ParquetUtils.readRowKeysFromParquet(new Path(filePath)));
|
||||
Collections.sort(rowKeysInFile);
|
||||
Collections.sort(rowKeys);
|
||||
|
||||
assertEquals("Did not read back the expected list of keys", rowKeys, rowKeysInFile);
|
||||
BloomFilter filterInFile = ParquetUtils.readBloomFilterFromParquetMetadata(new Path(filePath));
|
||||
for (String rowKey : rowKeys) {
|
||||
assertTrue("key should be found in bloom filter", filterInFile.mightContain(rowKey));
|
||||
}
|
||||
}
|
||||
}
|
||||
Reference in New Issue
Block a user