Import from Hoodie private repo: Part 1
This commit is contained in:
@@ -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;
|
||||
}
|
||||
}
|
||||
Reference in New Issue
Block a user