1
0

[HUDI-159] Redesigning bundles for lighter-weight integrations

- Documented principles applied for redesign at packaging/README.md
 - No longer depends on incl commons-codec, commons-io, commons-pool, commons-dbcp, commons-lang, commons-logging, avro-mapred
 - Introduce new FileIOUtils & added checkstyle rule for illegal import of above
 - Parquet, Avro dependencies moved to provided scope to enable being picked up from Hive/Spark/Presto instead
 - Pickup jackson jars for Hive sync tool from HIVE_HOME & unbundling jackson everywhere
 - Remove hive-jdbc standalone jar from being bundled in Spark/Hive/Utilities bundles
 - 6.5x reduced number of classes across bundles
This commit is contained in:
vinoth chandar
2019-09-02 16:15:55 -07:00
committed by Balaji Varadarajan
parent 0e6f078ec4
commit 7a973a6944
60 changed files with 689 additions and 1380 deletions

View File

@@ -25,15 +25,13 @@ import java.io.FileNotFoundException;
import java.io.IOException;
import java.io.Serializable;
import java.sql.Connection;
import java.sql.DriverManager;
import java.sql.ResultSet;
import java.sql.SQLException;
import java.sql.Statement;
import java.util.List;
import java.util.Scanner;
import java.util.stream.Collectors;
import javax.sql.DataSource;
import org.apache.commons.dbcp.BasicDataSource;
import org.apache.commons.io.IOUtils;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
@@ -41,6 +39,7 @@ import org.apache.hadoop.fs.permission.FsAction;
import org.apache.hadoop.fs.permission.FsPermission;
import org.apache.hudi.common.table.HoodieTableMetaClient;
import org.apache.hudi.common.table.timeline.HoodieInstant;
import org.apache.hudi.common.util.FileIOUtils;
import org.apache.hudi.common.util.Option;
import org.apache.hudi.exception.HoodieException;
import org.apache.hudi.utilities.exception.HoodieIncrementalPullException;
@@ -110,8 +109,8 @@ public class HiveIncrementalPuller {
public HiveIncrementalPuller(Config config) throws IOException {
this.config = config;
validateConfig(config);
String templateContent = IOUtils.toString(
this.getClass().getResourceAsStream("IncrementalPull.sqltemplate"), "UTF-8");
String templateContent = FileIOUtils.readAsUTFString(
this.getClass().getResourceAsStream("IncrementalPull.sqltemplate"));
incrementalPullSQLtemplate = new ST(templateContent);
}
@@ -344,22 +343,13 @@ public class HiveIncrementalPuller {
private Connection getConnection() throws SQLException {
if (connection == null) {
DataSource ds = getDatasource();
log.info("Getting Hive Connection from Datasource " + ds);
this.connection = ds.getConnection();
log.info("Getting Hive Connection to " + config.hiveJDBCUrl);
this.connection = DriverManager.getConnection(config.hiveJDBCUrl, config.hiveUsername, config.hivePassword);
}
return connection;
}
private DataSource getDatasource() {
BasicDataSource ds = new BasicDataSource();
ds.setDriverClassName(driverName);
ds.setUrl(config.hiveJDBCUrl);
ds.setUsername(config.hiveUsername);
ds.setPassword(config.hivePassword);
return ds;
}
public static void main(String[] args) throws IOException {
final Config cfg = new Config();
JCommander cmd = new JCommander(cfg, args);

View File

@@ -24,7 +24,6 @@ import java.io.IOException;
import java.util.HashMap;
import java.util.Map;
import java.util.UUID;
import org.apache.commons.lang.text.StrSubstitutor;
import org.apache.hudi.common.model.HoodieTableType;
import org.apache.hudi.common.util.Option;
import org.apache.log4j.LogManager;
@@ -44,47 +43,26 @@ public class SchedulerConfGenerator {
public static final String SPARK_SCHEDULER_MODE_KEY = "spark.scheduler.mode";
public static final String SPARK_SCHEDULER_ALLOCATION_FILE_KEY = "spark.scheduler.allocation.file";
private static final String DELTASYNC_POOL_KEY = "deltasync_pool";
private static final String COMPACT_POOL_KEY = "compact_pool";
private static final String DELTASYNC_POLICY_KEY = "deltasync_policy";
private static final String COMPACT_POLICY_KEY = "compact_policy";
private static final String DELTASYNC_WEIGHT_KEY = "deltasync_weight";
private static final String DELTASYNC_MINSHARE_KEY = "deltasync_minshare";
private static final String COMPACT_WEIGHT_KEY = "compact_weight";
private static final String COMPACT_MINSHARE_KEY = "compact_minshare";
private static String SPARK_SCHEDULING_PATTERN =
"<?xml version=\"1.0\"?>\n"
+ "<allocations>\n"
+ " <pool name=\"%(deltasync_pool)\">\n"
+ " <schedulingMode>%(deltasync_policy)</schedulingMode>\n"
+ " <weight>%(deltasync_weight)</weight>\n"
+ " <minShare>%(deltasync_minshare)</minShare>\n"
+ " <pool name=\"%s\">\n"
+ " <schedulingMode>%s</schedulingMode>\n"
+ " <weight>%s</weight>\n"
+ " <minShare>%s</minShare>\n"
+ " </pool>\n"
+ " <pool name=\"%(compact_pool)\">\n"
+ " <schedulingMode>%(compact_policy)</schedulingMode>\n"
+ " <weight>%(compact_weight)</weight>\n"
+ " <minShare>%(compact_minshare)</minShare>\n"
+ " <pool name=\"%s\">\n"
+ " <schedulingMode>%s</schedulingMode>\n"
+ " <weight>%s</weight>\n"
+ " <minShare>%s</minShare>\n"
+ " </pool>\n"
+ "</allocations>";
private static String generateConfig(Integer deltaSyncWeight, Integer compactionWeight, Integer deltaSyncMinShare,
Integer compactionMinShare) {
Map<String, String> schedulingProps = new HashMap<>();
schedulingProps.put(DELTASYNC_POOL_KEY, DELTASYNC_POOL_NAME);
schedulingProps.put(COMPACT_POOL_KEY, COMPACT_POOL_NAME);
schedulingProps.put(DELTASYNC_POLICY_KEY, "FAIR");
schedulingProps.put(COMPACT_POLICY_KEY, "FAIR");
schedulingProps.put(DELTASYNC_WEIGHT_KEY, deltaSyncWeight.toString());
schedulingProps.put(DELTASYNC_MINSHARE_KEY, deltaSyncMinShare.toString());
schedulingProps.put(COMPACT_WEIGHT_KEY, compactionWeight.toString());
schedulingProps.put(COMPACT_MINSHARE_KEY, compactionMinShare.toString());
StrSubstitutor sub = new StrSubstitutor(schedulingProps, "%(", ")");
String xmlString = sub.replace(SPARK_SCHEDULING_PATTERN);
log.info("Scheduling Configurations generated. Config=\n" + xmlString);
return xmlString;
return String.format(SPARK_SCHEDULING_PATTERN,
DELTASYNC_POOL_NAME, "FAIR", deltaSyncWeight.toString(), deltaSyncMinShare.toString(),
COMPACT_POOL_NAME, "FAIR", compactionWeight.toString(), compactionMinShare.toString());
}

View File

@@ -1,3 +1,21 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hudi.utilities;
import static org.junit.Assert.assertNotNull;