diff --git hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/HiveEndPoint.java hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/HiveEndPoint.java index 4c77842..3c24f19 100644 --- hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/HiveEndPoint.java +++ hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/HiveEndPoint.java @@ -387,7 +387,7 @@ public TransactionBatch run() throws StreamingException, InterruptedException { } private TransactionBatch fetchTransactionBatchImpl(int numTransactions, - RecordWriter recordWriter) + RecordWriter recordWriter) throws StreamingException, TransactionBatchUnAvailable, InterruptedException { return new TransactionBatchImpl(username, ugi, endPt, numTransactions, msClient , recordWriter); diff --git itests/capybara/pom.xml itests/capybara/pom.xml new file mode 100644 index 0000000..82aaee1 --- /dev/null +++ itests/capybara/pom.xml @@ -0,0 +1,231 @@ + + + + 4.0.0 + + org.apache.hive + hive-it + 2.1.0-SNAPSHOT + ../pom.xml + + + capybara-test + jar + Hive Integration - Capybara + + + ../.. + ${basedir}/${hive.path.to.root}/itests/hive-unit/target/spark + + + + + + org.apache.hive + hive-common + ${project.version} + + + org.apache.hive + hive-jdbc + ${project.version} + + + org.apache.hive + hive-service + ${project.version} + + + org.apache.hive + hive-exec + ${project.version} + + + org.apache.hive + hive-serde + ${project.version} + + + org.apache.hive + hive-it-unit + ${project.version} + + + org.apache.hive.hcatalog + hive-hcatalog-core + ${project.version} + + + org.apache.hive.hcatalog + hive-hcatalog-streaming + ${project.version} + + + + + + org.apache.hive + hive-beeline + ${project.version} + test + + + org.apache.hive + hive-metastore + ${project.version} + tests + + + org.apache.hive + hive-service + ${project.version} + tests + + + org.apache.hive + hive-exec + ${project.version} + tests + + + junit + junit + ${junit.version} + + + + org.mockito + mockito-all + ${mockito-all.version} + + + org.apache.hadoop + hadoop-common + ${hadoop.version} + + + org.apache.hadoop + hadoop-hdfs + ${hadoop.version} + tests + + + org.apache.hadoop + hadoop-mapreduce-client-jobclient + ${hadoop.version} + tests + + + org.apache.hadoop + hadoop-mapreduce-client-core + ${hadoop.version} + + + org.apache.hbase + hbase-server + ${hbase.version} + + + org.apache.hadoop + hadoop-minicluster + ${hadoop.version} + + + com.sun.jersey + jersey-servlet + test + + + org.apache.hadoop + hadoop-archives + ${hadoop.version} + test + + + org.apache.hadoop + hadoop-common + ${hadoop.version} + tests + + + org.apache.hadoop + hadoop-mapreduce-client-hs + ${hadoop.version} + + + org.apache.hadoop + hadoop-yarn-server-tests + ${hadoop.version} + tests + + + org.apache.hadoop + hadoop-yarn-client + ${hadoop.version} + + + org.apache.tez + tez-tests + ${tez.version} + test-jar + + + org.apache.tez + tez-api + ${tez.version} + + + org.apache.tez + tez-runtime-library + ${tez.version} + + + org.apache.tez + tez-mapreduce + ${tez.version} + + + org.apache.tez + tez-dag + ${tez.version} + + + postgresql + postgresql + 9.1-901.jdbc4 + test + + + + + + + org.apache.maven.plugins + maven-jar-plugin + ${maven.jar.plugin.version} + + + + test-jar + + + + + + + + diff --git itests/capybara/src/main/java/org/apache/hive/test/capybara/Explain.java itests/capybara/src/main/java/org/apache/hive/test/capybara/Explain.java new file mode 100644 index 0000000..cb222a7 --- /dev/null +++ itests/capybara/src/main/java/org/apache/hive/test/capybara/Explain.java @@ -0,0 +1,168 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hive.test.capybara; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import org.apache.hadoop.hive.ql.QueryPlan; +import org.apache.hadoop.hive.ql.exec.Operator; +import org.apache.hadoop.hive.ql.exec.Task; +import org.apache.hadoop.hive.ql.exec.tez.TezTask; +import org.apache.hadoop.hive.ql.plan.BaseWork; +import org.apache.hadoop.hive.ql.plan.OperatorDesc; +import org.junit.Assert; + +import java.io.Serializable; +import java.util.ArrayDeque; +import java.util.ArrayList; +import java.util.Collection; +import java.util.Deque; +import java.util.List; +import java.util.Set; + +/** + * Tools for working with the output of explain plan. You should never create this directly, + * rather it is generated by calling + * {@link org.apache.hive.test.capybara.IntegrationTest#explain}. This class provides tools + * for finding particular tasks or operators in the tree, or patterns of tasks or operators. + */ +public class Explain { + static final private Logger LOG = LoggerFactory.getLogger(Explain.class); + + final private QueryPlan plan; + + Explain(QueryPlan plan) { + this.plan = plan; + } + + /** + * Get the plan for this query. + * @return the plan. + */ + public QueryPlan getPlan() { + return plan; + } + + /** + * Doing a depth first traversal, find the first instance of the passed in Task. If no instance + * of this class is available Assert.fail will be called. + * @param expectedClass class of the expected task + * @return the first instance of the task to be found. + */ + public > T expect(Class expectedClass) { + Deque> toSearch = new ArrayDeque<>(plan.getRootTasks()); + toSearch.addLast(plan.getFetchTask()); + while (toSearch.size() > 0) { + Task task = toSearch.pollFirst(); + if (task.getClass().equals(expectedClass)) return (T)task; + if (task.getChildTasks() != null) { + for (Task child : task.getChildTasks()) { + toSearch.addFirst(child); + } + } + } + Assert.fail("Failed to find task of type " + expectedClass.getName()); + // I'll never get here, but it makes the compiler happy. + return null; + } + + /** + * Find all instances of a given task in the tree. + * @param lookFor class of the task we're looking for + * @return list of all tasks of this class. List will be empty if no tasks are found. + */ + public > List findAll(Class lookFor) { + List matches = new ArrayList<>(); + Deque> toSearch = new ArrayDeque<>(plan.getRootTasks()); + toSearch.addLast(plan.getFetchTask()); + while (toSearch.size() > 0) { + Task task = toSearch.pollFirst(); + if (task.getClass().equals(lookFor)) matches.add((T)task); + if (task.getChildTasks() != null) { + for (Task child : task.getChildTasks()) { + toSearch.addFirst(child); + } + } + } + return matches; + } + + /** + * Doing a depth first traversal, find the first instance of the passed in operator. If no + * instance of this class is found Assert.fail will be called. + * @param task Task to look for the operator in + * @param expectedOperator class of expected operator + * @return the first instance of the operator to be found. + */ + public > T expect(Task task, + Class expectedOperator) { + List ops = searchForOperator(task, expectedOperator, true); + if (ops.isEmpty()) Assert.fail("Failed to find operator of type " + expectedOperator.getName()); + else return ops.get(0); + // Don't need this, but it makes the compiler happy. + return null; + } + + /** + * Find all instances of a given operator in the task. + * @param task Task to look for the operators in + * @param lookFor list of all operators in the task. + * @return list of all operators of the given class. List will be empty if no matching + * operators are found. + */ + public > List findAll(Task task, + Class lookFor) { + return searchForOperator(task, lookFor, false); + } + + private > List + searchForOperator(Task task, Class lookFor, boolean returnFirst) { + List results = new ArrayList<>(); + Deque> toSearch = new ArrayDeque<>(); + Collection> ops = task.getTopOperators(); + if (!ops.isEmpty()) { + toSearch.addAll(ops); + } else if (task instanceof TezTask) { // Ok, this might be Tez + TezTask tezTask = (TezTask)task; + Set tezRoots = tezTask.getWork().getRoots(); + if (tezRoots != null) { + for (BaseWork root : tezRoots) { + if (root.getAllRootOperators() != null) { + for (Operator op : root.getAllRootOperators()) { + toSearch.addFirst(op); + } + } + } + } + } + while (toSearch.size() > 0) { + Operator op = toSearch.pollFirst(); + LOG.debug("XXX Looking at operator of type " + op.getClass().getName()); + if (op.getClass().equals(lookFor)) { + results.add((T)op); + if (returnFirst) return results; + } + if (op.getChildOperators() != null) { + for (Operator child : op.getChildOperators()) { + toSearch.addFirst(child); + } + } + } + return results; + } +} diff --git itests/capybara/src/main/java/org/apache/hive/test/capybara/IntegrationTest.java itests/capybara/src/main/java/org/apache/hive/test/capybara/IntegrationTest.java new file mode 100644 index 0000000..43c3883 --- /dev/null +++ itests/capybara/src/main/java/org/apache/hive/test/capybara/IntegrationTest.java @@ -0,0 +1,516 @@ +/** + * 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.hive.test.capybara; + +import com.google.common.annotations.VisibleForTesting; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hive.conf.HiveConf; +import org.apache.hadoop.hive.ql.QueryPlan; +import org.apache.hive.hcatalog.streaming.HiveEndPoint; +import org.apache.hive.test.capybara.annotations.AcidOn; +import org.apache.hive.test.capybara.annotations.MetadataOnly; +import org.apache.hive.test.capybara.annotations.SqlStdAuthOn; +import org.apache.hive.test.capybara.annotations.VectorOn; +import org.apache.hive.test.capybara.data.ResultCode; +import org.apache.hive.test.capybara.iface.Benchmark; +import org.apache.hive.test.capybara.infra.CapyEndPoint; +import org.apache.hive.test.capybara.iface.ClusterManager; +import org.apache.hive.test.capybara.data.DataSet; +import org.apache.hive.test.capybara.data.FetchResult; +import org.apache.hive.test.capybara.infra.HiveStore; +import org.apache.hive.test.capybara.infra.IntegrationRunner; +import org.apache.hive.test.capybara.infra.TestConf; +import org.apache.hive.test.capybara.infra.TestManager; +import org.apache.hive.test.capybara.iface.TestTable; +import org.junit.After; +import org.junit.AfterClass; +import org.junit.Assert; +import org.junit.Before; +import org.junit.BeforeClass; +import org.junit.Rule; +import org.junit.rules.TestName; +import org.junit.runner.RunWith; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.lang.annotation.Annotation; +import java.sql.SQLException; +import java.util.List; +import java.util.Map; + +/** + *

A super class for all integration tests. This class provides methods for running queries, + * generating expected results, comparing against expected results, etc.

+ * + *

The general expected flow is {@link #runQuery(String)} followed by {@link #compare}. In most + * cases the system can auto-generate the expected results. If auto-generation won't work you can + * instead use {@link #runHive} to run Hive and then get + * specifically tailored results by using {@link #runBenchmark} (for example, in + * the case where a feature is not supported by the Benchmark but can be produced by a different + * but semantically equivalent query). This is not generally required for syntax differences as + * the test framework can convert Hive SQL to ANSI SQL. You can also provide your own + * implementation of {@link org.apache.hive.test.capybara.iface.Benchmark} that will produce + * results that make sense for your test.

+ * + *

If your query requires any setting not handled in the general annotations you can set that + * using {@link #set}.

+ * + *

{@link #runQuery(String)}expects the query to return successfully. For testing negative + * queries use {@link #runQuery(String, org.apache.hive.test.capybara.data.ResultCode, Throwable)} which allows + * you to specify an expected result (success or failure) and potentially an expected exception.

+ * + *

You can explain a query using {@link #explain}. This will return an + * {@link org.apache.hive.test.capybara.Explain} object.

+ * + *

Users of this class should never directly create a + * {@link org.apache.hadoop.hive.conf.HiveConf} object. Special care has to be taken when + * creating these configuration objects as this class plays games with what is happening locally + * versus on the cluster, and these don't always play well with what the junit infrastructure is + * doing via maven. If you need a conf object call {@link #getConf}. + *

+ * + *

This class depends on being run by + * {@link org.apache.hive.test.capybara.infra.IntegrationRunner} to work properly. This is + * achieved with the @RunWith annotation. Don't override that.

+ */ +@RunWith(IntegrationRunner.class) +public abstract class IntegrationTest { + static final private Logger LOG = LoggerFactory.getLogger(IntegrationTest.class); + + // The config files that will be read from $HADOOP_HOME/conf + static final private String[] hadoopConfigFiles = {"core-site.xml", "hdfs-site.xml"}; + + private static TestManager testManager; + private static ClusterManager clusterManager; + // This is used to start up any mini-clusters. It is also used as a template to build new + // config files for each Hive instance. + private static Configuration baseConf; + + // This is the configuration used for a particular test. This will be re-created in the + // @Before method. + private HiveConf oneTestConf; + private HiveStore hive; + private Benchmark bench; + private FetchResult hiveResults; + private FetchResult benchmarkResults; + private Map> allAnnotations; + private boolean metadataOnly; + private String lastQuery; // Used to keep track of the last query sent to runQuery/runHive + + @Rule public TestName name = new TestName(); + + @BeforeClass + public static void initClass() throws IOException { + LOG.trace("Entering initClass"); + baseConf = getBaseConf(); + testManager = TestManager.getTestManager(); + testManager.setConf(baseConf); + // Start any necessary mini-clusters + clusterManager = testManager.getClusterManager(); + clusterManager.setup(); + LOG.trace("Leaving initClass"); + } + + @AfterClass + public static void teardownClass() { + LOG.trace("Entering teardownClass"); + // tear down any miniclusters we started + clusterManager.tearDown(); + LOG.trace("Leaving teardownClass"); + } + + private static HiveConf getBaseConf() { + HiveConf conf; + if (TestConf.onCluster()) { + LOG.debug("Choosing cluster for config file "); + // If we're on the cluster, we want our HiveConf to reflect the info for that cluster, not + // whatever is currently in our classpath. To make that happen we'll clear the contents of + // the configuration object and then force it to read exactly the files we want out of the + // HADOOP_HOME that has been passed to us. + String hadoopHome = System.getProperty("HADOOP_HOME"); + if (hadoopHome == null) { + throw new RuntimeException("You must define HADOOP_HOME to run on a cluster"); + } + String hadoopConf = hadoopHome + "/conf/"; + // Build a configuration that doesn't read the default resources. + Configuration base = new Configuration(false); + conf = new HiveConf(base, HiveConf.class); + for (String hadoopConfigFile : hadoopConfigFiles) { + Path p = new Path(hadoopConf + hadoopConfigFile); + conf.addResource(p); + } + } else { + LOG.debug("Choosing local for config file "); + conf = new HiveConf(); + } + return conf; + } + + @Before + public void initTest() throws SQLException, IOException { + LOG.trace("Entering initTest"); + metadataOnly = false; + + // Create a new configuration file that will be used for this test. + oneTestConf = new HiveConf(baseConf, HiveConf.class); + clusterManager.setConf(oneTestConf); + // Allow dynamic partitioning, as we need it. + set(HiveConf.ConfVars.DYNAMICPARTITIONINGMODE.varname, "nonstrict"); + + // Turn off strict mode, which stops certain queries. + set(HiveConf.ConfVars.HIVEMAPREDMODE.varname, "nonstrict"); + + // Set default file format to whatever has been chosen for this test, so we + // automatically get the right file format. + set(HiveConf.ConfVars.HIVEDEFAULTFILEFORMAT.varname, TestConf.fileFormat()); + + // This has to be done after we create a new config so that Hive get's the right + // config object. + hive = clusterManager.getHive(); + bench = testManager.getBenchmark(); + + // Handle any annotations that set values in the config file + List annotations = allAnnotations.get(name.getMethodName()); + if (annotations != null) { + for (Annotation annotation : annotations) { + // Find the values method and invoke it, then split the values to get what we should set. + if (annotation.annotationType().equals(AcidOn.class)) { + AcidOn ao = (AcidOn) annotation; + handleValueAnnotation(ao.values()); + } else if (annotation.annotationType().equals(SqlStdAuthOn.class)) { + SqlStdAuthOn ssao = (SqlStdAuthOn)annotation; + handleValueAnnotation(ssao.values()); + } else if (annotation.annotationType().equals(VectorOn.class)) { + VectorOn vo = (VectorOn)annotation; + handleValueAnnotation(vo.values()); + } else if (annotation.annotationType().equals(MetadataOnly.class)) { + metadataOnly = true; + } + } + } + + LOG.trace("Leaving initTest"); + } + + private void handleValueAnnotation(String[] values) throws SQLException, IOException { + Assert.assertEquals(0, values.length % 2); + for (int i = 0; i < values.length; i += 2) { + set(values[i], values[i + 1]); + } + + } + + @After + public void teardownTest() throws Exception { + LOG.trace("Entering teardownTest"); + clusterManager.unsetHive(); + testManager.resetBenchmark(); + LOG.trace("Leaving teardownTest"); + } + + /** + * Get a copy of the configuration object that is active for this test. + * @return conf + */ + protected HiveConf getConf() { + return new HiveConf(oneTestConf); + } + + /** + * Run a SQL query. This will be run against the configuration provided in the + * conf file. This will return an error if it does not succeed (that is, if the rc is not 0). + * @param sql SQL string to execute + * @throws SQLException + * @throws java.io.IOException + */ + protected void runQuery(String sql) throws SQLException, IOException { + runQuery(sql, ResultCode.SUCCESS, null, false); + } + + /** + * Run a SQL query. This will be run against the configuration provided in the conf file. + * @param sql SQL string to execute + * @param expectedResult expected result from running Hive. + * @param expectedException exception that it is expected the JDBC connection will throw. If + * this query is running in the CLI this value will be ignored. + * @throws SQLException + * @throws java.io.IOException + */ + protected void runQuery(String sql, final ResultCode expectedResult, + Throwable expectedException) throws SQLException, IOException { + runQuery(sql, expectedResult, expectedException, false); + } + + + private void runQuery(String sql, final ResultCode expectedResult, + Throwable expectedException, boolean hiveOnly) + throws SQLException, IOException { + lastQuery = sql; + try { + HiveRunner hiveRunner = new HiveRunner(sql, expectedException); + hiveRunner.start(); + if (!hiveOnly) { + // While Hive runs in another thread, run the benchmark. + runBenchmark(sql, expectedResult); + } + hiveRunner.join(); + if (expectedException != null && !hiveRunner.sawExpectedException) { + Assert.fail("Expected exception " + expectedException.getClass().getSimpleName() + " but " + + (hiveRunner.stashedException == null ? "got no exception " : " got " + + hiveRunner.stashedException.getClass().getSimpleName() + " instead.")); + } + // If an exception was thrown and we didn't expect it, go ahead and throw it on now. + if (hiveRunner.stashedException != null) { + if (SQLException.class.equals(hiveRunner.stashedException)) { + throw (SQLException)hiveRunner.stashedException; + } else if (IOException.class.equals(hiveRunner.stashedException)) { + throw (IOException)hiveRunner.stashedException; + } else { + throw new RuntimeException(hiveRunner.stashedException); + } + } + if (expectedResult != ResultCode.ANY) { + Assert.assertEquals("Unexpected fetch result", expectedResult, hiveResults.rc); + } + } catch (InterruptedException e) { + throw new RuntimeException(e); + } + } + + private class HiveRunner extends Thread { + Throwable stashedException; + boolean sawExpectedException = false; + final String sql; + final Throwable expectedException; + + HiveRunner(String s, Throwable ee) { + sql = s; + expectedException = ee; + } + + @Override + public void run() { + try { + hiveResults = hive.fetchData(sql); + } catch (Throwable e) { + if (e.equals(expectedException)) sawExpectedException = true; + else stashedException = e; + } + } + } + + /** + * Run a query only against Hive. Generally when you call this you need to call + * {@link #runBenchmark} yourself with different syntax. + * @param sql query to run + * @throws SQLException + * @throws IOException + */ + protected void runHive(String sql) throws SQLException, IOException { + runQuery(sql, ResultCode.SUCCESS, null, true); + } + + /** + * Fetch the query plan for the last query that was run. When running on the cluster this will + * return null, so your tests should handle that. (This is because it depends on being in the + * same memory space as the Hive client to work.) + * @return Explain object that contains the QueryPlan, or null when on the cluster. + * @throws IOException + * @throws SQLException + */ + protected Explain explain() throws IOException, SQLException { + if (lastQuery != null) return explain(lastQuery); + throw new RuntimeException("You must run a query before you explain it."); + } + + /** + * Fetch the query plan for a query. When running on the cluster this return null, so your + * tests should handle that. This is because it depends on being in the same memory space as + * the Hive client to work. Also on the cluster the client may not share the same + * configuration as the cluster and thus may not produce appropriate explain results. + * @param sql SQL string of the query + * @return Explain object that contains the QueryPlan. null will be returned if running on the + * cluster or the query has not been run. + * @throws SQLException + * @throws IOException + */ + protected Explain explain(String sql) throws IOException, SQLException { + LOG.debug("Going to explain query <" + sql + ">"); + QueryPlan plan = hive.explain(sql); + // plan may be null if we're on the cluster. + return plan == null ? null : new Explain(plan); + } + + /** + * Get results of the query. It is recommended that whenever possible you avoid using this and + * instead use {@link #compare}. For cases where comparing against a source of truth just + * won't work you can get back the {@link java.sql.ResultSet} and do the comparison yourself. + * @return resulting data set + */ + protected DataSet getResults() { + return hiveResults.data; + } + + /** + * Generate a benchmark for this data using the included SQL. This should only be used when + * the SQL has to be different than that passed to {@link #runQuery}. + * @param sql SQL to execute + * @param expectedResult expected result from running this query + */ + protected void runBenchmark(String sql, ResultCode expectedResult) + throws SQLException, IOException { + benchmarkResults = bench.getBenchDataStore().fetchData(sql); + if (expectedResult != ResultCode.ANY) { + Assert.assertEquals(expectedResult, benchmarkResults.rc); + } + } + + /** + * Sort and compare the results of running the query against Hive and the benchmark. If + * runQuery has not been invoked with a select query, this will result in an error. This + * expects the query to have returned results. If you expect the query to return no results + * call {@link #assertEmpty} instead. + */ + protected void sortAndCompare() throws SQLException, IOException { + compare(true); + } + + /** + * Compare the results of running the query against Hive and the benchmark without sorting. If + * runQuery has not been invoked with a select query, this will result in an error. This + * expects the query to have returned results. If you expect the query to return no results + * call {@link #assertEmpty} instead. + */ + protected void compare() throws SQLException, IOException { + compare(false); + } + + /** + * Compare the results of an insert query that creates a table in Hive and the benchmark. Note + * that this is not fast for small data sets because it has to go look at the data and decide + * whether to do the comparison locally or in the cluster. So if you can run the query as a + * select instead, or maybe even do the insert and then do a select yourself to compare on, it + * will be faster for small data sets. For large data sets (eg ETL queries) this will be much + * faster because it will do the comparison in the cluster. + * @param table table that data was inserted into. + * @throws IOException + * @throws SQLException + */ + protected void tableCompare(TestTable table) throws IOException, SQLException { + bench.getTableComparator().compare(hive, bench.getBenchDataStore(), table); + + } + + /** + * Check that the results of a query are empty. This does not check whether the query ran + * successfully, as that is controlled by how you call runQuery. + */ + protected void assertEmpty() { + Assert.assertNull("Expected results of query to be empty", hiveResults.data); + //LOG.debug("benchmark size = " + benchmarkResults.data.lengthInBytes()); + Assert.assertNull("Expected results of benchmark to be empty", benchmarkResults.data); + } + + private void compare(boolean sort) throws SQLException, IOException { + Assert.assertNotNull("Expected results of query to be non-empty", hiveResults.data); + Assert.assertNotNull("Expected results of benchmark to be non-empty", benchmarkResults.data); + if (hiveResults.data.getSchema() == null) { + // When Hive's working from the command line it doesn't know the schema of its output. To + // solve this cheat and grab the schema from the benchmark, so we know how to interpret the + // output. + hiveResults.data.setSchema(benchmarkResults.data.getSchema()); + } + bench.getResultComparator(sort).compare(hiveResults.data, benchmarkResults.data); + } + + /** + * Set a value in the configuration for this test. This value will not persist across tests. + * @param var variable to set + * @param val value to set it to + */ + protected void set(String var, String val) throws SQLException, IOException { + clusterManager.setConfVar(var, val); + } + + /** + * Set a value in the configuration for this test. This value will not persist across tests. + * the source of truth. + * @param var variable to set + * @param val value to set it to + */ + protected void set(String var, int val) throws SQLException, IOException { + set(var, Integer.toString(val)); + } + + /** + * Set a value in the configuration for this test. This value will not persist across tests. + * the source of truth. + * @param var variable to set + * @param val value to set it to + */ + protected void set(String var, boolean val) throws SQLException, IOException { + set(var, Boolean.toString(val)); + } + + /** + * Set a value in the configuration for this test. This value will not persist across tests. + * the source of truth. + * @param var variable to set + * @param val value to set it to + */ + protected void set(String var, double val) throws SQLException, IOException { + set(var, Double.toString(val)); + } + + /** + * Get a HiveEndPoint for streaming data too. To test Hive streaming you must call this rather + * than construct HiveEndPoint directly. This call gives a subclass of HiveEndPoint that + * splits the stream and passes it to both Hive and the benchmark. When you call + * @param testTable table to stream data to + * @param partVals partition values for this end point. + * @return a HiveEndPoint + */ + protected HiveEndPoint getHiveEndPoint(TestTable testTable, List partVals) { + return new CapyEndPoint(bench.getBenchDataStore(), testTable, getConf(), + hive.getMetastoreUri(), partVals); + } + + protected IntegrationTest() { + } + + public void phantomTest() { + // This does nothing. It is here as a place holder for the case where all tests are excluded + // because they've all been weeded out by the annotations. + LOG.debug("in phantomTest"); + } + + /** + * For use by {@link org.apache.hive.test.capybara.infra.IntegrationRunner} only. This sets + * the list so that we can determine values to set up for the test. + * @param annotations annotations for this test + */ + public void setAnnotations(Map> annotations) { + allAnnotations = annotations; + } + + @VisibleForTesting protected HiveConf getCurrentConf() { + return oneTestConf; + } +} diff --git itests/capybara/src/main/java/org/apache/hive/test/capybara/TableTool.java itests/capybara/src/main/java/org/apache/hive/test/capybara/TableTool.java new file mode 100644 index 0000000..4f3889a --- /dev/null +++ itests/capybara/src/main/java/org/apache/hive/test/capybara/TableTool.java @@ -0,0 +1,201 @@ +/** + * 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.hive.test.capybara; + +import org.apache.hive.test.capybara.iface.DataGenerator; +import org.apache.hive.test.capybara.data.DataSet; +import org.apache.hive.test.capybara.infra.RandomDataGenerator; +import org.apache.hive.test.capybara.infra.TestConf; +import org.apache.hive.test.capybara.iface.TestTable; +import org.apache.thrift.TException; + +import java.io.IOException; +import java.sql.SQLException; + +/** + * Standard tables used in integration tests. These tables tests are built to mimic the tests + * available in Hive's qfile infrastructure, though they are not 1-1. In particular the names + * have been changed because Hive now looks for those specific names to prevent dropping them. + */ +public class TableTool { + + /** + * Creates and populates the table alltypes. The schema of alltypes is + * (cvarchar varchar(120), cchar char(20), cstring string, cint int, cbigint bigint, csmallint + * smallint, ctinyint tinyint, cfloat float, cdouble double, cdecimal decimal(10,2), cdate + * date, ctimestamp timestamp, cboolean boolean). This is similar to qfiles' alltypesorc table. + * @throws TException + * @throws SQLException + */ + public static void createAllTypes() throws SQLException, IOException { + TestTable t = TestTable.getBuilder("alltypes") + .addCol("cvarchar", "varchar(120)") + .addCol("cchar", "char(20)") + .addCol("cstring", "string") + .addCol("cint", "int") + .addCol("cbigint", "bigint") + .addCol("csmallint", "smallint") + .addCol("ctinyint", "tinyint") + .addCol("cfloat", "float") + .addCol("cdouble", "double") + .addCol("cdecimal", "decimal(10,2)") + .addCol("cdate", "date") + .addCol("ctimestamp", "timestamp") + .addCol("cboolean", "boolean") + // .addCol("cbinary", "binary") Binary doesn't work with Derby yet. + .build(); + + DataGenerator generator = new RandomDataGenerator(1); + t.create(); + t.populate(generator); + } + + /** + * Creates and populates the table capysrc. This is equivalent to the qfile + * table src, but must be given a different name as Hive itself borks if you give it a table + * with that name. The schema is (k string, value string). + * @throws TException + * @throws SQLException + */ + public static void createCapySrc() throws SQLException, IOException { + TestTable t = TestTable.getBuilder("capysrc") + .addCol("k", "string") + .addCol("value", "string") + .build(); + + DataGenerator generator = new RandomDataGenerator(2); + t.create(); + t.populate(generator); + } + + /** + * Creates and populates the table capysrcpart. This is equilvalent to the + * qfile table srcpart, but must be given a different name as Hive itself borks if you try to + * create a table named srcpart. The schema is (k string, value string, ds string) where ds is + * the partition column. + * @throws SQLException + * @throws IOException + */ + public static void createCapySrcPart() throws SQLException, IOException { + TestTable t = TestTable.getBuilder("capysrcpart") + .addCol("k", "string") + .addCol("value", "string") + .addPartCol("ds", "string") + .addPartCol("hr", "string") + .setNumParts(partsFromScale()) + .build(); + + DataGenerator generator = new RandomDataGenerator(3); + t.create(); + t.populate(generator); + } + + /** + * Creates a populates a set of tables that are TPC-H like. (It is TPC-H like in that it is a + * very simplified version of TPC-H, with fewer tables, fewer columns per table, and only + * single columned primary keys and foreign keys. Also a few columns have been added or had + * their datatypes modified in order to cover more datatypes.) These have the following schemas + * and relationships: + * facttable: + * ph_lineitem (l_linenum bigint, l_orderkey bigint, l_partkey bigint, + * l_quantity integer, l_price decimal(10,2), l_shipdate date) + * primarykey: l_linenum + * foreignkeys: + * l_orderkey -> ph_order.o_orderkey + * l_partkey -> ph_part.p_partkey + * + * dimension tables: + * ph_customer (c_custkey bigint, c_name varchar(25), c_address varchar(100), + * c_acctbal decimal(20, 2)) + * primary key: c_custkey + * + * ph_order (o_orderkey bigint, o_custkey bigint, o_orderstatus char(1), o_orderdate timestamp, + * o_orderpriority tinyint) + * primary key: o_orderkey + * foreign keys: + * o_custkey -> ph_customer.c_custkey + * + * ph_part (p_partkey bigint, p_name varchar(55), p_size int, p_numinstock smallint, + * p_backordered boolean) + * primary key: p_partkey + */ + public static void createPseudoTpch() throws IOException, SQLException { + DataGenerator generator = new RandomDataGenerator('H'); + + int dimScale = getDimScale(); + + TestTable ph_part = TestTable.getBuilder("ph_part") + .addCol("p_partkey", "bigint") + .addCol("p_name", "varchar(55)") + .addCol("p_size", "int") + .addCol("p_numinstock", "smallint") + .addCol("p_backordered", "boolean") + .setPrimaryKey(new TestTable.Sequence(0)) + .build(); + ph_part.create(); + ph_part.setCacheData(true); + ph_part.populate(generator, dimScale, null); + DataSet partData = ph_part.getData(); + + TestTable ph_customer = TestTable.getBuilder("ph_customer") + .addCol("c_custkey", "bigint") + .addCol("c_name", "varchar(55)") + .addCol("c_address", "varchar(100)") + .addCol("c_acctbal", "decimal(10,2)") + .setPrimaryKey(new TestTable.Sequence(0)) + .build(); + ph_customer.create(); + ph_customer.populate(generator, dimScale, null); + DataSet customerData = ph_customer.getData(); + + TestTable ph_order = TestTable.getBuilder("ph_order") + .addCol("o_orderkey", "bigint") + .addCol("o_custkey", "bigint") + .addCol("o_orderstatus", "char(1)") + .addCol("o_orderdate", "timestamp") + .addCol("o_orderpriority", "tinyint") + .setPrimaryKey(new TestTable.Sequence(0)) + .addForeignKey(new TestTable.ForeignKey(customerData, 0, 1)) + .build(); + ph_order.create(); + ph_order.populate(generator, dimScale, null); + DataSet orderData = ph_order.getData(); + + TestTable ph_lineitem = TestTable.getBuilder("ph_lineitem") + .addCol("l_linenum", "bigint") + .addCol("l_orderkey", "bigint") + .addCol("l_partkey", "bigint") + .addCol("l_quantity", "int") + .addCol("l_price", "decimal(10,2)") + .addPartCol("l_shipdate", "date") + .setPrimaryKey(new TestTable.Sequence(0)) + .addForeignKey(new TestTable.ForeignKey(orderData, 0, 1)) + .addForeignKey(new TestTable.ForeignKey(partData, 0, 2)) + .build(); + ph_lineitem.create(); + ph_lineitem.populate(generator); + } + + private static int getDimScale() { + return TestConf.getScale() / 100 + 1; + } + + private static int partsFromScale() { + return Math.max(2, TestConf.getScale() / (1024 * 512)); + } +} diff --git itests/capybara/src/main/java/org/apache/hive/test/capybara/annotations/AcidOn.java itests/capybara/src/main/java/org/apache/hive/test/capybara/annotations/AcidOn.java new file mode 100644 index 0000000..aca9547 --- /dev/null +++ itests/capybara/src/main/java/org/apache/hive/test/capybara/annotations/AcidOn.java @@ -0,0 +1,34 @@ +/** + * 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.hive.test.capybara.annotations; + +import java.lang.annotation.Retention; +import java.lang.annotation.RetentionPolicy; + +/** + * Indicates that values should be set to turn on ACID for this test. + */ +@Retention(RetentionPolicy.RUNTIME) +public @interface AcidOn { + + public String[] values() default { + "hive.support.concurrency", "true", + "hive.txn.manager", "org.apache.hadoop.hive.ql.lockmgr.DbTxnManager", + "hive.enforce.bucketing", "true" + }; +} diff --git itests/capybara/src/main/java/org/apache/hive/test/capybara/annotations/AnnotationConfMap.java itests/capybara/src/main/java/org/apache/hive/test/capybara/annotations/AnnotationConfMap.java new file mode 100644 index 0000000..792b73e --- /dev/null +++ itests/capybara/src/main/java/org/apache/hive/test/capybara/annotations/AnnotationConfMap.java @@ -0,0 +1,62 @@ +/** + * 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.hive.test.capybara.annotations; + +import java.util.HashMap; +import java.util.Map; + +/** + * A helper class to map various annotations to the associated parameters. Ideally this would be + * done via a member in each annotation, but thanks to JDK-8013485 that doesn't work consistently. + */ +public class AnnotationConfMap { + + private static Map> settings; + + static { + settings = new HashMap<>(); + + // ACID properties + settings.put("AcidOn", mapMaker( + "hive.support.concurrency", "true", + "hive.txn.manager", "org.apache.hadoop.hive.ql.lockmgr.DbTxnManager", + "hive.enforce.bucketing", "true")); + + + // SQL Standard Auth properties + settings.put("SqlStdAuthOn", mapMaker( + "hive.test.authz.sstd.hs2.mode", "true", + "hive.security.authorization.manager", + "org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactoryForTest", + "hive.security.authenticator.manager", + "org.apache.hadoop.hive.ql.security.SessionStateConfigUserAuthenticator")); + + // Vectorization properties + settings.put("VectorOn", mapMaker( + "hive.vectorized.execution.enabled", "true")); + } + + private static Map mapMaker(String... properties) { + assert properties.length % 2 == 0; + Map map = new HashMap<>(properties.length / 2); + for (int i = 0; i < properties.length; i += 2) { + map.put(properties[i], properties[i + 1]); + } + return map; + } +} diff --git itests/capybara/src/main/java/org/apache/hive/test/capybara/annotations/MetadataOnly.java itests/capybara/src/main/java/org/apache/hive/test/capybara/annotations/MetadataOnly.java new file mode 100644 index 0000000..7fb19be --- /dev/null +++ itests/capybara/src/main/java/org/apache/hive/test/capybara/annotations/MetadataOnly.java @@ -0,0 +1,29 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hive.test.capybara.annotations; + +import java.lang.annotation.Retention; +import java.lang.annotation.RetentionPolicy; + +/** + * This test only does metadata commands, don't bother spinning up execution resources such as + * Tez or Spark. A metastore connection and an HDFS connection will still be established. + */ +@Retention(RetentionPolicy.RUNTIME) +public @interface MetadataOnly { +} diff --git itests/capybara/src/main/java/org/apache/hive/test/capybara/annotations/NoCli.java itests/capybara/src/main/java/org/apache/hive/test/capybara/annotations/NoCli.java new file mode 100644 index 0000000..2ae9b83 --- /dev/null +++ itests/capybara/src/main/java/org/apache/hive/test/capybara/annotations/NoCli.java @@ -0,0 +1,28 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hive.test.capybara.annotations; + +import java.lang.annotation.Retention; +import java.lang.annotation.RetentionPolicy; + +/** + * Tests annotated with this will not be run when testing via the CLI. + */ +@Retention(RetentionPolicy.RUNTIME) +public @interface NoCli { +} diff --git itests/capybara/src/main/java/org/apache/hive/test/capybara/annotations/NoHBaseMetastore.java itests/capybara/src/main/java/org/apache/hive/test/capybara/annotations/NoHBaseMetastore.java new file mode 100644 index 0000000..7ec3aea --- /dev/null +++ itests/capybara/src/main/java/org/apache/hive/test/capybara/annotations/NoHBaseMetastore.java @@ -0,0 +1,28 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hive.test.capybara.annotations; + +import java.lang.annotation.Retention; +import java.lang.annotation.RetentionPolicy; + +/** + * Tests annotated with this will not be run when testing with the HBaseMetastore. + */ +@Retention(RetentionPolicy.RUNTIME) +public @interface NoHBaseMetastore { +} diff --git itests/capybara/src/main/java/org/apache/hive/test/capybara/annotations/NoJdbc.java itests/capybara/src/main/java/org/apache/hive/test/capybara/annotations/NoJdbc.java new file mode 100644 index 0000000..779b1d3 --- /dev/null +++ itests/capybara/src/main/java/org/apache/hive/test/capybara/annotations/NoJdbc.java @@ -0,0 +1,28 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hive.test.capybara.annotations; + +import java.lang.annotation.Retention; +import java.lang.annotation.RetentionPolicy; + +/** + * Tests annotated with this will not be run when testing is being done via a JDBC connection. + */ +@Retention(RetentionPolicy.RUNTIME) +public @interface NoJdbc { +} diff --git itests/capybara/src/main/java/org/apache/hive/test/capybara/annotations/NoNonSecure.java itests/capybara/src/main/java/org/apache/hive/test/capybara/annotations/NoNonSecure.java new file mode 100644 index 0000000..ccaf398 --- /dev/null +++ itests/capybara/src/main/java/org/apache/hive/test/capybara/annotations/NoNonSecure.java @@ -0,0 +1,28 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hive.test.capybara.annotations; + +import java.lang.annotation.Retention; +import java.lang.annotation.RetentionPolicy; + +/** + * Tests annotated with this will not be run when testing in secure mode. + */ +@Retention(RetentionPolicy.RUNTIME) +public @interface NoNonSecure { +} diff --git itests/capybara/src/main/java/org/apache/hive/test/capybara/annotations/NoOrc.java itests/capybara/src/main/java/org/apache/hive/test/capybara/annotations/NoOrc.java new file mode 100644 index 0000000..26bde5d --- /dev/null +++ itests/capybara/src/main/java/org/apache/hive/test/capybara/annotations/NoOrc.java @@ -0,0 +1,28 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hive.test.capybara.annotations; + +import java.lang.annotation.Retention; +import java.lang.annotation.RetentionPolicy; + +/** + * Tests annotated with this will not be run when test data is stored in ORC. + */ +@Retention(RetentionPolicy.RUNTIME) +public @interface NoOrc { +} diff --git itests/capybara/src/main/java/org/apache/hive/test/capybara/annotations/NoParquet.java itests/capybara/src/main/java/org/apache/hive/test/capybara/annotations/NoParquet.java new file mode 100644 index 0000000..aa8ed50 --- /dev/null +++ itests/capybara/src/main/java/org/apache/hive/test/capybara/annotations/NoParquet.java @@ -0,0 +1,28 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hive.test.capybara.annotations; + +import java.lang.annotation.Retention; +import java.lang.annotation.RetentionPolicy; + +/** + * Tests with this annotation will not be run when test data is stored in Parquet. + */ +@Retention(RetentionPolicy.RUNTIME) +public @interface NoParquet { +} diff --git itests/capybara/src/main/java/org/apache/hive/test/capybara/annotations/NoRcFile.java itests/capybara/src/main/java/org/apache/hive/test/capybara/annotations/NoRcFile.java new file mode 100644 index 0000000..cb3c37b --- /dev/null +++ itests/capybara/src/main/java/org/apache/hive/test/capybara/annotations/NoRcFile.java @@ -0,0 +1,28 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hive.test.capybara.annotations; + +import java.lang.annotation.Retention; +import java.lang.annotation.RetentionPolicy; + +/** + * Tests with this annotation will not be run when test data is stored in RCFile. + */ +@Retention(RetentionPolicy.RUNTIME) +public @interface NoRcFile { +} diff --git itests/capybara/src/main/java/org/apache/hive/test/capybara/annotations/NoRdbmsMetastore.java itests/capybara/src/main/java/org/apache/hive/test/capybara/annotations/NoRdbmsMetastore.java new file mode 100644 index 0000000..9ec4b98 --- /dev/null +++ itests/capybara/src/main/java/org/apache/hive/test/capybara/annotations/NoRdbmsMetastore.java @@ -0,0 +1,28 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hive.test.capybara.annotations; + +import java.lang.annotation.Retention; +import java.lang.annotation.RetentionPolicy; + +/** + * Tests with this annotation will not be run when an RDBMS is being used for the metastore. + */ +@Retention(RetentionPolicy.RUNTIME) +public @interface NoRdbmsMetastore { +} diff --git itests/capybara/src/main/java/org/apache/hive/test/capybara/annotations/NoSecure.java itests/capybara/src/main/java/org/apache/hive/test/capybara/annotations/NoSecure.java new file mode 100644 index 0000000..9aa221b --- /dev/null +++ itests/capybara/src/main/java/org/apache/hive/test/capybara/annotations/NoSecure.java @@ -0,0 +1,28 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hive.test.capybara.annotations; + +import java.lang.annotation.Retention; +import java.lang.annotation.RetentionPolicy; + +/** + * Tests with this annotation will not be run when testing is being done in secure mode. + */ +@Retention(RetentionPolicy.RUNTIME) +public @interface NoSecure { +} diff --git itests/capybara/src/main/java/org/apache/hive/test/capybara/annotations/NoSpark.java itests/capybara/src/main/java/org/apache/hive/test/capybara/annotations/NoSpark.java new file mode 100644 index 0000000..eb0106f --- /dev/null +++ itests/capybara/src/main/java/org/apache/hive/test/capybara/annotations/NoSpark.java @@ -0,0 +1,28 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hive.test.capybara.annotations; + +import java.lang.annotation.Retention; +import java.lang.annotation.RetentionPolicy; + +/** + * Tests with this annotation will not be done when Spark is being used as the test engine. + */ +@Retention(value= RetentionPolicy.RUNTIME) +public @interface NoSpark { +} diff --git itests/capybara/src/main/java/org/apache/hive/test/capybara/annotations/NoTextFile.java itests/capybara/src/main/java/org/apache/hive/test/capybara/annotations/NoTextFile.java new file mode 100644 index 0000000..3737e07 --- /dev/null +++ itests/capybara/src/main/java/org/apache/hive/test/capybara/annotations/NoTextFile.java @@ -0,0 +1,28 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hive.test.capybara.annotations; + +import java.lang.annotation.Retention; +import java.lang.annotation.RetentionPolicy; + +/** + * Tests with this annotation will not be run when test data is stored in text file. + */ +@Retention(RetentionPolicy.RUNTIME) +public @interface NoTextFile { +} diff --git itests/capybara/src/main/java/org/apache/hive/test/capybara/annotations/NoTez.java itests/capybara/src/main/java/org/apache/hive/test/capybara/annotations/NoTez.java new file mode 100644 index 0000000..cf4c33e --- /dev/null +++ itests/capybara/src/main/java/org/apache/hive/test/capybara/annotations/NoTez.java @@ -0,0 +1,28 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hive.test.capybara.annotations; + +import java.lang.annotation.Retention; +import java.lang.annotation.RetentionPolicy; + +/** + * Tests with this annotation will not be run when Tez is being used as the execution engine. + */ +@Retention(RetentionPolicy.RUNTIME) +public @interface NoTez { +} diff --git itests/capybara/src/main/java/org/apache/hive/test/capybara/annotations/RequireCluster.java itests/capybara/src/main/java/org/apache/hive/test/capybara/annotations/RequireCluster.java new file mode 100644 index 0000000..5bbc099 --- /dev/null +++ itests/capybara/src/main/java/org/apache/hive/test/capybara/annotations/RequireCluster.java @@ -0,0 +1,28 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hive.test.capybara.annotations; + +import java.lang.annotation.Retention; +import java.lang.annotation.RetentionPolicy; + +/** + * Only run this test when connected to an external cluster. + */ +@Retention(RetentionPolicy.RUNTIME) +public @interface RequireCluster { +} diff --git itests/capybara/src/main/java/org/apache/hive/test/capybara/annotations/SqlStdAuthOn.java itests/capybara/src/main/java/org/apache/hive/test/capybara/annotations/SqlStdAuthOn.java new file mode 100644 index 0000000..ec89fd6 --- /dev/null +++ itests/capybara/src/main/java/org/apache/hive/test/capybara/annotations/SqlStdAuthOn.java @@ -0,0 +1,36 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hive.test.capybara.annotations; + +import java.lang.annotation.Retention; +import java.lang.annotation.RetentionPolicy; + +/** + * Set necessary values to run this test with SQL Standard Authorization + */ +@Retention(RetentionPolicy.RUNTIME) +public @interface SqlStdAuthOn { + + public String[] values() default { + "hive.test.authz.sstd.hs2.mode", "true", + "hive.security.authorization.manager", + "org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactoryForTest", + "hive.security.authenticator.manager", + "org.apache.hadoop.hive.ql.security.SessionStateConfigUserAuthenticator" + }; +} diff --git itests/capybara/src/main/java/org/apache/hive/test/capybara/annotations/VectorOn.java itests/capybara/src/main/java/org/apache/hive/test/capybara/annotations/VectorOn.java new file mode 100644 index 0000000..a288309 --- /dev/null +++ itests/capybara/src/main/java/org/apache/hive/test/capybara/annotations/VectorOn.java @@ -0,0 +1,32 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hive.test.capybara.annotations; + +import java.lang.annotation.Retention; +import java.lang.annotation.RetentionPolicy; + +/** + * Set necessary configuration values to turn on vectorization. + */ +@Retention(RetentionPolicy.RUNTIME) +public @interface VectorOn { + public String[] values() default { + "hive.vectorized.execution.enabled", "true" + }; + +} diff --git itests/capybara/src/main/java/org/apache/hive/test/capybara/data/BooleanColumn.java itests/capybara/src/main/java/org/apache/hive/test/capybara/data/BooleanColumn.java new file mode 100644 index 0000000..6a7b2e2 --- /dev/null +++ itests/capybara/src/main/java/org/apache/hive/test/capybara/data/BooleanColumn.java @@ -0,0 +1,67 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + *

+ * http://www.apache.org/licenses/LICENSE-2.0 + *

+ * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hive.test.capybara.data; + +import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector; +import org.apache.hadoop.hive.serde2.objectinspector.primitive.BooleanObjectInspector; + +import java.sql.PreparedStatement; +import java.sql.ResultSet; +import java.sql.SQLException; +import java.sql.Types; + +class BooleanColumn extends Column { + BooleanColumn(int colNum) { + super(colNum); + } + + @Override + public void load(PreparedStatement stmt) throws SQLException { + if (val == null) stmt.setNull(colNum, Types.BOOLEAN); + else stmt.setBoolean(colNum, (Boolean) val); + } + + @Override + public void fromString(String str, String nullIndicator) { + if (str.equals(nullIndicator)) val = null; + else val = Boolean.valueOf(str); + } + + @Override + public void fromResultSet(ResultSet rs) throws SQLException { + boolean hiveVal = rs.getBoolean(colNum); + if (rs.wasNull()) val = null; + else val = hiveVal; + } + + @Override + public void fromObject(ObjectInspector objectInspector, Object o) { + if (o == null) val = null; + else val = ((BooleanObjectInspector)objectInspector).get(o); + } + + @Override + public long length() { + return 1; + } + + @Override + public boolean asBoolean() { + return (Boolean)val; + } +} diff --git itests/capybara/src/main/java/org/apache/hive/test/capybara/data/ByteColumn.java itests/capybara/src/main/java/org/apache/hive/test/capybara/data/ByteColumn.java new file mode 100644 index 0000000..5d11943 --- /dev/null +++ itests/capybara/src/main/java/org/apache/hive/test/capybara/data/ByteColumn.java @@ -0,0 +1,67 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + *

+ * http://www.apache.org/licenses/LICENSE-2.0 + *

+ * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hive.test.capybara.data; + +import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector; +import org.apache.hadoop.hive.serde2.objectinspector.primitive.ByteObjectInspector; + +import java.sql.PreparedStatement; +import java.sql.ResultSet; +import java.sql.SQLException; +import java.sql.Types; + +class ByteColumn extends Column { + ByteColumn(int colNum) { + super(colNum); + } + + @Override + public void load(PreparedStatement stmt) throws SQLException { + if (val == null) stmt.setNull(colNum, Types.TINYINT); + else stmt.setByte(colNum, (Byte) val); + } + + @Override + public void fromString(String str, String nullIndicator) { + if (str.equals(nullIndicator)) val = null; + else val = Byte.valueOf(str); + } + + @Override + public void fromResultSet(ResultSet rs) throws SQLException { + byte hiveVal = rs.getByte(colNum); + if (rs.wasNull()) val = null; + else val = hiveVal; + } + + @Override + public void fromObject(ObjectInspector objectInspector, Object o) { + if (o == null) val = null; + else val = ((ByteObjectInspector)objectInspector).get(o); + } + + @Override + public long length() { + return Byte.SIZE / 8; + } + + @Override + public byte asByte() { + return (Byte)val; + } +} diff --git itests/capybara/src/main/java/org/apache/hive/test/capybara/data/BytesColumn.java itests/capybara/src/main/java/org/apache/hive/test/capybara/data/BytesColumn.java new file mode 100644 index 0000000..86c53c1 --- /dev/null +++ itests/capybara/src/main/java/org/apache/hive/test/capybara/data/BytesColumn.java @@ -0,0 +1,125 @@ +/** + * 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.hive.test.capybara.data; + +import org.apache.commons.codec.binary.Base64; +import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector; +import org.apache.hadoop.hive.serde2.objectinspector.primitive.BinaryObjectInspector; + +import java.sql.PreparedStatement; +import java.sql.ResultSet; +import java.sql.SQLException; +import java.sql.Types; +import java.util.Arrays; + +/** + * This one's a bit of a special case because byte[] isn't comparable. Also, we Base64 + * encode/decode when moving back and forth with String. If we don't, Hive has problems when + * we put the data in a text file because delimiters end up in the data. + */ +class BytesColumn extends Column { + byte[] bVal; + + BytesColumn(int colNum) { + super(colNum); + } + + @Override + public void load(PreparedStatement stmt) throws SQLException { + if (val == null) stmt.setNull(colNum, Types.BLOB); + else stmt.setBytes(colNum, bVal); + } + + @Override + public void fromString(String str, String nullIndicator) { + if (str.equals(nullIndicator)) bVal = null; + else bVal = Base64.decodeBase64(str); + } + + @Override + public void fromResultSet(ResultSet rs) throws SQLException { + byte[] hiveVal = rs.getBytes(colNum); + if (rs.wasNull()) bVal = null; + else bVal = hiveVal; + } + + @Override + public void fromObject(ObjectInspector objectInspector, Object o) { + if (o == null) bVal = null; + else bVal = ((BinaryObjectInspector)objectInspector).getPrimitiveJavaObject(o); + } + + @Override + public long length() { + if (bVal == null) return 0; + else return bVal.length; + } + + @Override + public byte[] asBytes() { + return bVal; + } + + @Override + public void set(Object val) { + if (val == null) bVal = null; + else if (val instanceof byte[]) bVal = (byte[])val; + else throw new RuntimeException("Attempt to set ByteColumn to non-byte[] value " + + val.getClass().getName()); + } + + @Override + public boolean isNull() { + return bVal == null; + } + + @Override + public int compareTo(Column o) { + if (o == null || !(o instanceof BytesColumn)) return 1; + BytesColumn that = (BytesColumn)o; + if (bVal == null && that.bVal == null) return 0; + if (bVal == null) return -1; + if (that.bVal == null) return 1; + for (int i = 0; i < bVal.length; i++) { + if (i >= that.bVal.length) return 1; + if (bVal[i] < that.bVal[i]) return -1; + if (bVal[i] > that.bVal[i]) return 1; + } + // We know the bytes are the same for the bytes they both have, but that might be longer + if (that.bVal.length > bVal.length) return -1; + else return 0; + } + + @Override + public boolean equals(Object o) { + if (o == null || !(o instanceof BytesColumn)) return false; + return Arrays.equals(bVal, ((BytesColumn) o).bVal); + } + + @Override + public int hashCode() { + if (bVal == null) return 0; + else return Arrays.hashCode(bVal); + } + + @Override + public String toString(String nullIndicator, String quotes) { + if (bVal == null) return nullIndicator; + else return Base64.encodeBase64URLSafeString(bVal); + } +} diff --git itests/capybara/src/main/java/org/apache/hive/test/capybara/data/CharColumn.java itests/capybara/src/main/java/org/apache/hive/test/capybara/data/CharColumn.java new file mode 100644 index 0000000..5de9d81 --- /dev/null +++ itests/capybara/src/main/java/org/apache/hive/test/capybara/data/CharColumn.java @@ -0,0 +1,34 @@ +/** + * 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. + */ // For the most part char and varchar can be handled identically to Strings. The one case +package org.apache.hive.test.capybara.data; + +import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector; +import org.apache.hadoop.hive.serde2.objectinspector.primitive.HiveCharObjectInspector; + +// where this isn't true is object inspectors. +class CharColumn extends StringColumn { + public CharColumn(int colNum) { + super(colNum); + } + + @Override + public void fromObject(ObjectInspector objectInspector, Object o) { + if (o == null) val = null; + else val = ((HiveCharObjectInspector)objectInspector).getPrimitiveJavaObject(o).toString(); + } +} diff --git itests/capybara/src/main/java/org/apache/hive/test/capybara/data/Column.java itests/capybara/src/main/java/org/apache/hive/test/capybara/data/Column.java new file mode 100644 index 0000000..ea9712e --- /dev/null +++ itests/capybara/src/main/java/org/apache/hive/test/capybara/data/Column.java @@ -0,0 +1,143 @@ +/** + * 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.hive.test.capybara.data; + +import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector; + +import java.math.BigDecimal; +import java.sql.Date; +import java.sql.PreparedStatement; +import java.sql.ResultSet; +import java.sql.SQLException; +import java.sql.Timestamp; + +/** + * A column value. There are implementations for each Java object type containing column data. + * This class provides methods for comparing, loading via JDBC, converting from JDBC result + * sets, parsing from a string, and turning into a string. For datatypes with differing + * requirements for equals (float, double) and for bytes (which needs different definitions of + * equals, sorting, hashing, and string parsing) this gives us a place to do it. + * + * Note that Columns themselves should + * never be null. Null in a column is represented by the contained val being null. + */ +public abstract class Column implements Comparable { + /** + * This column number is 1 based, since it is used for SQL calls. + */ + protected final int colNum; + protected Comparable val; + + protected Column(int cn) { + val = null; + colNum = cn; + } + + /** + * Load into a {@link java.sql.PreparedStatement}. + * @param stmt PreparedStatement to load column into + * @throws java.sql.SQLException + */ + public abstract void load(PreparedStatement stmt) throws SQLException; + + /** + * Parse this value from a string + * @param str string representation + * @param nullIndicator string value that indicates null + */ + public abstract void fromString(String str, String nullIndicator); + + /** + * Parse this value from a {@link java.sql.ResultSet}. + * @param rs ResultSet to parse the column from. + * @throws java.sql.SQLException + */ + public abstract void fromResultSet(ResultSet rs) throws SQLException; + + public abstract void fromObject(ObjectInspector objectInspector, Object o); + + /** + * Return the length of this column. For String based and binary columns this is the length of + * the actual data, not the max length possible (ie a varchar(32) columns with the value 'fred' + * will return 4, not 32). For fixed length columns it will return an estimate of the size of + * the column (ie 8 for long, 4 for int). + * @return length + */ + public abstract long length(); + + public void set(Object val) { + this.val = (Comparable)val; + } + + public Comparable get() { + return val; + } + + public boolean isNull() { + return val == null; + } + + @Override + public int compareTo(Column o) { + if (o == null) return 1; + if (val == null && o.val == null) return 0; + if (val == null) return -1; + if (o.val == null) return 1; + return val.compareTo(o.val); + } + + @Override + public boolean equals(Object o) { + if (o == null || !(o instanceof Column)) return false; + Column that = (Column)o; + if (val == null && that.val == null) return true; + else if (val == null || that.val == null) return false; + else return val.equals(that.val); + } + + @Override + public int hashCode() { + if (val == null) return 0; + else return val.hashCode(); + } + + public String toString(String nullIndicator, String quotes) { + if (val == null) return nullIndicator; + else return val.toString(); + } + + @Override + public String toString() { + return toString("NULL", ""); + } + + // For the most part I don't expect to need these, as hopefully we don't usually need to + // break things out by type. But they're useful for testing and a few other places. + public long asLong() { throw new UnsupportedOperationException("This is not a bigint column"); } + public int asInt() { throw new UnsupportedOperationException("This is not a int column"); } + public short asShort() { throw new UnsupportedOperationException("This is not a smallint column"); } + public byte asByte() { throw new UnsupportedOperationException("This is not a tinyint column"); } + public float asFloat() { throw new UnsupportedOperationException("This is not a float column"); } + public double asDouble() { throw new UnsupportedOperationException("This is not a double column"); } + public BigDecimal asBigDecimal() { throw new UnsupportedOperationException("This is not a decimal column"); } + public Date asDate() { throw new UnsupportedOperationException("This is not a date column"); } + public Timestamp asTimestamp() { throw new UnsupportedOperationException("This is not a timestamp column"); } + public String asString() { throw new UnsupportedOperationException("This is not a string column"); } + public boolean asBoolean() { throw new UnsupportedOperationException("This is not a boolean column"); } + public byte[] asBytes() { throw new UnsupportedOperationException("This is not a bytes column"); } +} diff --git itests/capybara/src/main/java/org/apache/hive/test/capybara/data/DataSet.java itests/capybara/src/main/java/org/apache/hive/test/capybara/data/DataSet.java new file mode 100644 index 0000000..a7590df --- /dev/null +++ itests/capybara/src/main/java/org/apache/hive/test/capybara/data/DataSet.java @@ -0,0 +1,178 @@ +/** + * 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.hive.test.capybara.data; + +import com.google.common.annotations.VisibleForTesting; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hive.metastore.api.FieldSchema; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.File; +import java.io.FileWriter; +import java.io.IOException; +import java.util.Iterator; +import java.util.List; + +/** + *

A data set. This class encapsulates a set of data. The data could be the result of a query + * against a {@link org.apache.hive.test.capybara.iface.DataStore} or generating data from a + * {@link org.apache.hive.test.capybara.iface.DataGenerator}.

+ * + *

Interally this class contains all the logic for handling rows, columns, and different + * datatypes. The intent is to factor all of the datatype specific code into this class so that + * other classes in capybara can be type agnostic. + *

+ */ +public abstract class DataSet implements Iterable { + static final private Logger LOG = LoggerFactory.getLogger(DataSet.class.getName()); + + static private int nextId = 1; + + protected List schema; + protected List rows; + // This value will only be set if the DataSet has been stored on the cluster. + private Path clusterLocation; + private transient int id = 0; // Unique identifier for this DataSet + + /** + * + * @param schema column schema for this dataset. + */ + protected DataSet(List schema) { + this.schema = schema; + } + + /** + * Get the schema for this data set. + * + * @return schema + */ + public List getSchema() { + return schema; + } + + /** + * Set the schema for a DataSet. This is necessary because we get some DataSets as a + * List<String> and thus we do not know the schema. + * @param schema the schema to set + */ + public void setSchema(List schema) { + this.schema = schema; + } + + /** + * Return the DataSet as a set of strings, one for each row. This is useful for printing to + * logs, etc. + * @param delimiter String to use to delimit columns + * @param nullIndicator String to write in for nulls. + * @param quotes String to use to quote String columns. + * @return iterator of Strings + */ + public Iterator stringIterator(final String delimiter, final String nullIndicator, + final String quotes) { + final Iterator outerIter = iterator(); + return new Iterator() { + @Override + public boolean hasNext() { + return outerIter.hasNext(); + } + + @Override + public String next() { + Row row = outerIter.next(); + return row == null ? null : row.toString(delimiter, nullIndicator, quotes); + } + + @Override + public void remove() { + } + }; + } + + /** + * Provide a Path where this data is stored. This will be the directory, there may be multiple + * files in the directory. A particular implemenation of DataSet may choose not to support + * writing itself out to HDFS. + * @param clusterLocation URI for this data. + */ + public void setClusterLocation(Path clusterLocation) { + this.clusterLocation = clusterLocation; + } + + /** + * Get the the Path where this is stored. If this is null, then the data is only in memory. + * @return storage location + */ + public Path getClusterLocation() { + return clusterLocation; + } + + /** + * Dump the contents of the DataSet into a local file in java.io.tmpdir. + * @param label A label to be used in the file name + * @return name of the file the data was dumped into. + */ + public String dumpToFile(String label) throws IOException { + String fileName = new StringBuilder(System.getProperty("java.io.tmpdir")) + .append(System.getProperty("file.separator")) + .append("capy_dataset_dump_") + .append(label) + .append('_') + .append(uniqueId()) + .toString(); + File file = new File(fileName); + FileWriter writer = new FileWriter(file); + Iterator iter = stringIterator(",", "NULL", "'"); + while (iter.hasNext()) writer.write(iter.next()); + writer.close(); + return fileName; + } + + /** + * Get a unique identifier for this DataSet. This is useful for classes handling multiple + * DataSets that need to be able to distinguish them (like in a Map). This identifier is only + * guaranteed to be unique within this JVM. Treat it as transient. + * @return a unique id for this DataSet + */ + public int uniqueId() { + if (id == 0) { + id = incrementId(); + } + return id; + } + + /** + * Get the total size of the dataset. This is for testing purposes and shouldn't be used in + * general. It can be expensive, as it forces an iteration through the data set. + * @return size of the dataset. + */ + @VisibleForTesting + public long lengthInBytes() { + long len = 0; + for (Row r : this) len += r.lengthInBytes(); + return len; + } + + private static synchronized int incrementId() { + return nextId++; + } + +} + + diff --git itests/capybara/src/main/java/org/apache/hive/test/capybara/data/DateColumn.java itests/capybara/src/main/java/org/apache/hive/test/capybara/data/DateColumn.java new file mode 100644 index 0000000..54aff36 --- /dev/null +++ itests/capybara/src/main/java/org/apache/hive/test/capybara/data/DateColumn.java @@ -0,0 +1,69 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + *

+ * http://www.apache.org/licenses/LICENSE-2.0 + *

+ * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hive.test.capybara.data; + +import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector; +import org.apache.hadoop.hive.serde2.objectinspector.primitive.DateObjectInspector; + +import java.sql.Date; +import java.sql.PreparedStatement; +import java.sql.ResultSet; +import java.sql.SQLException; +import java.sql.Types; + +class DateColumn extends Column { + DateColumn(int colNum) { + super(colNum); + } + + @Override + public void load(PreparedStatement stmt) throws SQLException { + if (val == null) stmt.setNull(colNum, Types.DATE); + else stmt.setDate(colNum, (Date) val); + } + + @Override + public void fromString(String str, String nullIndicator) { + if (str.equals(nullIndicator)) val = null; + else val = Date.valueOf(str); + } + + @Override + public void fromResultSet(ResultSet rs) throws SQLException { + Date hiveVal = rs.getDate(colNum); + if (rs.wasNull()) val = null; + else val = hiveVal; + } + + @Override + public void fromObject(ObjectInspector objectInspector, Object o) { + if (o == null) val = null; + else val = ((DateObjectInspector)objectInspector).getPrimitiveJavaObject(o); + } + + @Override + public long length() { + return Long.SIZE / 8; // Since we can express this as a long, that must be enough bytes to + // store it. + } + + @Override + public Date asDate() { + return (Date)val; + } +} diff --git itests/capybara/src/main/java/org/apache/hive/test/capybara/data/DecimalColumn.java itests/capybara/src/main/java/org/apache/hive/test/capybara/data/DecimalColumn.java new file mode 100644 index 0000000..2480a14 --- /dev/null +++ itests/capybara/src/main/java/org/apache/hive/test/capybara/data/DecimalColumn.java @@ -0,0 +1,98 @@ +/** + * 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.hive.test.capybara.data; + +import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector; +import org.apache.hadoop.hive.serde2.objectinspector.primitive.HiveDecimalObjectInspector; + +import java.math.BigDecimal; +import java.math.RoundingMode; +import java.sql.PreparedStatement; +import java.sql.ResultSet; +import java.sql.SQLException; +import java.sql.Types; + +class DecimalColumn extends Column { + DecimalColumn(int colNum) { + super(colNum); + } + + @Override + public void load(PreparedStatement stmt) throws SQLException { + if (val == null) stmt.setNull(colNum, Types.DECIMAL); + else stmt.setBigDecimal(colNum, (BigDecimal) val); + } + + @Override + public void fromString(String str, String nullIndicator) { + if (str.equals(nullIndicator)) val = null; + else val = new BigDecimal(str); + } + + @Override + public void fromResultSet(ResultSet rs) throws SQLException { + BigDecimal hiveVal = rs.getBigDecimal(colNum); + if (rs.wasNull()) val = null; + else val = hiveVal; + } + + @Override + public void fromObject(ObjectInspector objectInspector, Object o) { + if (o == null) val = null; + else val = ((HiveDecimalObjectInspector)objectInspector).getPrimitiveJavaObject(o).bigDecimalValue(); + } + + @Override + public long length() { + return val == null ? 0 : ((BigDecimal)val).unscaledValue().bitLength() / 8; + } + + @Override + public BigDecimal asBigDecimal() { + return (BigDecimal)val; + } + + @Override + public boolean equals(Object other) { + if (other == null || !(other instanceof DecimalColumn)) return false; + DecimalColumn that = (DecimalColumn)other; + if (val == null && that.val == null) return true; + else if (val == null || that.val == null) return false; + + // Handle the fact that values may have different scales, since Hive is sloppy about not + // appending trailing zeros. + BigDecimal thisVal = (BigDecimal)val; + BigDecimal thatVal = (BigDecimal)that.val; + if (thisVal.scale() == thatVal.scale()) { + return thisVal.equals(thatVal); + } else { + // TODO - I'm not sure this is the best choice. This picks the + // TODO - entry with the lowest scale, subtracts one, and compares both entries at that + // TODO - scale. (The subtraction of 1 is to handle rounding differences.) This handles + // TODO - the fact that Hive + // TODO - doesn't append trailing zeros and that for different expressions the two data + // TODO - stores may assign different scales (e.g. avg(decimal(10,2)) produces a + // TODO - decimal(10,6) in Hive and a decimal (10,4) in Derby.) But it will obscure + // TODO - cases where we'd like to check that scale is properly kept. + int newScale = Math.min(thisVal.scale(), thatVal.scale()) - 1; + BigDecimal newThisVal = thisVal.setScale(newScale, RoundingMode.FLOOR); + BigDecimal newThatVal = thatVal.setScale(newScale, RoundingMode.FLOOR); + return newThisVal.equals(newThatVal); + } + } +} diff --git itests/capybara/src/main/java/org/apache/hive/test/capybara/data/DoubleColumn.java itests/capybara/src/main/java/org/apache/hive/test/capybara/data/DoubleColumn.java new file mode 100644 index 0000000..1b4f876 --- /dev/null +++ itests/capybara/src/main/java/org/apache/hive/test/capybara/data/DoubleColumn.java @@ -0,0 +1,95 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + *

+ * http://www.apache.org/licenses/LICENSE-2.0 + *

+ * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hive.test.capybara.data; + +import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector; +import org.apache.hadoop.hive.serde2.objectinspector.primitive.DoubleObjectInspector; + +import java.sql.PreparedStatement; +import java.sql.ResultSet; +import java.sql.SQLException; +import java.sql.Types; + +class DoubleColumn extends Column { + DoubleColumn(int colNum) { + super(colNum); + } + + @Override + public void load(PreparedStatement stmt) throws SQLException { + if (val == null) stmt.setNull(colNum, Types.DOUBLE); + else stmt.setDouble(colNum, (Double) val); + } + + @Override + public void fromString(String str, String nullIndicator) { + if (str.equals(nullIndicator)) val = null; + else val = Double.valueOf(str); + } + + @Override + public void fromResultSet(ResultSet rs) throws SQLException { + double hiveVal = rs.getDouble(colNum); + if (rs.wasNull()) val = null; + else val = hiveVal; + } + + @Override + public void fromObject(ObjectInspector objectInspector, Object o) { + if (o == null) val = null; + else val = ((DoubleObjectInspector)objectInspector).get(o); + } + + @Override + public long length() { + return Double.SIZE / 8; + } + + @Override + public double asDouble() { + return (Double)val; + } + + @Override + public boolean equals(Object other) { + if (other == null || !(other instanceof DoubleColumn)) return false; + DoubleColumn that = (DoubleColumn)other; + if (val == null && that.val == null) return true; + else if (val == null || that.val == null) return false; + + // We want to be fuzzy in our comparisons, but just using a hard wired differential is hard + // because we don't know the scale. So look at the bits and mask out the last few, as + // these are where the difference is likely to be. + long thisBits = Double.doubleToLongBits((Double)val); + long thatBits = Double.doubleToLongBits((Double)that.val); + + // Make sure the sign is the same + if ((thisBits & 0x8000000000000000L) != (thatBits & 0x8000000000000000L)) return false; + // Check the exponent + if ((thisBits & 0x7ff0000000000000L) != (thatBits & 0x7ff0000000000000L)) return false; + // Check the mantissa, but leave off the last eight bits + return (thisBits & 0x000fffff00000000L) == (thatBits & 0x000fffff00000000L); + } + + @Override + public int compareTo(Column other) { + // Override this since we're playing a little fast and loose with equals + if (equals(other)) return 0; + else return super.compareTo(other); + } +} diff --git itests/capybara/src/main/java/org/apache/hive/test/capybara/data/FetchResult.java itests/capybara/src/main/java/org/apache/hive/test/capybara/data/FetchResult.java new file mode 100644 index 0000000..4d97d3f --- /dev/null +++ itests/capybara/src/main/java/org/apache/hive/test/capybara/data/FetchResult.java @@ -0,0 +1,58 @@ +/** + * 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.hive.test.capybara.data; + +/** + * Result of fetching data. This includes the + * {@link org.apache.hive.test.capybara.data.DataSet} and info on the result. If the result + * was a failure the data set will be null. + */ +public class FetchResult { + + ; + + final public DataSet data; + final public ResultCode rc; + + /** + * Create a FetchResult with SUCCESS as the ResultCode and a DataSet. + * @param d DataSet for this FetchResult. + */ + public FetchResult(DataSet d) { + data = d; + rc = ResultCode.SUCCESS; + } + + /** + * Create a FetchResult with no data. + * @param r ResultCode for this FetchResult. + */ + public FetchResult(ResultCode r) { + data = null; + rc = r; + } + + /** + * Determine whether this query returned results. Select queries will in general return + * results, while DML and DDL will not. + * @return true if this query has results. + */ + public boolean hasResults() { + return data != null; + } +} diff --git itests/capybara/src/main/java/org/apache/hive/test/capybara/data/FloatColumn.java itests/capybara/src/main/java/org/apache/hive/test/capybara/data/FloatColumn.java new file mode 100644 index 0000000..9f4c319 --- /dev/null +++ itests/capybara/src/main/java/org/apache/hive/test/capybara/data/FloatColumn.java @@ -0,0 +1,95 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + *

+ * http://www.apache.org/licenses/LICENSE-2.0 + *

+ * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hive.test.capybara.data; + +import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector; +import org.apache.hadoop.hive.serde2.objectinspector.primitive.FloatObjectInspector; + +import java.sql.PreparedStatement; +import java.sql.ResultSet; +import java.sql.SQLException; +import java.sql.Types; + +class FloatColumn extends Column { + FloatColumn(int colNum) { + super(colNum); + } + + @Override + public void load(PreparedStatement stmt) throws SQLException { + if (val == null) stmt.setNull(colNum, Types.FLOAT); + else stmt.setFloat(colNum, (Float) val); + } + + @Override + public void fromString(String str, String nullIndicator) { + if (str.equals(nullIndicator)) val = null; + else val = Float.valueOf(str); + } + + @Override + public void fromResultSet(ResultSet rs) throws SQLException { + float hiveVal = rs.getFloat(colNum); + if (rs.wasNull()) val = null; + else val = hiveVal; + } + + @Override + public void fromObject(ObjectInspector objectInspector, Object o) { + if (o == null) val = null; + else val = ((FloatObjectInspector)objectInspector).get(o); + } + + @Override + public long length() { + return Float.SIZE / 8; + } + + @Override + public float asFloat() { + return (Float)val; + } + + @Override + public boolean equals(Object other) { + if (other == null || !(other instanceof FloatColumn)) return false; + FloatColumn that = (FloatColumn)other; + if (val == null && that.val == null) return true; + else if (val == null || that.val == null) return false; + + // We want to be fuzzy in our comparisons, but just using a hard wired differential is hard + // because we don't know the scale. So look at the bits and mask out the last few, as + // these are where the difference is likely to be. + int thisBits = Float.floatToIntBits((Float)val); + int thatBits = Float.floatToIntBits((Float)that.val); + + // Make sure the sign is the same + if ((thisBits & 0x80000000) != (thatBits & 0x80000000)) return false; + // Check the exponent + if ((thisBits & 0x7f800000) != (thatBits & 0x7f800000)) return false; + // Check the mantissa, but leave off the last two bits + return (thisBits & 0x007fff00) == (thatBits & 0x007fff00); + } + + @Override + public int compareTo(Column other) { + // Override this since we're playing a little fast and loose with equals + if (equals(other)) return 0; + else return super.compareTo(other); + } +} diff --git itests/capybara/src/main/java/org/apache/hive/test/capybara/data/IntColumn.java itests/capybara/src/main/java/org/apache/hive/test/capybara/data/IntColumn.java new file mode 100644 index 0000000..4ce2ce9 --- /dev/null +++ itests/capybara/src/main/java/org/apache/hive/test/capybara/data/IntColumn.java @@ -0,0 +1,67 @@ +/** +* Licensed to the Apache Software Foundation (ASF) under one +* or more contributor license agreements. See the NOTICE file +* distributed with this work for additional information +* regarding copyright ownership. The ASF licenses this file +* to you under the Apache License, Version 2.0 (the +* "License"); you may not use this file except in compliance +* with the License. You may obtain a copy of the License at +*

+* http://www.apache.org/licenses/LICENSE-2.0 +*

+* Unless required by applicable law or agreed to in writing, software +* distributed under the License is distributed on an "AS IS" BASIS, +* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +* See the License for the specific language governing permissions and +* limitations under the License. +*/ +package org.apache.hive.test.capybara.data; + +import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector; +import org.apache.hadoop.hive.serde2.objectinspector.primitive.IntObjectInspector; + +import java.sql.PreparedStatement; +import java.sql.ResultSet; +import java.sql.SQLException; +import java.sql.Types; + +class IntColumn extends Column { + IntColumn(int colNum) { + super(colNum); + } + + @Override + public void load(PreparedStatement stmt) throws SQLException { + if (val == null) stmt.setNull(colNum, Types.INTEGER); + else stmt.setInt(colNum, (Integer) val); + } + + @Override + public void fromString(String str, String nullIndicator) { + if (str.equals(nullIndicator)) val = null; + else val = Integer.valueOf(str); + } + + @Override + public void fromResultSet(ResultSet rs) throws SQLException { + int hiveVal = rs.getInt(colNum); + if (rs.wasNull()) val = null; + else val = hiveVal; + } + + @Override + public void fromObject(ObjectInspector objectInspector, Object o) { + if (o == null) val = null; + else val = ((IntObjectInspector)objectInspector).get(o); + } + + @Override + public long length() { + return Integer.SIZE / 8; + } + + @Override + public int asInt() { + return (Integer)val; + } +} diff --git itests/capybara/src/main/java/org/apache/hive/test/capybara/data/LongColumn.java itests/capybara/src/main/java/org/apache/hive/test/capybara/data/LongColumn.java new file mode 100644 index 0000000..1d266c0 --- /dev/null +++ itests/capybara/src/main/java/org/apache/hive/test/capybara/data/LongColumn.java @@ -0,0 +1,68 @@ +/** +* 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.hive.test.capybara.data; + +import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector; +import org.apache.hadoop.hive.serde2.objectinspector.primitive.LongObjectInspector; + +import java.sql.PreparedStatement; +import java.sql.ResultSet; +import java.sql.SQLException; +import java.sql.Types; + + +class LongColumn extends Column { + LongColumn(int colNum) { + super(colNum); + } + + @Override + public void load(PreparedStatement stmt) throws SQLException { + if (val == null) stmt.setNull(colNum, Types.BIGINT); + else stmt.setLong(colNum, (Long)val); + } + + @Override + public void fromString(String str, String nullIndicator) { + if (str.equals(nullIndicator)) val = null; + else val = Long.valueOf(str); + } + + @Override + public void fromResultSet(ResultSet rs) throws SQLException { + long hiveVal = rs.getLong(colNum); + if (rs.wasNull()) val = null; + else val = hiveVal; + } + + @Override + public void fromObject(ObjectInspector objectInspector, Object o) { + if (o == null) val = null; + else val = ((LongObjectInspector)objectInspector).get(o); + } + + @Override + public long length() { + return Long.SIZE / 8; + } + + @Override + public long asLong() { + return (Long)val; + } +} diff --git itests/capybara/src/main/java/org/apache/hive/test/capybara/data/ResultCode.java itests/capybara/src/main/java/org/apache/hive/test/capybara/data/ResultCode.java new file mode 100644 index 0000000..56b83df --- /dev/null +++ itests/capybara/src/main/java/org/apache/hive/test/capybara/data/ResultCode.java @@ -0,0 +1,25 @@ +/** + * 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.hive.test.capybara.data; + +/** + * Result from Hive. This is done as an enum since values differ based on whether + * Hive is being accessed by command line or JDBC. ANY indicates that it's ok whether + * something passes or fails. + */ +public enum ResultCode { SUCCESS, RETRIABLE_FAILURE, NON_RETRIABLE_FAILURE, ANY } diff --git itests/capybara/src/main/java/org/apache/hive/test/capybara/data/Row.java itests/capybara/src/main/java/org/apache/hive/test/capybara/data/Row.java new file mode 100644 index 0000000..983750d --- /dev/null +++ itests/capybara/src/main/java/org/apache/hive/test/capybara/data/Row.java @@ -0,0 +1,121 @@ +/** + * 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.hive.test.capybara.data; + +import java.util.Iterator; +import java.util.List; + +/** + * A row of data. This is a list of Columns, along with methods for sorting, comparing, + * hashing, and turning a row to a string. + */ +public class Row implements Iterable, Comparable { + private List cols; + + Row(List c) { + cols = c; + } + + @Override + public Iterator iterator() { + return cols.iterator(); + } + + @Override + public int compareTo(Row o) { + for (int i = 0; i < cols.size(); i++) { + if (i > o.cols.size()) return 1; + Column thatCol = o.cols.get(i); + int rc = cols.get(i).compareTo(thatCol); + if (rc != 0) return rc; + } + // They may have been equal all along, but if there are more columns in the other row, + // declare it greater. + if (cols.size() < o.cols.size()) return -1; + else return 0; + } + + @Override + public String toString() { + return toString(",", "NULL", ""); + } + + /** + * Convert a row to a String, controlling how various aspects of the row are printed out. + * @param delimiter column delimiter + * @param nullIndicator value to return for null + * @param quotes quotes to use around string columns + * @return row as a String. + */ + public String toString(String delimiter, String nullIndicator, String quotes) { + StringBuilder builder = new StringBuilder(); + for (int i = 0; i < cols.size(); i++) { + if (i != 0) builder.append(delimiter); + builder.append(cols.get(i).toString(nullIndicator, quotes)); + } + return builder.toString(); + } + + /** + * Number of columns in this row. + * @return columns in row + */ + public int size() { + return cols.size(); + } + + /** + * Get the column. This is zero based even though column numbers are 1 based. + * @param i zero based column reference + * @return column + */ + public Column get(int i) { + return cols.get(i); + } + + /** + * Estimated length of this row. Note that this is a rough estimate on the number of bytes + * it takes to represent this in a binary format on disk. Each row takes + * significantly more space in memory due to container objects, etc. + * @return estimated length of the row. + */ + public int lengthInBytes() { + int len = 0; + for (Column col : cols) len += col.length(); + return len; + } + + @Override + public boolean equals(Object other) { + if (other == null || !(other instanceof Row)) return false; + return cols.equals(((Row)other).cols); + } + + @Override + public int hashCode() { + return cols.hashCode(); + } + + /** + * Append columns of another row to this row. + * @param that row to append + */ + public void append(Row that) { + cols.addAll(that.cols); + } +} diff --git itests/capybara/src/main/java/org/apache/hive/test/capybara/data/RowBuilder.java itests/capybara/src/main/java/org/apache/hive/test/capybara/data/RowBuilder.java new file mode 100644 index 0000000..a76c527 --- /dev/null +++ itests/capybara/src/main/java/org/apache/hive/test/capybara/data/RowBuilder.java @@ -0,0 +1,190 @@ +/** + * 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.hive.test.capybara.data; + +import org.apache.hadoop.hive.metastore.api.FieldSchema; + +import java.util.ArrayList; +import java.util.List; + +/** + * A class to build new rows. This understands the layout of the row it will build. It is + * useful because we can build the template once and then quickly generate new rows each time + * we need one. + */ +public class RowBuilder { + private final List builders; + private final int colOffset; // This is needed for partition columns, because their column number + // doesn't match their position in the schema that is passed. + + /** + * A constructor for regular columns. + * @param schema list of FieldSchemas that describe the row + */ + public RowBuilder(List schema) { + this(schema, 0); + } + + /** + * A constructor for building a RowBuilder for partition columns. + * @param schema list of FieldSchemas that describe the row + * @param offset number of non-partition columns in the table, used to figure out the column + * number for each partition column. + */ + public RowBuilder(List schema, int offset) { + builders = new ArrayList<>(schema.size()); + for (FieldSchema fs : schema) builders.add(getColBuilderFromFieldSchema(fs)); + colOffset = offset; + } + + public Row build() { + List cols = new ArrayList<>(builders.size()); + // Add 1 to the column number so that it works with SQL calls. + for (int i = 0; i < builders.size(); i++) cols.add(builders.get(i).build(i + colOffset + 1)); + return new Row(cols); + } + + /** + * Builders for individual column types. + */ + interface ColBuilder { + /** + * + * @param colNum 1 based reference to column number. + * @return A column instance of the appropriate type + */ + abstract Column build(int colNum); + } + + private static ColBuilder longColBuilder = new ColBuilder() { + @Override + public Column build(int colNum) { + return new LongColumn(colNum); + } + }; + + private static ColBuilder intColBuilder = new ColBuilder() { + @Override + public Column build(int colNum) { + return new IntColumn(colNum); + } + }; + + private static ColBuilder shortColBuilder = new ColBuilder() { + @Override + public Column build(int colNum) { + return new ShortColumn(colNum); + } + }; + + private static ColBuilder byteColBuilder = new ColBuilder() { + @Override + public Column build(int colNum) { + return new ByteColumn(colNum); + } + }; + + private static ColBuilder floatColBuilder = new ColBuilder() { + @Override + public Column build(int colNum) { + return new FloatColumn(colNum); + } + }; + + private static ColBuilder doubleColBuilder = new ColBuilder() { + @Override + public Column build(int colNum) { + return new DoubleColumn(colNum); + } + }; + + private static ColBuilder decimalColBuilder = new ColBuilder() { + @Override + public Column build(int colNum) { + return new DecimalColumn(colNum); + } + }; + + private static ColBuilder dateColBuilder = new ColBuilder() { + @Override + public Column build(int colNum) { + return new DateColumn(colNum); + } + }; + + private static ColBuilder timestampColBuilder = new ColBuilder() { + @Override + public Column build(int colNum) { + return new TimestampColumn(colNum); + } + }; + + private static ColBuilder stringColBuilder = new ColBuilder() { + @Override + public Column build(int colNum) { + return new StringColumn(colNum); + } + }; + + private static ColBuilder charColBuilder = new ColBuilder() { + @Override + public Column build(int colNum) { + return new CharColumn(colNum); + } + }; + + private static ColBuilder varcharColBuilder = new ColBuilder() { + @Override + public Column build(int colNum) { + return new VarcharColumn(colNum); + } + }; + + private static ColBuilder booleanColBuilder = new ColBuilder() { + @Override + public Column build(int colNum) { + return new BooleanColumn(colNum); + } + }; + + private static ColBuilder bytesColBuilder = new ColBuilder() { + @Override + public Column build(int colNum) { + return new BytesColumn(colNum); + } + }; + + private static ColBuilder getColBuilderFromFieldSchema(FieldSchema schema) { + String colType = schema.getType(); + if (colType.equalsIgnoreCase("bigint")) return longColBuilder; + else if (colType.toLowerCase().startsWith("int")) return intColBuilder; + else if (colType.equalsIgnoreCase("smallint")) return shortColBuilder; + else if (colType.equalsIgnoreCase("tinyint")) return byteColBuilder; + else if (colType.equalsIgnoreCase("float")) return floatColBuilder; + else if (colType.equalsIgnoreCase("double")) return doubleColBuilder; + else if (colType.toLowerCase().startsWith("decimal")) return decimalColBuilder; + else if (colType.equalsIgnoreCase("date")) return dateColBuilder; + else if (colType.equalsIgnoreCase("timestamp")) return timestampColBuilder; + else if (colType.equalsIgnoreCase("string")) return stringColBuilder; + else if (colType.toLowerCase().startsWith("char")) return charColBuilder; + else if (colType.toLowerCase().startsWith("varchar")) return varcharColBuilder; + else if (colType.equalsIgnoreCase("boolean")) return booleanColBuilder; + else if (colType.equalsIgnoreCase("binary")) return bytesColBuilder; + else throw new RuntimeException("Unknown column type " + colType); + } +} diff --git itests/capybara/src/main/java/org/apache/hive/test/capybara/data/ShortColumn.java itests/capybara/src/main/java/org/apache/hive/test/capybara/data/ShortColumn.java new file mode 100644 index 0000000..8b299a2 --- /dev/null +++ itests/capybara/src/main/java/org/apache/hive/test/capybara/data/ShortColumn.java @@ -0,0 +1,67 @@ +/** +* Licensed to the Apache Software Foundation (ASF) under one +* or more contributor license agreements. See the NOTICE file +* distributed with this work for additional information +* regarding copyright ownership. The ASF licenses this file +* to you under the Apache License, Version 2.0 (the +* "License"); you may not use this file except in compliance +* with the License. You may obtain a copy of the License at +*

+* http://www.apache.org/licenses/LICENSE-2.0 +*

+* Unless required by applicable law or agreed to in writing, software +* distributed under the License is distributed on an "AS IS" BASIS, +* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +* See the License for the specific language governing permissions and +* limitations under the License. +*/ +package org.apache.hive.test.capybara.data; + +import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector; +import org.apache.hadoop.hive.serde2.objectinspector.primitive.ShortObjectInspector; + +import java.sql.PreparedStatement; +import java.sql.ResultSet; +import java.sql.SQLException; +import java.sql.Types; + +class ShortColumn extends Column { + ShortColumn(int colNum) { + super(colNum); + } + + @Override + public void load(PreparedStatement stmt) throws SQLException { + if (val == null) stmt.setNull(colNum, Types.SMALLINT); + else stmt.setShort(colNum, (Short) val); + } + + @Override + public void fromString(String str, String nullIndicator) { + if (str.equals(nullIndicator)) val = null; + else val = Short.valueOf(str); + } + + @Override + public void fromResultSet(ResultSet rs) throws SQLException { + short hiveVal = rs.getShort(colNum); + if (rs.wasNull()) val = null; + else val = hiveVal; + } + + @Override + public void fromObject(ObjectInspector objectInspector, Object o) { + if (o == null) val = null; + else val = ((ShortObjectInspector)objectInspector).get(o); + } + + @Override + public long length() { + return Short.SIZE / 8; + } + + @Override + public short asShort() { + return (Short)val; + } +} diff --git itests/capybara/src/main/java/org/apache/hive/test/capybara/data/StringColumn.java itests/capybara/src/main/java/org/apache/hive/test/capybara/data/StringColumn.java new file mode 100644 index 0000000..9d219bc --- /dev/null +++ itests/capybara/src/main/java/org/apache/hive/test/capybara/data/StringColumn.java @@ -0,0 +1,77 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + *

+ * http://www.apache.org/licenses/LICENSE-2.0 + *

+ * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hive.test.capybara.data; + +import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector; +import org.apache.hadoop.hive.serde2.objectinspector.primitive.StringObjectInspector; + +import java.sql.PreparedStatement; +import java.sql.ResultSet; +import java.sql.SQLException; +import java.sql.Types; + +class StringColumn extends Column { + StringColumn(int colNum) { + super(colNum); + } + + @Override + public void load(PreparedStatement stmt) throws SQLException { + if (val == null) stmt.setNull(colNum, Types.VARCHAR); + else stmt.setString(colNum, (String) val); + } + + @Override + public void fromString(String str, String nullIndicator) { + if (str.equals(nullIndicator)) val = null; + else val = str; + } + + @Override + public void fromResultSet(ResultSet rs) throws SQLException { + String hiveVal = rs.getString(colNum); + if (rs.wasNull()) val = null; + else val = hiveVal; + } + + @Override + public void fromObject(ObjectInspector objectInspector, Object o) { + if (o == null) val = null; + else val = ((StringObjectInspector)objectInspector).getPrimitiveJavaObject(o); + } + + @Override + public long length() { + if (val == null) return 0; + else return ((String)val).length(); + } + + @Override + public String toString(String nullIndicator, String quotes) { + if (val == null) return nullIndicator; + return new StringBuilder(quotes) + .append(val.toString()) + .append(quotes) + .toString(); + } + + @Override + public String asString() { + return (String)val; + } +} diff --git itests/capybara/src/main/java/org/apache/hive/test/capybara/data/TimestampColumn.java itests/capybara/src/main/java/org/apache/hive/test/capybara/data/TimestampColumn.java new file mode 100644 index 0000000..8d3553d --- /dev/null +++ itests/capybara/src/main/java/org/apache/hive/test/capybara/data/TimestampColumn.java @@ -0,0 +1,68 @@ +/** + * 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.hive.test.capybara.data; + +import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector; +import org.apache.hadoop.hive.serde2.objectinspector.primitive.TimestampObjectInspector; + +import java.sql.PreparedStatement; +import java.sql.ResultSet; +import java.sql.SQLException; +import java.sql.Timestamp; +import java.sql.Types; + +class TimestampColumn extends Column { + TimestampColumn(int colNum) { + super(colNum); + } + + @Override + public void load(PreparedStatement stmt) throws SQLException { + if (val == null) stmt.setNull(colNum, Types.TIMESTAMP); + else stmt.setTimestamp(colNum, (Timestamp) val); + } + + @Override + public void fromString(String str, String nullIndicator) { + if (str.equals(nullIndicator)) val = null; + else val = Timestamp.valueOf(str); + } + + @Override + public void fromResultSet(ResultSet rs) throws SQLException { + Timestamp hiveVal = rs.getTimestamp(colNum); + if (rs.wasNull()) val = null; + else val = hiveVal; + } + + @Override + public void fromObject(ObjectInspector objectInspector, Object o) { + if (o == null) val = null; + else val = ((TimestampObjectInspector)objectInspector).getPrimitiveJavaObject(o); + } + + @Override + public long length() { + return Long.SIZE / 8; + } + + @Override + public Timestamp asTimestamp() { + return (Timestamp)val; + } +} diff --git itests/capybara/src/main/java/org/apache/hive/test/capybara/data/VarcharColumn.java itests/capybara/src/main/java/org/apache/hive/test/capybara/data/VarcharColumn.java new file mode 100644 index 0000000..5a329e6 --- /dev/null +++ itests/capybara/src/main/java/org/apache/hive/test/capybara/data/VarcharColumn.java @@ -0,0 +1,33 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + *

+ * http://www.apache.org/licenses/LICENSE-2.0 + *

+ * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hive.test.capybara.data; + +import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector; +import org.apache.hadoop.hive.serde2.objectinspector.primitive.HiveVarcharObjectInspector; + +class VarcharColumn extends StringColumn { + public VarcharColumn(int colNum) { + super(colNum); + } + + @Override + public void fromObject(ObjectInspector objectInspector, Object o) { + if (o == null) val = null; + else val = ((HiveVarcharObjectInspector)objectInspector).getPrimitiveJavaObject(o).toString(); + } +} diff --git itests/capybara/src/main/java/org/apache/hive/test/capybara/iface/Benchmark.java itests/capybara/src/main/java/org/apache/hive/test/capybara/iface/Benchmark.java new file mode 100644 index 0000000..da6467f --- /dev/null +++ itests/capybara/src/main/java/org/apache/hive/test/capybara/iface/Benchmark.java @@ -0,0 +1,52 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hive.test.capybara.iface; + +/** + * A benchmark for testing results against. This provides both the + * {@link DataStore} and the + * {@link ResultComparator} used to determine if a test was + * successful. Users can extend this, though in most cases one of the existing Benchmark + * implementations should be sufficient. + */ +public interface Benchmark { + + /** + * Return the DataStore to be used in this Benchmark. + * @return DataStore + */ + BenchmarkDataStore getBenchDataStore(); + + /** + * Get the ResultComparator to use with this Benchmark. + * @param sort Whether the result sets should be sorted as part of doing the comparison. If + * this is false it is assumed that the results are either already sorted or single + * valued. + * @return ResultComparator + */ + ResultComparator getResultComparator(boolean sort); + + /** + * Get a ResultComparator that will compare data already in a table. This is intended for use + * with insert statements. Note that it is generally slower for small data sets than + * {@link #getResultComparator(boolean)} but much faster for large ones since it can run in the + * cluster. + * @return TableComparator + */ + TableComparator getTableComparator(); +} diff --git itests/capybara/src/main/java/org/apache/hive/test/capybara/iface/BenchmarkDataStore.java itests/capybara/src/main/java/org/apache/hive/test/capybara/iface/BenchmarkDataStore.java new file mode 100644 index 0000000..c81cfcd --- /dev/null +++ itests/capybara/src/main/java/org/apache/hive/test/capybara/iface/BenchmarkDataStore.java @@ -0,0 +1,46 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + *

+ * http://www.apache.org/licenses/LICENSE-2.0 + *

+ * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hive.test.capybara.iface; + +import java.sql.Connection; +import java.sql.SQLException; + +/** + * Methods for benchmark data stores. These do not make sense for HiveStore to implement. This + * does not extend DataStore because all DataStores (including Hive) extend DataStoreBase. This + * interface is meant to be implemented by benchmark datastores in addition to extending + * DataStoreBase. + */ +public interface BenchmarkDataStore extends DataStore { + + /** + * Get the the class for the JDBC driver. This will be used to grab the associated jar and + * ship it to the cluster when needed. + * @return Driver class. + */ + Class getDriverClass(); + + /** + * Get a JDBC connection. The intended use for this is to write records into the benchmark + * that come from Hive streaming. + * @param autoCommit whether autoCommit should be set for this connection + * @return a JDBC connection. + * @throws java.sql.SQLException + */ + Connection getJdbcConnection(boolean autoCommit) throws SQLException; +} diff --git itests/capybara/src/main/java/org/apache/hive/test/capybara/iface/ClusterManager.java itests/capybara/src/main/java/org/apache/hive/test/capybara/iface/ClusterManager.java new file mode 100644 index 0000000..59051f5 --- /dev/null +++ itests/capybara/src/main/java/org/apache/hive/test/capybara/iface/ClusterManager.java @@ -0,0 +1,100 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hive.test.capybara.iface; + +import org.apache.hadoop.conf.Configurable; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hive.test.capybara.infra.HiveStore; + +import java.io.IOException; +import java.util.Map; + +/** + * Manage a connection to a cluster. This keeps track of all cluster oriented connections, such + * as DFS, Hive, the URL for HS2, etc. + */ +public interface ClusterManager extends Configurable { + + /** + * Prepare the cluster for testing. This will be called once at the beginning of a set of + * tests (in an @BeforeClass method). + */ + void setup() throws IOException; + + /** + * Tear down the cluster after testing. This will be called once at the end of a set of tests + * (in an @AfterClass method). + */ + void tearDown(); + + /** + * Indicate whether this is a remote cluster. + * @return true if it's remote, false if it's local using mini-clusters + */ + boolean remote(); + + /** + * Get the file system associated with this cluster. + * @return FileSystem handle + * @throws IOException + */ + FileSystem getFileSystem() throws IOException; + + /** + * Get a HiveStore that works for this cluster. + * @return connection to Hive + */ + HiveStore getHive(); + + /** + * Set the Hive connection to null. This is done after a test so that the next test doesn't + * inherit any state from the current one. This may be a NOP for some cluster managers. + */ + void unsetHive(); + + /** + * Get a JDBC URL to talk to Hive. If the access method is not set to "jdbc" the result of + * calling this method is undefined (i.e., it's likely to go up in flames on you). + * @return URL that can be used to connect to Hive. Note this is only the URL. The values to + * set for properties can be obtained by calling {@link #getConfVars}. + */ + String getJdbcURL(); + + /** + * Register that a table was created in a cluster. This is necessary because certain cluster + * types tear down the FS but leave the metastore (eg mini-clusters). To avoid issues we need + * to explicitly drop tables when we tear down the cluster. + * @param dbName name of the database the table is in, can be null if the table is in default + * @param tableName name of the table + */ + void registerTable(String dbName, String tableName); + + /** + * Set a configuration value that will be passed to Hive. How it is passed is up to the + * cluster manager. + * @param var variable to set + * @param val value to set it to. + */ + void setConfVar(String var, String val); + + /** + * Get the configuration variables set in this test. + * @return conf vars set. + */ + Map getConfVars(); +} diff --git itests/capybara/src/main/java/org/apache/hive/test/capybara/iface/DataGenerator.java itests/capybara/src/main/java/org/apache/hive/test/capybara/iface/DataGenerator.java new file mode 100644 index 0000000..8d2a1aa --- /dev/null +++ itests/capybara/src/main/java/org/apache/hive/test/capybara/iface/DataGenerator.java @@ -0,0 +1,76 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hive.test.capybara.iface; + +import org.apache.hive.test.capybara.data.DataSet; +import org.apache.hive.test.capybara.infra.TestConf; + +/** + * Generate data for use by Hive and benchmarks. + */ +public abstract class DataGenerator { + + /** + * Generate data. + * @param table table to generate for + * @param scale Scale of data to generate. The unit is kilobytes. As a general rule you + * should not set this as it prevents the system from scaling automatically. You + * may need it occasionally for dimension or fixed sized tables. + * @param pctNulls Percentage of values for each column that should be null. It is assumed + * there is an entry for every column in the table, not counting partition + * columns. Partition columns will not be given null values. Passing null will + * result in the default setting (1% of values per column) + * @return Data to be loaded. + */ + abstract public DataSet generateData(TestTable table, int scale, double[] pctNulls); + +/** + * Generate data. This is equivalent to calling {@link #generateData(TestTable,int,double[])} with + * scale fetched from {@link org.apache.hive.test.capybara.infra.TestConf#getScale} and pctNulls set + * to null. + * @param table table to generate for + * @return Data to be loaded. + */ + final public DataSet generateData(TestTable table) { + return generateData(table, TestConf.getScale(), null); + } + + /** + * Generate data. This is equivalent to calling {@link #generateData(TestTable,int,double[])} + * with pctNulls set to null. + * @param table table to generate for + * @param scale Scale of data to generate. The unit is kilobytes. As a general rule you + * should not set this as it prevents the system from scaling automatically. You + * may need it occasionally for dimension or fixed sized tables. + * @return Data to be loaded. + */ + final public DataSet generateData(TestTable table, int scale) { + return generateData(table, scale, null); + } + + /** + * Return a copy of the DataGenerator. The intent of this method is that it can be used by the + * {@link org.apache.hive.test.capybara.infra.ClusterDataGenerator} to make multiple copies of + * DataGenerator for use on the cluster. The default implementation throws an error. + * @param copyNum a unique number for this copy. + * @return A copy of the DataGenerator. + */ + protected DataGenerator copy(int copyNum) { + throw new UnsupportedOperationException("This DataGenerator does not support copy"); + } +} diff --git itests/capybara/src/main/java/org/apache/hive/test/capybara/iface/DataStore.java itests/capybara/src/main/java/org/apache/hive/test/capybara/iface/DataStore.java new file mode 100644 index 0000000..b536433 --- /dev/null +++ itests/capybara/src/main/java/org/apache/hive/test/capybara/iface/DataStore.java @@ -0,0 +1,96 @@ +/** + * 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.hive.test.capybara.iface; + +import org.apache.hive.test.capybara.data.DataSet; +import org.apache.hive.test.capybara.data.FetchResult; + +import java.io.IOException; +import java.sql.SQLException; + +/** + * Manage data stores for a test. This is used to manage Hive and benchmark stores. Classes that + * implement this interface are expected to be intelligent about whether a requested table already + * exists, and if so not re-create it. They are also expected to be intelligent about caching + * generated benchmarks and not re-creating them. Methods for doing this are provided in + * {@link org.apache.hive.test.capybara.infra.DataStoreBase}. Classes that implement + * DataStore will want to extend that rather than implement this interface directly. + */ +public interface DataStore { + + + /** + * Create a table. This is intended for use by common tables that will be used in many tests. + * If you are creating a small table for your single test or you want a table with special + * setting (like you want to control parameters or something) you should use + * {@link #fetchData} with a "create table" statement. + * @param table definition of table to create + * @return indicates whether the table was actually created, or if it already existed in the + * desired format. If true, the table was actually created. + * @throws SQLException + * @throws java.io.IOException + */ + boolean createTable(TestTable table) throws SQLException, IOException; + + /** + * Create a table, even if it already exists. If the table already exists it will be dropped + * and then re-created. Use this carefully as it removes the system's ability to re-use + * existing generated data. It should only be used for tables that will be constructed as part + * of the test rather than in prep for the test. + * @param table table to create. + * @throws SQLException + * @throws IOException + */ + void forceCreateTable(TestTable table) throws SQLException, IOException; + + /** + * Execute SQL against a the DataStore. This can be used for queries (as the name suggests) + * and for statements (which the name does not suggest). + * @param sql SQL to execute + * @return a FetchResult with a ResultCode and, if this was a query and it succeeded, a DataSet + * @throws SQLException anything thrown by the underlying implementaiton. + * @throws java.io.IOException + */ + FetchResult fetchData(String sql) throws SQLException, IOException; + + /** + * Load data into a data store. You should only call this if {@link #createTable} returned + * true. Otherwise you will double load data in the table. + * @param table Table to load the data into. + * @param rows data to be loaded into the table + * @throws SQLException anything thrown by the underlying implementation. + * @throws java.io.IOException + */ + void loadData(TestTable table, DataSet rows) throws SQLException, IOException; + + /** + * Dump a data set into a file in preparation for import to this DataStore. This method may be + * called more than once with the same DataSet. In that case it should append the new rows + * into the already existing file. + * @param rows DataSet to dump to a file. + * @throws IOException + */ + void dumpToFileForImport(DataSet rows) throws IOException; + + /** + * Get how this table is named in this store. Stores may choose to mangle table names in order + * to flatten out Hive's database structure. This method needs to return the mangled name. + * @return name of the table. + */ + String getTableName(TestTable table); +} diff --git itests/capybara/src/main/java/org/apache/hive/test/capybara/iface/ResultComparator.java itests/capybara/src/main/java/org/apache/hive/test/capybara/iface/ResultComparator.java new file mode 100644 index 0000000..feca12d --- /dev/null +++ itests/capybara/src/main/java/org/apache/hive/test/capybara/iface/ResultComparator.java @@ -0,0 +1,103 @@ +/** + * 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.hive.test.capybara.iface; + +import org.apache.hive.test.capybara.data.DataSet; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import org.junit.Assert; + +import java.io.IOException; +import java.sql.SQLException; + +/** + * Compare results from Hive and a benchmark. Any failures will result in a + * {@link java.lang.AssertionError}. If you expect a failure you should catch this error. + */ +public abstract class ResultComparator { + + public final static char DELIMITER = ','; + static final private Logger LOG = LoggerFactory.getLogger(ResultComparator.class.getName()); + + /** + * Compare two DataSets. The DataSets must have the same schemas (number of columns and type + * of columns, column names are ignored) and each DataSet must have the same data. Whether + * "same" includes sorted in the same order is implementation dependent. + * @param hiveResult DataSet from Hive. + * @param benchmarkResult DataSet from the benchmark. + * @throws SQLException + * @throws IOException + */ + public abstract void compare(DataSet hiveResult, DataSet benchmarkResult) throws SQLException, + IOException; + + /** + * Make sure the schemas of two results match. Match here is defined as having the same or + * compatible data types. Names of columns are not checked. + * @param hive DataSet from hive + * @param bench DataSet from the benchmark + * @throws java.sql.SQLException + */ + protected void compareSchemas(DataSet hive, DataSet bench) + throws SQLException { + Assert.assertEquals("Different number of columns", bench.getSchema().size(), + hive.getSchema().size()); + int numCols = hive.getSchema().size(); + for (int i = 0; i < numCols; i++) { + String hiveName = hive.getSchema().get(i).getType(); + String benchName = bench.getSchema().get(i).getType(); + Assert.assertTrue("Found discrepency in metadata at column " + i, + typeMatches(hiveName, benchName)); + } + } + + /** + * Determine whether types match. For types with lengths, precision, or scale, these values + * are ignored (ie varchar(32) == varchar(10) = true). All integer types (bigint, int, + * smallint, tinyint) are considered equal. All floating types (double, float) are considered + * equals. All string types (char, varchar, string) are considered equal. + * @param hiveType type name from hive + * @param benchType type name from benchmark + * @return true if equal, false otherwise + */ + static boolean typeMatches(String hiveType, String benchType) { + // Strip any length, scale, or precision off of the types. + int openParend = hiveType.indexOf('('); + if (openParend > -1) hiveType = hiveType.substring(0, openParend).toLowerCase(); + openParend = benchType.indexOf('('); + if (openParend > -1) benchType = benchType.substring(0, openParend).toLowerCase(); + + return hiveType.equals(benchType) || + isInteger(hiveType) && isInteger(benchType) || + isFloatingPoint(hiveType) && isFloatingPoint(benchType) || + isString(hiveType) && isString(benchType); + } + + private static boolean isInteger(String type) { + return type.equals("bigint") || type.equals("int") || type.equals("smallint") || + type.equals("tinyint"); + } + + private static boolean isFloatingPoint(String type) { + return type.equals("float") || type.equals("double"); + } + + private static boolean isString(String type) { + return type.equals("string") || type.equals("char") || type.equals("varchar"); + } +} diff --git itests/capybara/src/main/java/org/apache/hive/test/capybara/iface/TableComparator.java itests/capybara/src/main/java/org/apache/hive/test/capybara/iface/TableComparator.java new file mode 100644 index 0000000..5dd9a2a --- /dev/null +++ itests/capybara/src/main/java/org/apache/hive/test/capybara/iface/TableComparator.java @@ -0,0 +1,352 @@ +/** + * 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.hive.test.capybara.iface; + +import org.apache.hive.test.capybara.data.ResultCode; +import org.apache.hive.test.capybara.data.Row; +import org.apache.hive.test.capybara.infra.ClusterDataGenerator; +import org.apache.hive.test.capybara.data.FetchResult; +import org.apache.hive.test.capybara.infra.HiveStore; +import org.apache.hive.test.capybara.infra.NonSortingComparator; +import org.apache.hive.test.capybara.infra.TestConf; +import org.apache.hive.test.capybara.infra.TestManager; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.filecache.DistributedCache; +import org.apache.hadoop.fs.FSDataInputStream; +import org.apache.hadoop.fs.FSDataOutputStream; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hive.metastore.api.FieldSchema; +import org.apache.hadoop.io.LongWritable; +import org.apache.hadoop.io.NullWritable; +import org.apache.hadoop.io.Text; +import org.apache.hadoop.mapred.JobClient; +import org.apache.hadoop.mapred.JobConf; +import org.apache.hadoop.mapred.Mapper; +import org.apache.hadoop.mapred.OutputCollector; +import org.apache.hadoop.mapred.Reporter; +import org.apache.hadoop.mapred.RunningJob; +import org.apache.hadoop.mapred.TextOutputFormat; +import org.apache.hadoop.mapred.lib.NLineInputFormat; +import org.apache.hadoop.util.StringUtils; +import org.junit.Assert; + +import java.io.BufferedReader; +import java.io.IOException; +import java.io.InputStreamReader; +import java.sql.SQLException; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.HashMap; +import java.util.Iterator; +import java.util.List; +import java.util.Map; + +/** + * A comparator that works on tables in the cluster (and the benchmark). This is intended + * for use when the tests are ETL operations that result in tables on the cluster rather than + * select results. If the table size is small then the results will be pulled back and compared + * on the local machine. If they are large then they'll be compared in the cluster. Even in the + * former case this isn't fast because the system has to run a query to determine result size. + */ +public class TableComparator { + private static final Logger LOG = LoggerFactory.getLogger(ClusterDataGenerator.class); + private static final String FIELD_SEP = ":"; + private static final String NTILE_COL = "capyntile"; + private static final String OK = "OK RICK"; + + /** + * Compare tables in Hive and the benchmark. {@link java.lang.AssertionError} will be thrown + * if the comparison fails. + * @param hive HiveStore that contains the table. + * @param bench Benchmark that contains the table. + * @param table test table to compare. + * @throws IOException + * @throws SQLException + */ + public void compare(HiveStore hive, BenchmarkDataStore bench, TestTable table) + throws IOException, SQLException { + if (!TestConf.onCluster()) { + localCompare(hive, bench, table); + return; + } + + // Step 1, get an estimated size + FetchResult cnt = hive.fetchData("select count(*) from " + table.getFullName()); + if (cnt.rc != ResultCode.SUCCESS) { + throw new RuntimeException("Failed to count rows in " + table.getFullName()); + } + cnt.data.setSchema(Arrays.asList(new FieldSchema("col1", "bigint", ""))); + Iterator iter = cnt.data.iterator(); + assert iter.hasNext(); + long rowCnt = iter.next().get(0).asLong(); + + FetchResult lmt = hive.fetchData("select * from " + table.getFullName() + " limit 100"); + if (lmt.rc != ResultCode.SUCCESS) { + throw new RuntimeException("Failed to fetch first 100 rows in " + table.getFullName()); + } + lmt.data.setSchema(table.getCombinedSchema()); + Iterator strIter = lmt.data.stringIterator(",", "N", "\""); + int sum = 0; + while (strIter.hasNext()) sum += strIter.next().length(); + int avg = sum / 100; + + long size = rowCnt * avg; + int parallelism = (int) (2 * size * 1024 / TestConf.getClusterGenThreshold()); + + // If parallelism turns out to be less than or equal to 1, do this locally as two separate selects. Otherwise + // we'll do it in the cluster + if (parallelism <= 1) { + localCompare(hive, bench, table); + } else { + clusterCompare(hive, bench, table, parallelism); + } + } + + private void localCompare(HiveStore hive, DataStore bench, TestTable table) + throws IOException, SQLException { + List allCols = table.getCombinedSchema(); + StringBuilder colList = new StringBuilder(); + boolean first = true; + for (FieldSchema schema : allCols) { + if (first) first = false; + else colList.append(", "); + colList.append(schema.getName()); + } + StringBuilder sql = new StringBuilder("select ") + .append(colList) + .append(" from ") + .append(table.getFullName()) + .append(" order by ") + .append(colList); + + FetchResult hiveResult = hive.fetchData(sql.toString()); + Assert.assertEquals(ResultCode.SUCCESS, hiveResult.rc); + FetchResult benchResult = bench.fetchData(sql.toString()); + Assert.assertEquals(ResultCode.SUCCESS, benchResult.rc); + if (hiveResult.data.getSchema() == null) { + hiveResult.data.setSchema(benchResult.data.getSchema()); + } + ResultComparator comparator = new NonSortingComparator(); + comparator.compare(hiveResult.data, benchResult.data); + } + + private void clusterCompare(HiveStore hive, BenchmarkDataStore bench, TestTable table, + int parallelism) throws IOException, SQLException { + // Step 2 pick a set of columns to divide the table up by. We want columns likely to have a + // wide distribution of values. Thus we'll avoid boolean and tinyint. We will also avoid + // floating point because who knows how that plays across different platforms. Also, don't + // use partition columns as those will tend to have a limited set of values. + List orderByCols = new ArrayList<>(3); + + for (int i = 0; i < table.getCols().size() && orderByCols.size() < 3; i++) { + FieldSchema col = table.getCols().get(i); + if (col.getType().equalsIgnoreCase("tinyint") || + col.getType().equalsIgnoreCase("float") || + col.getType().equalsIgnoreCase("double") || + col.getType().equalsIgnoreCase("boolean") || + col.getType().equalsIgnoreCase("binary")) { + continue; + } + orderByCols.add(col.getName()); + } + // If we found less than the desired number, oh well. If we found none, then just take the + // first column and hope for the best. + if (orderByCols.size() == 0) orderByCols.add(table.getCols().get(0).getName()); + + // Step 3, create a table similar to our existing table but partitioned by an ntile column. + // Use TestTable to do this rather than doing it directly as it already has facilities for + // handling partitioning. + List allCols = table.getCombinedSchema(); + String comparisonTableName = table.getTableName() + "_comparison"; + TestTable comparisonTable = TestTable.getBuilder(comparisonTableName) + .setCols(allCols) + .addPartCol(NTILE_COL, "int") + .build(); + hive.forceCreateTable(comparisonTable); + bench.forceCreateTable(comparisonTable); + + StringBuilder sql = new StringBuilder("insert into ") + .append(comparisonTableName) + .append(" partition (") + .append(NTILE_COL) + .append(") select "); + for (FieldSchema col : allCols) { + sql.append(col.getName()) + .append(", "); + } + sql.append("ntile(") + .append(parallelism) + .append(") over (order by "); + boolean first = true; + for (String orderByCol : orderByCols) { + if (first) first = false; + else sql.append(", "); + sql.append(orderByCol); + } + sql.append(") from ") + .append(table.getFullName()); + FetchResult hiveResult = hive.fetchData(sql.toString()); + Assert.assertEquals(ResultCode.SUCCESS, hiveResult.rc); + FetchResult benchResult = bench.fetchData(sql.toString()); + Assert.assertEquals(ResultCode.SUCCESS, benchResult.rc); + + // Step 4, create an MR job that will do the comparison in pieces, with each task comparing + // one partition. The format of the file is one line per mapper each with four fields: + // 1) which partition to compare; + // 2) the class of HiveStore to use + // 3) the class of DataStore to use for bench. + // 4) the list of columns in the table + // 5) the name of the table + Configuration conf = TestManager.getTestManager().getConf(); + FileSystem fs = TestManager.getTestManager().getClusterManager().getFileSystem(); + Path inputFile = new Path(HiveStore.getDirForDumpFile()); + FSDataOutputStream out = fs.create(inputFile); + for (int i = 0; i < parallelism; i++) { + out.writeBytes(Integer.toString(i + 1)); // add 1 because SQL counts from 1 + out.writeBytes(FIELD_SEP); + out.writeBytes(hive.getClass().getName()); + out.writeBytes(FIELD_SEP); + out.writeBytes(bench.getClass().getName()); + out.writeBytes(FIELD_SEP); + first = true; + for (FieldSchema col : allCols) { + if (first) first = false; + else sql.append(", "); + sql.append(col.getName()); + } + out.writeBytes(FIELD_SEP); + out.writeBytes(table.getFullName()); + out.writeBytes(System.getProperty("line.separator")); + } + out.close(); + + Path outputFile = new Path(HiveStore.getDirForDumpFile()); + + JobConf job = new JobConf(conf); + job.setJobName("Capybara data compare for " + table.getTableName()); + job.setOutputKeyClass(NullWritable.class); + job.setOutputValueClass(Text.class); + job.setJarByClass(TableComparator.class); + job.setMapperClass(TableComparatorMapper.class); + job.setNumReduceTasks(0); + job.setInputFormat(NLineInputFormat.class); + NLineInputFormat.addInputPath(job, inputFile); + job.setOutputFormat(TextOutputFormat.class); + TextOutputFormat.setOutputPath(job, outputFile); + + // Put the benchmark's JDBC driver jar into the cache so it will available on the cluster. + String jar = + bench.getDriverClass().getProtectionDomain().getCodeSource().getLocation().getPath(); + Path hdfsJar = new Path(HiveStore.getDirForDumpFile() + "/" + "jdbc_driver.jar"); + fs.copyFromLocalFile(new Path(jar), hdfsJar); + // Add jar to distributed classPath + DistributedCache.addFileToClassPath(hdfsJar, job); + + RunningJob runningJob = JobClient.runJob(job); + LOG.debug("Submitted comparison job " + job.getJobName()); + runningJob.waitForCompletion(); + + // Now, read the resulting output file and see how it went. + FSDataInputStream input = fs.open(outputFile); + BufferedReader in = new BufferedReader(new InputStreamReader(input)); + int linenum = 0; + Map errors = new HashMap<>(); + String line; + while ((line = in.readLine()) != null) { + linenum++; + if (OK.equals(line)) continue; + errors.put(linenum, line); + } + Assert.assertTrue(buildErrorMessage(errors), errors.isEmpty()); + } + + private String buildErrorMessage(Map errors) { + StringBuilder msg = new StringBuilder("Found errors: "); + for (Map.Entry error : errors.entrySet()) { + msg.append("partition number ") + .append(error.getKey()) + .append(" message: <") + .append(error.getValue()) + .append(">") + .append(System.getProperty("line.separator")); + } + return msg.toString(); + } + + static class TableComparatorMapper implements Mapper { + private JobConf job; + + @Override + public void map(LongWritable longWritable, Text text, + OutputCollector outputCollector, Reporter reporter) + throws IOException { + String[] line = text.toString().split(FIELD_SEP); + assert line.length == 5; + + // Construct the appropriate classes. + try { + Class hiveClass = Class.forName(line[1]); + HiveStore hive = (HiveStore)hiveClass.newInstance(); + hive.setConf(job); + + Class benchClass = Class.forName(line[2]); + DataStore bench = (DataStore)benchClass.newInstance(); + + StringBuilder sql = new StringBuilder("select ") + .append(line[3]) + .append(" from ") + .append(line[4]) + .append(" where ") + .append(NTILE_COL) + .append(" = ") + .append(line[0]) + .append(" order by ") + .append(line[3]); + + FetchResult hiveResult = hive.fetchData(sql.toString()); + Assert.assertEquals(ResultCode.SUCCESS, hiveResult.rc); + FetchResult benchResult = bench.fetchData(sql.toString()); + Assert.assertEquals(ResultCode.SUCCESS, benchResult.rc); + if (hiveResult.data.getSchema() == null) { + hiveResult.data.setSchema(benchResult.data.getSchema()); + } + ResultComparator comparator = new NonSortingComparator(); + comparator.compare(hiveResult.data, benchResult.data); + // If the comparator asserts, then the assertion will get recorded. + outputCollector.collect(NullWritable.get(), new Text(OK)); + + } catch (Throwable t) { + outputCollector.collect(NullWritable.get(), new Text(StringUtils.stringifyException(t))); + } + + } + + @Override + public void close() throws IOException { + + } + + @Override + public void configure(JobConf jobConf) { + job = jobConf; + } + } +} diff --git itests/capybara/src/main/java/org/apache/hive/test/capybara/iface/TestTable.java itests/capybara/src/main/java/org/apache/hive/test/capybara/iface/TestTable.java new file mode 100644 index 0000000..bd57ee4 --- /dev/null +++ itests/capybara/src/main/java/org/apache/hive/test/capybara/iface/TestTable.java @@ -0,0 +1,624 @@ +/** + * 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.hive.test.capybara.iface; + +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hive.metastore.IMetaStoreClient; +import org.apache.hadoop.hive.metastore.api.FieldSchema; +import org.apache.hadoop.hive.metastore.api.Table; +import org.apache.hadoop.hive.ql.parse.SemanticAnalyzer; +import org.apache.hadoop.hive.ql.session.SessionState; +import org.apache.hive.test.capybara.data.DataSet; +import org.apache.hive.test.capybara.data.ResultCode; +import org.apache.hive.test.capybara.data.Row; +import org.apache.hive.test.capybara.data.FetchResult; +import org.apache.hive.test.capybara.infra.HiveStore; +import org.apache.hive.test.capybara.infra.TestConf; +import org.apache.hive.test.capybara.infra.TestManager; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.io.Serializable; +import java.sql.Connection; +import java.sql.PreparedStatement; +import java.sql.SQLException; +import java.util.ArrayList; +import java.util.List; + +/** + * A table to use in testing. This table may or may not yet exist in Hive or the benchmark. + */ +public class TestTable implements Serializable { + + static final private Logger LOG = LoggerFactory.getLogger(TestTable.class.getName()); + + private final String dbName; + private final String tableName; + private final List cols; + private final List partCols; + private final PrimaryKey primaryKey; + private final List foreignKeys; + private final boolean isAcid; + private final int numBuckets; + private final String[] bucketCols; + private List partVals; + private int numParts; + private boolean hiveCreated, benchCreated; + // If true, cache generated data. This is used for generating data that will be referenced in + // a foreign key. + private boolean cacheData; + private DataSet cachedData; + + /** + * Build a TestTable based on an already existing Hive table. This is useful if you want to + * build the table yourself (for example if it needs DDL beyond the columns and partitions) and + * then use it as a TestTable. The way to use this is to first use + * {@link org.apache.hive.test.capybara.IntegrationTest#runQuery} to drop the table, and then + * create the table also using runQuery(). This causes the table to be dropped and created in + * both Hive and the Benchmark. Then when you call + * {@link #populate(org.apache.hive.test.capybara.iface.DataGenerator)} below it will load data + * into + * both. + * @param dbName database table is located in. + * @param tableName name of the table + * @return TestTable + * @throws java.io.IOException + */ + public static TestTable fromHiveMetastore(String dbName, String tableName) throws IOException { + try { + HiveStore hive = TestManager.getTestManager().getClusterManager().getHive(); + IMetaStoreClient msClient = hive.getMetastoreConnection(); + Table msTable = msClient.getTable(dbName, tableName); + // If we haven't started a session, start one, as we'll need it + if (SessionState.get() == null) SessionState.start(hive.getHiveConf()); + // If we haven't set the transaction manager, we need to do it + if (SessionState.get().getTxnMgr() == null) SessionState.get().initTxnMgr(hive.getHiveConf()); + TestTable testTable = new TestTable(dbName, tableName, msTable.getSd().getCols(), + msTable.getPartitionKeysSize() > 0 ? msTable.getPartitionKeys() : null, + null, null, null, 0, + SemanticAnalyzer.isAcidTable(new org.apache.hadoop.hive.ql.metadata.Table(msTable)), + msTable.getSd().getBucketCols().toArray(new String[msTable.getSd().getBucketColsSize()]), + msTable.getSd().getNumBuckets()); + testTable.benchCreated = testTable.hiveCreated = true; + return testTable; + } catch (Exception e) { + throw new IOException(e); + } + } + + public static class Builder { + private String dbName; + private String tableName; + private List cols; + private List partCols; + private DataGenerator partValsGenerator; + private int numParts = 0; + private PrimaryKey pk; + private List fks; + private boolean isAcid = false; + private String[] bucketCols; + private int numBuckets = 0; + + private Builder(String tableName) { + this.tableName = tableName; + } + + /** + * Set the database name for this table. If this is not set, "default" will be used. + * @param dbName dbname + * @return this + */ + public Builder setDbName(String dbName) { + this.dbName = dbName; + return this; + } + + /** + * Set the columns for this table. You must set the columns before calling {@link #build}, + * either with this or {@link #addCol}. + * @param cols list of FieldSchemas + * @return this + */ + public Builder setCols(List cols) { + this.cols = cols; + return this; + } + + /** + * Add a column to this table. You must set the columns before calling {@link #build}, + * either with call(s) to this or via {@link #setCols}. + * @param name column name + * @param type column data type + * @return this + */ + public Builder addCol(String name, String type) { + if (cols == null) cols = new ArrayList<>(); + cols.add(new FieldSchema(name, type, "")); + return this; + } + + /** + * Set partition columns for this table. If you do not call this or {@link #addPartCol} then + * the table will be created non-partitioned. + * @param partCols list of FieldSchemas + * @return this + */ + public Builder setPartCols(List partCols) { + this.partCols = partCols; + return this; + } + + /** + * Add a partition columns to the table. If you do not call this or {@link #setPartCols} + * then the table will be created non-partitioned. + * @param name column name + * @param type column data type + * @return + */ + public Builder addPartCol(String name, String type) { + if (partCols == null) partCols = new ArrayList<>(); + partCols.add(new FieldSchema(name, type, "")); + return this; + } + + /** + * Provide a DataGenerator to give values for the partitions. This is not required. By + * setting this you can control how many partitions are created and their values. You cannot + * call this and {@link #setNumParts}. + * @param partValsGenerator DataGenerator that will provide partition values + * @return this + */ + public Builder setPartValsGenerator(DataGenerator partValsGenerator) { + this.partValsGenerator = partValsGenerator; + return this; + } + + /** + * Set the number of partitions this table should have without specifying the values. You + * cannot call this and {@link #setPartValsGenerator}. + * @param numParts number of partitions the table should have + * @return this + */ + public Builder setNumParts(int numParts) { + this.numParts = numParts; + return this; + } + + /** + * Set a primary key on the table. This does not affect how the table is created in Hive or + * the benchmark, but it will be used in data generation. + * @param pk primary key + * @return this + */ + public Builder setPrimaryKey(PrimaryKey pk) { + this.pk = pk; + return this; + } + + /** + * Add a new foreign key on this table. This does not affect how the table is created in + * Hive or the benchmark, but it will be used in data generation. + * @param fk foreign key + * @return this + */ + public Builder addForeignKey(ForeignKey fk) { + if (fks == null) fks = new ArrayList<>(); + fks.add(fk); + return this; + } + + /** + * Set up this table to work with ACID. + * @param isAcid true for acid, false for non-transactional + * @return this + */ + public Builder setAcid(boolean isAcid) { + this.isAcid = isAcid; + return this; + } + + /** + * Set the columns to be used in bucketing. + * @param bucketCols column names + * @return this + */ + public Builder setBucketCols(String... bucketCols) { + this.bucketCols = bucketCols; + return this; + } + + /** + * Set number of buckets + * @param numBuckets number of buckets + * @return this + */ + public Builder setNumBuckets(int numBuckets) { + this.numBuckets = numBuckets; + return this; + } + + /** + * Create the TestTable. This only creates the Java object. The table is not created in + * Hive or the benchmark. + * @return the table + */ + public TestTable build() { + if (dbName == null) dbName = "default"; + if (tableName == null || cols == null) { + throw new RuntimeException("You must provide a tablename and columns before building"); + } + return new TestTable(dbName, tableName, cols, partCols, pk, fks, partValsGenerator, + numParts, isAcid, bucketCols, numBuckets); + } + } + + /** + * Get a builder to construct the table. + * @param tableName name of the table. + * @return builder + */ + public static Builder getBuilder(String tableName) { + return new Builder(tableName); + } + + private TestTable(String dbName, String tableName, List cols, + List partCols, PrimaryKey pk, List fk, + DataGenerator partValsGenerator, int numParts, boolean isAcid, + String[] bucketCols, int numBuckets) { + this.dbName = dbName == null ? "default" : dbName; + this.tableName = tableName; + this.cols = cols; + this.partCols = partCols; + primaryKey = pk; + foreignKeys = fk; + this.isAcid = isAcid; + this.bucketCols = bucketCols; + this.numBuckets = numBuckets; + if (partValsGenerator != null) { + assert numParts == 0; + partVals = new ArrayList<>(); + // This is very meta, but we need a TestTable definition just to build these few rows. + // That TestTable needs to match the schema of the partition columns, not the regular + // columns, thus we can't use 'this' here. + TestTable meta = new TestTable("fake", "fake", partCols, null, null, null, null, 0, false, + null, 0); + DataSet ds = partValsGenerator.generateData(meta); + for (Row row : ds) partVals.add(row); + this.numParts = partVals.size(); + } else { + partVals = null; + if (numParts != 0) { + this.numParts = numParts; + } + } + } + + /** + * Create the test table in Hive and the Benchmark. This will only be done if the table hasn't + * already been created with the proper characteristics (scale, storage format, etc.). + * @throws SQLException + * @throws IOException + */ + public void create() throws SQLException, IOException { + HiveStore hive = TestManager.getTestManager().getClusterManager().getHive(); + DataStore bench = TestManager.getTestManager().getBenchmark().getBenchDataStore(); + + hiveCreated = hive.createTable(this); + benchCreated = bench.createTable(this); + } + + /** + * Populate a table. You must call {@link #create} before calling this. Data will only + * written into the table if the table was created by create. This will use default values for + * scale and pctnull. + * @param generator generator to use to generate the data. + * @throws SQLException any exception thrown from the benchmark + * @throws java.io.IOException + */ + public void populate(DataGenerator generator) throws SQLException, IOException { + populate(generator, -1, null); + } + + /** + * Populate a table. You must call {@link #create} before calling this. Data will only + * written into the table if the table was created by create. + * @param generator generator to use to generate the data. + * @param scale scale to populate table at. To get the scale from the configuration pass -1 + * @param pctNull array with one element for each column excluding partition columns which + * describes the percentage of entries that are null. 0.0 is no nulls, 1.0 is + * all nulls, 0.5 is 50% nulls. If null is passed, then the default value of 1% + * for each column will be used. + * @throws SQLException any exception thrown from the benchmark + * @throws java.io.IOException + */ + public void populate(DataGenerator generator, int scale, double[] pctNull) + throws SQLException, IOException { + HiveStore hive = TestManager.getTestManager().getClusterManager().getHive(); + DataStore bench = TestManager.getTestManager().getBenchmark().getBenchDataStore(); + + scale = scale == -1 ? TestConf.getScale() : scale; + + DataSet data = null; + if (hiveCreated || benchCreated) { + data = generator.generateData(this, scale, pctNull); + } + HiveLoader hiveLoader = null; + if (hiveCreated) { + hiveLoader = new HiveLoader(this, data, hive); + hiveLoader.start(); + } + if (benchCreated) { + bench.loadData(this, data); + } + if (hiveLoader != null) { + try { + hiveLoader.join(); + } catch (InterruptedException e) { + throw new RuntimeException(e); + } + if (hiveLoader.stashedIoException != null) throw hiveLoader.stashedIoException; + if (hiveLoader.stashedSqlException != null) throw hiveLoader.stashedSqlException; + } + /* + */ + if (cacheData) cachedData = data; + } + + /** + * Create the table, even if it already exists. If the table already exists it will be + * dropped and re-created. This will not populate the table. This should + * not be used for tables to read from (for that use {@link #create}). This should + * only be used for tables to be written to as part of the query. If you are calling this and + * then {@link #populate} you are most likely doing it wrong. + * @throws SQLException + * @throws java.io.IOException + */ + public void createTargetTable() throws IOException, SQLException { + HiveStore hive = TestManager.getTestManager().getClusterManager().getHive(); + DataStore bench = TestManager.getTestManager().getBenchmark().getBenchDataStore(); + + hive.forceCreateTable(this); + bench.forceCreateTable(this); + } + + private class HiveLoader extends Thread { + final TestTable table; + final DataSet data; + final HiveStore hive; + SQLException stashedSqlException; + IOException stashedIoException; + + public HiveLoader(TestTable table, DataSet data, HiveStore hive) { + this.table = table; + this.data = data; + this.hive = hive; + } + + @Override + public void run() { + try { + hive.loadData(table, data); + } catch (SQLException e) { + stashedSqlException = e; + } catch (IOException e) { + stashedIoException = e; + } + } + } + + public String getDbName() { + return dbName; + } + + /** + * Get just the table name without the database. + * @return table name + */ + public String getTableName() { + return tableName; + } + + /** + * Get fully qualified table name. + * @return dbname.tablename + */ + public String getFullName() { + return new StringBuilder(dbName).append('.').append(tableName).toString(); + } + + public List getCols() { + return cols; + } + + /** + * Get the partition columns + * @return part cols + */ + public List getPartCols() { + return partCols; + } + + public PrimaryKey getPrimaryKey() { + return primaryKey; + } + + public List getForeignKeys() { + return foreignKeys; + } + + /** + * Get the partition values. This may be null, which indicates that partitions values have not + * been determined yet. + * @return part vals + */ + public List getPartVals() { + return partVals; + } + + /** + * Get the number of partitions. Even for a partitioned table this may be 0, which indicates + * that the number of partitions will be determined by the test scale. + * @return num parts + */ + public int getNumParts() { + return numParts; + } + + public void setPartVals(List partVals) { + this.partVals = partVals; + } + + public boolean isAcid() { + return isAcid; + } + + public int getNumBuckets() { + return numBuckets; + } + + public String[] getBucketCols() { + return bucketCols; + } + + /** + * Get a PreparedStatement that can be used to load data into this table. This statement can + * be passed to {@link org.apache.hive.test.capybara.data.Column#load}. + * @param conn JDBC connection + * @param store DataStore this statement is being prepared for + * @return a prepared statement + * @throws SQLException + */ + public PreparedStatement getLoadingStatement(Connection conn, DataStore store) + throws SQLException { + StringBuilder sql = new StringBuilder("insert into ") + .append(store.getTableName(this)) + .append(" ("); + boolean first = true; + for (FieldSchema col : cols) { + if (first) first = false; + else sql.append(", "); + sql.append(col.getName()); + } + sql.append(" ) values ("); + first = true; + for (FieldSchema col : cols) { + if (first) first = false; + else sql.append(", "); + sql.append("?"); + } + sql.append(")"); + LOG.debug("Going to prepare statement <" + sql.toString() + ">"); + return conn.prepareStatement(sql.toString()); + } + + @Override + public String toString() { + return getFullName(); + } + + /** + * Build a schema that combines the columns with the partition columns. This is not guaranteed + * to be a copy, so don't mess with it (I wish Java had const). + * @return combined schema. + */ + public List getCombinedSchema() { + if (partCols == null) return cols; + List combined = new ArrayList<>(cols); + combined.addAll(partCols); + return combined; + } + + + /** + * A primary key. For now it is limited to a single column. + */ + public static class PrimaryKey implements Serializable { + public final int colNum; + + public PrimaryKey(int colNum) { + this.colNum = colNum; + } + + public boolean isSequence() { return false; } + } + + /** + * A sequence, guaranteed to be monotonically increasing. + */ + public static class Sequence extends PrimaryKey implements Serializable { + public Sequence(int colNum) { + super(colNum); + } + + @Override + public boolean isSequence() { return true; } + } + + /** + * A foreign key. This takes a reference to a + * {@link TestTable.PrimaryKey} in a target table. + */ + public static class ForeignKey implements Serializable { + public transient DataSet targetTable; + public final int colNumInTarget; + public final int colNumInSrc; + public Path pkFile; // if non-null, points to the file that contains the primary key data. + + public ForeignKey(DataSet targetTable, int targetColNum, int srcColNum) { + this.targetTable = targetTable; + this.colNumInTarget = targetColNum; + this.colNumInSrc = srcColNum; + } + + public ForeignKey(String pkFile, int targetColNum, int srcColNum) { + // TODO populate the dataset from the file. + this.colNumInTarget = targetColNum; + this.colNumInSrc = srcColNum; + } + } + + /** + * Tell TestTable to keep a hold of the data after it is generated. In general you don't want + * to do this because it will eat memory. But it's useful if you're planning on building a + * foreign key in another table referencing this table. + * @param cacheData whether to cache the data. + */ + public void setCacheData(boolean cacheData) { + this.cacheData = cacheData; + } + + /** + * Get a DataSet that represents the contents of this table. + * @return data for this data + */ + public DataSet getData() throws IOException, SQLException { + if (cachedData == null) { + // TODO - create a FetchingDataSet that only fetches the data when requested. Because it's + // quite likely we won't actually need to fetch the data. + DataStore bench = TestManager.getTestManager().getBenchmark().getBenchDataStore(); + FetchResult fetch = bench.fetchData("select * from " + toString()); + if (fetch.rc != ResultCode.SUCCESS) { + throw new RuntimeException("Unable to fetch results from already populated table " + + toString()); + } + cachedData = fetch.data; + } + return cachedData; + } +} diff --git itests/capybara/src/main/java/org/apache/hive/test/capybara/infra/AnsiSqlStore.java itests/capybara/src/main/java/org/apache/hive/test/capybara/infra/AnsiSqlStore.java new file mode 100644 index 0000000..1adac56 --- /dev/null +++ itests/capybara/src/main/java/org/apache/hive/test/capybara/infra/AnsiSqlStore.java @@ -0,0 +1,212 @@ +/** + * 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.hive.test.capybara.infra; + +import org.apache.hadoop.hive.metastore.api.FieldSchema; +import org.apache.hive.test.capybara.data.DataSet; +import org.apache.hive.test.capybara.data.FetchResult; +import org.apache.hive.test.capybara.data.ResultCode; +import org.apache.hive.test.capybara.iface.BenchmarkDataStore; +import org.apache.hive.test.capybara.iface.TestTable; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.File; +import java.io.FileWriter; +import java.io.IOException; +import java.sql.Connection; +import java.sql.SQLException; +import java.sql.Statement; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.Iterator; +import java.util.List; +import java.util.Map; + +/** + * An abstract class for all DataStores that are ANSI SQL compliant. It contains convenience + * methods for converting Hive SQL into ANSI SQL. It also contains implementations of common + * methods that don't change between JDBC drivers. + */ +abstract class AnsiSqlStore extends DataStoreBase implements BenchmarkDataStore { + private static final Logger LOG = LoggerFactory.getLogger(AnsiSqlStore.class.getName()); + + /** + * A map from DataSets to places they've been dumped. The key is obtained by calling + * DataStore.uniqueId(). The value is a File where the data is stored. + */ + protected Map dataSetDumps; + + /** + * Certain statements might fail for an ANSI SQL store where they would not for Hive, such as + * 'create table if not exists'. If we detect this condition note it so that the ANSI store + * knows that failure is acceptable. + */ + protected boolean failureOk; + + + protected AnsiSqlStore() { + dataSetDumps = new HashMap<>(); + } + + @Override + public boolean createTable(TestTable table) throws SQLException { + return createTableInternal(table, false); + } + + @Override + public void forceCreateTable(TestTable table) throws SQLException, IOException { + createTableInternal(table, true); + } + + @Override + public FetchResult fetchData(String hiveSql) throws SQLException, IOException { + String sql = hiveSqlToAnsiSql(hiveSql); + if (sql.equals("")) { + // This means this is a NOP for the benchmark + LOG.debug("Hive SQL <" + hiveSql + "> is NOP on benchmark"); + return new FetchResult(ResultCode.SUCCESS); + } + LOG.debug("Translated Hive SQL <" + hiveSql + "> to <" + sql + + ">, going to run against benchmark"); + return jdbcFetch(sql, getLimit(), failureOk); + } + + @Override + public void dumpToFileForImport(DataSet rows) throws IOException { + File dumpFile = dataSetDumps.get(rows.uniqueId()); + if (dumpFile == null) { + dumpFile = File.createTempFile("capybara_", "_data_set_" + rows.uniqueId()); + dataSetDumps.put(rows.uniqueId(), dumpFile); + } + FileWriter writer = new FileWriter(dumpFile, true); + Iterator iter = + rows.stringIterator(fileColumnDelimiter(), fileNull(), fileStringQuotes()); + String lineSep = System.getProperty("line.separator"); + while (iter.hasNext()) { + writer.write(iter.next() + lineSep); + } + writer.close(); + } + + @Override + public Connection getJdbcConnection(boolean autoCommit) throws SQLException { + return connect(autoCommit); + } + + /** + * Get the string used to delimit columns in a file that will be loaded to the database. + * @return delimiter + */ + protected abstract String fileColumnDelimiter(); + + /** + * Get the string that represents null in a file that will be loaded into the database. + * @return null string + */ + protected abstract String fileNull(); + + /** + * Get the string used to quote values in a file that will be loaded into the database. + * @return quote + */ + protected abstract String fileStringQuotes(); + + /** + * Get the appropriate SQL translator for this database. + * @return translator + */ + protected abstract SQLTranslator getTranslator(); + + /** + * Get the limit that should be applied to a query. This is to deal with the fact that derby + * doesn't support limit. If a database supports limit it should not override this method. + * @return limit value. + */ + protected long getLimit() { + return Long.MAX_VALUE; + } + + /** + * Convert Hive SQL to ANSI SQL. This works via a fuzzy parser, so it's not fool proof. + * @param hiveSql Hive SQL string to convert + * @return ANSI SQL + */ + protected String hiveSqlToAnsiSql(String hiveSql) throws IOException, SQLException { + + try { + SQLTranslator translator = getTranslator(); + String benchSql = translator.translate(hiveSql); + failureOk = translator.isFailureOk(); + return benchSql; + } catch (TranslationException e) { + LOG.warn("Unable to translate Hive SQL <" + hiveSql + + "> for your benchmark, risking it using original Hive SQL"); + return hiveSql; + } + } + + protected String convertType(String hiveType) { + if (hiveType.equalsIgnoreCase("string")) return "varchar(255)"; + else if (hiveType.equalsIgnoreCase("tinyint")) return "smallint"; + else if (hiveType.equalsIgnoreCase("binary")) return "blob"; + else return hiveType; + } + + private boolean createTableInternal(TestTable table, boolean force) throws SQLException { + if (!force && tableExistsInCorrectState(table)) return false; + + try { + // if exists isn't proper ANSI SQL, but it signals the system to allow failure here if the + // table already exists. + fetchData("drop table if exists " + table.toString()); + } catch (IOException e) { + throw new RuntimeException(e); + } + + List allCols = new ArrayList<>(); + allCols.addAll(table.getCols()); + // If there are any partition columns add those at the end + if (table.getPartCols() != null) { + allCols.addAll(table.getPartCols()); + } + + Statement stmt = null; + try (Connection conn = connect(true)) { + StringBuilder sql = new StringBuilder("create table ") + .append(getTableName(table)) + .append(" ("); + for (FieldSchema col : allCols) { + sql.append(col.getName()) + .append(" ") + .append(convertType(col.getType())) + .append(","); + } + sql.deleteCharAt(sql.length() - 1).append(") "); + stmt = conn.createStatement(); + LOG.debug("Going to run <" + sql.toString() + "> on ANSI SQL store"); + stmt.execute(sql.toString()); + if (!force) recordTableCreation(table); + } finally { + if (stmt != null) stmt.close(); + + } + return true; + } + +} diff --git itests/capybara/src/main/java/org/apache/hive/test/capybara/infra/BoundedQueue.java itests/capybara/src/main/java/org/apache/hive/test/capybara/infra/BoundedQueue.java new file mode 100644 index 0000000..5223bc1 --- /dev/null +++ itests/capybara/src/main/java/org/apache/hive/test/capybara/infra/BoundedQueue.java @@ -0,0 +1,62 @@ +/** + * 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.hive.test.capybara.infra; + +import com.google.common.collect.ForwardingQueue; + +import java.util.ArrayDeque; +import java.util.Collection; +import java.util.Queue; + +/** + * An impelmentation of queue that is bounded in capacity. When capacity is reached and new + * elements are added, old elements are dropped off the back. + */ +public class BoundedQueue extends ForwardingQueue { + private final int size; + Queue elements; + + public BoundedQueue(int size) { + this.size = size; + elements = new ArrayDeque<>(size); + } + + @Override + protected Queue delegate() { + return elements; + } + + @Override + public boolean add(T t) { + while (elements.size() >= size) elements.poll(); + elements.add(t); + return true; + } + + @Override + public boolean addAll(Collection c) { + while (elements.size() + c.size() > size) elements.poll(); + elements.addAll(c); + return true; + } + + @Override + public boolean offer(T t) { + return add(t); + } +} diff --git itests/capybara/src/main/java/org/apache/hive/test/capybara/infra/CapyEndPoint.java itests/capybara/src/main/java/org/apache/hive/test/capybara/infra/CapyEndPoint.java new file mode 100644 index 0000000..88da873 --- /dev/null +++ itests/capybara/src/main/java/org/apache/hive/test/capybara/infra/CapyEndPoint.java @@ -0,0 +1,252 @@ +/** + * 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.hive.test.capybara.infra; + +import org.apache.hadoop.hive.conf.HiveConf; +import org.apache.hadoop.hive.serde2.SerDeException; +import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector; +import org.apache.hadoop.hive.serde2.objectinspector.StructField; +import org.apache.hadoop.hive.serde2.objectinspector.StructObjectInspector; +import org.apache.hadoop.security.UserGroupInformation; +import org.apache.hive.hcatalog.streaming.AbstractRecordWriter; +import org.apache.hive.hcatalog.streaming.ConnectionError; +import org.apache.hive.hcatalog.streaming.HiveEndPoint; +import org.apache.hive.hcatalog.streaming.ImpersonationFailed; +import org.apache.hive.hcatalog.streaming.InvalidPartition; +import org.apache.hive.hcatalog.streaming.InvalidTable; +import org.apache.hive.hcatalog.streaming.PartitionCreationFailed; +import org.apache.hive.hcatalog.streaming.RecordWriter; +import org.apache.hive.hcatalog.streaming.SerializationError; +import org.apache.hive.hcatalog.streaming.StreamingConnection; +import org.apache.hive.hcatalog.streaming.StreamingException; +import org.apache.hive.hcatalog.streaming.TransactionBatch; +import org.apache.hive.test.capybara.data.Column; +import org.apache.hive.test.capybara.data.Row; +import org.apache.hive.test.capybara.data.RowBuilder; +import org.apache.hive.test.capybara.iface.BenchmarkDataStore; +import org.apache.hive.test.capybara.iface.TestTable; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.sql.Connection; +import java.sql.PreparedStatement; +import java.sql.SQLException; +import java.util.Collection; +import java.util.List; + +public class CapyEndPoint extends HiveEndPoint { + static final private Logger LOG = LoggerFactory.getLogger(CapyEndPoint.class.getName()); + + // A copy of the conf from IntegrationTest, used when conf is null so we don't create a new one. + private final HiveConf testConf; + private final BenchmarkDataStore bench; + private final TestTable testTable; + + public CapyEndPoint(BenchmarkDataStore bench, TestTable testTable, HiveConf testConf, + String metaStoreUri, List partitionVals) { + super(metaStoreUri, testTable.getDbName(), testTable.getTableName(), partitionVals); + this.bench = bench; + this.testTable = testTable; + this.testConf = testConf; + } + + @Override + public StreamingConnection newConnection(boolean createPartIfNotExists) throws + ConnectionError, InvalidPartition, InvalidTable, PartitionCreationFailed, ImpersonationFailed, + InterruptedException { + return this.newConnection(createPartIfNotExists, testConf, null); + } + + @Override + public StreamingConnection newConnection(boolean createPartIfNotExists, HiveConf conf, + UserGroupInformation authenticatedUser) throws + ConnectionError, InvalidPartition, InvalidTable, PartitionCreationFailed, ImpersonationFailed, + InterruptedException { + StreamingConnection hiveStream = + super.newConnection(createPartIfNotExists, conf, authenticatedUser); + return new CapyStreamingConnection(hiveStream); + } + + private class CapyStreamingConnection implements StreamingConnection { + private final StreamingConnection hiveStream; + private Connection conn; + + public CapyStreamingConnection(StreamingConnection hiveStream) { + this.hiveStream = hiveStream; + } + + @Override + public TransactionBatch fetchTransactionBatch(int numTransactionsHint, RecordWriter writer) + throws ConnectionError, StreamingException, InterruptedException { + TransactionBatch hiveBatch = hiveStream.fetchTransactionBatch(numTransactionsHint, writer); + try { + conn = bench.getJdbcConnection(false); + return new CapyTransactionBatch(hiveBatch, conn, writer); + } catch (Exception e) { + throw new ConnectionError("Unable to get connection to benchmark or instantiate object " + + "inspectors", e); + } + } + + @Override + public void close() { + hiveStream.close(); + try { + conn.close(); + } catch (SQLException e) { + throw new RuntimeException(e); + } + } + } + + private class CapyTransactionBatch implements TransactionBatch { + private final TransactionBatch hiveBatch; + private final Connection conn; + private final AbstractRecordWriter writer; + private final RowBuilder rowBuilder; + private final StructObjectInspector rowInspector; + private final ObjectInspector[] colInspectors; + private final Row partRow; + private final PreparedStatement preparedStatement; + private boolean isClosed; + + public CapyTransactionBatch(TransactionBatch hiveBatch, Connection conn, + RecordWriter writer) throws SerializationError, SerDeException { + this.hiveBatch = hiveBatch; + this.conn = conn; + this.writer = (AbstractRecordWriter)writer; + rowBuilder = new RowBuilder(testTable.getCombinedSchema()); + rowInspector = + (StructObjectInspector)((AbstractRecordWriter) writer).getSerde().getObjectInspector(); + List fields = rowInspector.getAllStructFieldRefs(); + colInspectors = new ObjectInspector[fields.size()]; + for (int i = 0; i < colInspectors.length; i++) { + colInspectors[i] = fields.get(i).getFieldObjectInspector(); + } + + if (partitionVals != null && partitionVals.size() > 0) { + RowBuilder partRowBuilder = new RowBuilder(testTable.getPartCols()); + partRow = partRowBuilder.build(); + for (int i = 0; i < partRow.size(); i++) { + String val = partitionVals.get(i) == null ? "NULL" : partitionVals.get(i); + partRow.get(i).fromString(val, "NULL"); + } + } else { + partRow = null; + } + try { + preparedStatement = testTable.getLoadingStatement(conn, bench); + } catch (SQLException e) { + throw new RuntimeException(e); + } + isClosed = false; + } + + @Override + public void beginNextTransaction() throws StreamingException, InterruptedException { + hiveBatch.beginNextTransaction(); + } + + @Override + public Long getCurrentTxnId() { + return hiveBatch.getCurrentTxnId(); + } + + @Override + public TxnState getCurrentTransactionState() { + return hiveBatch.getCurrentTransactionState(); + } + + @Override + public void commit() throws StreamingException, InterruptedException { + hiveBatch.commit(); + try { + conn.commit(); + } catch (SQLException e) { + throw new StreamingException("Couldn't commit exception", e); + } + } + + @Override + public void abort() throws StreamingException, InterruptedException { + hiveBatch.abort(); + try { + conn.rollback(); + } catch (SQLException e) { + throw new StreamingException("Couldn't rollback exception", e); + } + } + + @Override + public int remainingTransactions() { + return hiveBatch.remainingTransactions(); + } + + @Override + public void write(byte[] record) throws StreamingException, InterruptedException { + hiveBatch.write(record); + writeBenchmark(record); + } + + @Override + public void write(Collection records) throws StreamingException, InterruptedException { + hiveBatch.write(records); + for (byte[] record : records) writeBenchmark(record); + } + + private void writeBenchmark(byte[] record) throws StreamingException { + // Convert the record from whatever format it came in (we don't know what it is) to an + // Object that Hive can parse with ObjectInspectors + Object objRow = writer.encode(record); + + // Convert the Object to our Row format. This may not fill up all of the columns because + // it won't have the partitions columns. + Row row = rowBuilder.build(); + List objCols = rowInspector.getStructFieldsDataAsList(objRow); + for (int i = 0; i < colInspectors.length; i++) { + row.get(i).fromObject(colInspectors[i], objCols.get(i)); + } + + // If there are any partition values, append them here + if (partRow != null) row.append(partRow); + + // Load it in via a prepared statement + try { + for (Column col : row) col.load(preparedStatement); + preparedStatement.executeUpdate(); + } catch (SQLException e) { + throw new StreamingException("Unable to load into benchmark", e); + } + } + + @Override + public void heartbeat() throws StreamingException { + hiveBatch.heartbeat(); + } + + @Override + public void close() throws StreamingException, InterruptedException { + hiveBatch.close(); + } + + @Override + public boolean isClosed() { + return isClosed; + } + } +} diff --git itests/capybara/src/main/java/org/apache/hive/test/capybara/infra/ClusterCliHiveStore.java itests/capybara/src/main/java/org/apache/hive/test/capybara/infra/ClusterCliHiveStore.java new file mode 100644 index 0000000..942e865 --- /dev/null +++ itests/capybara/src/main/java/org/apache/hive/test/capybara/infra/ClusterCliHiveStore.java @@ -0,0 +1,164 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hive.test.capybara.infra; + +import org.apache.commons.lang.StringUtils; +import org.apache.hadoop.hive.conf.HiveConf; +import org.apache.hive.test.capybara.data.FetchResult; +import org.apache.hive.test.capybara.data.ResultCode; +import org.apache.hive.test.capybara.iface.ClusterManager; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import org.apache.hadoop.hive.ql.QueryPlan; + +import java.io.BufferedReader; +import java.io.IOException; +import java.io.InputStreamReader; +import java.sql.SQLException; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Properties; + +/** + * An implementation of HiveStore that works on external clusters using the CLI. + */ +class ClusterCliHiveStore extends HiveStore { + static final private Logger LOG = LoggerFactory.getLogger(ClusterCliHiveStore.class.getName()); + Map envMap; + + /** + * No-args constructor for use on the cluster. + */ + ClusterCliHiveStore() { + + } + + ClusterCliHiveStore(ClusterManager clusterManager) { + super(clusterManager); + envMap = new HashMap<>(); + getEnv("HADOOP_HOME"); + getEnv("HIVE_HOME"); + envMap.put("JAVA_HOME", System.getProperty("java.home")); + } + + @Override + public FetchResult fetchData(String sql) throws SQLException, IOException { + List cmd = new ArrayList<>(); + cmd.add(new StringBuilder(envMap.get("HIVE_HOME")) + .append(System.getProperty("file.separator")) + .append("bin") + .append(System.getProperty("file.separator")) + .append("hive") + .toString()); + // Add in the configuration values + for (Map.Entry entry : clusterManager.getConfVars().entrySet()) { + cmd.add("--hiveconf"); + cmd.add(new StringBuilder(entry.getKey()) + .append('=') + .append(entry.getValue()) + .toString()); + } + cmd.add("-e"); + cmd.add(sql); + + String[] env = new String[envMap.size()]; + int i = 0; + for (Map.Entry entry : envMap.entrySet()) { + env[i++] = new StringBuilder(entry.getKey()) + .append("=") + .append(entry.getValue()) + .toString(); + } + if (LOG.isDebugEnabled()) { + LOG.debug("Going to exec: " + StringUtils.join(cmd, " ") + " with env " + + StringUtils.join(env, " ")); + } + Process proc = Runtime.getRuntime().exec(cmd.toArray(new String[cmd.size()]), env); + List stdoutLines = new ArrayList<>(); + List stderrLines = new ArrayList<>(); + Thread stdout = new Thread(getStreamPumper(new BufferedReader(new InputStreamReader( + proc.getInputStream())), stdoutLines)); + Thread stderr = new Thread(getStreamPumper(new BufferedReader(new InputStreamReader( + proc.getErrorStream())), stderrLines)); + stdout.start(); + stderr.start(); + try { + proc.waitFor(); + } catch (InterruptedException e) { + throw new IOException(e); + } + int rc = proc.exitValue(); + if (rc != 0) { + LOG.warn("Got non-zero return code from Hive " + rc + ", stderr is <" + + StringUtils.join(stderrLines, "\n") + ">"); + return new FetchResult(ResultCode.NON_RETRIABLE_FAILURE); + } + LOG.debug("stderr from Hive invocation: " + StringUtils.join(stderrLines, "\n")); + if (stdoutLines.size() > 0) { + return new FetchResult(new StringDataSet(stdoutLines, "\t", "NULL")); + } else { + return new FetchResult(ResultCode.SUCCESS); + } + } + + @Override + public QueryPlan explain(String sql) { + return null; + } + + @Override + protected String connectionURL() { + return null; + } + + @Override + protected Properties connectionProperties() { + return null; + } + + @Override + public String getMetastoreUri() { + return conf.getVar(HiveConf.ConfVars.METASTOREURIS); + } + + private void getEnv(String var) { + String val = System.getProperty(var); + if (val == null || val.equals("")) { + throw new RuntimeException("You must set " + var + " in your properties before using " + + "Hive on a cluster"); + } + envMap.put(var, val); + } + + private Runnable getStreamPumper(final BufferedReader reader, final List lines) { + return new Runnable() { + @Override + public void run() { + try { + String line; + while ((line = reader.readLine()) != null) lines.add(line); + } catch (IOException e) { + throw new RuntimeException(e); + } + + } + }; + } +} diff --git itests/capybara/src/main/java/org/apache/hive/test/capybara/infra/ClusterDataGenerator.java itests/capybara/src/main/java/org/apache/hive/test/capybara/infra/ClusterDataGenerator.java new file mode 100644 index 0000000..73f79e9 --- /dev/null +++ itests/capybara/src/main/java/org/apache/hive/test/capybara/infra/ClusterDataGenerator.java @@ -0,0 +1,220 @@ +/** + * 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.hive.test.capybara.infra; + +import org.apache.commons.codec.binary.Base64; +import org.apache.hive.test.capybara.data.DataSet; +import org.apache.hive.test.capybara.iface.TestTable; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.filecache.DistributedCache; +import org.apache.hadoop.fs.FSDataInputStream; +import org.apache.hadoop.fs.FSDataOutputStream; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.io.LongWritable; +import org.apache.hadoop.io.NullWritable; +import org.apache.hadoop.io.Text; +import org.apache.hadoop.mapred.JobClient; +import org.apache.hadoop.mapred.JobConf; +import org.apache.hadoop.mapred.Mapper; +import org.apache.hadoop.mapred.OutputCollector; +import org.apache.hadoop.mapred.Reporter; +import org.apache.hadoop.mapred.RunningJob; +import org.apache.hadoop.mapred.lib.NLineInputFormat; +import org.apache.hadoop.mapred.lib.NullOutputFormat; +import org.apache.hive.test.capybara.iface.DataGenerator; + +import java.io.BufferedReader; +import java.io.ByteArrayInputStream; +import java.io.ByteArrayOutputStream; +import java.io.IOException; +import java.io.InputStreamReader; +import java.io.ObjectInputStream; +import java.io.ObjectOutputStream; +import java.io.Serializable; +import java.util.ArrayList; +import java.util.Iterator; +import java.util.List; + +/** + * A class that takes a generator and makes many copies of it run on a cluster. + */ +public class ClusterDataGenerator extends DataGenerator { + private static final Logger LOG = LoggerFactory.getLogger(ClusterDataGenerator.class); + private static final String NUM_SPLITS = "hive.test.capybara.cluster.data.gen.numsplits"; + private DataGeneratorImpl wrapped; + + public ClusterDataGenerator(DataGeneratorImpl base) { + wrapped = base; + } + + @Override + public DataSet generateData(TestTable table, int scale, double[] pctNulls) { + LOG.debug("Generating data on cluster"); + if (table.getPartCols() != null) { + // Decide on our part vals before we generate. In the random case this prevents each + // generator from generating different random values. The number of discrete values should + // be relatively small. + table.setPartVals(wrapped.determinePartVals(table, scale, pctNulls)); + } + + try { + Configuration conf = TestManager.getTestManager().getConf(); + FileSystem fs = TestManager.getTestManager().getClusterManager().getFileSystem(); + + // If the table has one or more foreign keys don't try to encode that information in each + // row. Write each foreign key into a separate file and add the file to the distributed cache. + if (table.getForeignKeys() != null) { + for (int i = 0; i < table.getForeignKeys().size(); i++) { + TestTable.ForeignKey fk = table.getForeignKeys().get(i); + Path fkPath = new Path(HiveStore.getDirForDumpFile()); + FSDataOutputStream out = fs.create(fkPath); + // Write the index in the first line of the file. + out.writeBytes(Integer.toString(i)); + out.writeBytes(System.getProperty("line.separator")); + Iterator iter = fk.targetTable.stringIterator("\u0001", "NULL", "\""); + while (iter.hasNext()) { + out.writeBytes(iter.next()); + out.writeBytes(System.getProperty("line.separator")); + } + out.close(); + fk.pkFile = fkPath; + // TODO - not sure this is the correct way to get something in the DC. Check once I + // have network. + DistributedCache.addCacheFile(fkPath.toUri(), conf); + } + } + + // Figure out how parallel to go. + int parallelism = 2 * scale * 1024 / TestConf.getClusterGenThreshold(); + + // The algorithm here is to write a line for each mapper we're going to start and then use + // the NLineInputFormat to create one mapper per line. This creates quite a lot of + // duplication, but the entries are only a few K so it isn't that bad. The exception is if + // we have a foreign key it needs to be written into a side file and then put in the DC. + String hiveDumpDir = HiveStore.getDirForDumpFile(); + Path inputFile = new Path(HiveStore.getDirForDumpFile()); + FSDataOutputStream out = fs.create(inputFile); + for (int i = 0; i < parallelism; i++) { + GeneratorInfo gi = + new GeneratorInfo(scale / parallelism, pctNulls, table, wrapped, hiveDumpDir); + ByteArrayOutputStream baos = new ByteArrayOutputStream(); + ObjectOutputStream oos = new ObjectOutputStream(baos); + oos.writeObject(gi); + out.writeBytes(Base64.encodeBase64URLSafeString(baos.toByteArray())); + out.writeBytes(System.getProperty("line.separator")); + } + out.close(); + + // Now, construct an MR job to run this. + JobConf job = new JobConf(conf); + job.setJobName("Capybara data gen for " + table.getTableName()); + job.setOutputKeyClass(NullWritable.class); + job.setOutputValueClass(NullWritable.class); + job.setJarByClass(ClusterDataGenerator.class); + job.setMapperClass(ClusterDataGeneratorMap.class); + job.setNumReduceTasks(0); + job.setInputFormat(NLineInputFormat.class); + NLineInputFormat.addInputPath(job, inputFile); + job.setOutputFormat(NullOutputFormat.class); + + RunningJob runningJob = JobClient.runJob(job); + LOG.debug("Submitted generation job " + job.getJobName()); + runningJob.waitForCompletion(); + // Now, we write the value of the directory we dumped data to directly into the table -> + // directory map for HiveStore to use in building temp tables. This enables it to just + // find the files we've created. + DataSet clustered = new ClusteredDataSet(table.getCombinedSchema()); + clustered.setClusterLocation(new Path(hiveDumpDir)); + return clustered; + } catch (IOException e) { + throw new RuntimeException(e); + } + } + + static class ClusterDataGeneratorMap + implements Mapper { + private JobConf conf; + + @Override + public void map(LongWritable key, Text value, + OutputCollector outputCollector, Reporter reporter) + throws IOException { + try { + FileSystem fs = FileSystem.get(conf); + // Our info is in the Text value, reconstruct it from there. + ObjectInputStream ois = new ObjectInputStream(new ByteArrayInputStream( + Base64.decodeBase64(value.toString()))); + GeneratorInfo ginfo = (GeneratorInfo) ois.readObject(); + DataSet rows = ginfo.gen.generateData(ginfo.table, ginfo.scale, ginfo.pctNulls); + + // Deal with foreign key reconstruction if needed + Path[] cached = DistributedCache.getLocalCacheFiles(conf); + for (Path p : cached) { + FSDataInputStream input = fs.open(cached[0]); + BufferedReader in = new BufferedReader(new InputStreamReader(input)); + List lines = new ArrayList<>(); + // The first line contains the foreign key entry number + int entryNum = Integer.valueOf(in.readLine()); + String line; + while ((line = in.readLine()) != null) lines.add(line); + ginfo.table.getForeignKeys().get(entryNum).targetTable = + new StringDataSet(ginfo.table.getCombinedSchema(), lines, "\u0001", "NULL"); + input.close(); + + } + + // Now generate the data + String filename = HiveStore.getFileForDump(); + Path file = new Path(ginfo.dumpDir, filename); + FSDataOutputStream output = fs.create(file); + HiveStore.writeToFile(output, rows); + } catch (ClassNotFoundException e) { + throw new IOException(e); + } + } + + @Override + public void close() throws IOException { + + } + + @Override + public void configure(JobConf jobConf) { + conf = jobConf; + } + } + + static class GeneratorInfo implements Serializable { + final int scale; + final double[] pctNulls; + final TestTable table; + final DataGenerator gen; + final String dumpDir; + + GeneratorInfo(int scale, double[] pctNulls, TestTable table, DataGenerator gen, String dumpDir) { + this.scale = scale; + this.pctNulls = pctNulls; + this.table = table; + this.gen = gen; + this.dumpDir = dumpDir; + } + } +} diff --git itests/capybara/src/main/java/org/apache/hive/test/capybara/infra/ClusteredDataSet.java itests/capybara/src/main/java/org/apache/hive/test/capybara/infra/ClusteredDataSet.java new file mode 100644 index 0000000..5d58942 --- /dev/null +++ itests/capybara/src/main/java/org/apache/hive/test/capybara/infra/ClusteredDataSet.java @@ -0,0 +1,41 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hive.test.capybara.infra; + +import org.apache.hadoop.hive.metastore.api.FieldSchema; +import org.apache.hive.test.capybara.data.DataSet; +import org.apache.hive.test.capybara.data.Row; + +import java.util.Collections; +import java.util.Iterator; +import java.util.List; + +/** + * A DataSet that has already been written to the cluster. + */ +public class ClusteredDataSet extends DataSet { + public ClusteredDataSet(List schema) { + super(schema); + } + + @Override + public Iterator iterator() { + // We always want this to return nothing, not that anyone should be calling it. + return Collections.emptyIterator(); + } +} diff --git itests/capybara/src/main/java/org/apache/hive/test/capybara/infra/DataGeneratorImpl.java itests/capybara/src/main/java/org/apache/hive/test/capybara/infra/DataGeneratorImpl.java new file mode 100644 index 0000000..3e71777 --- /dev/null +++ itests/capybara/src/main/java/org/apache/hive/test/capybara/infra/DataGeneratorImpl.java @@ -0,0 +1,78 @@ +/** + * 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.hive.test.capybara.infra; + +import org.apache.hadoop.hive.metastore.api.FieldSchema; +import org.apache.hive.test.capybara.data.Row; +import org.apache.hive.test.capybara.data.RowBuilder; +import org.apache.hive.test.capybara.iface.DataGenerator; +import org.apache.hive.test.capybara.iface.TestTable; + +import java.util.ArrayList; +import java.util.HashSet; +import java.util.List; +import java.util.Set; + +/** + * Base class for DataGenerators with commonly used methods. + */ +abstract class DataGeneratorImpl extends DataGenerator { + + protected abstract ColGenerator getGenerator(FieldSchema col, TestTable.PrimaryKey pk, + TestTable.ForeignKey fk); + + protected List determinePartVals(TestTable table, int scale, double[] pctNulls) { + // There are three options for partition values. The user could have specified the values + // they want, or they could have just specified how many partitions they want or they could + // have let us decide based one scale. + List partCols = table.getPartCols(); + List cols = table.getCols(); + List partVals = table.getPartVals(); + int numPartitions = table.getNumParts(); + + // If the user did not preset the partVals, it will be null + if (partVals == null) { + partVals = new ArrayList<>(); + RowBuilder partBuilder = new RowBuilder(partCols, cols.size()); + // If the user did not specify a number of partitions, set it based on scale. + if (numPartitions == 0) numPartitions = (int) Math.log10(scale) + 1; + RandomDataGenerator.ColGenerator[] partGenerators = new RandomDataGenerator.ColGenerator[partCols.size()]; + for (int i = 0; i < partGenerators.length; i++) { + partGenerators[i] = getGenerator(partCols.get(i), null, null); + } + Set usedPartVals = new HashSet<>(); + for (int i = 0; i < numPartitions; i++) { + Row partVal = partBuilder.build(); + for (int j = 0; j < partGenerators.length; j++) { + partVal.get(j).set(partGenerators[j].generate(pctNulls[j])); + } + if (!usedPartVals.add(partVal)) { + // We've seen this partition value before, so try again. + i--; + } else { + partVals.add(partVal); + } + } + } + return partVals; + } + + interface ColGenerator { + Object generate(double pctNull); + } +} diff --git itests/capybara/src/main/java/org/apache/hive/test/capybara/infra/DataStoreBase.java itests/capybara/src/main/java/org/apache/hive/test/capybara/infra/DataStoreBase.java new file mode 100644 index 0000000..880350f --- /dev/null +++ itests/capybara/src/main/java/org/apache/hive/test/capybara/infra/DataStoreBase.java @@ -0,0 +1,289 @@ +/** + * 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.hive.test.capybara.infra; + +import com.google.common.annotations.VisibleForTesting; +import org.apache.hive.test.capybara.data.FetchResult; +import org.apache.hive.test.capybara.data.ResultCode; +import org.apache.hive.test.capybara.data.Row; +import org.apache.hive.test.capybara.iface.ClusterManager; +import org.apache.hive.test.capybara.iface.DataStore; +import org.apache.hive.test.capybara.iface.TestTable; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import org.apache.hadoop.hive.metastore.api.FieldSchema; + +import java.io.IOException; +import java.sql.Connection; +import java.sql.Driver; +import java.sql.ResultSet; +import java.sql.SQLException; +import java.sql.Statement; +import java.util.Arrays; +import java.util.Iterator; +import java.util.Properties; + +/** + * Base methods that most DataStores will want to use. + */ +abstract class DataStoreBase implements DataStore { + static final private Logger LOG = LoggerFactory.getLogger(DataStoreBase.class.getName()); + + protected final ClusterManager clusterManager; + /** + * SQL driver. It is the responsibility of the sub-class to set this value. + */ + protected Driver jdbcDriver; + + /** + * Constructor for DataStores that will not be running on the cluster. This should be all + * databases, such as Derby and Postges. + */ + protected DataStoreBase() { + clusterManager = null; + } + + /** + * Constructor for DataStores that will be running on the cluster, including mini-clusters. In + * general this should just be Hive. + * @param clusterManager ClusterManager for this test + */ + protected DataStoreBase(ClusterManager clusterManager) { + this.clusterManager = clusterManager; + } + + /** + * Check if a table for use in testing already exists and is in the correct state. Being in + * the correct state means that the scale is correct for this test, the storage format is + * the correct on for this test (Hive tables only), and it is stored in the right type of + * metastore (Hive tables only). + * @param table table to check. + * @return true if the table already exists, false if it does not or is in the wrong state. + * @throws SQLException + */ + protected boolean tableExistsInCorrectState(TestTable table) throws SQLException { + return tableExistsInCorrectState(table, false); + } + + private boolean tableExistsInCorrectState(TestTable table, boolean beenThereDoneThat) + throws SQLException { + StringBuilder sql = new StringBuilder(); + sql.append("select metastore, fileformat, scale from testtables where tablename = '") + .append(getTableName(table)) + .append('\''); + try { + FetchResult fetch = fetchData(sql.toString()); + if (fetch.rc != ResultCode.SUCCESS) { + if (beenThereDoneThat) { + throw new RuntimeException("Unable to instantiate metadata table for testing."); + } else { + createTestTableTracker(); + return tableExistsInCorrectState(table, true); + } + } else { + if (fetch.hasResults()) { + if (fetch.data.getSchema() == null) { + // If this is Hive we have to tell it the schema we expect in the query. + fetch.data.setSchema(Arrays.asList( + new FieldSchema("metastore", "varchar(5)", ""), + new FieldSchema("fileformat", "varchar(10)", ""), + new FieldSchema("scale", "int", "") + )); + } + Iterator iter = fetch.data.iterator(); + if (iter.hasNext()) { + Row row = iter.next(); + return TestConf.fileFormat().equalsIgnoreCase(row.get(1).asString()) && + TestConf.getScale() == row.get(2).asInt(); + } else { + return false; + } + } else { + return false; + } + } + } catch (IOException e) { + throw new RuntimeException(e); + } + } + + private void createTestTableTracker() throws SQLException, IOException { + FetchResult fetch = fetchData("create table testtables (tablename varchar(100) " + + markColumnPrimaryKey() + ", metastore varchar(5), fileformat varchar(10), scale int)"); + if (fetch.rc != ResultCode.SUCCESS) { + throw new RuntimeException("Unable to create test metadata table"); + } + if (clusterManager != null) clusterManager.registerTable(null, "testtables"); + } + + /** + * Record that a table was created for testing. The table name, scale, file format, and which + * type of metastore it is stored in will be recorded. + * @param table table that was created. + * @throws SQLException + */ + protected void recordTableCreation(TestTable table) throws SQLException { + StringBuilder sql = new StringBuilder(); + sql.append("insert into testtables (tablename, metastore, fileformat, scale) values ('") + .append(getTableName(table)) + .append("', '") + .append(TestConf.metastore()) + .append("', '") + .append(TestConf.fileFormat()) + .append("', ") + .append(TestConf.getScale()) + .append(')'); + try { + FetchResult fetch = fetchData(sql.toString()); + if (fetch.rc != ResultCode.SUCCESS) { + throw new RuntimeException("Unable to record data in test metadata table"); + } + if (clusterManager != null) clusterManager.registerTable(table.getDbName(), table.getTableName()); + } catch (IOException e) { + throw new RuntimeException(e); + } + } + + /** + * Get the proper string for "if exists" for a database. May be empty. Should not be null. + * @return if exists string + */ + protected abstract String ifExists(); + + /** + * Get the proper string for marking a single column a primary key. May be empty. Should not + * be null. + * @return SQL string to mark a column as a primary key. + */ + protected abstract String markColumnPrimaryKey(); + + /** + * Drop a table and remember that it's dropped. This is for testing. + * @param table table to drop + * @throws SQLException + * @throws IOException + */ + @VisibleForTesting + void dropTable(TestTable table) throws SQLException, IOException { + StringBuilder sql = new StringBuilder(); + sql.append("delete from testtables where tablename = '") + .append(getTableName(table)) + .append('\''); + try { + FetchResult dropFetch = fetchData("drop table " + ifExists() + " " + getTableName(table)); + if (dropFetch.rc != ResultCode.SUCCESS) { + LOG.debug("Failed to drop table, most likely means it does not exist yet" + dropFetch.rc); + } + FetchResult metaFetch = fetchData(sql.toString()); + if (metaFetch.rc != ResultCode.SUCCESS) { + LOG.debug("Failed to drop table from metadata, most likely means we don't know about it " + + "yet, code is " + metaFetch.rc); + } + } catch (SQLException | IOException e) { + LOG.debug("Failed to drop table from metadata, most likely means we don't know about it " + + "yet", e); + } + } + + /** + * Get the URL to connect to the JDBC driver. Subclasses that do not connect via JDBC can + * return null for this. + * @return URL + */ + protected abstract String connectionURL(); + + /** + * Get any properties that need to be passed as part of connecting to JDBC. Subclasses that do + * not connect via JDBC can return null for this. + * @return properties + */ + protected abstract Properties connectionProperties(); + + /** + * Get a JDBC connection. + * @param autoCommit whether or not the connection should be set to autoCommit + * @return JDBC Connection object + * @throws SQLException + */ + protected Connection connect(boolean autoCommit) throws SQLException { + Connection conn = jdbcDriver.connect(connectionURL(), connectionProperties()); + conn.setAutoCommit(autoCommit); + return conn; + } + + /** + * Do a fetch via JDBC. This assumes that {@link #connect} has already been called. + * @param sql SQL string to be executed. If it is a select then executeQuery() will be called, + * otherwise execute(). + * @param limit Limit the number of rows returned. This is for implementations like Derby that + * do not support a LIMIT clause. + * @param failureOk whether it's ok for this to fail. If true, SUCCESS will be returned + * regardless of the results of the operation. This is useful to replicate + * the functionality of "create table if not exists" + * @return A FetchResult, which contains information on success/failure (assuming failureOk is + * false) and the data (if there is a query) + * @throws SQLException + * @throws IOException + */ + protected FetchResult jdbcFetch(String sql, long limit, boolean failureOk) + throws SQLException, IOException { + // If this is a DML or DDL statement we have to handle it separately because you can't use + // stmt.executeQuery with those. + if (!sql.toLowerCase().startsWith("select")) return executeStatement(sql, failureOk); + Connection conn = connect(true); + Statement stmt = null; + try { + + stmt = conn.createStatement(); + ResultSet rs = stmt.executeQuery(sql); + return new FetchResult(new ResultSetDataSet(rs, limit)); + } catch (SQLException e) { + if (failureOk) { + LOG.info("Failed to execute query <" + sql + "> but continuing anyway because we " + + "think that's ok."); + return new FetchResult(ResultCode.SUCCESS); + } + LOG.debug("Failed to run SQL query <" + sql + ">, got exception ", e); + return new FetchResult(ResultCode.NON_RETRIABLE_FAILURE); + } finally { + if (stmt != null) stmt.close(); + conn.close(); + } + } + + private FetchResult executeStatement(String sql, boolean failureOk) throws SQLException { + Connection conn = connect(true); + Statement stmt = null; + try { + stmt = conn.createStatement(); + stmt.execute(sql); + return new FetchResult(ResultCode.SUCCESS); + } catch (SQLException e) { + if (failureOk) { + LOG.info("Failed to execute statement <" + sql + "> but continuing anyway because we " + + "think that's ok."); + return new FetchResult(ResultCode.SUCCESS); + } + LOG.debug("Failed to run SQL query <" + sql + ">, got exception ", e); + return new FetchResult(ResultCode.NON_RETRIABLE_FAILURE); + } finally { + if (stmt != null) stmt.close(); + conn.close(); + } + } +} diff --git itests/capybara/src/main/java/org/apache/hive/test/capybara/infra/DerbyBenchmark.java itests/capybara/src/main/java/org/apache/hive/test/capybara/infra/DerbyBenchmark.java new file mode 100644 index 0000000..bfe70c6 --- /dev/null +++ itests/capybara/src/main/java/org/apache/hive/test/capybara/infra/DerbyBenchmark.java @@ -0,0 +1,49 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hive.test.capybara.infra; + +import org.apache.hive.test.capybara.iface.Benchmark; +import org.apache.hive.test.capybara.iface.BenchmarkDataStore; +import org.apache.hive.test.capybara.iface.ResultComparator; +import org.apache.hive.test.capybara.iface.TableComparator; + +/** + * A Benchmark that uses Derby to store the results and standard ResultComparators for comparison. + */ +public class DerbyBenchmark implements Benchmark { + BenchmarkDataStore store; + + @Override + public BenchmarkDataStore getBenchDataStore() { + if (store == null) { + store = new DerbyStore(); + } + return store; + } + + @Override + public ResultComparator getResultComparator(boolean sort) { + if (sort) return new SortingComparator(); + else return new NonSortingComparator(); + } + + @Override + public TableComparator getTableComparator() { + return new TableComparator(); + } +} diff --git itests/capybara/src/main/java/org/apache/hive/test/capybara/infra/DerbyStore.java itests/capybara/src/main/java/org/apache/hive/test/capybara/infra/DerbyStore.java new file mode 100644 index 0000000..524f92e --- /dev/null +++ itests/capybara/src/main/java/org/apache/hive/test/capybara/infra/DerbyStore.java @@ -0,0 +1,302 @@ +/** + * 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.hive.test.capybara.infra; + +import org.apache.hive.test.capybara.data.DataSet; +import org.apache.hive.test.capybara.iface.TestTable; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import org.apache.derby.jdbc.EmbeddedDriver; + +import java.io.File; +import java.io.IOException; +import java.sql.Connection; +import java.sql.SQLException; +import java.sql.Statement; +import java.util.Properties; +import java.util.regex.Matcher; +import java.util.regex.Pattern; + +/** + * Controller for Derby. + */ +public class DerbyStore extends AnsiSqlStore { + private static final Logger LOG = LoggerFactory.getLogger(DerbyStore.class.getName()); + private static final String DELIMITER_STR = ","; + private static final String QUOTES = "\""; + private static final String NULL_STR = ""; + private static final String DERBY_URL; + + private long limit; + + static { + DERBY_URL = new StringBuilder("jdbc:derby:") + .append(System.getProperty("java.io.tmpdir")) + .append(System.getProperty("file.separator")) + .append("hiveitestdb;create=true") + .toString(); + } + + DerbyStore() { + jdbcDriver = new EmbeddedDriver(); + } + + @Override + public void loadData(TestTable table, DataSet rows) + throws SQLException, IOException { + dumpToFileForImport(rows); + File file = dataSetDumps.get(rows.uniqueId()); + StringBuilder sql = new StringBuilder("CALL SYSCS_UTIL.SYSCS_IMPORT_TABLE ("); + if (table.getDbName().equalsIgnoreCase("default")) sql.append("null, '"); + else sql.append('\'').append(table.getDbName().toUpperCase()).append("', '"); + // Table name must be upper cased. + sql.append(table.getTableName().toUpperCase()) + .append("', '") + .append(file.getAbsolutePath()) + .append("', '") + .append(DELIMITER_STR) + .append("', '") + .append(QUOTES) + .append("', null, 0)"); + + if (LOG.isDebugEnabled()) { + LOG.debug("Going to import into Derby with command " + sql.toString()); + } + Connection conn = connect(false); + Statement stmt = null; + try { + stmt = conn.createStatement(); + stmt.execute(sql.toString()); + } finally { + if (stmt != null) stmt.close(); + conn.close(); + } + } + + @Override + public String getTableName(TestTable table) { + return "default".equals(table.getDbName()) ? table.getTableName() : table.toString(); + } + + @Override + protected String hiveSqlToAnsiSql(String hiveSql) throws IOException, SQLException { + // TODO this will not return the right result for subqueries with limits + Matcher matcher = + Pattern.compile(".*\\slimit\\s+([0-9]+).*", Pattern.CASE_INSENSITIVE).matcher(hiveSql); + if (matcher.matches()) { + limit = Long.valueOf(matcher.group(1)); + // Now remove the limit. + matcher = Pattern.compile("\\slimit\\s+[0-9]+", Pattern.CASE_INSENSITIVE).matcher(hiveSql); + hiveSql = matcher.replaceAll(" "); + } else { + limit = Long.MAX_VALUE; + } + return super.hiveSqlToAnsiSql(hiveSql); + } + + @Override + protected String ifExists() { + return ""; + } + + @Override + protected String markColumnPrimaryKey() { + return "not null primary key"; + } + + @Override + protected String connectionURL() { + return DERBY_URL; + } + + @Override + protected Properties connectionProperties() { + return new Properties(); + } + + @Override + protected String fileColumnDelimiter() { + return DELIMITER_STR; + } + + @Override + protected String fileNull() { + return NULL_STR; + } + + @Override + protected String fileStringQuotes() { + return QUOTES; + } + + @Override + public Class getDriverClass() { + return jdbcDriver.getClass(); + } + + @Override + protected long getLimit() { + return limit; + } + + private SQLTranslator derbyTranslator = new SQLTranslator() { + @Override + protected String translateCreateDatabase(String hiveSql) throws + TranslationException { + String benchSql = super.translateCreateDatabase(hiveSql); + // Remove 'if not exists' + Matcher m = Pattern.compile("create schema if not exists").matcher(benchSql); + if (m.find()) { + failureOk = true; + return m.replaceAll("create schema"); + } + return benchSql; + } + + @Override + protected String translateDropDatabase(String hiveSql) throws TranslationException { + String benchSql = super.translateDropDatabase(hiveSql); + // Remove 'if exists' + Matcher m = Pattern.compile("drop schema if exists").matcher(benchSql); + if (m.find()) { + failureOk = true; + benchSql = m.replaceAll("drop schema"); + } + + // Derby requires restrict, so put it there if it's missing + m = Pattern.compile("cascade$").matcher(benchSql); + benchSql = m.replaceAll("restrict"); + if (!benchSql.contains("restrict")) { + benchSql = benchSql + " restrict"; + } + return benchSql; + } + + @Override + protected String translateCreateTableLike(Matcher matcher) { + StringBuilder sql = new StringBuilder(); + if (matcher.group(1) != null && matcher.group(1).equals("temporary ")) { + sql.append("declare global temporary table "); + } else { + sql.append("create table "); + } + if (matcher.group(2) != null) failureOk = true; + sql.append(matcher.group(3)) + .append(" as select * from ") + .append(matcher.group(4)); + return sql.toString(); + } + + @Override + protected String translateCreateTableAs(Matcher matcher) throws TranslationException { + StringBuilder sql = new StringBuilder(); + if (matcher.group(1) != null && matcher.group(1).equals("temporary ")) { + sql.append("declare global temporary table "); + } else { + sql.append("create table "); + } + if (matcher.group(2) != null) failureOk = true; + sql.append(matcher.group(3)) + .append(" as ") + .append(translateSelect(matcher.group(4))); + return sql.toString(); + } + + @Override + protected String translateCreateTableWithColDefs(Matcher matcher) { + StringBuilder sql = new StringBuilder(); + if (matcher.group(1) != null && matcher.group(1).equals("temporary ")) { + sql.append("declare global temporary table "); + } else { + sql.append("create table "); + } + if (matcher.group(2) != null) failureOk = true; + sql.append(matcher.group(3)) + .append(" (") + .append(translateDataTypes(parseOutColDefs(matcher.group(4)))); + return sql.toString(); + } + + @Override + protected String translateDataTypes(String hiveSql) { + Matcher m = Pattern.compile(" string").matcher(hiveSql); + hiveSql = m.replaceAll(" varchar(255)"); + m = Pattern.compile(" tinyint").matcher(hiveSql); + hiveSql = m.replaceAll(" smallint"); + m = Pattern.compile(" binary").matcher(hiveSql); + hiveSql = m.replaceAll(" blob"); + return hiveSql; + } + + @Override + protected String translateDropTable(String hiveSql) throws TranslationException { + // Need to remove purge and 'if exists' if they are there + Matcher m = Pattern.compile("drop table (if exists )?(" + tableNameRegex + ")").matcher(hiveSql); + if (m.lookingAt()) { + if (m.group(1) != null) failureOk = true; + return "drop table " + m.group(2); + } else { + throw new TranslationException("drop table", hiveSql); + } + } + + @Override + protected String translateAlterTableRename(String tableName, String remainder) throws + TranslationException { + LOG.error("Derby does not support alter table rename"); + throw new TranslationException("alter table rename", remainder); + } + + @Override + protected String translateConstants(String hiveSql) throws TranslationException { + String benchSql = super.translateConstants(hiveSql); + Matcher m = Pattern.compile("date " + QUOTE_START).matcher(benchSql); + benchSql = m.replaceAll(QUOTE_START); + m = Pattern.compile("timestamp " + QUOTE_START).matcher(benchSql); + benchSql = m.replaceAll(QUOTE_START); + return benchSql; + } + + @Override + protected String translateLimit(String hiveSql) throws TranslationException { + Matcher m = Pattern.compile("([0-9]+)").matcher(hiveSql); + if (m.find()) { + limit = Integer.valueOf(m.group(1)); + return ""; + } else { + throw new TranslationException("limit", hiveSql); + } + } + + @Override + protected void fillOutUdfMapping() { + super.fillOutUdfMapping(); + udfMapping.put("substring", "substr"); + } + + @Override + protected char identifierQuote() { + return '"'; + } + }; + + @Override + protected SQLTranslator getTranslator() { + return derbyTranslator; + } +} diff --git itests/capybara/src/main/java/org/apache/hive/test/capybara/infra/ExternalClusterManager.java itests/capybara/src/main/java/org/apache/hive/test/capybara/infra/ExternalClusterManager.java new file mode 100644 index 0000000..b4eac19 --- /dev/null +++ itests/capybara/src/main/java/org/apache/hive/test/capybara/infra/ExternalClusterManager.java @@ -0,0 +1,115 @@ +/** + * 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.hive.test.capybara.infra; + +import org.apache.hive.test.capybara.iface.ClusterManager; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileSystem; + +import java.io.IOException; +import java.util.HashMap; +import java.util.Map; + +/** + * Manage external clusters. + */ +public class ExternalClusterManager implements ClusterManager { + static final private Logger LOG = LoggerFactory.getLogger(ExternalClusterManager.class); + + private Configuration conf; + private FileSystem fs; + private HiveStore hive; + private Map confVars; + + @Override + public void setup() { + // NOP + } + + @Override + public void tearDown() { + // NOP + } + + @Override + public boolean remote() { + return true; + } + + @Override + public FileSystem getFileSystem() throws IOException { + assert conf != null; + if (fs == null) { + fs = FileSystem.get(conf); + LOG.debug("Returning file system, fs.defaultFS is " + conf.get("fs.defaultFS")); + } + return fs; + } + + @Override + public void setConf(Configuration conf) { + this.conf = conf; + } + + @Override + public Configuration getConf() { + return conf; + } + + @Override + public HiveStore getHive() { + assert conf != null; + if (hive == null) { + hive = new ClusterCliHiveStore(this); + } + // Reset the conf file each time, because it may have changed + hive.setConf(conf); + return hive; + } + + @Override + public String getJdbcURL() { + // TODO + return null; + } + + @Override + public void setConfVar(String var, String val) { + if (confVars == null) confVars = new HashMap<>(); + confVars.put(var, val); + } + + @Override + public Map getConfVars() { + return confVars; + } + + @Override + public void unsetHive() { + // NOP, since we're not directly connected to Hive we don't need to build a new instance each + // time to reset the config. + + } + + @Override + public void registerTable(String dbName, String tableName) { + // NOP, as we want tables to be long lived on external clusters. + } +} diff --git itests/capybara/src/main/java/org/apache/hive/test/capybara/infra/GeneratedDataSet.java itests/capybara/src/main/java/org/apache/hive/test/capybara/infra/GeneratedDataSet.java new file mode 100644 index 0000000..90185fd --- /dev/null +++ itests/capybara/src/main/java/org/apache/hive/test/capybara/infra/GeneratedDataSet.java @@ -0,0 +1,129 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hive.test.capybara.infra; + +import org.apache.hive.test.capybara.data.DataSet; +import org.apache.hive.test.capybara.data.Row; +import org.apache.hive.test.capybara.iface.DataStore; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import org.apache.hadoop.hive.metastore.api.FieldSchema; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Iterator; +import java.util.List; + +/** + * A generated DataSet. This should never be used for data read out of a + * {@link org.apache.hive.test.capybara.iface.DataStore}, but only for data generated by a + * {@link org.apache.hive.test.capybara.DataGenerator}. It makes assumptions about spilling and + * remembering what rows it has that will go all wrong if you try to store data + * from a DataStore in it. + */ +class GeneratedDataSet extends DataSet { + + static final private Logger LOG = LoggerFactory.getLogger(GeneratedDataSet.class.getName()); + + private int currentSize; + private int spillSize; // Size at which we spill the contents to disk. + private DataStore hive; + private DataStore bench; + private Thread hiveDumper; + private Thread benchDumper; + + + GeneratedDataSet(List schema) { + super(schema); + rows = new ArrayList<>(); + spillSize = TestConf.getSpillSize(); + } + + @Override + public Iterator iterator() { + // If we've started spilling, we need to complete that before anyone reads from this. + try { + if (hiveDumper != null) hiveDumper.join(); + if (benchDumper != null) benchDumper.join(); + } catch (InterruptedException e) { + throw new RuntimeException(e); + } + return rows.iterator(); + } + + void addRow(Row row) { + rows.add(row); + currentSize += row.lengthInBytes(); + if (currentSize > spillSize) spill(); + } + + private void spill() { + LOG.debug("Spilling rows because currentSize got to " + currentSize); + try { + if (hive == null) hive = TestManager.getTestManager().getClusterManager().getHive(); + if (bench == null) bench = TestManager.getTestManager().getBenchmark().getBenchDataStore(); + // Wait for previous instances of the spilling to complete before we start it again. This + // avoids building up too much in memory. + if (hiveDumper != null) hiveDumper.join(); + if (benchDumper != null) benchDumper.join(); + hiveDumper = new Dumper(hive, new QueuedDataSet(this)); + benchDumper = new Dumper(bench, new QueuedDataSet(this)); + hiveDumper.start(); + benchDumper.start(); + currentSize = 0; + // Don't clear the list, as we're using it to spill. + rows = new ArrayList<>(rows.size()); + } catch (InterruptedException e) { + throw new RuntimeException(e); + } + } + + private class Dumper extends Thread { + final DataStore store; + final DataSet data; + IOException stashed; + + public Dumper(DataStore store, DataSet data) { + this.store = store; + this.data = data; + } + + @Override + public void run() { + try { + store.dumpToFileForImport(data); + } catch (IOException e) { + stashed = e; + } + } + } + + private class QueuedDataSet extends DataSet { + public QueuedDataSet(GeneratedDataSet outer) { + super(outer.schema); + rows = outer.rows; + } + + @Override + public Iterator iterator() { + return rows.iterator(); + } + } + + +} diff --git itests/capybara/src/main/java/org/apache/hive/test/capybara/infra/HiveStore.java itests/capybara/src/main/java/org/apache/hive/test/capybara/infra/HiveStore.java new file mode 100644 index 0000000..47c91f8 --- /dev/null +++ itests/capybara/src/main/java/org/apache/hive/test/capybara/infra/HiveStore.java @@ -0,0 +1,320 @@ +/** + * 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.hive.test.capybara.infra; + +import org.apache.hive.test.capybara.data.DataSet; +import org.apache.hive.test.capybara.data.FetchResult; +import org.apache.hive.test.capybara.data.ResultCode; +import org.apache.hive.test.capybara.iface.ClusterManager; +import org.apache.hive.test.capybara.iface.TestTable; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import org.apache.hadoop.conf.Configurable; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FSDataOutputStream; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hive.common.ObjectPair; +import org.apache.hadoop.hive.conf.HiveConf; +import org.apache.hadoop.hive.metastore.HiveMetaStoreClient; +import org.apache.hadoop.hive.metastore.IMetaStoreClient; +import org.apache.hadoop.hive.metastore.api.FieldSchema; +import org.apache.hadoop.hive.metastore.api.MetaException; +import org.apache.hadoop.hive.ql.QueryPlan; +import org.junit.Assert; + +import java.io.IOException; +import java.sql.SQLException; +import java.util.HashMap; +import java.util.Iterator; +import java.util.Map; +import java.util.UUID; + +/** + * A class for a Hive store. All implementations of this that will run on the cluster must have + * a no-args constructor. + */ +public abstract class HiveStore extends DataStoreBase implements Configurable { + public static final char DELIMITER = '\u0001'; + public static final String DELIMITER_STR = new String(new char[]{DELIMITER}); + public static final String NULL_STR = "\u0005"; + public static final String QUOTES = ""; + private static final Logger LOG = LoggerFactory.getLogger(HiveStore.class.getName()); + + protected HiveConf conf; + protected IMetaStoreClient msClient; + /** + * A map from DataSets to places they've been dumped. The key is obtained by calling + * DataStore.uniqueId(). The value is a pair, with the first element being the HDFS directory + * the data is dumped in, and the second being the actual file. + */ + private Map> dataSetDumps; + + /** + * A constructor for use on the cluster when we need don't have a clusterManager. + */ + HiveStore() { + + } + + protected HiveStore(ClusterManager clusterManager) { + super(clusterManager); + dataSetDumps = new HashMap<>(); + } + + public IMetaStoreClient getMetastoreConnection() throws MetaException { + assert conf != null; + if (msClient == null) { + msClient = new HiveMetaStoreClient(conf); + } + return msClient; + } + + @Override + public boolean createTable(TestTable table) throws SQLException, IOException { + return createTableInternal(table, false); + } + + @Override + public void forceCreateTable(TestTable table) throws SQLException, IOException { + createTableInternal(table, true); + } + + @Override + public void loadData(TestTable table, DataSet rows) + throws SQLException, IOException { + // Dump the data into an HDFS file. Then create a table with that as it's location. + // Finally run a SQL statement to insert into the destination table from there. + String tmpTableName = table.getDbName() + "_" + table.getTableName() + "_tmp_load"; + + // Check whether this data has already been dumped. If so, don't do it again. + Path dir = rows.getClusterLocation(); + if (dir == null) { + // It hasn't been dumped, so do it now. + dumpToFileForImport(rows); + dir = dataSetDumps.get(rows.uniqueId()).getFirst(); + } + + + // Drop the temp table in case we've previously created it. Ignore any errors. + fetchData("drop table " + tmpTableName); + + // Create a temp table with the file in the right location. + StringBuilder sql = new StringBuilder("create table ").append(tmpTableName).append(" ("); + boolean first = true; + for (FieldSchema colSchema : table.getCols()) { + if (first) first = false; + else sql.append(", "); + sql.append(colSchema.getName()) + .append(' ') + .append(colSchema.getType()); + } + if (table.getPartCols() != null) { + for (FieldSchema partSchema : table.getPartCols()) { + sql.append(", ") + .append(partSchema.getName()) + .append(' ') + .append(partSchema.getType()); + } + } + sql.append(") row format delimited fields terminated by '\\001' null defined as '\\005' ") + .append("stored as textfile ") + .append("location '").append(dir.toUri().toString()).append("'"); + + FetchResult res = fetchData(sql.toString()); + Assert.assertEquals(ResultCode.SUCCESS, res.rc); + + // Now, insert from the temp table to the target table. + sql = new StringBuilder("insert into ") + .append(table.toString()); + if (table.getPartCols() != null) { + sql.append(" partition ("); + first = true; + for (FieldSchema partCol : table.getPartCols()) { + if (first) first = false; + else sql.append(", "); + sql.append(partCol.getName()); + } + sql.append(")"); + } + sql.append(" select * from ").append(tmpTableName); + + LOG.debug("Going to send to Hive: " + sql.toString()); + res = fetchData(sql.toString()); + Assert.assertEquals(ResultCode.SUCCESS, res.rc); + } + + @Override + public void dumpToFileForImport(DataSet rows) throws IOException { + ClusterManager clusterMgr = TestManager.getTestManager().getClusterManager(); + FileSystem fs = clusterMgr.getFileSystem(); + ObjectPair pathPair = dataSetDumps.get(rows.uniqueId()); + if (pathPair == null) { + Path dir = new Path(getDirForDumpFile()); + LOG.debug("Going to dump data for import to " + dir.toUri().toString()); + fs.mkdirs(dir); + Path file = new Path(dir, "file"); + FSDataOutputStream output = fs.create(file); + writeToFile(output, rows); + dataSetDumps.put(rows.uniqueId(), new ObjectPair<>(dir, file)); + } else { + Path file = pathPair.getSecond(); + FSDataOutputStream output = fs.append(file); + writeToFile(output, rows); + } + } + + @Override + public void setConf(Configuration conf) { + this.conf = (HiveConf)conf; + + } + + @Override + public Configuration getConf() { + return conf; + } + + public HiveConf getHiveConf() { + return conf; + } + + @Override + public String getTableName(TestTable table) { + return table.toString(); + } + + /** + * Explain a SQL query. This should only be implemented by local implementations, not on the + * cluster when the test harness may not share the configuration with the cluster. + * @param sql SQL to explain + * @return plan for this SQL + */ + public abstract QueryPlan explain(String sql); + + /** + * Get a URI to connect to the metastore. In the local case this should be null. + * @return uri + */ + public abstract String getMetastoreUri(); + + /** + * Generate a random directory to dump generated data in. + * @return directory name + */ + public static String getDirForDumpFile() { + // Use '/' explicitly rather than file.separator property as HDFS always uses forward slash. + String filename = new StringBuilder(TestManager.getTestManager().getConf().get("hadoop.tmp.dir")) + .append('/') + .append("capybara_") + .append(UUID.randomUUID().toString()) + .toString(); + return filename; + } + + /** + * Generate a random file name to dump generated data in. + * @return filename + */ + static String getFileForDump() { + return UUID.randomUUID().toString(); + } + + /** + * Write generated data to a file. + * @param output output stream to use. + * @param data DataSet to dump. + * @throws IOException + */ + static void writeToFile(FSDataOutputStream output, DataSet data) throws IOException { + Iterator iter = data.stringIterator(DELIMITER_STR, NULL_STR, QUOTES); + String lineSep = System.getProperty("line.separator"); + while (iter.hasNext()) { + output.writeBytes(iter.next() + lineSep); + } + output.close(); + } + + @Override + protected String ifExists() { + return " if exists "; + } + + @Override + protected String markColumnPrimaryKey() { + return ""; + } + + private boolean createTableInternal(TestTable table, boolean force) + throws SQLException, IOException { + if (!force && tableExistsInCorrectState(table)) return false; + FetchResult rc = fetchData("drop table if exists " + table.toString()); + Assert.assertEquals(ResultCode.SUCCESS, rc.rc); + + StringBuilder builder = new StringBuilder(); + builder.append("create table ") + .append(table.toString()) + .append(" ("); + boolean first = true; + for (FieldSchema col : table.getCols()) { + if (first) first = false; + else builder.append(", "); + builder.append(col.getName()) + .append(' ') + .append(col.getType()); + } + builder.append(')'); + if (table.getPartCols() != null) { + builder.append(" partitioned by ("); + first = true; + for (FieldSchema partCol : table.getPartCols()) { + if (first) first = false; + else builder.append(", "); + builder.append(partCol.getName()) + .append(' ') + .append(partCol.getType()); + } + builder.append(')'); + } + + if (table.getNumBuckets() > 0) { + builder.append(" clustered by ("); + first = true; + for (String bucketCol : table.getBucketCols()) { + if (first) first = false; + else builder.append(", "); + builder.append(bucketCol); + } + builder.append(") into ") + .append(table.getNumBuckets()) + .append(" buckets"); + } + + if (table.isAcid()) { + if (!TestConf.fileFormat().equals(TestConf.FILE_FORMAT_ORC)) { + builder.append(" stored as orc "); + } + builder.append(" tblproperties ('transactional'='true')"); + } + + rc = fetchData(builder.toString()); + Assert.assertEquals(ResultCode.SUCCESS, rc.rc); + if (!force) recordTableCreation(table); + return true; + } +} diff --git itests/capybara/src/main/java/org/apache/hive/test/capybara/infra/IntegrationRunner.java itests/capybara/src/main/java/org/apache/hive/test/capybara/infra/IntegrationRunner.java new file mode 100644 index 0000000..4ea5207 --- /dev/null +++ itests/capybara/src/main/java/org/apache/hive/test/capybara/infra/IntegrationRunner.java @@ -0,0 +1,132 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hive.test.capybara.infra; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import org.apache.hive.test.capybara.IntegrationTest; +import org.junit.runners.BlockJUnit4ClassRunner; +import org.junit.runners.model.FrameworkMethod; +import org.junit.runners.model.InitializationError; +import org.junit.Assert; + +import java.lang.annotation.Annotation; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +/** + * Runner for integration tests. This extends the standard junit4 runner, but pays attention to + * our annotations to weed out some tests. + */ +public class IntegrationRunner extends BlockJUnit4ClassRunner { + static final private Logger LOG = LoggerFactory.getLogger(IntegrationRunner.class); + + private Map> testVisibleAnnotations; + + public IntegrationRunner(Class klass) throws InitializationError { + super(klass); + } + + @Override + protected List computeTestMethods() { + if (testVisibleAnnotations == null) testVisibleAnnotations = new HashMap<>(); + + // Get the list from our parent, then weed out any that don't match what we're currently + // running. + List toReturn = new ArrayList<>(); + List methods = super.computeTestMethods(); + for (FrameworkMethod method : methods) { + boolean skipIt = false; + List annotations = new ArrayList<>(); + annotations.addAll(Arrays.asList(getTestClass().getAnnotations())); + annotations.addAll(Arrays.asList(method.getAnnotations())); + for (Annotation annotation : annotations) { + LOG.trace("Considering annotation " + annotation.annotationType().getSimpleName()); + if ("NoCli".equals(annotation.annotationType().getSimpleName()) && + TestConf.access().equals("cli") || + "NoJdbc".equals(annotation.annotationType().getSimpleName()) && + TestConf.access().equals("jdbc") || + "NoNonSecure".equals(annotation.annotationType().getSimpleName()) && + TestConf.security().equals("nonsecure") || + "NoSecure".equals(annotation.annotationType().getSimpleName()) && + TestConf.security().equals("secure") || + "NoOrc".equals(annotation.annotationType().getSimpleName()) && + TestConf.fileFormat().equals("orc") || + "NoParquet".equals(annotation.annotationType().getSimpleName()) && + TestConf.fileFormat().equals("parquet") || + "NoRcFile".equals(annotation.annotationType().getSimpleName()) && + TestConf.fileFormat().equals("rc") || + "NoTextFile".equals(annotation.annotationType().getSimpleName()) && + TestConf.fileFormat().equals("text") || + "NoSpark".equals(annotation.annotationType().getSimpleName()) && + TestConf.engine().equals("spark") || + "NoTez".equals(annotation.annotationType().getSimpleName()) && + TestConf.engine().equals("tez") || + "NoRdbmsMetastore".equals(annotation.annotationType().getSimpleName()) && + TestConf.metastore().equals("metastore") || + "NoHBaseMetastore".equals(annotation.annotationType().getSimpleName()) && + TestConf.metastore().equals("hbase") || + "RequireCluster".equals(annotation.annotationType().getSimpleName()) && + !TestConf.onCluster()) { + skipIt = true; + LOG.debug("Skipping test " + method.getName() + " because it is annotated with " + + annotation.annotationType().getSimpleName()); + break; + } else { + LOG.trace("Not skipping test " + method.getName()); + if (annotation.annotationType().getSimpleName().matches(".*On$") || + annotation.annotationType().getSimpleName().equals("MetadataOnly")) { + List a = testVisibleAnnotations.get(method.getName()); + if (a == null) { + a = new ArrayList<>(); + testVisibleAnnotations.put(method.getName(), a); + } + a.add(annotation); + } + } + } + if (!skipIt) toReturn.add(method); + } + if (toReturn.size() == 0) { + // JUnit complains when we give it no tests. To avoid this put in a phantom test that does + // nothing + try { + toReturn.add(new FrameworkMethod(IntegrationTest.class.getMethod("phantomTest"))); + } catch (NoSuchMethodException e) { + throw new RuntimeException("Couldn't find phantomTest!", e); + } + } + return toReturn; + } + + @Override + protected Object createTest() throws Exception { + Object o = super.createTest(); + if (!(o instanceof IntegrationTest)) { + throw new RuntimeException("Unexpected test type " + o.getClass().getName()); + } + Assert.assertNotNull(testVisibleAnnotations); + ((IntegrationTest)o).setAnnotations(testVisibleAnnotations); + return o; + } + + +} diff --git itests/capybara/src/main/java/org/apache/hive/test/capybara/infra/MiniCliHiveStore.java itests/capybara/src/main/java/org/apache/hive/test/capybara/infra/MiniCliHiveStore.java new file mode 100644 index 0000000..2f4f9bc --- /dev/null +++ itests/capybara/src/main/java/org/apache/hive/test/capybara/infra/MiniCliHiveStore.java @@ -0,0 +1,78 @@ +/** + * 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.hive.test.capybara.infra; + +import org.apache.hive.test.capybara.data.FetchResult; +import org.apache.hive.test.capybara.data.ResultCode; +import org.apache.hive.test.capybara.iface.ClusterManager; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import org.apache.hadoop.hive.ql.CommandNeedRetryException; +import org.apache.hadoop.hive.ql.processors.CommandProcessorResponse; + +import java.io.IOException; +import java.sql.SQLException; +import java.util.ArrayList; +import java.util.List; +import java.util.Properties; + +/** + * An implementation of HiveStore that works with mini clusters and the cli. + */ +class MiniCliHiveStore extends MiniHiveStoreBase { + + static final private Logger LOG = LoggerFactory.getLogger(MiniCliHiveStore.class.getName()); + + + MiniCliHiveStore(ClusterManager clusterManager) { + super(clusterManager); + } + + @Override + public FetchResult fetchData(String sql) throws SQLException, IOException { + try { + LOG.debug("Going to send to Hive: " + sql); + CommandProcessorResponse rsp = getDriver().run(sql); + if (rsp.getResponseCode() != 0) { + LOG.info("Response code from query <" + sql + "> is " + rsp.getResponseCode() + ", error" + + " message <" + rsp.getErrorMessage() + "> SQLState <" + rsp.getSQLState() + ">"); + return new FetchResult(ResultCode.NON_RETRIABLE_FAILURE); + } + List rows = new ArrayList<>(); + if (getDriver().getResults(rows)) { + return new FetchResult(new StringDataSet(rows, "\t", "NULL")); + } else { + return new FetchResult(ResultCode.SUCCESS); + } + } catch (CommandNeedRetryException e) { + LOG.info("Query <" + sql + "> failed with retriable error"); + return new FetchResult(ResultCode.RETRIABLE_FAILURE); + } + } + + @Override + protected String connectionURL() { + return null; + } + + @Override + protected Properties connectionProperties() { + return null; + } +} + diff --git itests/capybara/src/main/java/org/apache/hive/test/capybara/infra/MiniClusterManager.java itests/capybara/src/main/java/org/apache/hive/test/capybara/infra/MiniClusterManager.java new file mode 100644 index 0000000..2515f18 --- /dev/null +++ itests/capybara/src/main/java/org/apache/hive/test/capybara/infra/MiniClusterManager.java @@ -0,0 +1,265 @@ +/** + * 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.hive.test.capybara.infra; + +import org.apache.hadoop.crypto.key.KeyProviderCryptoExtension; +import org.apache.hadoop.fs.CommonConfigurationKeysPublic; +import org.apache.hadoop.hive.shims.Utils; + +import org.apache.hive.test.capybara.iface.ClusterManager; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.FileUtil; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hdfs.MiniDFSCluster; +import org.apache.hadoop.hive.conf.HiveConf; +import org.apache.hadoop.hive.ql.WindowsPathUtil; +import org.apache.hadoop.mapreduce.MRJobConfig; +import org.apache.hive.jdbc.miniHS2.MiniHS2; +import org.apache.tez.test.MiniTezCluster; + +import java.io.File; +import java.io.IOException; +import java.sql.SQLException; +import java.util.ArrayList; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +/** + * Manager for mini-clusters + */ +public class MiniClusterManager implements ClusterManager { + static final private Logger LOG = LoggerFactory.getLogger(MiniClusterManager.class.getName()); + + private static final String DFS_DIR = "minidfs"; + private static final String TEZ_DIR = "apps_staging_dir"; + private Configuration conf; + private MiniDFSCluster dfs; + private MiniTezCluster tez; + private HiveStore hive; + private MiniHS2 hs2; + private List registeredTables = new ArrayList<>(); + private Map confVars = new HashMap<>(); + + @Override + public void setup() throws IOException { + File base = new File(baseDir() + DFS_DIR).getAbsoluteFile(); + + FileUtil.fullyDelete(base); + conf.set(MiniDFSCluster.HDFS_MINIDFS_BASEDIR, base.getAbsolutePath()); + // If we are running with security off, we need to turn off permissions in HDFS as it messes + // up MiniHS2 + // TODO - not clear if I need this. + /* + if (TestConf.security().equals(TestConf.SECURITY_NONSECURE)) { + conf.setBoolean("dfs.permissions.enabled", false); + } + */ + + String user; + try { + user = Utils.getUGI().getShortUserName(); + } catch (Exception e) { + String msg = "Cannot obtain username: " + e; + throw new IllegalStateException(msg, e); + } + conf.set("hadoop.proxyuser." + user + ".groups", "*"); + conf.set("hadoop.proxyuser." + user + ".hosts", "*"); + + dfs = new MiniDFSCluster.Builder(conf).numDataNodes(4).format(true).build(); + + // Need to set the client's KeyProvider to the NN's for JKS, + // else the updates do not get flushed properly + KeyProviderCryptoExtension keyProvider = dfs.getNameNode().getNamesystem().getProvider(); + if (keyProvider != null) { + dfs.getFileSystem().getClient().setKeyProvider(keyProvider); + } + + String nameNodeUri = WindowsPathUtil.getHdfsUriString(getFileSystem().getUri().toString()); + Path warehouseDir = new Path(nameNodeUri,"/user/hive/warehouse"); + dfs.getFileSystem().mkdirs(warehouseDir); + + // Now, set the location for the metastore so it doesn't end up in the goofy pfile stuff + LOG.debug("Setting warhouse dir to " + warehouseDir.toUri()); + HiveConf.setVar(conf, HiveConf.ConfVars.METASTOREWAREHOUSE, warehouseDir.toUri().toString()); + HiveConf.setBoolVar(conf, HiveConf.ConfVars.HIVE_SUPPORT_CONCURRENCY, false); + // Don't think I need to remember the URI, but keeping this here in case I do. + // dfsURI = "hdfs://localhost:"+ dfs.getNameNodePort(); + + // if this is running Tez, construct Tez mini-cluster + if (TestConf.engine().equals(TestConf.ENGINE_TEZ)) { + tez = new MiniTezCluster("hive", TestConf.numTezTasks()); + conf.set("mapred.tez.java.opts","-Xmx128m"); + conf.setInt("hive.tez.container.size", 128); + conf.setBoolean("hive.merge.tezfiles", false); + conf.set("hive.tez.java.opts", "-Dlog4j.configurationFile=tez-container-log4j2.xml -Dtez.container.log.level=INFO -Dtez.container.root.logger=CLA"); + conf.set("tez.am.launch.cmd-opts", "-Dlog4j.configurationFile=tez-container-log4j2.xml -Dtez.container.log.level=INFO -Dtez.container.root.logger=CLA"); + conf.set("hive.tez.input.format", "org.apache.hadoop.hive.ql.io.HiveInputFormat"); + conf.set("fs.defaultFS", nameNodeUri); + conf.set("tez.am.log.level", "DEBUG"); + conf.set(MRJobConfig.MR_AM_STAGING_DIR, baseDir() + TEZ_DIR); + HiveConf.setVar(conf, HiveConf.ConfVars.HIVE_EXECUTION_ENGINE, "tez"); + + LOG.debug("Starting mini tez cluster"); + tez.init(conf); + tez.start(); + LOG.debug("Starting complete"); + + Configuration config = tez.getConfig(); + for (Map.Entry pair: config) { + conf.set(pair.getKey(), pair.getValue()); + } + + Path jarPath = new Path("hdfs:///user/hive"); + Path hdfsPath = new Path("hdfs:///user/"); + + try { + FileSystem fs = getFileSystem(); + jarPath = fs.makeQualified(jarPath); + conf.set("hive.jar.directory", jarPath.toString()); + fs.mkdirs(jarPath); + hdfsPath = fs.makeQualified(hdfsPath); + conf.set("hive.user.install.directory", hdfsPath.toString()); + fs.mkdirs(hdfsPath); + } catch (Exception e) { + LOG.error("failed setup: ", e); + } + + conf.set(CommonConfigurationKeysPublic.FS_DEFAULT_NAME_KEY, nameNodeUri); + } else if (TestConf.engine().equals(TestConf.ENGINE_UNSPECIFIED)) { + // ok, hope you know what you're doing + } else { + throw new RuntimeException("Unknown engine: " + TestConf.engine()); + } + + if (TestConf.access().equals(TestConf.ACCESS_JDBC)) { + try { + hs2 = new MiniHS2((HiveConf)conf, dfs.getFileSystem()); + hs2.start(Collections.emptyMap()); + } catch (Exception e) { + throw new IOException(e); + } + } + + // TODO if this is running Hbase metastore, construct HBase mini-cluster + // TODO if this is running secure, construct mini kdc + } + + @Override + public void tearDown() { + try { + for (String dbNameTableName : registeredTables) { + try { + getHive().fetchData("drop table " + dbNameTableName); + } catch (SQLException | IOException e) { + LOG.error("Unable to drop table " + dbNameTableName, e); + throw new RuntimeException(e); + } + } + } finally { + // tear down any mini-clusters we've constructed. + if (tez != null) tez.stop(); + if (dfs != null) dfs.shutdown(); + } + // tear down any mini-clusters we've constructed. + if (dfs != null) dfs.shutdown(); + if (tez != null) tez.stop(); + if (hs2 != null) hs2.stop(); + } + + @Override + public boolean remote() { + return false; + } + + @Override + public FileSystem getFileSystem() throws IOException { + return dfs.getFileSystem(); + } + + @Override + public HiveStore getHive() { + assert conf != null; + if (hive == null) { + String access = TestConf.access(); + if (access.equals(TestConf.ACCESS_CLI)) { + hive = new MiniCliHiveStore(this); + } else if (access.equals(TestConf.ACCESS_JDBC)) { + hive = new MiniHS2HiveStore(this); + } else { + throw new RuntimeException("Unknown access method " + access); + } + hive.setConf(conf); + } + return hive; + } + + @Override + public String getJdbcURL() { + if (hs2 == null) { + throw new RuntimeException("No in JDBC mode!"); + } + return hs2.getJdbcURL(); + } + + @Override + public void unsetHive() { + hive = null; + } + + @Override + public void registerTable(String dbName, String tableName) { + StringBuilder builder = new StringBuilder(); + if (dbName != null) { + builder.append(dbName) + .append('.'); + } + builder.append(tableName); + registeredTables.add(builder.toString()); + } + + @Override + public void setConfVar(String var, String val) { + confVars.put(var, val); + conf.set(var, val); + } + + @Override + public Map getConfVars() { + return confVars; + } + + @Override + public void setConf(Configuration conf) { + this.conf = conf; + + } + + @Override + public Configuration getConf() { + return conf; + } + + private String baseDir() { + return System.getProperty("java.io.tmpdir") + System.getProperty("file.separator"); + } +} diff --git itests/capybara/src/main/java/org/apache/hive/test/capybara/infra/MiniHS2HiveStore.java itests/capybara/src/main/java/org/apache/hive/test/capybara/infra/MiniHS2HiveStore.java new file mode 100644 index 0000000..078a5b3 --- /dev/null +++ itests/capybara/src/main/java/org/apache/hive/test/capybara/infra/MiniHS2HiveStore.java @@ -0,0 +1,66 @@ +/** + * 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.hive.test.capybara.infra; + +import org.apache.hive.test.capybara.data.FetchResult; +import org.apache.hive.test.capybara.iface.ClusterManager; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.sql.Driver; +import java.sql.SQLException; +import java.util.Map; +import java.util.Properties; + +/** + * A HiveStore for JDBC connections (via MiniHS2) on the local machine. + */ +class MiniHS2HiveStore extends MiniHiveStoreBase { + private static final Logger LOG = LoggerFactory.getLogger(AnsiSqlStore.class.getName()); + + public MiniHS2HiveStore(ClusterManager clusterManager) { + super(clusterManager); + try { + Class cls = Class.forName("org.apache.hive.jdbc.HiveDriver"); + jdbcDriver = (Driver)cls.newInstance(); + } catch (ClassNotFoundException|IllegalAccessException|InstantiationException e) { + throw new RuntimeException(e); + } + } + + @Override + public FetchResult fetchData(String sql) throws SQLException, IOException { + LOG.debug("Going to run query <" + sql + "> against MiniHS2"); + return jdbcFetch(sql, Long.MAX_VALUE, false); + } + + @Override + protected String connectionURL() { + return clusterManager.getJdbcURL(); + } + + @Override + protected Properties connectionProperties() { + Properties p = new Properties(); + for (Map.Entry entry : clusterManager.getConfVars().entrySet()) { + p.setProperty(entry.getKey(), entry.getValue()); + } + return p; + } +} diff --git itests/capybara/src/main/java/org/apache/hive/test/capybara/infra/MiniHiveStoreBase.java itests/capybara/src/main/java/org/apache/hive/test/capybara/infra/MiniHiveStoreBase.java new file mode 100644 index 0000000..6b2b2dd --- /dev/null +++ itests/capybara/src/main/java/org/apache/hive/test/capybara/infra/MiniHiveStoreBase.java @@ -0,0 +1,62 @@ +/** + * 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.hive.test.capybara.infra; + +import org.apache.hadoop.hive.ql.Driver; +import org.apache.hadoop.hive.ql.QueryPlan; +import org.apache.hadoop.hive.ql.session.SessionState; +import org.apache.hive.test.capybara.iface.ClusterManager; +import org.junit.Assert; + +abstract class MiniHiveStoreBase extends HiveStore { + private ThreadLocal driver; + + public MiniHiveStoreBase(ClusterManager clusterManager) { + super(clusterManager); + } + + protected Driver getDriver() { + if (driver == null) { + driver = new ThreadLocal() { + @Override + protected Driver initialValue() { + assert conf != null; + // Make sure our conf file gets set + SessionState.start(conf); + assert SessionState.get() != null; + return new Driver(conf); + } + }; + } + return driver.get(); + } + + @Override + public QueryPlan explain(String sql) { + getDriver(); + if (driver.get().compile(sql) != 0) { + Assert.fail("Failed to compile <" + sql + ">"); + } + return driver.get().getPlan(); + } + + @Override + public String getMetastoreUri() { + return null; + } +} diff --git itests/capybara/src/main/java/org/apache/hive/test/capybara/infra/NonSortingComparator.java itests/capybara/src/main/java/org/apache/hive/test/capybara/infra/NonSortingComparator.java new file mode 100644 index 0000000..816e25d --- /dev/null +++ itests/capybara/src/main/java/org/apache/hive/test/capybara/infra/NonSortingComparator.java @@ -0,0 +1,101 @@ +/** + * 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.hive.test.capybara.infra; + +import org.apache.hive.test.capybara.data.DataSet; +import org.apache.hive.test.capybara.data.Row; +import org.apache.hive.test.capybara.iface.ResultComparator; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import org.junit.Assert; + +import java.io.IOException; +import java.sql.SQLException; +import java.util.Iterator; +import java.util.PriorityQueue; + +/** + * Compare two data sets without sorting Hive. We still sort the benchmark because every + * database sorts things just a little differently (like NULLs first or last, collation, etc.). + */ +public class NonSortingComparator extends ResultComparator { + static final private Logger LOG = LoggerFactory.getLogger(NonSortingComparator.class.getName()); + + @Override + public void compare(DataSet hive, DataSet bench) throws SQLException, IOException { + compareSchemas(hive, bench); + + compare(hive.iterator(), sort(bench)); + } + + protected void compare(Iterator hiveIter, Iterator benchIter) { + // Keep a bounded queue of records so that if there's a failure we can print some context + // around the failed lines rather than just the individual lines. + BoundedQueue hiveQueue = new BoundedQueue<>(25); + BoundedQueue benchQueue = new BoundedQueue<>(25); + + int rowNum = 0; + while (hiveIter.hasNext()) { + Assert.assertTrue("Benchmark ran out of rows at " + rowNum + " but hive still has rows", + benchIter.hasNext()); + rowNum++; + Row hiveRow = hiveIter.next(); + Row benchRow = benchIter.next(); + hiveQueue.add(hiveRow); + benchQueue.add(benchRow); + + if (!hiveRow.equals(benchRow)) { + + LOG.debug("Last 25 Hive rows:"); + while (!hiveQueue.isEmpty()) LOG.debug(hiveQueue.poll().toString(",", "NULL", "'")); + LOG.debug("Last 25 Bench rows:"); + while (!benchQueue.isEmpty()) LOG.debug(benchQueue.poll().toString(",", "NULL", "'")); + + Assert.fail("Mismatch at row " + rowNum + " hive row is <" + + hiveRow.toString(",", "NULL", "") + "> bench row is <" + + benchRow.toString(",", "NULL", "") + ">"); + } + } + Assert.assertFalse("Hive ran out of rows at " + rowNum + " but benchmark still has rows", + benchIter.hasNext()); + } + + protected Iterator sort(DataSet data) { + final PriorityQueue sorted = new PriorityQueue<>(); + Iterator iter = data.iterator(); + while (iter.hasNext()) { + sorted.add(iter.next()); + } + return new Iterator() { + @Override + public boolean hasNext() { + return sorted.size() > 0; + } + + @Override + public Row next() { + return sorted.poll(); + } + + @Override + public void remove() { + + } + }; + } +} diff --git itests/capybara/src/main/java/org/apache/hive/test/capybara/infra/PostgresBenchmark.java itests/capybara/src/main/java/org/apache/hive/test/capybara/infra/PostgresBenchmark.java new file mode 100644 index 0000000..ef4f51f --- /dev/null +++ itests/capybara/src/main/java/org/apache/hive/test/capybara/infra/PostgresBenchmark.java @@ -0,0 +1,51 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hive.test.capybara.infra; + +import org.apache.hive.test.capybara.iface.Benchmark; +import org.apache.hive.test.capybara.iface.BenchmarkDataStore; +import org.apache.hive.test.capybara.iface.ResultComparator; +import org.apache.hive.test.capybara.iface.TableComparator; + +/** + * A Benchmark that uses Postgres to store the results and standard ResultComparators for + * comparison. + */ +public class PostgresBenchmark implements Benchmark { + BenchmarkDataStore store; + + @Override + public BenchmarkDataStore getBenchDataStore() { + if (store == null) { + store = new PostgresStore(); + } + return store; + } + + @Override + public ResultComparator getResultComparator(boolean sort) { + if (sort) return new SortingComparator(); + else return new NonSortingComparator(); + } + + @Override + public TableComparator getTableComparator() { + return new TableComparator(); + } +} + diff --git itests/capybara/src/main/java/org/apache/hive/test/capybara/infra/PostgresStore.java itests/capybara/src/main/java/org/apache/hive/test/capybara/infra/PostgresStore.java new file mode 100644 index 0000000..a270ee9 --- /dev/null +++ itests/capybara/src/main/java/org/apache/hive/test/capybara/infra/PostgresStore.java @@ -0,0 +1,276 @@ +/** + * 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.hive.test.capybara.infra; + +import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hive.test.capybara.data.DataSet; +import org.apache.hive.test.capybara.iface.TestTable; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.File; +import java.io.IOException; +import java.sql.Connection; +import java.sql.Driver; +import java.sql.SQLException; +import java.sql.Statement; +import java.util.Properties; +import java.util.concurrent.ConcurrentLinkedDeque; +import java.util.regex.Matcher; +import java.util.regex.Pattern; + +/** + * DataStore for Postgres. The way data is loaded in this class makes the assumption that the + * postgres server is local to the machine running the tests. Not sure if this is reasonable or + * not. + */ +class PostgresStore extends AnsiSqlStore { + private static final Logger LOG = LoggerFactory.getLogger(PostgresStore.class.getName()); + + private static final String DELIMITER_STR = "\t"; + private static final String NULL_STR = "\\N"; + private static final String QUOTES = ""; + + private String connectionUrl; + private Properties connectionProperties; + + PostgresStore() { + try { + Class cls = Class.forName("org.postgresql.Driver"); + jdbcDriver = (Driver)cls.newInstance(); + } catch (ClassNotFoundException|IllegalAccessException|InstantiationException e) { + throw new RuntimeException(e); + } + } + + @Override + public String getTableName(TestTable table) { + return "default".equals(table.getDbName()) ? table.getTableName() : table.toString(); + } + + @Override + protected String ifExists() { + return ""; + } + + @Override + protected String markColumnPrimaryKey() { + return "primary key"; + } + + @Override + public void loadData(TestTable table, DataSet rows) throws SQLException, IOException { + if (rows.getClusterLocation() != null) { + // The data has already been dumped onto the cluster. Fetch it back from there and load it. + // This will be slow, but I suspect it's still faster than loading it via JDBC from the + // cluster. + loadFromCluster(table, rows); + } else { + dumpToFileForImport(rows); + File file = dataSetDumps.get(rows.uniqueId()); + loadFromFile(table, file); + } + } + + @Override + protected String connectionURL() { + if (connectionUrl == null) { + StringBuilder url = new StringBuilder("jdbc:postgresql:"); + String host = System.getProperty("hive.test.capybara.postgres.host"); + if (host != null) url.append("//").append(host); + String port = System.getProperty("hive.test.capybara.postgres.port"); + if (port != null) url.append(":").append(port); + String db = System.getProperty("hive.test.capybara.postgres.database"); + if (db != null) { + if (host != null) url.append("/"); + url.append(db); + } else { + url.append("/"); + } + connectionUrl = url.toString(); + } + return connectionUrl; + } + + @Override + protected Properties connectionProperties() { + if (connectionProperties == null) { + connectionProperties = new Properties(); + connectionProperties.setProperty("user", + System.getProperty("hive.test.capybara.postgres.user", System.getProperty("user.name"))); + connectionProperties.setProperty("password", + System.getProperty("hive.test.capybara.postgres.password", "")); + } + return connectionProperties; + } + + @Override + protected String fileColumnDelimiter() { + return DELIMITER_STR; + } + + @Override + protected String fileNull() { + return NULL_STR; + } + + @Override + protected String fileStringQuotes() { + return QUOTES; + } + + private static SQLTranslator postgresTranslator = new SQLTranslator() { + @Override + protected String translateDataTypes(String hiveSql) { + Matcher m = Pattern.compile(" string").matcher(hiveSql); + hiveSql = m.replaceAll(" varchar(255)"); + m = Pattern.compile(" double").matcher(hiveSql); + hiveSql = m.replaceAll(" double precision"); + m = Pattern.compile(" float").matcher(hiveSql); + hiveSql = m.replaceAll(" real"); + m = Pattern.compile(" tinyint").matcher(hiveSql); + hiveSql = m.replaceAll(" smallint"); + m = Pattern.compile(" binary").matcher(hiveSql); + hiveSql = m.replaceAll(" blob"); + return hiveSql; + } + + @Override + protected String translateAlterTableRename(String tableName, String remainder) + throws TranslationException { + Matcher m = + Pattern.compile("rename to (" + SQLTranslator.tableNameRegex + ")").matcher(remainder); + if (m.lookingAt()) { + return "alter table " + tableName + " rename to " + translateTableNames(m.group(1)); + } else { + throw new TranslationException("alter table rename", remainder); + } + } + + @Override + protected char identifierQuote() { + return '"'; + } + }; + + @Override + protected SQLTranslator getTranslator() { + return postgresTranslator; + } + + @Override + protected String convertType(String hiveType) { + if (hiveType.equalsIgnoreCase("double")) return "double precision"; + else if (hiveType.equalsIgnoreCase("float")) return "real"; + else if (hiveType.equalsIgnoreCase("string")) return "varchar(255)"; + else if (hiveType.toLowerCase().startsWith("varchar")) return hiveType + " collate \"POSIX\""; + else if (hiveType.toLowerCase().startsWith("char")) return hiveType + " collate \"POSIX\""; + else return super.convertType(hiveType); + } + + private void loadFromFile(TestTable table, File file) throws SQLException { + StringBuilder sql = new StringBuilder("COPY ") + .append(getTableName(table)) + .append(" FROM '") + .append(file.getAbsolutePath()) + .append("' WITH (DELIMITER '") + .append(DELIMITER_STR) + .append("', NULL '") + .append(NULL_STR) + .append("')"); + if (LOG.isDebugEnabled()) { + LOG.debug("Going to import into Postgres with command " + sql.toString()); + } + Connection conn = connect(true); + Statement stmt = null; + try { + stmt = conn.createStatement(); + stmt.execute(sql.toString()); + } finally { + if (stmt != null) stmt.close(); + conn.close(); + } + } + + private void loadFromCluster(TestTable table, DataSet rows) throws IOException, SQLException { + // Find each file in the directory, bring each back, and then load it. Since we're making + // the assumption that the postgres instance is local to the test machine, pulling it back + // from HDFS via copyToLocal is no worse than loading it from the cluster and less likely to + // overwhelm postgres. + // The data is pulled back by one thread while another thread loads the data into postgres. + ConcurrentLinkedDeque localFiles = new ConcurrentLinkedDeque<>(); + HdfsFilePuller puller = new HdfsFilePuller(localFiles, rows); + puller.start(); + while ((puller.isAlive() || localFiles.peek() != null) && puller.stashed == null) { + File f = localFiles.poll(); + if (f != null) { + loadFromFile(table, f); + // Delete the local file to save space + f.delete(); + } else { + try { + Thread.sleep(250); + } catch (InterruptedException e) { + // Not sure what to do about this, going to ignore it. + } + } + } + if (puller.stashed != null) { + throw puller.stashed; + } + } + + private class HdfsFilePuller extends Thread { + private final ConcurrentLinkedDeque localFiles; + private final DataSet rows; + IOException stashed; + + HdfsFilePuller(ConcurrentLinkedDeque localFiles, DataSet rows) { + this.localFiles = localFiles; + this.rows = rows; + } + + @Override + public void run() { + try { + Path dir = rows.getClusterLocation(); + FileSystem fs = FileSystem.get(TestManager.getTestManager().getConf()); + FileStatus[] stats = fs.listStatus(dir); + for (FileStatus stat : stats) { + Path clusterFile = stat.getPath(); + File f = File.createTempFile("capybara_", "_data_set_" + rows.uniqueId()); + Path localFile = new Path(f.getAbsolutePath()); + LOG.debug("Copying cluster file " + clusterFile.getName() + " to local file " + + localFile.getName()); + fs.copyToLocalFile(clusterFile, localFile); + localFiles.add(f); + } + } catch (IOException e) { + LOG.debug("HdfsFilePuller thread got IOException, giving up", e); + stashed = e; + } + } + } + + @Override + public Class getDriverClass() { + return jdbcDriver.getClass(); + } +} diff --git itests/capybara/src/main/java/org/apache/hive/test/capybara/infra/RandomDataGenerator.java itests/capybara/src/main/java/org/apache/hive/test/capybara/infra/RandomDataGenerator.java new file mode 100644 index 0000000..09ac8ad --- /dev/null +++ itests/capybara/src/main/java/org/apache/hive/test/capybara/infra/RandomDataGenerator.java @@ -0,0 +1,432 @@ +/** + * 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.hive.test.capybara.infra; + +import org.apache.hive.test.capybara.data.DataSet; +import org.apache.hive.test.capybara.data.Row; +import org.apache.hive.test.capybara.data.RowBuilder; +import org.apache.hive.test.capybara.iface.TestTable; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import org.apache.hadoop.hive.metastore.api.FieldSchema; +import org.apache.hive.test.capybara.iface.DataGenerator; + +import java.io.Serializable; +import java.math.BigDecimal; +import java.math.BigInteger; +import java.sql.Date; +import java.sql.Timestamp; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Random; +import java.util.Set; +import java.util.regex.Matcher; +import java.util.regex.Pattern; + +/** + * A data generator that creates random data. If you want the data generated to be the same each + * time (that is, you're passing in a constant seed value) you should instantiate a new version + * of RandomDataGenerator for each table. Otherwise tables after the first one will inherit the + * first one's rand, which won't generate the same data as if you generated that table first. + */ +public class RandomDataGenerator extends DataGeneratorImpl implements Serializable { + static final private Logger LOG = LoggerFactory.getLogger(RandomDataGenerator.class); + + protected Random rand; + protected long seed; + + /** + * Constructor for default number of partitions (log10(scale)) and null values (1% of values + * per column) + * @param seed Seed to use for generating random numbers. By setting this constant you can + * make sure the same data is generated each time. + */ + public RandomDataGenerator(long seed) { + this.seed = seed; + rand = new Random(seed); + } + + @Override + protected DataGenerator copy(int copyNum) { + return new RandomDataGenerator(seed + copyNum); + } + + @Override + public DataSet generateData(TestTable table, int scale, double[] pctNulls) { + // Decide whether we're going to generate locally or on the cluster. Remember that scale is + // in K, while other values are in b. + if (scale * 1024 > TestConf.getClusterGenThreshold() && TestConf.onCluster()) { + return new ClusterDataGenerator(this).generateData(table, scale, pctNulls); + } + + LOG.debug("Generating data locally"); + + // We'll handle partition columns and regular columns separately, as we want to control the + // number of partitions. + List cols = table.getCols(); + ColGenerator[] generators = new ColGenerator[cols.size()]; + for (int i = 0; i < generators.length; i++) { + TestTable.PrimaryKey pk = null; + if (table.getPrimaryKey() != null && table.getPrimaryKey().colNum == i) { + pk = table.getPrimaryKey(); + } + TestTable.ForeignKey fk = null; + if (table.getForeignKeys() != null) { + for (TestTable.ForeignKey f : table.getForeignKeys()) { + if (f.colNumInSrc == i) { + fk = f; + break; + } + } + } + generators[i] = getGenerator(cols.get(i), pk, fk); + } + + if (pctNulls == null) { + pctNulls = new double[cols.size()]; + for (int i = 0; i < pctNulls.length; i++) pctNulls[i] = 0.01; + } else { + if (pctNulls.length != cols.size()) { + throw new RuntimeException("Your null array must be the same size as the number of " + + "columns in the table, excluding partition columns"); + } + } + + List partCols = table.getPartCols(); + GeneratedDataSet rows; + RowBuilder colBuilder = new RowBuilder(cols); + rows = new GeneratedDataSet(table.getCombinedSchema()); + if (partCols == null || partCols.size() == 0) { + generateData(rows, colBuilder, generators, null, scale, pctNulls); + } else { + List partVals = determinePartVals(table, scale, pctNulls); + + for (Row partVal : partVals) { + // Divide scale by number of partitions so we still get the right amount of data. + generateData(rows, colBuilder, generators, partVal, scale / partVals.size() + 1, pctNulls); + } + } + return rows; + } + + private void generateData(GeneratedDataSet rows, RowBuilder builder, + ColGenerator[] generators, Row partVals, int scale, + double[] pctNulls) { + long generatedSize = 0; + while (generatedSize < scale * 1024) { + Row row = builder.build(); + for (int i = 0; i < generators.length; i++) { + row.get(i).set(generators[i].generate(pctNulls[i])); + } + if (partVals != null) row.append(partVals); + generatedSize += row.lengthInBytes(); + rows.addRow(row); + } + } + + @Override + protected ColGenerator getGenerator(FieldSchema col, TestTable.PrimaryKey pk, + TestTable.ForeignKey fk) { + + String colType = col.getType(); + if (fk != null) { + return new ForeignKeyGenerator(fk); + } + + if (pk != null && pk.isSequence()) { + if (!colType.equalsIgnoreCase("bigint")) { + throw new RuntimeException("You tried to create a sequence with something other than a " + + "bigint. That can't end well."); + } + return new SequenceGenerator(); + } + + ColGenerator cg; + if (colType.equalsIgnoreCase("bigint")) { + cg = getLongGenerator(col); + } else if (colType.substring(0, 3).equalsIgnoreCase("int")) { + cg = getIntGenerator(col); + } else if (colType.equalsIgnoreCase("smallint")) { + cg = getShortGenerator(col); + } else if (colType.equalsIgnoreCase("tinyint")) { + cg = getByteGenerator(col); + } else if (colType.equalsIgnoreCase("float")) { + cg = getFloatGenerator(col); + } else if (colType.equalsIgnoreCase("double")) { + cg = getDoubleGenerator(col); + } else if (colType.toLowerCase().startsWith("decimal")) { + int precision, scale; + if (colType.equalsIgnoreCase("decimal")) { + // They didn't give a precision and scale, so pick the default + precision = 10; + scale = 2; + } else { + if (colType.contains(",")) { + Pattern regex = Pattern.compile(".*\\(([0-9]+),([0-9]+)\\).*"); + Matcher matcher = regex.matcher(colType); + if (!matcher.matches()) { + throw new RuntimeException("Expected type to match decimal([0-9]+,[0-9]+) but it " + + "doesn't appear to: " + colType); + } + precision = Integer.valueOf(matcher.group(1)); + scale = Integer.valueOf(matcher.group(2)); + } else { + precision = getPrecision(colType); + scale = 2; + } + } + cg = getBigDecimalGenerator(col, precision, scale); + } else if (colType.equalsIgnoreCase("date")) { + cg = getDateGenerator(col); + } else if (colType.equalsIgnoreCase("timestamp")) { + cg = getTimestampGenerator(col); + } else if (colType.toLowerCase().startsWith("varchar") || + colType.toLowerCase().startsWith("char")) { + cg = getStringGenerator(col, getPrecision(colType)); + } else if (colType.equalsIgnoreCase("string")) { + cg = getStringGenerator(col, 20); + } else if (colType.equalsIgnoreCase("boolean")) { + cg = getBoolGenerator(col); + } else if (colType.equalsIgnoreCase("binary")) { + cg = getBinaryGenerator(col, 100); + } else { + throw new RuntimeException("How'd we get here? Unknown type"); + } + + if (pk != null) return new PrimaryKeyGenerator(cg); + else return cg; + } + + protected int getPrecision(String colType) { + Pattern regex = Pattern.compile(".*\\(([0-9]+)\\).*"); + Matcher matcher = regex.matcher(colType); + if (!matcher.matches()) { + throw new RuntimeException("Expected type to match datatype([0-9]+) but it " + + "doesn't appear to: " + colType); + } + return Integer.valueOf(matcher.group(1)); + } + + /** + * If you override this class you MUST remember to updated generatedSize as you add values, + * otherwise the generator will go into an infinite loop. + * @return column value generator for long (bigint) values. + */ + protected ColGenerator getLongGenerator(FieldSchema col) { + return new ColGenerator() { + @Override + public Object generate(double pctNull) { + if (rand.nextDouble() < pctNull) return null; + return rand.nextLong(); + } + }; + } + + protected ColGenerator getIntGenerator(FieldSchema col) { + return new ColGenerator() { + @Override + public Object generate(double pctNull) { + if (rand.nextDouble() < pctNull) return null; + return rand.nextInt(); + } + }; + } + + protected ColGenerator getShortGenerator(FieldSchema col) { + return new ColGenerator() { + @Override + public Object generate(double pctNull) { + if (rand.nextDouble() < pctNull) return null; + return (short) rand.nextInt(Short.MAX_VALUE); + } + }; + } + + protected ColGenerator getByteGenerator(FieldSchema col) { + return new ColGenerator() { + @Override + public Object generate(double pctNull) { + if (rand.nextDouble() < pctNull) return null; + return (byte) rand.nextInt(Byte.MAX_VALUE); + } + }; + } + + protected ColGenerator getFloatGenerator(FieldSchema col) { + return new ColGenerator() { + @Override + public Object generate(double pctNull) { + if (rand.nextDouble() < pctNull) return null; + return rand.nextFloat() + rand.nextInt(); + } + }; + } + + protected ColGenerator getDoubleGenerator(FieldSchema col) { + return new ColGenerator() { + @Override + public Object generate(double pctNull) { + if (rand.nextDouble() < pctNull) return null; + return rand.nextDouble() + rand.nextLong(); + } + }; + } + + protected ColGenerator getBigDecimalGenerator(FieldSchema col, final int precision, + final int scale) { + return new ColGenerator() { + @Override + public Object generate(double pctNull) { + if (rand.nextDouble() < pctNull) return null; + BigInteger bi = new BigInteger(precision * 3, rand); + return new BigDecimal(bi, scale); + } + }; + + } + + protected ColGenerator getDateGenerator(FieldSchema col) { + return new ColGenerator() { + @Override + public Object generate(double pctNull) { + if (rand.nextDouble() < pctNull) return null; + // Limit the range of dates, otherwise the databases get bent out of shape when dates in + // the year 20,000AD show up. + return new Date(rand.nextLong() % (Integer.MAX_VALUE * 1000L)); + } + }; + } + + protected ColGenerator getTimestampGenerator(FieldSchema col) { + return new ColGenerator() { + @Override + public Object generate(double pctNull) { + if (rand.nextDouble() < pctNull) return null; + // Limit the range of timestamps, otherwise the databases get bent out of shape when + // timestamps in the year 20,000AD show up. + return new Timestamp(rand.nextLong() % (Integer.MAX_VALUE * 1000L)); + } + }; + } + + protected ColGenerator getStringGenerator(FieldSchema col, final int maxLength) { + + return new ColGenerator() { + @Override + public Object generate(double pctNull) { + if (rand.nextDouble() < pctNull) return null; + int length = rand.nextInt(maxLength) + 1; + StringBuilder builder = new StringBuilder(length); + for (int i = 0; i < length; i++) { + // Generate alpha-characters only for now. Hive uses POSIX sorting for strings and + // other databases use other ones, so including symbols, numbers, etc. goes all wrong + // on us in comparisons otherwise. + char nextChar = (char)(rand.nextInt('z' - 'a') + 'a'); + if (nextChar == '\\') nextChar = '/'; + builder.append(nextChar); + } + return builder.toString(); + } + }; + } + + protected ColGenerator getBoolGenerator(FieldSchema col) { + return new ColGenerator() { + @Override + public Object generate(double pctNull) { + if (rand.nextDouble() < pctNull) return null; + return rand.nextBoolean(); + } + }; + } + + protected ColGenerator getBinaryGenerator(FieldSchema col, final int maxLength) { + + return new ColGenerator() { + @Override + public Object generate(double pctNull) { + if (rand.nextDouble() < pctNull) return null; + int length = rand.nextInt(maxLength) + 1; + byte[] bytes = new byte[length]; + rand.nextBytes(bytes); + return bytes; + } + }; + } + + /** + * A class to generate foreign keys. Given a primary key in another table, this class will + * generate foreign keys that are guaranteed to match a row in the target table's primary key. + * The primary key is assumed to be single columned for now. Pctnull is ignored by this generator. + */ + protected class ForeignKeyGenerator implements ColGenerator { + private final Map vals; + + ForeignKeyGenerator(TestTable.ForeignKey fk) { + vals = new HashMap<>(); + int nextVal = 0; + for (Row row : fk.targetTable) { + vals.put(nextVal++, row.get(fk.colNumInTarget).get()); + } + } + + @Override + public Object generate(double pctNull) { + return vals.get(rand.nextInt(vals.size())); + } + } + + /** + * Generate a bigint sequence. Pctnull is ignored by this generator. + */ + protected class SequenceGenerator implements ColGenerator { + long nextSequence = 1; + + @Override + public Object generate(double pctNull) { + return nextSequence++; + } + } + + /** + * Generate a primary key for a table. The values are guaranteed to be unique. You can get a + * sequence primary key by passing a SequenceGenerator as the wrapped ColGenerator. Primary + * keys are currently restricted to a single column. Pctnull is ignored by this generator. + */ + protected class PrimaryKeyGenerator implements ColGenerator { + private final ColGenerator wrapped; + private Set usedVals; + + PrimaryKeyGenerator(ColGenerator wrapped) { + this.wrapped = wrapped; + usedVals = new HashSet<>(); + } + + @Override + public Object generate(double pctNull) { + Object val; + do { + val = wrapped.generate(0.0); + } while (!usedVals.add(val)); + return val; + } + } + +} diff --git itests/capybara/src/main/java/org/apache/hive/test/capybara/infra/ResultSetDataSet.java itests/capybara/src/main/java/org/apache/hive/test/capybara/infra/ResultSetDataSet.java new file mode 100644 index 0000000..890ce15 --- /dev/null +++ itests/capybara/src/main/java/org/apache/hive/test/capybara/infra/ResultSetDataSet.java @@ -0,0 +1,136 @@ +/** + * 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.hive.test.capybara.infra; + +import org.apache.hadoop.hive.metastore.api.FieldSchema; +import org.apache.hive.test.capybara.data.Column; +import org.apache.hive.test.capybara.data.DataSet; +import org.apache.hive.test.capybara.data.Row; +import org.apache.hive.test.capybara.data.RowBuilder; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.sql.ResultSet; +import java.sql.ResultSetMetaData; +import java.sql.SQLException; +import java.sql.Types; +import java.util.ArrayList; +import java.util.Iterator; +import java.util.List; + +/** + * A DataSet wrapped around a {@link java.sql.ResultSet}. Unforunately we cannot be lazy in + * turning the provided ResultSet into a List<Object[]> because the caller may close the + * statement after handing us this, which will close the ResultSet. + */ +public class ResultSetDataSet extends DataSet { + static final private Logger LOG = LoggerFactory.getLogger(ResultSetDataSet.class.getName()); + + public ResultSetDataSet(ResultSet rs) throws SQLException { + this(rs, Long.MAX_VALUE); + } + + public ResultSetDataSet(ResultSet rs, long limit) throws SQLException { + super(resultSetMetaDataToSchema(rs)); + rows = new ArrayList<>(); + + RowBuilder builder = new RowBuilder(schema); + for (long rownum = 0; rownum < limit && rs.next(); rownum++) { + Row row = builder.build(); + for (Column col : row) { + col.fromResultSet(rs); + } + rows.add(row); + } + } + + private static List resultSetMetaDataToSchema(ResultSet rs) throws SQLException { + ResultSetMetaData meta = rs.getMetaData(); + List cols = new ArrayList<>(meta.getColumnCount()); + StringBuilder logMsg = new StringBuilder(); + for (int i = 0; i < meta.getColumnCount(); i++) { + String type; + switch (meta.getColumnType(i+1)) { + case Types.BIGINT: + type = "bigint"; + break; + case Types.INTEGER: + type = "int"; + break; + case Types.SMALLINT: + type = "smallint"; + break; + case Types.TINYINT: + type = "tinyint"; + break; + case Types.REAL: + case Types.FLOAT: + type = "float"; + break; + case Types.DOUBLE: + type = "double"; + break; + case Types.NUMERIC: + case Types.DECIMAL: + type = "decimal(" + meta.getPrecision(i+1) + "," + meta.getScale(i+1) + ")"; + break; + case Types.DATE: + type = "date"; + break; + case Types.TIMESTAMP: + type = "timestamp"; + break; + case Types.VARCHAR: + type = "varchar(" + meta.getPrecision(i+1) + ")"; + break; + case Types.CHAR: + type = "char(" + meta.getPrecision(i+1) + ")"; + break; + case Types.BOOLEAN: + case Types.BIT: // postgress maps boolean to this + type = "boolean"; + break; + case Types.BLOB: + case Types.BINARY: + type = "binary"; + break; + + default: + throw new RuntimeException("Unknown data type: " + meta.getColumnType(i+1) + " name: " + + meta.getColumnTypeName(i + 1)); + } + FieldSchema fs = new FieldSchema(meta.getColumnName(i+1), type, ""); + if (LOG.isDebugEnabled()) { + logMsg.append(fs.getName()) + .append(':') + .append(fs.getType()) + .append(','); + } + + cols.add(fs); + } + LOG.debug("Obtained schema from ResultSetMetaData: " + logMsg.toString()); + return cols; + + } + + @Override + public Iterator iterator() { + return rows.iterator(); + } +} diff --git itests/capybara/src/main/java/org/apache/hive/test/capybara/infra/SQLTranslator.java itests/capybara/src/main/java/org/apache/hive/test/capybara/infra/SQLTranslator.java new file mode 100644 index 0000000..ad2ff42 --- /dev/null +++ itests/capybara/src/main/java/org/apache/hive/test/capybara/infra/SQLTranslator.java @@ -0,0 +1,832 @@ +/** + * 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.hive.test.capybara.infra; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.Arrays; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.regex.Matcher; +import java.util.regex.Pattern; + +/** + * A class to transform Hive SQL to the SQL dialect of the benchmark. This base class handles + * the most common issues, but each type of benchmark will need to extend this to handle quirks + * of its own SQL dialect. The class is built to be mostly stateless so that {@link #translate} + * can be called repeatedly with different SQL statements on the same object. The one bit of + * state kept is when "use database" is translated, so that the current database is tracked. + */ +abstract class SQLTranslator { + private static final Logger LOG = LoggerFactory.getLogger(SQLTranslator.class.getName()); + + protected static final String idRegex = "[a-zA-Z0-9_]+"; + protected static final String tableNameRegex = "(?:" + idRegex + "\\.)?" + idRegex; + protected static final String QUOTE_START = "CQ_"; + protected static final String QUOTE_END = "_QC"; + + /** + * If true, it's ok if this SQL fails to run on the benchmark. This is used to mask the fact + * that the database may not have 'if not exists' or 'if exists' which can obviously cause + * failures. + */ + protected boolean failureOk; + + /** + * Current database, used in table names + */ + protected String currentDb = "default"; + + /** + * Translate the Hive SQL to the appropraite dialect. This call breaks up the Hive SQL into + * separate sections. It is final because sub-classes shouldn't change the way it is broken up. + * @param hiveSql SQL in Hive dialect + * @return SQL in benchmark dialect. + */ + final String translate(String hiveSql) throws TranslationException { + failureOk = false; + String trimmed = hiveSql.trim().toLowerCase(); + + // Remove any hints + Matcher matcher = Pattern.compile("/\\*.*\\*/").matcher(trimmed); + trimmed = matcher.replaceAll(""); + + trimmed = deQuote(trimmed); + // Convert all white space to single spaces so we don't have to keep matching \\s+ everywhere + Matcher m = Pattern.compile("\\s+").matcher(trimmed); + trimmed = m.replaceAll(" "); + + String benchSql; + if (Pattern.compile("select").matcher(trimmed).lookingAt()) { + benchSql = translateSelect(trimmed); + } else if (Pattern.compile("insert").matcher(trimmed).lookingAt()) { + benchSql = translateInsert(trimmed); + } else if (Pattern.compile("explain").matcher(trimmed).lookingAt()) { + benchSql = ""; + } else if (Pattern.compile("update").matcher(trimmed).lookingAt()) { + benchSql = translateUpdate(trimmed); + } else if (Pattern.compile("delete from").matcher(trimmed).lookingAt()) { + benchSql = translateDelete(trimmed); + } else if (Pattern.compile("create (temporary |external )?table").matcher(trimmed).lookingAt()) { + benchSql = translateCreateTable(trimmed); + } else if (Pattern.compile("drop table").matcher(trimmed).lookingAt()) { + benchSql = translateDropTable(trimmed); + } else if (Pattern.compile("alter table").matcher(trimmed).lookingAt()) { + benchSql = translateAlterTable(trimmed); + } else if (Pattern.compile("show").matcher(trimmed).lookingAt()) { + benchSql = ""; + } else if (Pattern.compile("describe").matcher(trimmed).lookingAt()) { + benchSql = ""; + } else if (Pattern.compile("create (database|schema)").matcher(trimmed).lookingAt()) { + benchSql = translateCreateDatabase(trimmed); + } else if (Pattern.compile("alter (database|schema)").matcher(trimmed).lookingAt()) { + benchSql = ""; + } else if (Pattern.compile("drop (database|schema)").matcher(trimmed).lookingAt()) { + benchSql = translateDropDatabase(trimmed); + } else if (Pattern.compile("use (" + idRegex +")").matcher(trimmed).lookingAt()) { + benchSql = translateUseDatabase(m.group(1)); + } else if (Pattern.compile("analyze").matcher(trimmed).lookingAt()) { + benchSql = ""; + } else if (Pattern.compile("create role").matcher(trimmed).lookingAt()) { + benchSql = ""; + } else if (Pattern.compile("drop role").matcher(trimmed).lookingAt()) { + benchSql = ""; + } else if (Pattern.compile("set role").matcher(trimmed).lookingAt()) { + benchSql = ""; + } else if (Pattern.compile("grant").matcher(trimmed).lookingAt()) { + benchSql = ""; + } else if (Pattern.compile("revoke").matcher(trimmed).lookingAt()) { + benchSql = ""; + } else if (Pattern.compile("create index").matcher(trimmed).lookingAt()) { + benchSql = ""; + } else if (Pattern.compile("alter index").matcher(trimmed).lookingAt()) { + benchSql = ""; + } else if (Pattern.compile("drop index").matcher(trimmed).lookingAt()) { + benchSql = ""; + } else if (Pattern.compile("create (temporary )?function").matcher(trimmed).lookingAt()) { + // This won't end well. We can't translate functions + throw new TranslationException("create function", hiveSql); + } else if (Pattern.compile("drop (temporary )?function").matcher(trimmed).lookingAt()) { + throw new TranslationException("drop function", hiveSql); + } else if (Pattern.compile("reload function").matcher(trimmed).lookingAt()) { + throw new TranslationException("reload function", hiveSql); + } else { + throw new TranslationException("Unrecognized", hiveSql); + } + return reQuote(benchSql); + + // TODO: + // alter table + // alter view + // create view + // drop view + // truncate table - have to handle truncate partition and weird partition casts + // with + } + + public boolean isFailureOk() { + return failureOk; + } + + /********************************************************************************************** + * DB related + **********************************************************************************************/ + /** + * Translate create database. + * @param hiveSql hiveSql, already trimmed and coverted to lower case. + * @return benchmark sql + */ + protected String translateCreateDatabase(String hiveSql) throws + TranslationException { + // Convert 'create database' to 'create schema' + Matcher m = Pattern.compile("create database").matcher(hiveSql); + String benchSql = m.replaceAll("create schema"); + + // Remove any other modifiers (location, comment, dbproperties)... + //m = Pattern.compile("create schema (?:if not exists )?([a-zA-Z0-9][a-zA-Z0-9_]*) .*").matcher(benchSql); + m = Pattern.compile("create schema (if not exists )?(" + idRegex + ")(?: .*)?").matcher( + benchSql); + if (m.lookingAt()) { + return "create schema " + (m.group(1) == null ? "" : m.group(1)) + m.group(2); + } + + throw new TranslationException("create database", hiveSql); + } + + protected String translateDropDatabase(String hiveSql) throws TranslationException { + // Convert 'drop database' to 'create schema' + Matcher m = Pattern.compile("drop database").matcher(hiveSql); + return m.replaceAll("drop schema"); + } + + protected String translateUseDatabase(String dbName) throws TranslationException { + // Find the database name and set that as the default. + currentDb = dbName; + // Use database doesn't translate. + return ""; + } + + /********************************************************************************************** + * Table related + **********************************************************************************************/ + private String translateCreateTable(String hiveSql) throws TranslationException { + + // Look for like + Matcher m = Pattern.compile("create (temporary |external )?table (if not exists )?(" + + tableNameRegex + ") like (" + tableNameRegex + ")").matcher(hiveSql); + if (m.lookingAt()) return translateCreateTableLike(m); + + // Look for as + m = Pattern.compile("create (temporary |external )?table (if not exists )?(" + + tableNameRegex + ") (?:.* )?as (select.*)").matcher(hiveSql); + if (m.lookingAt()) return translateCreateTableAs(m); + + // Must be your basic create table foo (x int ...) type + m = Pattern.compile("create (temporary |external )?table (if not exists )?(" + + tableNameRegex + ") ?\\((.*)").matcher(hiveSql); + if (m.lookingAt()) return translateCreateTableWithColDefs(m); + + throw new TranslationException("create table", hiveSql); + } + + /** + * Translate a create table like + * @param matcher A matcher for the statement, + * group 1 is temporary or external (may be null), + * group 2 is 'if not exists', (may be null) + * group 3 is the new table name + * group 4 is the source table name + * @return translated SQL + */ + protected String translateCreateTableLike(Matcher matcher) { + StringBuilder sql = new StringBuilder("create "); + if (matcher.group(1) != null && matcher.group(1).equals("temporary ")) sql.append("temporary "); + sql.append("table "); + if (matcher.group(2) != null) sql.append(matcher.group(2)); + sql.append(matcher.group(3)) + .append(" like ") + .append(matcher.group(4)); + return sql.toString(); + } + + /** + * Translate a create table as + * @param matcher A matcher for the statement + * group 1 is temporary or external (may be null) + * group 2 is 'if not exists', may be null + * group 3 is the table name + * group 4 is the select query + * @return translated SQL + */ + protected String translateCreateTableAs(Matcher matcher) throws TranslationException { + StringBuilder sql = new StringBuilder("create "); + if (matcher.group(1) != null && matcher.group(1).equals("temporary ")) sql.append("temporary "); + sql.append("table "); + if (matcher.group(2) != null) sql.append(matcher.group(2)); + sql.append(matcher.group(3)) + .append(" as ") + .append(translateSelect(matcher.group(4))); + return sql.toString(); + } + + /** + * Translate create table iwth column definitions + * @param matcher A matcher for the statement + * group 1 is temporary or external (may be null) + * group 2 is 'if not exists', may be null + * group 3 is the table name + * group 4 column definitions and the rest of the query + * @return bench sql + */ + protected String translateCreateTableWithColDefs(Matcher matcher) { + StringBuilder sql = new StringBuilder("create "); + if (matcher.group(1) != null && matcher.group(1).equals("temporary ")) sql.append("temporary "); + sql.append("table "); + if (matcher.group(2) != null) sql.append(matcher.group(2)); + sql.append(matcher.group(3)) + .append(" (") + .append(translateDataTypes(parseOutColDefs(matcher.group(4)))); + return sql.toString(); + } + + protected String parseOutColDefs(String restOfQuery) { + StringBuilder cols = new StringBuilder(); + int level = 1; + for (int i = 0; i < restOfQuery.length() && level > 0; i++) { + char current = restOfQuery.charAt(i); + cols.append(current); + if (current == '(') level++; + else if (current == ')') level--; + } + return cols.toString(); + } + + /** + * Translate data types + * @param hiveSql hive data types, may contain other text as well + * @return transformed text, with datatypes changed and other contents untouched. + */ + protected abstract String translateDataTypes(String hiveSql); + + protected String translateDropTable(String hiveSql) throws TranslationException { + // Need to remove purge if its there + Matcher m = Pattern.compile("drop table (if exists )?(" + tableNameRegex + ")").matcher(hiveSql); + if (m.lookingAt()) { + StringBuilder sql = new StringBuilder("drop table "); + if (m.group(1) != null) sql.append(m.group(1)); + sql.append(m.group(2)); + return sql.toString(); + } else { + throw new TranslationException("drop table", hiveSql); + } + } + + private String translateAlterTable(String hiveSql) throws TranslationException { + // It has been said that enough monkeys pounding on typewriters would eventually produce the + // complete works of Shakespeare. In the case of Hive's alter table syntax someone gave the + // monkeys whiskey first. + Matcher m = Pattern.compile("alter table (" + tableNameRegex + ") ").matcher(hiveSql); + if (m.lookingAt()) { + String tableName = translateTableNames(m.group(1)); + String remainder = hiveSql.substring(m.end()); + if (remainder.startsWith("rename to")) return translateAlterTableRename(tableName, remainder); + if (remainder.startsWith("set")) return ""; + if (remainder.startsWith("clustered by")) return ""; + if (remainder.startsWith("skewed by")) return ""; + if (remainder.startsWith("not skewed")) return ""; + if (remainder.startsWith("not stored as directories")) return ""; + if (remainder.startsWith("add partition") || + remainder.startsWith("add if not exists partition")) { + return ""; + } + + // This is really an alter partition. We need to parse through the partition and possibly + // translate it. + if (remainder.startsWith("partition (")) { + String partition = parsePartition(remainder.substring("partition ".length())); + String afterPartition = + remainder.substring("partition ".length() + partition.length()).trim(); + + if (afterPartition.startsWith("rename to partition ")) { + String secondPartition = + parsePartition(afterPartition.substring("rename to partition ".length())); + } + + + } + + + // TODO - Partitions ones to deal with + // TODO - rename + // TODO - set + + + return "oops"; + + } else { + throw new TranslationException("alter table", hiveSql); + } + + } + + private String parsePartition(String hiveSql) throws TranslationException { + StringBuilder sql = new StringBuilder(); + int level = 0; + for (int i = 0; i < hiveSql.length(); i++) { + if (hiveSql.charAt(i) == '(') { + if (level++ > 0) sql.append('('); + } else if (hiveSql.charAt(i) == ')') { + if (--level == 0) return sql.toString(); + else sql.append(')'); + } else { + sql.append(hiveSql.charAt(i)); + } + } + throw new TranslationException("partition", hiveSql); + } + + /** + * Translate alter table rename + * @param tableName translated table name + * @param remainder remainder of the Hive SQL, commencing after the table name. It should + * start with 'rename to' + * @return translated SQL, this should include the entire SQL, not just the remainder (ie it + * should start with 'alter table' + * @throws TranslationException + */ + protected abstract String translateAlterTableRename(String tableName, String remainder) + throws TranslationException; + + /********************************************************************************************** + * Query related + **********************************************************************************************/ + final protected String translateSelect(String hiveSql) throws TranslationException { + // First, find where the from starts. Hive doesn't support subqueries in the projection + // list, so we don't need to worry about hitting the wrong from + StringBuilder benchSql = new StringBuilder("select "); + int current = 7; // pointer to our place in the SQL stream + + // Handle all|distinct + if (hiveSql.substring(current).startsWith("all ")) { + benchSql.append("all "); + current += 4; + } else if (hiveSql.substring(current).startsWith("distinct ")) { + benchSql.append("distinct "); + current+= 9; + } + + // Move past the projection list + int nextKeyword = findNextKeyword(hiveSql, current, Arrays.asList(" from ", " where ", + " group by ", " having ", " union ", " order by ", " limit ")); + benchSql.append(translateExpressions(hiveSql.substring(current, nextKeyword))); + current = nextKeyword; + + // Handle a from if it's there + if (current < hiveSql.length() && hiveSql.substring(current).startsWith(" from ")) { + // We can't just look for the next keyword, as there can be subqueries in the from clause, + // so we need to handle any parenthesis + current += 6; // move past " from " + nextKeyword = findNextKeyword(hiveSql, current, Arrays.asList(" where ", " group by ", + " having ", " union ", " order by ", " limit ")); + benchSql.append(" from ") + .append(translateFrom(hiveSql.substring(current, nextKeyword))); + current = nextKeyword; + } + + // We may be on where + if (current < hiveSql.length() && hiveSql.substring(current).startsWith(" where ")) { + // Again, we have to be aware of parenthesis + current += 7; // move past " where " + nextKeyword = findNextKeyword(hiveSql, current, Arrays.asList(" group by ", " having ", + " union ", " order by ", " limit ")); + benchSql.append(" where ") + .append(translateExpressions(hiveSql.substring(current, nextKeyword))); + current = nextKeyword; + } + + // Now we might be on group by, shouldn't be anything to translate here + if (current < hiveSql.length() && hiveSql.substring(current).startsWith(" group by ")) { + current += 10; // move past " group by " + nextKeyword = findNextKeyword(hiveSql, current, Arrays.asList(" having ", " union ", + " order by ", " limit ")); + benchSql.append(" group by ") + .append(hiveSql.substring(current, nextKeyword)); + current = nextKeyword; + } + + // Maybe having + if (current < hiveSql.length() && hiveSql.substring(current).startsWith(" having ")) { + // Again, we have to be aware of parenthesis + current += 8; // move past " having " + nextKeyword = findNextKeyword(hiveSql, current, Arrays.asList(" union ", " order by ", + " limit ")); + benchSql.append(" having ") + .append(translateExpressions(hiveSql.substring(current, nextKeyword))); + current = nextKeyword; + } + + // Maybe union + if (current < hiveSql.length() && hiveSql.substring(current).startsWith(" union ")) { + // Again, we have to be aware of parenthesis + current += 7; // move past " union " + benchSql.append(" union "); + if (hiveSql.substring(current).startsWith("all ")) { + benchSql.append("all "); + current += 4; + } else if (hiveSql.substring(current).startsWith("distinct ")) { + benchSql.append("distinct "); + current += 9; + } + benchSql.append(translateSelect(hiveSql.substring(current))); + // Any trailing orders or limits will get attached to the last select, so just return here. + return benchSql.toString(); + } + + // Maybe order by, shouldn't be anything to translate here + if (current < hiveSql.length() && hiveSql.substring(current).startsWith(" order by ")) { + current += 10; // move past " group by " + nextKeyword = findNextKeyword(hiveSql, current, Arrays.asList(" limit ")); + benchSql.append(" order by ") + .append(hiveSql.substring(current, nextKeyword)); + current = nextKeyword; + } + + // Maybe limit, this has to be handled specially since some databases don't handle it + if (current < hiveSql.length() && hiveSql.substring(current).startsWith(" limit ")) { + benchSql.append(translateLimit(hiveSql.substring(current))); + } + + return benchSql.toString(); + } + + private int findNextKeyword(String hiveSql, int current, List keywords) { + int level = 0; + for (int i = current; i < hiveSql.length(); i++) { + if (hiveSql.charAt(i) == '(') { + level++; + } else if (hiveSql.charAt(i) == ')') { + level--; + } else if (level == 0) { + // Check whether we've hit a keyword. This is O(n^2), but I can't see a way to make it + // better. + for (String keyword : keywords) { + if (hiveSql.substring(i).startsWith(keyword)) return i; + } + } + } + return hiveSql.length(); + } + + protected String translateLimit(String hiveSql) throws TranslationException { + // Have to add in the limit here, because it wasn't + return hiveSql; + } + + private String translateExpressions(String hiveSql) throws TranslationException { + String benchSql = translateConstants(hiveSql); + benchSql = translateCasts(benchSql); + benchSql = translateUDFs(benchSql); + return translateSubqueries(benchSql); + } + + protected String translateConstants(String hiveSql) throws TranslationException { + return hiveSql; + /* + if (hiveSql.contains(" interval ")) { + LOG.error("Interval type not yet supported."); + throw new TranslationException("interval", hiveSql); + } + // Remove the annotations Hive uses for bigint, etc. + String benchSql = hiveSql; + Pattern p = Pattern.compile("([0-9]+)(l|s|y|bd)"); + Matcher m = p.matcher(benchSql); + while (m.find()) { + benchSql = m.replaceFirst(m.group(1)); // Get rid of the letter qualifier + m = p.matcher(benchSql); + } + + // Make sure all dates and timestamps have 2 digit months + p = Pattern.compile("(date|timestamp) (" + QUOTE_START + "[0-9]+" + QUOTE_END + ")"); + Pattern qpm = Pattern.compile("(date|timestamp) '([0-9]{4})-([0-9])"); + m = p.matcher(benchSql); + int current = 0; + while (m.find(current)) { + // The quotes have been replaced for safety. We have to go into the quote map and check + // that this quote is ok + Quote quote = quotes.get(m.group(1)); + Matcher qm = qpm.matcher(quote.value); + + // TODO fix the month + + // TODO fix the day + + benchSql = m.replaceFirst(m.group(1) + " '" + m.group(2) + "-0" + m.group(3)); + current = m.end(); + } + + // Make sure all dates and timestamps have 2 digit days + p = Pattern.compile("(date|timestamp) '([0-9]{4})-([0-9]{2})-([0-9])"); + m = p.matcher(benchSql); + while (m.find()) { + benchSql = + m.replaceFirst(m.group(1) + " '" + m.group(2) + "-" + m.group(3) + "-0" + m.group(4)); + m = p.matcher(benchSql); + } + return benchSql; + */ + } + + private String translateCasts(String hiveSql) throws TranslationException { + // We need to look for data type conversions in casts + StringBuilder benchSql = new StringBuilder(); + int current = 0; + Matcher m = Pattern.compile("cast\\((.*?)\\)").matcher(hiveSql); + while (m.find(current)) { + if (m.start() - current > 0) benchSql.append(hiveSql.substring(current, m.start())); + benchSql.append("cast (") + .append(translateDataTypes(m.group(1))) + .append(')'); + current = m.end(); + } + + if (current != 0) { + benchSql.append(hiveSql.substring(current)); + return benchSql.toString(); + } else { + return hiveSql; + } + } + + static Map udfMapping = null; + + /** + * A method to fill out the mapping of Hive UDF names to benchmark UDF names. Overrides of + * this should always call super.fillOutUdfMapping first as this one creates the object. + */ + protected void fillOutUdfMapping() { + udfMapping = new HashMap<>(); + // Entries that aren't UDFs but look like one to the regex matcher + udfMapping.put("cast", "cast"); + udfMapping.put("char", "char"); + udfMapping.put("decimal", "decimal"); + udfMapping.put("exists", "exists "); + udfMapping.put("in", "in "); + udfMapping.put("on", "on"); + udfMapping.put("varchar", "varchar"); + + // Actual UDFs + udfMapping.put("acos", "acos"); + udfMapping.put("asin", "asin"); + udfMapping.put("atan", "atan"); + udfMapping.put("avg", "avg"); + udfMapping.put("cos", "cos"); + udfMapping.put("count", "count"); + udfMapping.put("length", "length"); + udfMapping.put("max", "max"); + udfMapping.put("min", "min"); + udfMapping.put("sin", "sin"); + udfMapping.put("substring", "substring"); + udfMapping.put("sum", "sum"); + udfMapping.put("tan", "tan"); + + // TODO go through all the Hive UDFs. + + } + + private String translateUDFs(String hiveSql) throws TranslationException { + if (udfMapping == null) fillOutUdfMapping(); + Matcher m = Pattern.compile("(" + idRegex + ") ?\\(").matcher(hiveSql); + StringBuilder benchSql = new StringBuilder(); + int current = 0; + while (m.find(current)) { + if (m.start() - current > 0) benchSql.append(hiveSql.substring(current, m.start())); + benchSql.append(translateUDF(m.group(1))) + .append('('); + current = m.end(); + } + // Pick up whatever is left after the last match. + if (current != 0) { + benchSql.append(hiveSql.substring(current)); + return benchSql.toString(); + } else { + return hiveSql; + } + } + + private String translateUDF(String udfName) throws TranslationException { + String benchName = udfMapping.get(udfName); + if (benchName == null) { + throw new TranslationException("UDF name translation", udfName); + } + return benchName; + } + + private String translateFrom(String hiveSql) throws TranslationException { + hiveSql = translateTableNames(hiveSql); + return translateSubqueries(hiveSql); + } + + protected String translateTableNames(String hiveSql) { + // For now just strain out the default if it's there. Eventually need to add non-default + // name if it's been set. + Matcher matcher = Pattern.compile("default\\.").matcher(hiveSql); + return matcher.replaceAll(""); + } + + private String translateSubqueries(String hiveSql) throws TranslationException { + StringBuilder sql = new StringBuilder(); + Matcher m = Pattern.compile("\\( ?select").matcher(hiveSql); + int current = 0; + while (m.find(current)) { + int level = 1; + for (int i = m.start() + 1; i < hiveSql.length() && level > 0; i++) { + if (hiveSql.charAt(i) == '(') { + level++; + } else if (hiveSql.charAt(i) == ')' && --level == 0) { + if (m.start() - current > 0) sql.append(hiveSql.substring(current, m.start())); + sql.append('(') + .append(translateSelect(hiveSql.substring(m.start() + 1, i).trim())); + // Don't append the final ')', it will get picked up by the append of the next section + // of the query. + current = i; + } + } + } + // We still need to copy the last bit in. + if (current > 0 && current < hiveSql.length()) sql.append(hiveSql.substring(current)); + if (sql.length() > 0) return sql.toString(); + else return hiveSql; + } + + /********************************************************************************************** + * DML releated + **********************************************************************************************/ + protected String translateInsert(String hiveSql) throws TranslationException { + StringBuilder sql = new StringBuilder(); + Matcher m = Pattern.compile("insert (?:overwrite )?(?:into )?(?:table )?(" + tableNameRegex + + ") (partition)?").matcher(hiveSql); + if (m.lookingAt()) { + sql.append("insert into ") + .append(translateTableNames(m.group(1))) + .append(' '); + int current = m.end(); + if (m.group(2) != null) { + // chew through the partition definition, we don't care about it + // TODO - handle adding partition values to list if it makes sense. + int level = 0; + for (; current < hiveSql.length(); current++) { + if (hiveSql.charAt(current) == '(') { + level++; + } else if (hiveSql.charAt(current) == ')') { + if (--level == 0) { + // increment current so we move past the ) + current++; + break; + } + } + } + } + // TODO handle column names listed in insert + // We might have a select, or we might have a values + String remaining = hiveSql.substring(current).trim(); + if (remaining.startsWith("values")) { + sql.append(remaining); + } else { + sql.append(translateSelect(remaining)); + } + return sql.toString(); + } else { + throw new TranslationException("insert", hiveSql); + } + } + + protected String translateUpdate(String hiveSql) throws TranslationException { + StringBuilder sql = new StringBuilder(); + Matcher m = Pattern.compile("update (" + tableNameRegex + ") set ").matcher(hiveSql); + if (m.lookingAt()) { + sql.append("update ") + .append(translateTableNames(m.group(1))) + .append(" set ") + .append(translateExpressions(hiveSql.substring(m.end()))); + return sql.toString(); + } else { + throw new TranslationException("update", hiveSql); + } + } + + protected String translateDelete(String hiveSql) throws TranslationException { + StringBuilder sql = new StringBuilder(); + Matcher m = Pattern.compile("delete from (" + tableNameRegex + ")").matcher(hiveSql); + if (m.lookingAt()) { + sql.append("delete from ") + .append(translateTableNames(m.group(1))); + if (m.end() < hiveSql.length()) { + sql.append(translateExpressions(hiveSql.substring(m.end()))); + } + return sql.toString(); + } else { + throw new TranslationException("delete", hiveSql); + } + } + + /********************************************************************************************** + * Utility functions + **********************************************************************************************/ + + private class Quote { + final int number; + final char quoteType; + final String value; + + public Quote(int number, char quoteType, String value) { + this.number = number; + this.quoteType = quoteType; + this.value = new String(value); + } + } + + private Map quotes = new HashMap<>(); + private static int numQuotes = 1; + + private String deQuote(String hiveSql) { + // Take all of the quoted strings out, replacing them with non-sense strings but saving them + // in a map. We'll put them back at the end. This avoids the need to modify regex's for + // them and it keeps us from getting bogus hits when we're looking for keywords. + char prev = 0; + char currentQuote = 0; + StringBuilder output = new StringBuilder(); + StringBuilder quoteValue = new StringBuilder(); + for (char c : hiveSql.toCharArray()) { + if (currentQuote == 0) { + if (c == '`' || c == '"' || c == '\'') { + currentQuote = c; + } else { + output.append(c); + } + } else if (c == currentQuote && prev == 0) { + // First instance of a quote, can't close it yet as the next character may be a quote to, + // so this is just an escape + prev = c; // set up for next pass + quoteValue.append(c); + } else if (c != currentQuote && prev == currentQuote) { + // okay, now we've seen a closing quote and the next char is not another quote, so close it. + String marker = QUOTE_START + numQuotes + QUOTE_END; + output.append(marker); + quoteValue.setLength(quoteValue.length() - 1); // trim the final quote + quotes.put(marker, new Quote(numQuotes++, currentQuote, quoteValue.toString())); + quoteValue = new StringBuilder(); + currentQuote = 0; + prev = 0; + output.append(c); + } else if (c == currentQuote && prev == currentQuote) { + // it was an escaped quote. + prev = 0; + quoteValue.append(c); + } else { + quoteValue.append(c); + } + } + // We could have ended on the quote + if (currentQuote != 0 && (prev == '`' || prev == '"' || prev == '\'')) { + String marker = QUOTE_START + numQuotes + QUOTE_END; + output.append(marker); + quoteValue.setLength(quoteValue.length() - 1); // trim the final quote + quotes.put(marker, new Quote(numQuotes, currentQuote, quoteValue.toString())); + } + + return output.toString(); + } + + private String reQuote(String benchSql) { + for (Quote quote : quotes.values()) { + Matcher m = Pattern.compile(QUOTE_START + quote.number + QUOTE_END).matcher(benchSql); + if (quote.quoteType == '`') { + benchSql = m.replaceAll(identifierQuote() + quote.value + identifierQuote()); + } else { + benchSql = m.replaceAll(quote.quoteType + quote.value + quote.quoteType); + } + } + quotes.clear(); + return benchSql; + } + + protected abstract char identifierQuote(); + +} diff --git itests/capybara/src/main/java/org/apache/hive/test/capybara/infra/SortingComparator.java itests/capybara/src/main/java/org/apache/hive/test/capybara/infra/SortingComparator.java new file mode 100644 index 0000000..8c86f78 --- /dev/null +++ itests/capybara/src/main/java/org/apache/hive/test/capybara/infra/SortingComparator.java @@ -0,0 +1,41 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hive.test.capybara.infra; + +import org.apache.hive.test.capybara.data.DataSet; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.sql.SQLException; + +/** + * Sort results and then compare them. + */ +class SortingComparator extends NonSortingComparator { + + static final private Logger LOG = LoggerFactory.getLogger(SortingComparator.class.getName()); + + @Override + public void compare(DataSet hive, DataSet bench) throws SQLException, IOException { + compareSchemas(hive, bench); + + compare(sort(hive), sort(bench)); + } + +} diff --git itests/capybara/src/main/java/org/apache/hive/test/capybara/infra/StaticDataGenerator.java itests/capybara/src/main/java/org/apache/hive/test/capybara/infra/StaticDataGenerator.java new file mode 100644 index 0000000..f37b609 --- /dev/null +++ itests/capybara/src/main/java/org/apache/hive/test/capybara/infra/StaticDataGenerator.java @@ -0,0 +1,65 @@ +/** + * 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.hive.test.capybara.infra; + +import org.apache.hadoop.hive.metastore.api.FieldSchema; +import org.apache.hive.test.capybara.data.DataSet; +import org.apache.hive.test.capybara.iface.DataGenerator; +import org.apache.hive.test.capybara.iface.TestTable; + +import java.util.List; + +/** + * A DataGenerator that takes canned data a user gives and creates a table. This data generator + * ignores the scale and pctNulls passed to {@link #generateData}. + */ +public class StaticDataGenerator extends DataGenerator { + + private List rows; + private final String delimiter; + + /** + * + * @param rows The data, as strings. Integer types (bigint, int, etc.) need to be valid + * integers. Floating point and fixed point types need to be valid numbers + * (scientific notation is ok). Boolean should be true or false. String length + * is not checked for varchar and char data. Dates should be in the format + * 'yyyy-mm-dd' and timestamps 'yyyy-mm-dd hh:mm:ss[.f]' where f is a fractional + * component of the seconds. Binary types should be Base64 encoded strings, they + * will be decoded by {@link org.apache.commons.codec.binary.Base64}. + * @param delimiter column delimiter + */ + public StaticDataGenerator(List rows, String delimiter) { + this.rows = rows; + this.delimiter = delimiter; + } + + @Override + public DataSet generateData(TestTable table, int scale, double[] pctNulls) { + // This method can be used with or without a schema. Without one it just creates a bunch of + // string columns. This is used by IntegrationTest.compareAgainst. + List cols = null; + if (table != null) { + // Figure out the schema of the table + cols = table.getCombinedSchema(); + } + + return (cols == null) ? new StringDataSet(rows, delimiter, "NULL") : + new StringDataSet(cols, rows, delimiter, "NULL"); + } +} diff --git itests/capybara/src/main/java/org/apache/hive/test/capybara/infra/StatsDataGenerator.java itests/capybara/src/main/java/org/apache/hive/test/capybara/infra/StatsDataGenerator.java new file mode 100644 index 0000000..bc1c6505 --- /dev/null +++ itests/capybara/src/main/java/org/apache/hive/test/capybara/infra/StatsDataGenerator.java @@ -0,0 +1,578 @@ +/** + * 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.hive.test.capybara.infra; + +import org.apache.hive.test.capybara.data.DataSet; +import org.apache.hive.test.capybara.iface.TestTable; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import org.apache.hadoop.hive.metastore.api.FieldSchema; +import org.apache.hive.test.capybara.iface.DataGenerator; + +import java.io.Serializable; +import java.math.BigDecimal; +import java.sql.Date; +import java.sql.Timestamp; +import java.util.HashMap; +import java.util.Map; + +/** + * A data generator that generates data based on the stats produced by Hive. This works by + * provided {@link org.apache.hive.test.capybara.infra.DataGeneratorImpl.ColGenerator}s that take + * the statistics gathered from Hive into account. + */ +public class StatsDataGenerator extends RandomDataGenerator implements Serializable { + static final private Logger LOG = LoggerFactory.getLogger(StatsDataGenerator.class); + + private TableStats tableStats; + + public StatsDataGenerator(TableStats tableStats, long seed) { + super(seed); + this.tableStats = tableStats; + } + + @Override + protected DataGenerator copy(int copyNum) { + return new StatsDataGenerator(tableStats, seed + copyNum); + } + + @Override + public DataSet generateData(TestTable table, int scale, double[] pctNulls) { + tableStats.scale(scale); + return super.generateData(table, scale, pctNulls); + } + + public static class ColStats implements Serializable { + String colName; + String dataType; + Object min; + Object max; + long numNulls; + long distinctCount; + double avgColLen; + long maxColLen; + long numTrues; + long numFalses; + double pctNull; + double pctTrue; + double pctFalse; + + public ColStats(String colName, String dataType, Object min, Object max, double avgColLen, + long maxColLen, long numFalses, long numTrues, long distinctCount, + long numNulls) { + this.colName = colName; + this.dataType = dataType; + this.min = min; + this.max = max; + this.avgColLen = avgColLen; + this.maxColLen = maxColLen; + this.numFalses = numFalses; + this.numTrues = numTrues; + this.distinctCount = distinctCount; + this.numNulls = numNulls; + } + + /** + * Scale up or down the counts for this column + * @param multiplier multiplier to use in the scaling + * @param numRows total number of rows in the original + */ + void scale(double multiplier, long numRows, long origNumRows) { + + // When the scale is small relative to the original data set we can get a very small + // multiplier which ends up in everything else being rounded down to zero. Prevent that. + if (distinctCount != 0) { + distinctCount = Math.max(10L, (long)(distinctCount * multiplier)); + } + if (numTrues != 0) { + pctTrue = numTrues / (double)origNumRows; + } + if (numFalses != 0) { + pctFalse = numFalses / (double)origNumRows; + } + pctNull = (double)numNulls / (double)origNumRows; + } + + } + + public static class TableStats implements Serializable { + String dbName; + String tableName; + long totalSize; + long numRows; + int numPartitions; + Map colStats; + + public TableStats(Map colStats, int numPartitions, String dbName, + String tableName, long totalSize, long numRows) { + this.colStats = colStats; + this.numPartitions = numPartitions; + this.dbName = dbName; + this.tableName = tableName; + this.totalSize = totalSize; + this.numRows = numRows; + } + + /** + * Modify values based on the scale. + * @param scale Scale used for this test + */ + TableStats scale(int scale) { + double multiplier = (scale * 1024)/ (double)totalSize ; + + // When the scale is small relative to the original data set we can get a very small + // multiplier which ends up in everything else being rounded down to zero. Prevent that. + if (numPartitions != 0) { + numPartitions = Math.max(2, (int)(numPartitions * multiplier)); + } + long origNumRows = numRows; + numRows = Math.max(100L, (long)(numRows * multiplier)); + + for (Map.Entry entry : colStats.entrySet()) { + entry.getValue().scale(multiplier, numRows, origNumRows); + } + return this; + } + } + + @Override + protected ColGenerator getLongGenerator(FieldSchema col) { + final ColStats colStats = tableStats.colStats.get(col.getName()); + if (colStats == null) { + LOG.warn("Unable to find column stats entry for " + col.getName() + ", using basic random " + + "generation for that column."); + return super.getLongGenerator(col); + } + final Long max = (Long)colStats.max; + final Long min = (Long)colStats.min; + final Map distinctVals = new HashMap<>(); + + return new ColGenerator() { + private long range = 0; + + @Override + public Object generate(double pctNull) { + // Always check whether we should return null before checking for distinct values + if (rand.nextDouble() < colStats.pctNull) return null; + + if (distinctVals.size() >= colStats.distinctCount) { + return distinctVals.get(rand.nextInt(distinctVals.size())); + } + if (range == 0) { + range = max - min; + if (range < 2) range = 2; + } + Long generated = Math.abs(rand.nextLong() % range) + min; + distinctVals.put(distinctVals.size(), generated); + return generated; + } + }; + } + + @Override + protected ColGenerator getIntGenerator(FieldSchema col) { + final ColStats colStats = tableStats.colStats.get(col.getName()); + if (colStats == null) { + LOG.warn("Unable to find column stats entry for " + col.getName() + ", using basic random " + + "generation for that column."); + return super.getIntGenerator(col); + } + final Integer max = (Integer)colStats.max; + final Integer min = (Integer)colStats.min; + final Map distinctVals = new HashMap<>(); + + return new ColGenerator() { + private int range = 0; + + @Override + public Object generate(double pctNull) { + // Always check whether we should return null before checking for distinct values + if (rand.nextDouble() < colStats.pctNull) return null; + + if (distinctVals.size() >= colStats.distinctCount) { + return distinctVals.get(rand.nextInt(distinctVals.size())); + } + if (range == 0) { + range = max - min; + if (range < 2) range = 2; + } + Integer generated = Math.abs(rand.nextInt() % range) + min; + distinctVals.put(distinctVals.size(), generated); + return generated; + } + }; + } + + @Override + protected ColGenerator getShortGenerator(FieldSchema col) { + final ColStats colStats = tableStats.colStats.get(col.getName()); + if (colStats == null) { + LOG.warn("Unable to find column stats entry for " + col.getName() + ", using basic random " + + "generation for that column."); + return super.getShortGenerator(col); + } + final Short max = (Short)colStats.max; + final Short min = (Short)colStats.min; + final Map distinctVals = new HashMap<>(); + + return new ColGenerator() { + private short range = 0; + + @Override + public Object generate(double pctNull) { + // Always check whether we should return null before checking for distinct values + if (rand.nextDouble() < colStats.pctNull) return null; + + if (distinctVals.size() >= colStats.distinctCount) { + return distinctVals.get(rand.nextInt(distinctVals.size())); + } + if (range == 0) { + range = (short)(max - min); // short - short = int? seriously? + if (range < 2) range = 2; + } + Short generated = (short)(rand.nextInt(range) + min); + distinctVals.put(distinctVals.size(), generated); + return generated; + } + }; + } + + @Override + protected ColGenerator getByteGenerator(FieldSchema col) { + final ColStats colStats = tableStats.colStats.get(col.getName()); + if (colStats == null) { + LOG.warn("Unable to find column stats entry for " + col.getName() + ", using basic random " + + "generation for that column."); + return super.getByteGenerator(col); + } + final Byte max = (Byte)colStats.max; + final Byte min = (Byte)colStats.min; + final Map distinctVals = new HashMap<>(); + + return new ColGenerator() { + private byte range = 0; + + @Override + public Object generate(double pctNull) { + // Always check whether we should return null before checking for distinct values + if (rand.nextDouble() < colStats.pctNull) return null; + + if (distinctVals.size() >= colStats.distinctCount) { + return distinctVals.get(rand.nextInt(distinctVals.size())); + } + if (range == 0) { + range = (byte)(max - min); + if (range < 2) range = 2; + } + Byte generated = (byte)(rand.nextInt(range) + min); + distinctVals.put(distinctVals.size(), generated); + return generated; + } + }; + } + + @Override + protected ColGenerator getFloatGenerator(FieldSchema col) { + final ColStats colStats = tableStats.colStats.get(col.getName()); + if (colStats == null) { + LOG.warn("Unable to find column stats entry for " + col.getName() + ", using basic random " + + "generation for that column."); + return super.getFloatGenerator(col); + } + final Float max = (Float)colStats.max; + final Float min = (Float)colStats.min; + final Map distinctVals = new HashMap<>(); + + return new ColGenerator() { + @Override + public Object generate(double pctNull) { + // Always check whether we should return null before checking for distinct values + if (rand.nextDouble() < colStats.pctNull) return null; + + if (distinctVals.size() >= colStats.distinctCount) { + return distinctVals.get(rand.nextInt(distinctVals.size())); + } + float range = max - min; + float tmp; + if (range > 2) { + tmp = Math.abs(rand.nextLong() % range) + rand.nextFloat() + min; + } else { + tmp = rand.nextFloat() + min; + } + // Take the min here because I could have a range of 0.5 and then have generated 0.75. + Float generated = Math.min(max, tmp); + distinctVals.put(distinctVals.size(), generated); + return generated; + } + }; + } + + @Override + protected ColGenerator getDoubleGenerator(FieldSchema col) { + final ColStats colStats = tableStats.colStats.get(col.getName()); + if (colStats == null) { + LOG.warn("Unable to find column stats entry for " + col.getName() + ", using basic random " + + "generation for that column."); + return super.getDoubleGenerator(col); + } + final Double max = (Double)colStats.max; + final Double min = (Double)colStats.min; + final Map distinctVals = new HashMap<>(); + + return new ColGenerator() { + @Override + public Object generate(double pctNull) { + // Always check whether we should return null before checking for distinct values + if (rand.nextDouble() < colStats.pctNull) return null; + + if (distinctVals.size() >= colStats.distinctCount) { + return distinctVals.get(rand.nextInt(distinctVals.size())); + } + double range = max - min; + double tmp; + if (range > 2) { + tmp = Math.abs(rand.nextLong() % range) + rand.nextDouble() + min; + } else { + tmp = rand.nextDouble() + min; + } + // Take the min here because I could have a range of 0.5 and then have generated 0.75. + Double generated = Math.min(max, tmp); + distinctVals.put(distinctVals.size(), generated); + return generated; + } + }; + } + + @Override + protected ColGenerator getBigDecimalGenerator(FieldSchema col, final int precision, + final int scale) { + final ColStats colStats = tableStats.colStats.get(col.getName()); + if (colStats == null) { + LOG.warn("Unable to find column stats entry for " + col.getName() + ", using basic random " + + "generation for that column."); + return super.getBigDecimalGenerator(col, precision, scale); + } + final BigDecimal max = (BigDecimal)colStats.max; + final BigDecimal min = (BigDecimal)colStats.min; + final Map distinctVals = new HashMap<>(); + + return new ColGenerator() { + @Override + public Object generate(double pctNull) { + // Always check whether we should return null before checking for distinct values + if (rand.nextDouble() < colStats.pctNull) return null; + + if (distinctVals.size() >= colStats.distinctCount) { + return distinctVals.get(rand.nextInt(distinctVals.size())); + } + double range = max.doubleValue() - min.doubleValue(); + double tmp; + if (range > 2) { + tmp = Math.abs(rand.nextLong() % range) + rand.nextDouble() + min.doubleValue(); + } else { + tmp = rand.nextDouble() + min.doubleValue(); + } + tmp = Math.min(max.doubleValue(), tmp); + BigDecimal generated = new BigDecimal(tmp).setScale(scale, BigDecimal.ROUND_DOWN); + distinctVals.put(distinctVals.size(), generated); + return generated; + } + }; + } + + @Override + protected ColGenerator getDateGenerator(FieldSchema col) { + final ColStats colStats = tableStats.colStats.get(col.getName()); + if (colStats == null) { + LOG.warn("Unable to find column stats entry for " + col.getName() + ", using basic random " + + "generation for that column."); + return super.getDateGenerator(col); + } + final Date max = (Date)colStats.max; + final Date min = (Date)colStats.min; + final Map distinctVals = new HashMap<>(); + + return new ColGenerator() { + private long range = 0; + + @Override + public Object generate(double pctNull) { + // Always check whether we should return null before checking for distinct values + if (rand.nextDouble() < colStats.pctNull) return null; + + if (distinctVals.size() >= colStats.distinctCount) { + return distinctVals.get(rand.nextInt(distinctVals.size())); + } + if (range == 0) { + range = max.getTime() - min.getTime(); + if (range < 2) range = 2; + } + Date generated = new Date(Math.abs(rand.nextLong() % range) + min.getTime()); + distinctVals.put(distinctVals.size(), generated); + return generated; + } + }; + } + + @Override + protected ColGenerator getTimestampGenerator(FieldSchema col) { + final ColStats colStats = tableStats.colStats.get(col.getName()); + if (colStats == null) { + LOG.warn("Unable to find column stats entry for " + col.getName() + ", using basic random " + + "generation for that column."); + return super.getTimestampGenerator(col); + } + final Timestamp max = (Timestamp)colStats.max; + final Timestamp min = (Timestamp)colStats.min; + final Map distinctVals = new HashMap<>(); + + return new ColGenerator() { + private long range = 0; + + @Override + public Object generate(double pctNull) { + // Always check whether we should return null before checking for distinct values + if (rand.nextDouble() < colStats.pctNull) return null; + + if (distinctVals.size() >= colStats.distinctCount) { + return distinctVals.get(rand.nextInt(distinctVals.size())); + } + if (range == 0) { + range = max.getTime() - min.getTime(); + if (range < 2) range = 2; + } + Timestamp generated = new Timestamp(Math.abs(rand.nextLong() % range) + min.getTime()); + distinctVals.put(distinctVals.size(), generated); + return generated; + } + }; + } + + @Override + protected ColGenerator getStringGenerator(FieldSchema col, final int maxLength) { + final ColStats colStats = tableStats.colStats.get(col.getName()); + if (colStats == null) { + LOG.warn("Unable to find column stats entry for " + col.getName() + ", using basic random " + + "generation for that column."); + return super.getStringGenerator(col, maxLength); + } + final Map distinctVals = new HashMap<>(); + + return new ColGenerator() { + long totalLen = 0L; + + @Override + public Object generate(double pctNull) { + if (rand.nextDouble() < colStats.pctNull) return null; + if (distinctVals.size() >= colStats.distinctCount) { + return distinctVals.get(rand.nextInt(distinctVals.size())); + } + + // If the average length is trending too low, produce a larger string. If it's trending + // too high, produce a smaller string. If it's within 10% of the requested average then + // just take what we get. + int length; + if (colStats.maxColLen <= colStats.avgColLen + 1) { + length = (int)colStats.avgColLen; + } else if (distinctVals.size() > 0 && + totalLen / distinctVals.size() > colStats.avgColLen * 1.1) { + if (colStats.avgColLen < 2.0) length = 1; + else length = rand.nextInt((int)colStats.avgColLen) + 1; + } else if (distinctVals.size() > 0 && + totalLen / distinctVals.size() < colStats.avgColLen * 0.9) { + length = rand.nextInt((int)colStats.maxColLen - (int)colStats.avgColLen) + + (int)colStats.avgColLen; + } else { + length = rand.nextInt((int)colStats.maxColLen) + 1; + } + + totalLen += length; + StringBuilder builder = new StringBuilder(length); + for (int i = 0; i < length; i++) { + builder.append((char)(rand.nextInt('z' - 'a') + 'a')); + } + String generated = builder.toString(); + distinctVals.put(distinctVals.size(), generated); + return generated; + } + }; + } + + @Override + protected ColGenerator getBinaryGenerator(FieldSchema col, final int maxLength) { + final ColStats colStats = tableStats.colStats.get(col.getName()); + if (colStats == null) { + LOG.warn("Unable to find column stats entry for " + col.getName() + ", using basic random " + + "generation for that column."); + return super.getBinaryGenerator(col, maxLength); + } + + return new ColGenerator() { + long totalLen = 0L; + long numGenerated = 0L; + + @Override + public Object generate(double pctNull) { + if (rand.nextDouble() < colStats.pctNull) return null; + + // If the average length is trending too low, produce a larger string. If it's trending + // too high, produce a smaller string. If it's within 10% of the requested average then + // just take what we get. + int length; + if (colStats.maxColLen <= colStats.avgColLen + 1) { + length = (int)colStats.avgColLen; + } else if (numGenerated > 0 && totalLen / numGenerated > colStats.avgColLen * 1.1) { + if (colStats.avgColLen < 2.0) length = 1; + else length = rand.nextInt((int)colStats.avgColLen) + 1; + } else if (numGenerated > 0 && totalLen / numGenerated < colStats.avgColLen * 0.9) { + length = rand.nextInt((int)colStats.maxColLen - (int)colStats.avgColLen) + + (int)colStats.avgColLen; + } else { + length = rand.nextInt((int)colStats.maxColLen) + 1; + } + + totalLen += length; + numGenerated++; + byte[] bytes = new byte[length]; + rand.nextBytes(bytes); + return bytes; + } + }; + } + + @Override + protected ColGenerator getBoolGenerator(FieldSchema col) { + final ColStats colStats = tableStats.colStats.get(col.getName()); + if (colStats == null) { + LOG.warn("Unable to find column stats entry for " + col.getName() + ", using basic random " + + "generation for that column."); + return super.getBoolGenerator(col); + } + + return new ColGenerator() { + + @Override + public Object generate(double pctNull) { + double d = rand.nextDouble(); + if (d < colStats.pctTrue) return true; + else if (d < colStats.pctTrue + colStats.pctFalse) return false; + else return null; + } + }; + } +} diff --git itests/capybara/src/main/java/org/apache/hive/test/capybara/infra/StringDataSet.java itests/capybara/src/main/java/org/apache/hive/test/capybara/infra/StringDataSet.java new file mode 100644 index 0000000..402b6cc --- /dev/null +++ itests/capybara/src/main/java/org/apache/hive/test/capybara/infra/StringDataSet.java @@ -0,0 +1,105 @@ +/** + * 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.hive.test.capybara.infra; + +import org.apache.hadoop.hive.metastore.api.FieldSchema; +import org.apache.hive.test.capybara.data.DataSet; +import org.apache.hive.test.capybara.data.Row; +import org.apache.hive.test.capybara.data.RowBuilder; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.ArrayList; +import java.util.Iterator; +import java.util.List; + +/** + * A string based DataSet. This can be generated by static data the user has given or by Hive + * returning data via the CLI. Sometimes at construction time the schema will not be known (such + * as when data comes from Hive's CLI). + * + * Conversion from Strings to Object[] is done lazily in the iterator to avoid unnecessary + * conversions. + */ +class StringDataSet extends DataSet { + static final private Logger LOG = LoggerFactory.getLogger(StringDataSet.class.getName()); + + final List stringRows; + final String delimiter; + final String nullIndicator; + + /** + * + * @param cols schema + * @param rows rows, one per String. + * @param delimiter value that delimits columns in the rows + */ + StringDataSet(List cols, List rows, String delimiter, String nullIndicator) { + super(cols); + stringRows = rows; + this.delimiter = delimiter; + this.nullIndicator = nullIndicator; + } + + StringDataSet(List rows, String delimiter, String nullIndicator) { + this(null, rows, delimiter, nullIndicator); + } + + @Override + public Iterator iterator() { + // If we've already done the conversion, don't redo it. + if (rows != null) return rows.iterator(); + + if (schema == null) { + throw new RuntimeException("You must set the schema before you try to iterate through the " + + "data set."); + } + rows = new ArrayList<>(); + final RowBuilder builder = new RowBuilder(schema); + final Iterator outerIter = stringRows.iterator(); + + return new Iterator() { + @Override + public boolean hasNext() { + return outerIter.hasNext(); + } + + @Override + public Row next() { + String strRow = outerIter.next(); + if (strRow != null) { + String[] strCols = strRow.split(delimiter); + Row row = builder.build(); + for (int i = 0; i < row.size(); i++) { + row.get(i).fromString(strCols[i], nullIndicator); + } + // Add this to the rows so we don't have to do the conversion again if asked. + rows.add(row); + return row; + } else { + return null; + } + } + + @Override + public void remove() { + outerIter.remove(); + } + }; + } +} diff --git itests/capybara/src/main/java/org/apache/hive/test/capybara/infra/TestConf.java itests/capybara/src/main/java/org/apache/hive/test/capybara/infra/TestConf.java new file mode 100644 index 0000000..cf82cf7 --- /dev/null +++ itests/capybara/src/main/java/org/apache/hive/test/capybara/infra/TestConf.java @@ -0,0 +1,268 @@ +/** + * 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.hive.test.capybara.infra; + +import com.google.common.annotations.VisibleForTesting; +import org.apache.hadoop.hive.conf.HiveConf; +import org.junit.Assert; + +/** + * Configuration for integration tests. Configuration is done as a set of system properties. + */ +public class TestConf { + + /** + * Property that controls how Hive is accessed in the tests. + */ + public static final String ACCESS_PROPERTY = "hive.test.capybara.access"; + + /** + * Value to set to access Hive via the CLI. + */ + public static final String ACCESS_CLI = "cli"; + + /** + * Value to set to access Hive via JDBC + */ + public static final String ACCESS_JDBC = "jdbc"; + + /** + * Default access method for the tests. + */ + public static final String ACCESS_DEFAULT = ACCESS_CLI; + + /** + * Determine whether this test is being executed via the CLI or JDBC + * @return cli or jdbc, depending on how this is being executed. + */ + public static String access() { + return System.getProperty(ACCESS_PROPERTY, ACCESS_DEFAULT).toLowerCase(); + } + + @VisibleForTesting static void setAccess(String access) { + System.setProperty(ACCESS_PROPERTY, access); + } + + /** + * Property that controls which engine is used to execute Hive queries. + */ + public static final String ENGINE_PROPERTY = "hive.test.capybara.engine"; + + /** + * Value to set to execute Hive queries using Tez. (Currently local tests fail when set.) + */ + public static final String ENGINE_TEZ = "tez"; + + /** + * Value to set to get the default engine from Hive (currently MapReduce). + */ + public static final String ENGINE_UNSPECIFIED = "default"; + + /** + * Default engine. + */ + public static final String ENGINE_DEFAULT = ENGINE_UNSPECIFIED; + + /** + * Determine execution engine for this test + * @return default, tez or spark + */ + public static String engine() { + return System.getProperty(ENGINE_PROPERTY, ENGINE_DEFAULT).toLowerCase(); + } + + @VisibleForTesting + static void setEngine(String engine) { + System.setProperty(ENGINE_PROPERTY, engine); + } + + /** + * Property to set to control how many tasks Tez runs. This only controls Tez in the local + * (mini-cluster). Tez on the cluster will be controlled by the configuration of the cluster. + */ + public static final String TEZ_NUM_TASKS_PROPERTY = "hive.test.capybara.tez.num.tasks"; + + /** + * Default number of Tez tasks when run in the minicluster. + */ + public static final String TEZ_NUM_TASKS_DEFAULT = "2"; + + /** + * Determine number of tasks to start in MiniTezCluster + * @return numer of tasks + */ + static int numTezTasks() { + return Integer.valueOf(System.getProperty(TEZ_NUM_TASKS_PROPERTY, TEZ_NUM_TASKS_DEFAULT)); + } + + /** + * Property to set to control which file format Hive uses by default. + */ + public static final String FILE_FORMAT_PROPERTY = "hive.test.capybara.file.format"; + + /** + * Value to set to use ORC as the default file format. + */ + public static final String FILE_FORMAT_ORC = "ORC"; + + /** + * Value to set to use Sequence as the default file format. + */ + public static final String FILE_FORMAT_SEQUENCE = "SequenceFile"; + + /** + * Value to set to use Text as the default file format. + */ + public static final String FILE_FORMAT_TEXT = "TextFile"; + + /** + * Value to set to use RCFile as the default file format. + */ + public static final String FILE_FORMAT_RCFILE = "RCfile"; + + /** + * Default value to use for file format. + */ + public static final String FILE_FORMAT_DEFAULT = FILE_FORMAT_ORC; + + /** + * Determine default storage format for this test + * @return orc, parquet, text, rcfile + */ + public static String fileFormat() { + String format = System.getProperty(FILE_FORMAT_PROPERTY, FILE_FORMAT_DEFAULT); + // So Validator.validate returns null if everything is good, or error string if there's an + // issue. Wow, that's the weirdest interface I've seen in a while. + Assert.assertNull(HiveConf.ConfVars.HIVEDEFAULTFILEFORMAT.getValidator().validate(format)); + return format; + } + + /** + * Property to set to control whether these tests use a secure cluster. + */ + public static final String SECURITY_PROPERTY = "hive.test.capybara.security"; + + /** + * Value for running with security off. + */ + public static final String SECURITY_NONSECURE = "nonsecure"; + + /** + * Default setting for security. + */ + public static final String SECURITY_DEFAULT = SECURITY_NONSECURE; + + /** + * Determine whether this test is being executed in secure or non-secure mode + * @return secure or nonsecure + */ + public static String security() { + return System.getProperty(SECURITY_PROPERTY, SECURITY_DEFAULT).toLowerCase(); + } + + /** + * Property to set to control which metastore implementation Hive uses. This will only affect + * the minicluster setup, as the metastore on a real cluster will be controlled by the cluster. + */ + public static final String METASTORE_PROPERTY = "hive.test.capybara.metastore"; + + /** + * Value for running with a RDBMS metastore. + */ + public static final String METASTORE_RDBMS = "rdbms"; + + /** + * Default value for the metastore. + */ + public static final String METASTORE_DEFAULT = METASTORE_RDBMS; + + /** + * Determine which metastore implementation to use. + * @return rdbms or hbase. + */ + public static String metastore() { + return System.getProperty(METASTORE_PROPERTY, METASTORE_DEFAULT).toLowerCase(); + } + + /** + * Property to set to control whether tests run locally in a minicluster or on a real cluster. + * The values are boolean. If set to true, you must also provide values for HADOOP_HOME and + * HIVE_HOME via system properties (e.g. add -DHADOOP_HOME=/cluster/test/hadoop to your command + * line). + */ + public static final String USE_CLUSTER_PROPERTY ="hive.test.capybara.use.cluster"; + + /** + * Default value for running on a cluster. + */ + public static final String USE_CLUSTER_DEFAULT = "false"; + + /** + * Determine whether the tests should run on a cluster. + * @return true if they should run on a cluster. + */ + public static boolean onCluster() { + return Boolean.valueOf(System.getProperty(USE_CLUSTER_PROPERTY, USE_CLUSTER_DEFAULT)); + } + + /** + * Property to set to control the scale the tests run at. The unit of scale is a kilobyte. + */ + public static final String SCALE_PROPERTY = "hive.test.capybara.scale"; + + /** + * Default value for the scale. This is set for the local case and should definitely be set + * higher if being run on a cluster. + */ + public static final String SCALE_DEFAULT = "1"; + + public static int getScale() { + return Integer.valueOf(System.getProperty(SCALE_PROPERTY, SCALE_DEFAULT)); + } + + /** + * Property to set to control when generated data is spilled to disk. In bytes. + */ + public static final String SPILL_SIZE_PROPERTY = "hive.test.capybara.data.spill.size"; + + /** + * Default spill size. + */ + public static final String SPILL_SIZE_DEFAULT = Integer.toString(1024 * 1024 * 256); + + public static int getSpillSize() { + // Keep in mind that twice the spill size may be in memory at a time as it will be spilling + // one batch while it is generating the next. + return Integer.valueOf(System.getProperty(SPILL_SIZE_PROPERTY, SPILL_SIZE_DEFAULT)); + } + + /** + * Property to set to control when data is generated on the cluster instead of on the local + * machine. This only applies when running tests on the cluster. + */ + public static final String CLUSTERGEN_SIZE_PROPERTY = "hive.test.capybara.data.clustergen.threshold"; + + /** + * Default size to switch to generating data on the cluster. In bytes. + */ + public static final String CLUSTERGEN_SIZE_DEFAULT = Integer.toString(1024 * 1024 * 1024); + + public static int getClusterGenThreshold() { + return Integer.valueOf(System.getProperty(CLUSTERGEN_SIZE_PROPERTY, CLUSTERGEN_SIZE_DEFAULT)); + } +} diff --git itests/capybara/src/main/java/org/apache/hive/test/capybara/infra/TestManager.java itests/capybara/src/main/java/org/apache/hive/test/capybara/infra/TestManager.java new file mode 100644 index 0000000..89d3ab5 --- /dev/null +++ itests/capybara/src/main/java/org/apache/hive/test/capybara/infra/TestManager.java @@ -0,0 +1,107 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hive.test.capybara.infra; + +import com.google.common.annotations.VisibleForTesting; +import org.apache.hadoop.conf.Configurable; +import org.apache.hadoop.conf.Configuration; +import org.apache.hive.test.capybara.iface.Benchmark; +import org.apache.hive.test.capybara.iface.ClusterManager; + +/** + * Manage all aspects of the test. This is the glue that holds all the pieces together. The two + * most important pieces here are the ClusterManager, which contains references to the cluster + * being run on, and Benchmark, which references the generator of expected results for the tests. + */ +public class TestManager implements Configurable { + + private static TestManager self = null; + + private Configuration conf; + private ClusterManager cluster; + private Benchmark bench; + private Benchmark oneTimeBench; + + public static TestManager getTestManager() { + if (self == null) { + self = new TestManager(); + } + return self; + } + + private TestManager() { + + } + + /** + * Get the cluster manager for this test. + * @return cluster manager + */ + public ClusterManager getClusterManager() { + if (cluster == null) { + cluster = TestConf.onCluster() ? new ExternalClusterManager() : new MiniClusterManager(); + cluster.setConf(conf); + } + return cluster; + } + + /** + * Get the benchmark for this test. + * @return benchmark + */ + public Benchmark getBenchmark() { + if (oneTimeBench != null) return oneTimeBench; + if (bench == null) { + bench = TestConf.onCluster() ? new PostgresBenchmark() : new DerbyBenchmark(); + } + return bench; + } + + /** + * Set up a special Benchmark for this test. This gives the user an opportunity to inject a + * special Benchmark for a particular test, rather than using whatever is standard for the + * current configuration. This will be reset at the end of the test. + * @param bench special Benchmark to use + */ + public void setOneTimeBenchmark(Benchmark bench) { + oneTimeBench = bench; + } + + /** + * Reset the Benchmark to the standard for the current configuration. This will be called at + * the end of each test by the system. + */ + public void resetBenchmark() { + oneTimeBench = null; + } + + @Override + public void setConf(Configuration conf) { + this.conf = conf; + } + + @Override + public Configuration getConf() { + return conf; + } + + @VisibleForTesting + void setClusterManager(ClusterManager clusterMgr) { + cluster = clusterMgr; + } +} diff --git itests/capybara/src/main/java/org/apache/hive/test/capybara/infra/TranslationException.java itests/capybara/src/main/java/org/apache/hive/test/capybara/infra/TranslationException.java new file mode 100644 index 0000000..0960aa0 --- /dev/null +++ itests/capybara/src/main/java/org/apache/hive/test/capybara/infra/TranslationException.java @@ -0,0 +1,24 @@ +/** +* 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.hive.test.capybara.infra; + +class TranslationException extends Exception { + TranslationException(String section, String hiveSql) { + super("Could not translate " + section + ", Hive SQL: <" + hiveSql + ">"); + } +} diff --git itests/capybara/src/main/java/org/apache/hive/test/capybara/package-info.java itests/capybara/src/main/java/org/apache/hive/test/capybara/package-info.java new file mode 100644 index 0000000..ffdfd49 --- /dev/null +++ itests/capybara/src/main/java/org/apache/hive/test/capybara/package-info.java @@ -0,0 +1,118 @@ +/* + * 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. + */ + +/** + * Capybara is a integration test framework for Hive. It has several goals: + *

    + *
  • Separate writing and running tests from permutations of features in Hive. That is, one + * test should be able to run in beeline, CLI, and JDBC, with ORC, Parquet, RCFile, etc., with + * Tez or Spark as an engine, etc.
  • + *
  • The same tests should run locally on developer machines and against a cluster. These + * tests should be able to automatically scale as appropriate to their location. This allows + * simple testing for devs on their boxes with a few K of data and for users on their clusters + * with G or T of data.
  • + *
  • Expected results of the queries should be auto-generated by the system. It's insane to + * rely on dev's eyeballing results.
  • + *
  • Access to the query plan should be provided in code, rather than requiring string compares + * to see if the output of explain looks the same today as it did yesterday.
  • + *
  • The tool should leverage JUnit and Maven. It should be in Java so that developers can + * test a variety of scenarios beyond just SQL queries.
  • + *
  • The tool should be able to simulate user data. Given users queries and access to the + * user's tables, it should generate data like the user data and then build tests for the user's + * queries.
  • + * + *
+ * + *

Capybara works by managing an instance of Hive and a Benchmark. When running locally the + * Hive instance will be run in process using a DFSMiniCluster (and other mini-clusters as + * appropriate). When running on a cluster the Hive instance will connect to Hive via CLI or + * JDBC (depending on how the system is configured). The Benchmark usually uses a RDBMS (by + * default Derby locally and Postgres on the cluster) to run the same query and compare the + * results. To some extent the system can smooth over the differences between Hive SQL and ANSI + * standard SQL. If you need to run a completely different query against the benchmark (say + * you're testing a UDF not supported in the Benchmark) then you can run separate queries against + * Hive and the Benchmark. For extreme cases you can also provide your own Benchmark + * implementation. + *

+ * + *

To use the framework create a JUnit4 test (i.e. one that uses @Test) and have it extend + * {@link org.apache.hive.test.capybara.IntegrationTest}. IntegrationTest handles setup and + * teardown of cluster and benchmark resources.

+ * + *

{@link org.apache.hive.test.capybara.TableTool} + * provides methods to build a number of commonly used test tables. You can also use + * {@link org.apache.hive.test.capybara.iface.TestTable} to build your own table and populate it. + * The system keeps track of tables created and populated in both Hive and the Benchmark and will + * not re-create the tables if they already exist. It will detect changes in scale, file format, + * etc. that will require a re-creation and then handle dropping and re-creating the table.

+ * + *

Once your tables are created and populated you can run queries against them using + * {@link org.apache.hive.test.capybara.IntegrationTest#runQuery(String)} and related methods. + * You can run any number of queries desired in the test. All operations will be run against + * both Hive and the Benchmark. If you need to set any configuartion values, this should be done + * via calls to {@link org.apache.hive.test.capybara.IntegrationTest#set}. The configuration is + * reset for each test so that set calls from one test do not affect any other tests. If you + * have a set of values you would like to have set for all tests in a file you can do that in a + * @Before method. + *

+ * + *

Some features require a set of configurtion values to be set. Rather than requiring test + * writers to set these up each time, annotations are provided that will tell the system to set + * the appropriate configuration values for a test. For example, all of the values for testing + * SQL Standard Authorization can be turned on by annotating a test with @SqlStdAuthOn. See the + * {@link org.apache.hive.test.capybara.annotations} package for a full list.

+ * + *

Once you have produced a result you would like to compare you can use one of the + * comparison functions to check your results. For select queries + * {@link org.apache.hive.test.capybara.IntegrationTest#compare} will compare results in the + * order they are returned. This should only be used if you expect your data to be sorted. + * {@link org.apache.hive.test.capybara.IntegrationTest#sortAndCompare} will sort results and + * compare them. {@link org.apache.hive.test.capybara.IntegrationTest#tableCompare} will + * compare entries in two tables. This is useful for insert queries.

+ * + *

Which features are being tested (e.g. which file format is used for tables, which + * execution engine, how Hive is accessed, etc.) are controlled by system properties. You can + * find the system properties to set as well as default values by looking in + * {@link org.apache.hive.test.capybara.infra.TestConf}. To change these values you pass + * properties to JUnit as part of your maven build command. For example, to use Tez as your + * execution engine instead of the default (currently unspecified, which means Hive's local mode + * on your machine and the cluster default on yoru cluster), you would give a command like: + * mvn test -Dtest=ExampleTest -Dhive.test.capybara.engine=tez + *

+ * + *

When running on a cluster you must tell capybara explicitly that it is on a cluster, and + * where to find configuration information for that cluster. The system property to tell it to + * run on the cluster is hive.test.capybara.use.cluster. To tell it where to find + * Hadoop you need to define the property HADOOP_HOME. To find Hive you need to set + * the property HIVE_HOME. If your postgres database is password protected, you can + * pass that via the property hive.test.capybara.postgres.password. So to run + * ExampleTest on a cluster the command is + * mvn test -Dtest=ExampleTest -Dhive.test.capybara.postgres.password=yourpasswd + * -Dhive.test.capybara.use.cluster=true -DHADOOP_HOME=hadoop_location + * -DHIVE_HOME=hive_location + *

+ * + *

Some tests do not make sense in some contexts. For example, currently ACID features are + * only supported when using ORC file format. Therefore tests making use of ACID features + * should not be run when the file format being tested is anything other than ORC. To control + * this you can annotate your tests to indicate when they should not be run. In the ACID case + * you would mark it @NoParquet, @NoTextFile, @NoRcFile. You can see the complete list of + * annotations in {@link org.apache.hive.test.capybara.annotations}.

+ * + */ +package org.apache.hive.test.capybara; \ No newline at end of file diff --git itests/capybara/src/main/java/org/apache/hive/test/capybara/tools/UserQueryGenerator.java itests/capybara/src/main/java/org/apache/hive/test/capybara/tools/UserQueryGenerator.java new file mode 100644 index 0000000..69e44b9 --- /dev/null +++ itests/capybara/src/main/java/org/apache/hive/test/capybara/tools/UserQueryGenerator.java @@ -0,0 +1,715 @@ +/** + * 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.hive.test.capybara.tools; + +import com.google.common.base.Function; +import com.google.common.collect.Lists; +import org.apache.commons.cli.CommandLine; +import org.apache.commons.cli.GnuParser; +import org.apache.commons.cli.HelpFormatter; +import org.apache.commons.cli.OptionBuilder; +import org.apache.commons.cli.Options; +import org.apache.commons.cli.ParseException; +import org.apache.commons.codec.binary.Base64; +import org.apache.commons.lang.StringUtils; +import org.apache.hadoop.hive.metastore.api.NoSuchObjectException; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import org.apache.hadoop.hive.common.StatsSetupConst; +import org.apache.hadoop.hive.conf.HiveConf; +import org.apache.hadoop.hive.metastore.HiveMetaStoreClient; +import org.apache.hadoop.hive.metastore.IMetaStoreClient; +import org.apache.hadoop.hive.metastore.api.BinaryColumnStatsData; +import org.apache.hadoop.hive.metastore.api.BooleanColumnStatsData; +import org.apache.hadoop.hive.metastore.api.ColumnStatisticsData; +import org.apache.hadoop.hive.metastore.api.ColumnStatisticsObj; +import org.apache.hadoop.hive.metastore.api.DateColumnStatsData; +import org.apache.hadoop.hive.metastore.api.Decimal; +import org.apache.hadoop.hive.metastore.api.DecimalColumnStatsData; +import org.apache.hadoop.hive.metastore.api.DoubleColumnStatsData; +import org.apache.hadoop.hive.metastore.api.FieldSchema; +import org.apache.hadoop.hive.metastore.api.LongColumnStatsData; +import org.apache.hadoop.hive.metastore.api.Partition; +import org.apache.hadoop.hive.metastore.api.StringColumnStatsData; +import org.apache.hadoop.hive.metastore.api.Table; +import org.apache.thrift.TException; + +import java.io.BufferedReader; +import java.io.BufferedWriter; +import java.io.FileReader; +import java.io.FileWriter; +import java.io.IOException; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Random; +import java.util.Set; +import java.util.regex.Matcher; +import java.util.regex.Pattern; + +/** + *

A tool to collect information on tables based on user queries and then write code to test + * those queries.

+ * + *

To use this tool:

+ *
    + *
  1. If this is the first time you have used the tool, in your git directory do cd + * itests/capybara; mvn package then copy target/capybara-test-version + * .jar to $HIVE_HOME/lib/ and + * itests/capybara/src/scripts/capygen.sh to $HIVE_HOME/bin/ext.
  2. + *
  3. For each test you wish to run, write all of the SQL for that test into a file, one + * line for each SQL command. Thus if you have 5 tests to run, you will have 5 separate + * files.
  4. + *
  5. Run analyze table yourtable compute statistics for columns; for + * each table referenced in the queries you wish to test.
  6. + *
  7. On a machine that has access to the metastore for your cluster run this class, passing + * in each of your input files and the name of the Java class for your test. The command + * runs as a Hive service. If you had two input files, test.sql and + * anothertest.sql and you wanted to create a class called TestMyApp the command line + * would be $HIVE_HOME/bin/hive --service capygen -i test.sql anothertest.sql -o TestMyApp + *
  8. + *
  9. The resulting class will be in the package org.apache.hive.test.capybara + * .generated. You can put it in the appropriate directory in your source code. At this + * point doing cd itests/capybara; mvn test will run your test(s). If you like you + * can change the package the test is in, though you will then also need to make sure and put + * it in a place with an appropriate pom file and infrastructure to run the tests. + * These tests should not be pushed back to Apache as they will have your queries in them. + *
  10. + *
+ */ +public class UserQueryGenerator { + static final private Logger LOG = LoggerFactory.getLogger(UserQueryGenerator.class); + + private BufferedWriter writer; + private int indent; + + public static void main(String[] args) { + + Options options = new Options(); + + options.addOption(OptionBuilder + .withLongOpt("help") + .withDescription("You're looking at it") + .create('h')); + + options.addOption(OptionBuilder + .withLongOpt("input") + .withDescription("Input files") + .hasArgs() + .isRequired() + .create('i')); + + options.addOption(OptionBuilder + .withLongOpt("output") + .withDescription("Output class") + .hasArg() + .isRequired() + .create('o')); + + try { + CommandLine cli = new GnuParser().parse(options, args); + + if (cli.hasOption('h')) { + HelpFormatter formatter = new HelpFormatter(); + formatter.printHelp("userquerygenerator", options); + } + + UserQueryGenerator gen = new UserQueryGenerator(); + gen.run(cli.getOptionValues('i'), cli.getOptionValue('o')); + } catch (ParseException|TException|IOException e) { + System.err.println("Failed to run, " + e.getMessage()); + LOG.error("Failure", e); + } + } + + private UserQueryGenerator() { + + } + + private void run(String[] inputFiles, String outputClass) throws IOException, TException { + Map> tests = readStatementsFromFile(inputFiles); + Set tables = determineTables(tests); + getTableInfo(tables); + writeCode(tests, tables, outputClass); + } + + private Map> readStatementsFromFile(String[] filenames) throws IOException { + Map> files = new HashMap<>(); + for (String filename : filenames) { + List stmts = new ArrayList<>(); + BufferedReader reader = new BufferedReader(new FileReader(filename)); + String line; + List statement = new ArrayList<>(); + while ((line = reader.readLine()) != null) { + LOG.info("Evaluating line: " + line); + if (statement.size() == 0 && line.toLowerCase().matches(".*;\\s*")) { + // It's a statement on one line + stmts.add(new SQLStatement(Arrays.asList(line))); + LOG.info("Found one line query <" + line + ">"); + } else if (statement.size() == 0 && line.toLowerCase().matches(".*\\S+.*")) { + LOG.info("Starting new statement"); + statement.add(line); + } else if (statement.size() > 0) { + statement.add(line); + LOG.info("Appending " + line + " to existing statement"); + if (line.matches(".*;\\s*")) { + SQLStatement stmt = new SQLStatement(statement); + stmts.add(stmt); + LOG.info("Found multi-line query <" + stmt.toString() + ">"); + statement = new ArrayList<>(); + } + } + } + reader.close(); + if (files.put(cleanFilename(filename), stmts) != null) { + throw new IOException("Two files both map to " + cleanFilename(filename)); + }; + } + return files; + } + + private String cleanFilename(String original) { + if (original.contains(System.getProperty("file.separator"))) { + original = original.substring(original.lastIndexOf(System.getProperty("file.separator")) + 1); + } + return original.replace(".sql", "").replaceAll("[^a-zA-Z0-9_]", "_"); + } + + private Set determineTables(Map> statements) { + Set tables = new HashSet<>(); + + // Look for the table name at the head of the block. + // Each of these should either have a table name or possibly a subquery. Only worry about + // the table names + // TODO - doesn't handle from a, b + // TODO - doesn't handle use db + Pattern fromAndjoin = Pattern.compile("([A-Za-z0-9_\\.]+).*"); + Pattern insert = Pattern.compile("insert\\s+into\\s+([A-Za-z0-9_\\.]+).*"); + Pattern insertTable = Pattern.compile("insert\\s+into\\s+table\\s+([A-Za-z0-9_\\.]+).*"); + Pattern insertOverwrite = Pattern.compile("insert\\s+overwrite\\s+table\\s+([A-Za-z0-9_\\.]+).*"); + Pattern update = Pattern.compile("update\\s+([A-Za-z0-9_\\.]+).*"); + Pattern delete = Pattern.compile("delete\\s+from\\s+([A-Za-z0-9_\\.]+).*"); + for (List stmts : statements.values()) { + for (SQLStatement stmt : stmts) { + if (stmt.toString().startsWith("set")) continue; + findTableName(fromAndjoin, stmt.toString().split("\\s*from\\s*"), stmt, tables, true); + findTableName(fromAndjoin, stmt.toString().split("\\s*join\\s*"), stmt, tables, true); + findTableName(insert, new String[]{"", stmt.toString()}, stmt, tables, false); + findTableName(insertTable, new String[]{"", stmt.toString()}, stmt, tables, false); + findTableName(insertOverwrite, new String[]{"", stmt.toString()}, stmt, tables, false); + findTableName(update, new String[]{"", stmt.toString()}, stmt, tables, false); + findTableName(delete, new String[]{"", stmt.toString()}, stmt, tables, false); + } + } + return tables; + } + + private void findTableName(Pattern pattern, String[] segments, SQLStatement stmt, + Set tables, boolean needsPopulated) { + // Skip the first segment since it will be the part before 'from' or 'join' + for (int i = 1; i < segments.length; i++) { + Matcher matcher = pattern.matcher(segments[i]); + if (matcher.matches()) { + String tableName = matcher.group(1); + LOG.info("Found table " + tableName + ", needsPopulated is " + needsPopulated); + TableInfo ti = new TableInfo(needsPopulated, tableName); + if (!needsPopulated) stmt.targetTable = ti; + tables.add(ti); + } + } + } + + private void getTableInfo(Set tables) throws TException { + HiveConf conf = new HiveConf(); + IMetaStoreClient msClient = new HiveMetaStoreClient(conf); + for (TableInfo table : tables) { + String dbName, tableOnlyName; + if (table.nameFromStatement.contains(".")) { + String[] compoundName = table.nameFromStatement.split("\\."); + dbName = compoundName[0]; + tableOnlyName = compoundName[1]; + } else { + dbName = "default"; + tableOnlyName = table.nameFromStatement; + } + + try { + table.msTable = msClient.getTable(dbName, tableOnlyName); + } catch (NoSuchObjectException e) { + // This might be ok, because the table might be created as part of the test. + LOG.warn("Failed to find table " + dbName + "." + tableOnlyName); + continue; + } + + List colNames = Lists.transform(table.msTable.getSd().getCols(), + new Function() { + @Override + public String apply(FieldSchema input) { + return input.getName(); + } + }); + + if (table.msTable.getPartitionKeys() != null && table.msTable.getPartitionKeys().size() > 0) { + List partNames = msClient.listPartitionNames(dbName, tableOnlyName, (short)-1); + LOG.info("Found partitions: " + StringUtils.join(partNames, ",")); + table.numParts = partNames.size(); + Map> partStats = msClient.getPartitionColumnStatistics(dbName, + tableOnlyName, partNames, colNames); + table.colStats = combineColStats(partStats); + // To get the sizes we need to fetch indivudal partitions and look at the aggregate stats + // . I don't want to fetch all of them at once for fear of blowing out the metastore + // memory. So we fetch them a thousand at a time. + List> partitionGroups = new ArrayList<>(); + int numPartsInGroup = 0; + List currentGroup = null; + for (String partName : partNames) { + if (numPartsInGroup % 1000 == 0) { + if (currentGroup != null) { + partitionGroups.add(currentGroup); + } + currentGroup = new ArrayList<>(1000); + } + currentGroup.add(partName); + numPartsInGroup++; + } + // Add the last group + partitionGroups.add(currentGroup); + + for (List oneGroup : partitionGroups) { + LOG.info("Fetching stats for partitions: " + StringUtils.join(oneGroup, ",")); + List parts = msClient.getPartitionsByNames(dbName, tableOnlyName, oneGroup); + LOG.info("Got " + parts.size() + " partitions"); + for (Partition part : parts) { + table.dataSize += Long.valueOf(part.getParameters().get(StatsSetupConst.RAW_DATA_SIZE)); + table.rowCount += Long.valueOf(part.getParameters().get(StatsSetupConst.ROW_COUNT)); + } + } + } else { + table.numParts = 0; + table.colStats = msClient.getTableColumnStatistics(dbName, tableOnlyName, colNames); + table.dataSize = Long.valueOf(table.msTable.getParameters().get(StatsSetupConst.RAW_DATA_SIZE)); + table.rowCount = Long.valueOf(table.msTable.getParameters().get(StatsSetupConst.ROW_COUNT)); + } + } + } + + private List combineColStats(Map> partStats) { + Map colToStatsMap = new HashMap<>(); + + for (List statsObjs : partStats.values()) { + for (ColumnStatisticsObj latest : statsObjs) { + ColumnStatisticsObj existing = colToStatsMap.get(latest.getColName()); + colToStatsMap.put(latest.getColName(), combineTwoStats(existing, latest)); + } + } + return new ArrayList<>(colToStatsMap.values()); + } + + private ColumnStatisticsObj combineTwoStats(ColumnStatisticsObj existing, + ColumnStatisticsObj latest) { + if (existing == null) return latest; + + ColumnStatisticsData newStatsData = new ColumnStatisticsData(); + if (existing.getStatsData().isSetLongStats()) { + LongColumnStatsData existingStats = existing.getStatsData().getLongStats(); + LongColumnStatsData latestStats = latest.getStatsData().getLongStats(); + LongColumnStatsData newStats = + new LongColumnStatsData(existingStats.getNumNulls() + latestStats.getNumNulls(), + (long)(Math.max(existingStats.getNumDVs(), latestStats.getNumDVs()) * 1.2)); + newStats.setHighValue(Math.max(existingStats.getHighValue(), latestStats.getHighValue())); + newStats.setLowValue(Math.max(existingStats.getLowValue(), latestStats.getLowValue())); + newStatsData.setLongStats(newStats); + } else if (existing.getStatsData().isSetDoubleStats()) { + DoubleColumnStatsData existingStats = existing.getStatsData().getDoubleStats(); + DoubleColumnStatsData latestStats = latest.getStatsData().getDoubleStats(); + DoubleColumnStatsData newStats = + new DoubleColumnStatsData(existingStats.getNumNulls() + latestStats.getNumNulls(), + (long)(Math.max(existingStats.getNumDVs(), latestStats.getNumDVs()) * 1.2)); + newStats.setHighValue(Math.max(existingStats.getHighValue(), latestStats.getHighValue())); + newStats.setLowValue(Math.max(existingStats.getLowValue(), latestStats.getLowValue())); + newStatsData.setDoubleStats(newStats); + } else if (existing.getStatsData().isSetDecimalStats()) { + DecimalColumnStatsData existingStats = existing.getStatsData().getDecimalStats(); + DecimalColumnStatsData latestStats = latest.getStatsData().getDecimalStats(); + DecimalColumnStatsData newStats = + new DecimalColumnStatsData(existingStats.getNumNulls() + latestStats.getNumNulls(), + (long) (Math.max(existingStats.getNumDVs(), latestStats.getNumDVs()) * 1.2)); + newStats.setHighValue(existingStats.getHighValue().compareTo(latestStats.getHighValue()) > 1 ? + existingStats.getHighValue() : latestStats.getHighValue()); + newStats.setLowValue(existingStats.getLowValue().compareTo(latestStats.getLowValue()) < 1 ? + existingStats.getLowValue() : latestStats.getLowValue()); + newStatsData.setDecimalStats(newStats); + } else if (existing.getStatsData().isSetDateStats()) { + DateColumnStatsData existingStats = existing.getStatsData().getDateStats(); + DateColumnStatsData latestStats = latest.getStatsData().getDateStats(); + DateColumnStatsData newStats = + new DateColumnStatsData(existingStats.getNumNulls() + latestStats.getNumNulls(), + (long) (Math.max(existingStats.getNumDVs(), latestStats.getNumDVs()) * 1.2)); + newStats.setHighValue(existingStats.getHighValue().compareTo(latestStats.getHighValue()) > 1 ? + existingStats.getHighValue() : latestStats.getHighValue()); + newStats.setLowValue(existingStats.getLowValue().compareTo(latestStats.getLowValue()) < 1 ? + existingStats.getLowValue() : latestStats.getLowValue()); + newStatsData.setDateStats(newStats); + } else if (existing.getStatsData().isSetBooleanStats()) { + BooleanColumnStatsData existingStats = existing.getStatsData().getBooleanStats(); + BooleanColumnStatsData latestStats = latest.getStatsData().getBooleanStats(); + BooleanColumnStatsData newStats = + new BooleanColumnStatsData( existingStats.getNumTrues() + latestStats.getNumTrues(), + existingStats.getNumFalses() + latestStats.getNumFalses(), + existingStats.getNumNulls() + latestStats.getNumNulls()); + newStatsData.setBooleanStats(newStats); + } else if (existing.getStatsData().isSetStringStats()) { + StringColumnStatsData existingStats = existing.getStatsData().getStringStats(); + StringColumnStatsData latestStats = latest.getStatsData().getStringStats(); + StringColumnStatsData newStats = + new StringColumnStatsData( + Math.max(existingStats.getMaxColLen(), latestStats.getMaxColLen()), + // TODO improve this + existingStats.getAvgColLen() + latestStats.getAvgColLen() / 2.0, + existingStats.getNumNulls() + latestStats.getNumNulls(), + (long) (Math.max(existingStats.getNumDVs(), latestStats.getNumDVs()) * 1.2)); + newStatsData.setStringStats(newStats); + } else if (existing.getStatsData().isSetBinaryStats()) { + BinaryColumnStatsData existingStats = existing.getStatsData().getBinaryStats(); + BinaryColumnStatsData latestStats = latest.getStatsData().getBinaryStats(); + BinaryColumnStatsData newStats = + new BinaryColumnStatsData( + Math.max(existingStats.getMaxColLen(), latestStats.getMaxColLen()), + // TODO improve this + existingStats.getAvgColLen() + latestStats.getAvgColLen() / 2.0, + existingStats.getNumNulls() + latestStats.getNumNulls()); + newStatsData.setBinaryStats(newStats); + } + + return new ColumnStatisticsObj(latest.getColName(), latest.getColType(), newStatsData); + } + + private void writeCode(Map> statements, Set tables, + String outputClass) + throws IOException { + writer = new BufferedWriter(new FileWriter(outputClass + ".java")); + indent = 0; + for (String line : prologue) writeALine(line); + writeALine("public class " + outputClass + " extends IntegrationTest {"); + indent++; + + // Create set of tables so we can fetch them in the tests if we need them. + writeALine("private Map targetTables = new HashMap<>();"); + + // Write the methods that build the tables, marking each with @Before + for (TableInfo info : tables) writeBefore(info); + + // Write the tests, naming them with the filenames + for (Map.Entry> entry : statements.entrySet()) { + writeTest(entry.getKey(), entry.getValue()); + } + + indent--; + writeALine("}"); + writer.close(); + } + + private void writeBefore(TableInfo info) throws IOException { + if (info.msTable == null) { + // If we don't have info from the metastore on the table we can't build it. We will assume + // it will be created somewhere as part of the test. + LOG.info("No metastore info for table " + info.nameFromStatement + + ", not creating @Before method for it."); + return; + } + writeALine("@Before"); + writeALine("public void createTable" + safeTableName(info) + "() throws Exception {"); + indent++; + writeALine("TestTable tTable = TestTable.getBuilder(\"" + info.msTable.getTableName() + "\")"); + indent++; indent++; + writeALine(".setDbName(\"" + info.msTable.getDbName() + "\")"); + for (FieldSchema fs : info.msTable.getSd().getCols()) { + writeALine(".addCol(\"" + fs.getName() + "\", \"" + fs.getType() + "\")"); + } + + if (info.msTable.getPartitionKeys() != null && info.msTable.getPartitionKeysSize() > 0) { + for (FieldSchema partCol : info.msTable.getPartitionKeys()) { + writeALine(".addPartCol(\"" + partCol.getName() + "\", \"" + partCol.getType() + "\")"); + + } + writeALine(".setNumParts(" + info.numParts + ")"); + } + writeALine(".build();"); + indent--; indent--; + + writeALine("Map colStats = new HashMap<>();"); + for (ColumnStatisticsObj cso : info.colStats) { + writeALine("colStats.put(\"" + cso.getColName() + "\","); + indent++; + writeALine("new StatsDataGenerator.ColStats(\"" + cso.getColName() + "\", \"" + + cso.getColType() + "\", "); + indent++; + if (cso.getColType().equalsIgnoreCase("bigint")) { + LongColumnStatsData lcsd = cso.getStatsData().getLongStats(); + writeALine(lcsd.getLowValue() + "L, " + lcsd.getHighValue() + "L, 0, 0, 0, 0, " + + lcsd.getNumDVs() + ", " + lcsd.getNumNulls() + "));"); + + } else if (cso.getColType().toLowerCase().startsWith("int")) { + LongColumnStatsData lcsd = cso.getStatsData().getLongStats(); + writeALine(lcsd.getLowValue() + ", " + lcsd.getHighValue() + ", 0, 0, 0, 0, " + + lcsd.getNumDVs() + ", " + lcsd.getNumNulls() + "));"); + + } else if (cso.getColType().equalsIgnoreCase("smallint")) { + LongColumnStatsData lcsd = cso.getStatsData().getLongStats(); + writeALine("(short)" + lcsd.getLowValue() + ", (short)" + lcsd.getHighValue() + + ", 0, 0, 0, 0, " + lcsd.getNumDVs() + ", " + lcsd.getNumNulls() + "));"); + + } else if (cso.getColType().equalsIgnoreCase("tinyint")) { + LongColumnStatsData lcsd = cso.getStatsData().getLongStats(); + writeALine("(byte)" + lcsd.getLowValue() + ", (byte)" + lcsd.getHighValue() + + ", 0, 0, 0, 0, " + lcsd.getNumDVs() + ", " + lcsd.getNumNulls() + "));"); + } else if (cso.getColType().equalsIgnoreCase("float")) { + DoubleColumnStatsData dcsd = cso.getStatsData().getDoubleStats(); + writeALine(dcsd.getLowValue() + "f, " + dcsd.getHighValue() + "f, 0, 0, 0, 0, " + + dcsd.getNumDVs() + ", " + dcsd.getNumNulls() + "));"); + + } else if (cso.getColType().equalsIgnoreCase("double")) { + DoubleColumnStatsData dcsd = cso.getStatsData().getDoubleStats(); + writeALine(dcsd.getLowValue() + ", " + dcsd.getHighValue() + ", 0, 0, 0, 0, " + + dcsd.getNumDVs() + ", " + dcsd.getNumNulls() + "));"); + } else if (cso.getColType().toLowerCase().startsWith("decimal")) { + DecimalColumnStatsData dcsd = cso.getStatsData().getDecimalStats(); + Decimal low = dcsd.getLowValue(); + Decimal high = dcsd.getHighValue(); + writeALine("new BigDecimal(new BigInteger(Base64.decodeBase64(\"" + + Base64.encodeBase64URLSafeString(low.getUnscaled()) + "\")), " + low.getScale() + + "), new BigDecimal(new BigInteger(Base64.decodeBase64(\"" + + Base64.encodeBase64URLSafeString(high.getUnscaled()) + "\")), " + high.getScale() + + "), 0, 0, 0, 0, " + dcsd.getNumDVs() + ", " + dcsd.getNumNulls() + "));"); + } else if (cso.getColType().equalsIgnoreCase("date")) { + DateColumnStatsData dcsd = cso.getStatsData().getDateStats(); + writeALine("new Date(" + (dcsd.getLowValue().getDaysSinceEpoch() * 86400) + "), " + + "new Date(" + (dcsd.getHighValue().getDaysSinceEpoch() * 86400) + "), 0, 0, 0, 0, " + + dcsd.getNumDVs() + ", " + dcsd.getNumNulls() + "));"); + } else if (cso.getColType().equalsIgnoreCase("timestamp")) { + LongColumnStatsData lcsd = cso.getStatsData().getLongStats(); + writeALine("new Timestamp(" + (lcsd.getLowValue()) + "), " + + "new Timestamp(" + (lcsd.getHighValue()) + "), 0, 0, 0, 0, " + lcsd.getNumDVs() + ", " + + lcsd.getNumNulls() + "));"); + } else if (cso.getColType().equalsIgnoreCase("string") || + cso.getColType().toLowerCase().startsWith("varchar") || + cso.getColType().toLowerCase().startsWith("char")) { + StringColumnStatsData scsd = cso.getStatsData().getStringStats(); + writeALine( + "null, null, " + scsd.getAvgColLen() + ", " + scsd.getMaxColLen() + ", 0, 0, " + + scsd.getNumDVs() + ", " + scsd.getNumNulls() + "));"); + } else if (cso.getColType().equalsIgnoreCase("boolean")) { + BooleanColumnStatsData bcsd = cso.getStatsData().getBooleanStats(); + writeALine("0, 0, 0, 0, " + bcsd.getNumFalses() + ", " + bcsd.getNumTrues() + ", 0, " + + bcsd.getNumNulls() + "));"); + } else if (cso.getColType().equalsIgnoreCase("binary")) { + BinaryColumnStatsData bcsd = cso.getStatsData().getBinaryStats(); + writeALine( + "null, null, " + bcsd.getAvgColLen() + ", " + bcsd.getMaxColLen() + ", 0, 0, 0," + + bcsd.getNumNulls() + "));"); + } else { + throw new RuntimeException("Unknown column type " + cso.getColType()); + } + indent--; + indent--; + } + writeALine("StatsDataGenerator.TableStats tableStats = new StatsDataGenerator.TableStats" + + "(colStats, " + info.numParts + ", \"" + info.msTable.getDbName() + "\", \"" + + info.msTable.getTableName() + "\", " + info.dataSize + ", " + info.rowCount + ");"); + writeALine("StatsDataGenerator gen = new StatsDataGenerator(tableStats, " + + new Random().nextInt() + ");"); + if (info.needsPopulated) { + writeALine("tTable.create();"); + writeALine("tTable.populate(gen);"); + } else { + writeALine("tTable.createTargetTable();"); + writeALine("targetTables.put(\"" + info.nameFromStatement + "\", tTable);"); + } + indent--; + writeALine("}"); + writeALine(""); + } + + private String safeTableName(TableInfo info) { + return info.msTable == null ? info.nameFromStatement : + info.msTable.getDbName() + "_" + info.msTable.getTableName(); + } + + private void writeTest(String name, List statements) throws IOException { + writeALine("@Test"); + writeALine("public void " + name + "() throws Exception {"); + indent++; + SQLStatement lastStmt = null; + for (SQLStatement stmt : statements) { + if (stmt.toString().startsWith("set")) { + String[] setParts = stmt.toString().substring(3).split("="); + writeALine("set(\"" + setParts[0].trim() + "\", \"" + setParts[1].trim() + "\");"); + } else { + writeALine("runQuery("); + stmt.writeOut(); + writeALine(");"); + } + lastStmt = stmt; + } + // Only compare based on the last statement. + if (lastStmt.toString().startsWith("select")) { + if (lastStmt.toString().contains("order by")) { + writeALine("compare();"); + } else { + writeALine("sortAndCompare();"); + } + } else if (lastStmt.toString().startsWith("insert") || lastStmt.toString().startsWith("update") || + lastStmt.toString().startsWith("delete")) { + TableInfo targetTable = lastStmt.targetTable; + if (targetTable.msTable == null) { + // We didn't build the table before because we didn't have metastore info. But by now + // the table must exist, so we need to go build it. + writeALine("String dbName, tableOnlyName;"); + if (targetTable.nameFromStatement.contains(".")) { + String[] compoundName = targetTable.nameFromStatement.split("\\."); + writeALine("dbName = \"" + compoundName[0] + "\";"); + writeALine("tableOnlyName = \"" + compoundName[1] + "\";"); + } else { + writeALine("dbName = \"default\";"); + writeALine("tableOnlyName = \"" + targetTable.nameFromStatement + "\";"); + } + + writeALine("TestTable tTable = TestTable.fromHiveMetastore(dbName, tableOnlyName);"); + } else { + writeALine("TestTable tTable = targetTables.get(\"" + targetTable.nameFromStatement + "\");"); + } + writeALine("tableCompare(tTable);"); + } else { + throw new IOException("Unclear how to do comparison for statement " + lastStmt.toString()); + } + indent--; + writeALine("}"); + writeALine(""); + } + + private void writeALine(String line) throws IOException { + for (int i = 0; i < indent; i++) writer.write(" "); + writer.write(line); + writer.newLine(); + } + + private static class TableInfo { + Table msTable; + int numParts; + long dataSize; + long rowCount; + List colStats; + final boolean needsPopulated; + final String nameFromStatement; + + TableInfo(boolean needsPopulated, String nameFromStatement) { + this.needsPopulated = needsPopulated; + this.nameFromStatement = nameFromStatement; + } + + @Override + public boolean equals(Object other) { + if (!(other instanceof TableInfo)) return false; + return nameFromStatement.equals(((TableInfo)other).nameFromStatement); + } + + @Override + public int hashCode() { + return nameFromStatement.hashCode(); + } + } + + private class SQLStatement { + final String[] stmt; + TableInfo targetTable; + private String asOneLine = null; + + SQLStatement(List lines) { + // Trim anything after a comment indicator + stmt = new String[lines.size()]; + for (int i = 0; i < lines.size(); i++) { + if (lines.get(i).contains("--")) { + if (lines.get(i).indexOf("--") != 0) { + stmt[i] = lines.get(i).substring(0, lines.get(i).indexOf("--")).trim(); + } else { + stmt[i] = ""; + } + } else { + stmt[i] = lines.get(i).trim(); + } + + if (stmt[i].length() == 0) continue; + + stmt[i] = stmt[i].toLowerCase(); + + // Trim any ending ;s off + if (stmt[i].charAt(stmt[i].length() - 1) == ';') { + stmt[i] = stmt[i].substring(0, stmt[i].length() - 1); + } + // Escape any quotes + stmt[i] = stmt[i].replaceAll("\"", "\\\""); + } + } + + void writeOut() throws IOException { + indent += 2; + boolean first = true; + for (String line : stmt) { + String plus = ""; + if (first) first = false; + else plus = "+ "; + writeALine(plus + "\"" + line + "\""); + } + indent -= 2; + } + + @Override + public String toString() { + if (asOneLine == null) asOneLine = StringUtils.join(stmt, " "); + return asOneLine; + } + } + + private static final String[] prologue = { + "package org.apache.hive.test.capybara.generated;", + " ", + "import org.apache.commons.codec.binary.Base64;", + "import org.apache.hadoop.hive.conf.HiveConf;", + "import org.apache.hive.test.capybara.IntegrationTest;", + "import org.apache.hive.test.capybara.infra.StatsDataGenerator;", + "import org.apache.hive.test.capybara.iface.TestTable;", + "import org.junit.Assert;", + "import org.junit.Before;", + "import org.junit.Test;", + "import java.math.BigDecimal;", + "import java.math.BigInteger;", + "import java.sql.Date;", + "import java.sql.Timestamp;", + "import java.util.ArrayList;", + "import java.util.HashMap;", + "import java.util.List;", + "import java.util.Map;", + " " + }; + + // Problems: + // q2, q6 - grabbing the wrong table name +} diff --git itests/capybara/src/scripts/capygen.sh itests/capybara/src/scripts/capygen.sh new file mode 100644 index 0000000..71aed4c --- /dev/null +++ itests/capybara/src/scripts/capygen.sh @@ -0,0 +1,31 @@ +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +THISSERVICE=capygen +export SERVICE_LIST="${SERVICE_LIST}${THISSERVICE} " + +capygen () { + CLASS=org.apache.hive.test.capybara.tools.UserQueryGenerator + HIVE_OPTS='' + execHiveCmd $CLASS "$@" +} + +capygen_help () { + echo "usage ./hive capygen [-h] -i input_file ... -o output_file" + echo "" + echo " --input (-i) Input files to read queries from. Eache file becomes a separate JUnit test." + echo " --output (-o) Output file to write tests to. All tests will be placed in one class" + echo " --help (-h) Print help message" +} diff --git itests/capybara/src/test/java/org/apache/hive/test/capybara/data/TestDataSet.java itests/capybara/src/test/java/org/apache/hive/test/capybara/data/TestDataSet.java new file mode 100644 index 0000000..7b01b7b --- /dev/null +++ itests/capybara/src/test/java/org/apache/hive/test/capybara/data/TestDataSet.java @@ -0,0 +1,94 @@ +/** + * 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.hive.test.capybara.data; + +import org.junit.Assert; +import org.junit.Test; + +public class TestDataSet { + + @Test + public void fuzzyFloat() { + FloatColumn one = new FloatColumn(0); + one.set(new Float(1.0f)); + Assert.assertEquals(one, one); + + FloatColumn two = new FloatColumn(0); + two.set(new Float(2.0f)); + Assert.assertNotEquals(one, two); + + FloatColumn pointOne = new FloatColumn(0); + pointOne.set(new Float(0.1f)); + Assert.assertNotEquals(one, pointOne); + + FloatColumn quintillion = new FloatColumn(0); + quintillion.set(new Float(1000000000000000000.0f)); + Assert.assertEquals(quintillion, quintillion); + + FloatColumn quintillionOne = new FloatColumn(0); + quintillionOne.set(new Float(1000000000000000001.0f)); + Assert.assertEquals(quintillion, quintillionOne); + + FloatColumn fiveQuintillion = new FloatColumn(0); + fiveQuintillion.set(new Float(5000000000000000000.0f)); + Assert.assertNotEquals(quintillion, fiveQuintillion); + + FloatColumn verySmall = new FloatColumn(0); + verySmall.set(new Float(0.0000000000000001f)); + Assert.assertEquals(verySmall, verySmall); + + FloatColumn justOverOne = new FloatColumn(0); + justOverOne.set(new Float(1.0000000000000001f)); + Assert.assertEquals(one, justOverOne); + } + + @Test + public void fuzzyDouble() { + DoubleColumn one = new DoubleColumn(0); + one.set(new Double(1.0)); + Assert.assertEquals(one, one); + + DoubleColumn two = new DoubleColumn(0); + two.set(new Double(2.0)); + Assert.assertNotEquals(one, two); + + DoubleColumn pointOne = new DoubleColumn(0); + pointOne.set(new Double(0.1)); + Assert.assertNotEquals(one, pointOne); + + DoubleColumn quintillion = new DoubleColumn(0); + quintillion.set(new Double(1000000000000000000.0)); + Assert.assertEquals(quintillion, quintillion); + + DoubleColumn quintillionOne = new DoubleColumn(0); + quintillionOne.set(new Double(1000000000000000001.0)); + Assert.assertEquals(quintillion, quintillionOne); + + DoubleColumn fiveQuintillion = new DoubleColumn(0); + fiveQuintillion.set(new Double(5000000000000000000.0)); + Assert.assertNotEquals(quintillion, fiveQuintillion); + + DoubleColumn verySmall = new DoubleColumn(0); + verySmall.set(new Double(0.0000000000000001)); + Assert.assertEquals(verySmall, verySmall); + + DoubleColumn justOverOne = new DoubleColumn(0); + justOverOne.set(new Double(1.0000000000000001)); + Assert.assertEquals(one, justOverOne); + } +} diff --git itests/capybara/src/test/java/org/apache/hive/test/capybara/examples/ExampleStreamingTest.java itests/capybara/src/test/java/org/apache/hive/test/capybara/examples/ExampleStreamingTest.java new file mode 100644 index 0000000..6f2d048 --- /dev/null +++ itests/capybara/src/test/java/org/apache/hive/test/capybara/examples/ExampleStreamingTest.java @@ -0,0 +1,96 @@ +/** + * 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.hive.test.capybara.examples; + +import org.apache.hive.hcatalog.streaming.DelimitedInputWriter; +import org.apache.hive.hcatalog.streaming.HiveEndPoint; +import org.apache.hive.hcatalog.streaming.StreamingConnection; +import org.apache.hive.hcatalog.streaming.TransactionBatch; +import org.apache.hive.test.capybara.IntegrationTest; +import org.apache.hive.test.capybara.annotations.AcidOn; +import org.apache.hive.test.capybara.iface.TestTable; +import org.junit.Test; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class ExampleStreamingTest extends IntegrationTest { + static final private Logger LOG = LoggerFactory.getLogger(ExampleStreamingTest.class.getName()); + + private boolean running; + + @Test + @AcidOn + public void stream() throws Exception { + // There's no way to validate results while streaming (since there's no guarantee that each + // database is at the same point in committing the results. But we can stream data in while + // running queries (just to make sure the queries work) and occasionally pause and run + // queries to verify we are getting the same results. + TestTable target = TestTable.getBuilder("streamtarget") + .addCol("a", "varchar(10)") + .addCol("b", "int") + .setBucketCols("a") + .setNumBuckets(2) + .setAcid(true) + .build(); + String[] colNames = {"a", "b"}; + target.createTargetTable(); + String[] rows = new String[] {"abc,1", "def,2", "ghi,3", "jkl,4", "mno,5", "pqr,6", "stu,7", + "vwx,8", "yz,9", "alpha,10"}; + + HiveEndPoint endPoint = getHiveEndPoint(target, null); + StreamingConnection conn = endPoint.newConnection(true, getConf()); + running = true; + Runnable queryRunner = new Runnable() { + @Override + public void run() { + while (running) { + try { + // Just run it on Hive, we're not interested in testing that the benchmark can read + // while writing. + runHive("select count(*) from streamtarget"); + Thread.sleep(1000); + } catch (Exception e) { + throw new RuntimeException(e); + } + } + } + }; + + Thread queryThread = new Thread(queryRunner); + queryThread.start(); + + for (int i = 0; i < 5; i++) { + TransactionBatch txnBatch = conn.fetchTransactionBatch(5, + new DelimitedInputWriter(colNames, ",", endPoint, getConf(), ',')); + while (txnBatch.remainingTransactions() > 0) { + txnBatch.beginNextTransaction(); + for (String row : rows) txnBatch.write(row.getBytes()); + txnBatch.commit(); + } + txnBatch.close(); + runQuery("select count(*) from streamtarget"); + sortAndCompare(); + } + running = false; + conn.close(); + queryThread.join(); + runQuery("select count(*) from streamtarget"); + compare(); + //tableCompare(target); + } +} diff --git itests/capybara/src/test/java/org/apache/hive/test/capybara/examples/ExampleTest.java itests/capybara/src/test/java/org/apache/hive/test/capybara/examples/ExampleTest.java new file mode 100644 index 0000000..4b5e7a9 --- /dev/null +++ itests/capybara/src/test/java/org/apache/hive/test/capybara/examples/ExampleTest.java @@ -0,0 +1,119 @@ +/** + * 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.hive.test.capybara.examples; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import org.apache.hive.test.capybara.IntegrationTest; +import org.apache.hive.test.capybara.TableTool; +import org.apache.hive.test.capybara.annotations.AcidOn; +import org.apache.hive.test.capybara.annotations.NoParquet; +import org.apache.hive.test.capybara.annotations.NoRcFile; +import org.apache.hive.test.capybara.annotations.NoSpark; +import org.apache.hive.test.capybara.annotations.NoTextFile; +import org.junit.Test; + +@NoSpark // These tests don't make sense when Spark is the engine +public class ExampleTest extends IntegrationTest { + static final private Logger LOG = LoggerFactory.getLogger(ExampleTest.class); + + @Test + public void simple() throws Exception { + TableTool.createAllTypes(); + + runQuery("select cvarchar, cchar, cstring, cint, cbigint, csmallint, ctinyint, " + + "cfloat, cdecimal, cdate, ctimestamp, cboolean from alltypes"); + sortAndCompare(); + } + + @Test + public void countStar() throws Exception { + TableTool.createAllTypes(); + + runQuery("select count(*) from alltypes"); + sortAndCompare(); + } + + @Test + public void groupBy() throws Exception { + TableTool.createCapySrcPart(); + + runQuery("select k, count(*) as cnt from capysrcpart where value is not null group by k order" + + " by cnt, k"); + compare(); + } + + @Test + public void simpleJoin() throws Exception { + TableTool.createPseudoTpch(); + + runQuery("select p_name, avg(l_price) from ph_lineitem join ph_part on (l_partkey = " + + "p_partkey) group by p_name order by p_name"); + compare(); + } + + // TODO + // * Make it work with HS2 + // * Make it work with security + // * Make it work with HBase metastore + // * Make work for multi-user + // * Should scale move to M on the cluster instead of K? + // * Add qfile translator. + // * Add default scale (-1) to DataGenerator.generateData so people can set pctNull without + // messing with the scale. + // * Rename DataStore.fetchData to something that reflects what it actually does (like + // executeStmt). Rename FetchResult to StmtResult or something. + // * Make is so the user can change the package for tests created by UserQueryGenerator + // * Split up the infra package into interface and impl, it's getting too big and confusing. + // Move DataGenerator from top to interface. + + // TODO - needs tested + // * Test ability to generate data in parallel (on cluster) for large data + // * Test ability to compare data in tables, for ETL type queries + + // FIXME + // * Make decimal work with default precision and scale + // * Make binary work with Derby + // * We don't properly drop old records in the testtables when we discover a wrong version of + // the table. + // * We don't do anything to assure that joined user tables generate records that will join. + // This is somewhat hard in that Hive statistics don't help us, but we may want to at least + // detect the join conditions and do something to make sure we don't get null results. In + // particular we could infer pk/fk relationships in star schemas. + + @AcidOn // Turn acid on for this test (ie set the appropriate config values) + @NoParquet @NoRcFile @NoTextFile + @Test + public void updateAllNonPartitioned() throws Exception { + TableTool.createAllTypes(); + + // Run a query. Complain if it fails. + runQuery("drop table if exists acid_uanp"); + runQuery("create table acid_uanp(a int, b varchar(128)) clustered by (a) into 2 buckets " + + "stored as orc TBLPROPERTIES ('transactional'='true')"); + runQuery("insert into acid_uanp select cint, cast(cstring as varchar(128)) " + + "from alltypes where cint < 0"); + runQuery("select a,b from acid_uanp order by a"); + compare(); // compare the results of the previous query against the source of truth. + runQuery("update acid_uanp set b = 'fred'"); + runQuery("select a,b from acid_uanp"); + sortAndCompare(); + } + + +} diff --git itests/capybara/src/test/java/org/apache/hive/test/capybara/examples/SecondExampleTest.java itests/capybara/src/test/java/org/apache/hive/test/capybara/examples/SecondExampleTest.java new file mode 100644 index 0000000..11e0049 --- /dev/null +++ itests/capybara/src/test/java/org/apache/hive/test/capybara/examples/SecondExampleTest.java @@ -0,0 +1,49 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hive.test.capybara.examples; + +import org.apache.hive.test.capybara.IntegrationTest; +import org.apache.hive.test.capybara.TableTool; +import org.apache.hive.test.capybara.iface.TestTable; +import org.junit.Test; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * More example tests. + */ +public class SecondExampleTest extends IntegrationTest { + static final private Logger LOG = LoggerFactory.getLogger(SecondExampleTest.class); + + @Test + public void insert() throws Exception { + TableTool.createAllTypes(); + + TestTable target = TestTable.getBuilder("INSERT_EXAMPLE") + .addCol("cstring", "varchar(120)") + .addCol("cbool", "boolean") + .addCol("clong", "bigint") + .addCol("cfloat", "float") + .addCol("cint", "int") + .build(); + target.createTargetTable(); + runQuery("insert into INSERT_EXAMPLE select cvarchar, cboolean, cbigint, cfloat, cint " + + "from alltypes"); + tableCompare(target); + } +} diff --git itests/capybara/src/test/java/org/apache/hive/test/capybara/examples/TestExplain.java itests/capybara/src/test/java/org/apache/hive/test/capybara/examples/TestExplain.java new file mode 100644 index 0000000..a145858 --- /dev/null +++ itests/capybara/src/test/java/org/apache/hive/test/capybara/examples/TestExplain.java @@ -0,0 +1,75 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + *

+ * http://www.apache.org/licenses/LICENSE-2.0 + *

+ * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hive.test.capybara.examples; + +import org.apache.hadoop.hive.ql.exec.TableScanOperator; +import org.apache.hadoop.hive.ql.exec.mr.MapRedTask; +import org.apache.hive.test.capybara.Explain; +import org.apache.hive.test.capybara.IntegrationTest; +import org.apache.hive.test.capybara.TableTool; +import org.junit.Assert; +import org.junit.Test; + +import java.util.List; + +public class TestExplain extends IntegrationTest { + + @Test + public void limitPushdown() throws Exception { + // Set configuration values for this test + set("hive.explain.user", false); + set("hive.limit.pushdown.memory.usage", 0.3f); + set("hive.optimize.reducededuplication.min.reducer", 1); + + // TableTool provides a set of standard table for testing. This particular one gets the + // src table, which has two string fields, k and value. + TableTool.createCapySrc(); + + runQuery("select k,value from capysrc order by k limit 20"); + + // Explain fetches an Explain object for the query, which contains the QueryPlan. It also + // includes tools for validating the tree. + Explain explain = explain(); + + // Expect that somewhere in the plan is a TezTask. Fetch that task. This will assert if it + // can't find the task. + MapRedTask mrTask = explain.expect(MapRedTask.class); + // Expect that somewhere in the TezTask there's a limit operator. Fetch that operator. This + // will assert if it can't find the operator. + TableScanOperator scan = explain.expect(mrTask, TableScanOperator.class); + Assert.assertNotNull(scan); + compare(); + } + + @Test + public void noRunQuery() throws Exception { + // TableTool provides a set of standard table for testing. This particular one gets the + // src table, which has two string fields, k and value. + TableTool.createCapySrc(); + + Explain explain = explain("select k,value from capysrc order by k"); + + // Expect that somewhere in the plan is a TezTask. Fetch that task. This will assert if it + // can't find the task. + MapRedTask mrTask = explain.expect(MapRedTask.class); + // Expect that somewhere in the MapRedTask there's a limit operator. Fetch that operator. This + // will assert if it can't find the operator. + List scans = explain.findAll(mrTask, TableScanOperator.class); + Assert.assertEquals(1, scans.size()); + } +} diff --git itests/capybara/src/test/java/org/apache/hive/test/capybara/infra/TestAnsiSqlStore.java itests/capybara/src/test/java/org/apache/hive/test/capybara/infra/TestAnsiSqlStore.java new file mode 100644 index 0000000..4395f60 --- /dev/null +++ itests/capybara/src/test/java/org/apache/hive/test/capybara/infra/TestAnsiSqlStore.java @@ -0,0 +1,139 @@ +/** + * 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.hive.test.capybara.infra; + +import org.apache.hive.test.capybara.data.DataSet; +import org.apache.hive.test.capybara.data.ResultCode; +import org.apache.hive.test.capybara.iface.TestTable; +import org.junit.Assert; +import org.junit.Test; + +import java.io.IOException; +import java.sql.SQLException; +import java.util.Properties; + +public class TestAnsiSqlStore { + + private AnsiSqlStore store = new AnsiSqlStore() { + @Override + protected String ifExists() { + return null; + } + + @Override + public String getTableName(TestTable table) { + return null; + } + + @Override + protected String markColumnPrimaryKey() { + return null; + } + + @Override + public void loadData(TestTable table, DataSet rows) throws SQLException, IOException { + + } + + @Override + protected String fileColumnDelimiter() { + return null; + } + + @Override + protected String fileNull() { + return null; + } + + @Override + protected String fileStringQuotes() { + return null; + } + + @Override + protected Properties connectionProperties() { + return null; + } + + @Override + protected String connectionURL() { + return null; + } + + @Override + public Class getDriverClass() { + return null; + } + + @Override + protected SQLTranslator getTranslator() { + return new SQLTranslator() { + @Override + protected String translateDataTypes(String hiveSql) { + return hiveSql; + } + + @Override + protected String translateAlterTableRename(String tableName, String remainder) throws + TranslationException { + return null; + } + + @Override + protected char identifierQuote() { + return '"'; + } + }; + } + + @Override + public void forceCreateTable(TestTable table) throws SQLException, IOException { + + } + }; + + @Test + public void createTable() throws Exception { + String hiveSql = "create table if not exists acid_uanp(a int, b varchar(128)) partitioned by " + + "(c string) clustered by (a) into 2 buckets stored as orc TBLPROPERTIES ('transactional'='true')"; + Assert.assertEquals("create table if not exists acid_uanp (a int, b varchar(128))", + store.hiveSqlToAnsiSql(hiveSql)); + } + + @Test + public void insert() throws Exception { + String hiveSql = "insert into table acid_uanp partition (c = 'fred') values (1, 'boy')"; + Assert.assertEquals("insert into acid_uanp values (1, 'boy')", + store.hiveSqlToAnsiSql(hiveSql)); + Assert.assertFalse(store.failureOk); + } + + @Test + public void dropTable() throws Exception { + String hiveSql = "drop table if exists acid_uanp"; + Assert.assertEquals("drop table if exists acid_uanp", store.hiveSqlToAnsiSql(hiveSql)); + } + + @Test + public void emptySqlSucceeds() throws Exception { + // Make sure a Hive SQL statement like alter database which is a NOP for the benchmark succeeds + String hiveSQL = "alter database fred set owner user user1"; + Assert.assertEquals("", store.hiveSqlToAnsiSql(hiveSQL)); + Assert.assertEquals(ResultCode.SUCCESS, store.fetchData(hiveSQL).rc); + } +} diff --git itests/capybara/src/test/java/org/apache/hive/test/capybara/infra/TestBoundedQueue.java itests/capybara/src/test/java/org/apache/hive/test/capybara/infra/TestBoundedQueue.java new file mode 100644 index 0000000..5774a7b --- /dev/null +++ itests/capybara/src/test/java/org/apache/hive/test/capybara/infra/TestBoundedQueue.java @@ -0,0 +1,123 @@ +/** + * 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.hive.test.capybara.infra; + +import org.junit.Assert; +import org.junit.Test; + +import java.util.Arrays; +import java.util.Iterator; + +public class TestBoundedQueue { + + @Test + public void underCapacity() { + BoundedQueue queue = new BoundedQueue<>(10); + queue.add(1); + queue.add(2); + + Assert.assertEquals(2, queue.size()); + Iterator iter = queue.iterator(); + Assert.assertEquals(1, (int)iter.next()); + Assert.assertEquals(2, (int)iter.next()); + Assert.assertFalse(iter.hasNext()); + + Assert.assertEquals(1, (int)queue.poll()); + Assert.assertEquals(2, (int)queue.peek()); + Assert.assertEquals(2, (int)queue.poll()); + Assert.assertTrue(queue.isEmpty()); + } + + @Test + public void atCapacity() { + BoundedQueue queue = new BoundedQueue<>(2); + queue.add(1); + queue.add(2); + + Assert.assertEquals(2, queue.size()); + Iterator iter = queue.iterator(); + Assert.assertEquals(1, (int)iter.next()); + Assert.assertEquals(2, (int)iter.next()); + Assert.assertFalse(iter.hasNext()); + + Assert.assertEquals(1, (int)queue.poll()); + Assert.assertEquals(2, (int)queue.poll()); + Assert.assertTrue(queue.isEmpty()); + } + + @Test + public void overCapacity() { + BoundedQueue queue = new BoundedQueue<>(2); + queue.add(1); + queue.add(2); + queue.add(3); + queue.add(4); + + Assert.assertEquals(2, queue.size()); + Iterator iter = queue.iterator(); + Assert.assertEquals(3, (int)iter.next()); + Assert.assertEquals(4, (int)iter.next()); + Assert.assertFalse(iter.hasNext()); + + Assert.assertEquals(3, (int)queue.poll()); + Assert.assertEquals(4, (int)queue.poll()); + Assert.assertTrue(queue.isEmpty()); + } + + @Test + public void addAll() { + BoundedQueue queue = new BoundedQueue<>(5); + queue.addAll(Arrays.asList(1, 2, 3, 4)); + Assert.assertEquals(4, queue.size()); + Iterator iter = queue.iterator(); + Assert.assertEquals(1, (int)iter.next()); + Assert.assertEquals(2, (int)iter.next()); + Assert.assertEquals(3, (int)iter.next()); + Assert.assertEquals(4, (int)iter.next()); + Assert.assertFalse(iter.hasNext()); + + queue.addAll(Arrays.asList(5, 6, 7, 8)); + Assert.assertEquals(5, queue.size()); + Assert.assertEquals(4, (int)queue.poll()); + Assert.assertEquals(5, (int)queue.poll()); + Assert.assertEquals(6, (int)queue.poll()); + Assert.assertEquals(7, (int)queue.poll()); + Assert.assertEquals(8, (int)queue.poll()); + Assert.assertTrue(queue.isEmpty()); + } + + @Test + public void offer() { + BoundedQueue queue = new BoundedQueue<>(2); + queue.offer(1); + queue.offer(2); + + Assert.assertEquals(2, queue.size()); + Iterator iter = queue.iterator(); + Assert.assertEquals(1, (int)iter.next()); + Assert.assertEquals(2, (int)iter.next()); + Assert.assertFalse(iter.hasNext()); + + queue.offer(3); + queue.offer(4); + + Assert.assertEquals(3, (int)queue.poll()); + Assert.assertEquals(4, (int)queue.poll()); + Assert.assertTrue(queue.isEmpty()); + } +} diff --git itests/capybara/src/test/java/org/apache/hive/test/capybara/infra/TestDerbyStore.java itests/capybara/src/test/java/org/apache/hive/test/capybara/infra/TestDerbyStore.java new file mode 100644 index 0000000..4d9c3cc --- /dev/null +++ itests/capybara/src/test/java/org/apache/hive/test/capybara/infra/TestDerbyStore.java @@ -0,0 +1,197 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hive.test.capybara.infra; + +import org.apache.hive.test.capybara.data.DataSet; +import org.apache.hive.test.capybara.data.FetchResult; +import org.apache.hive.test.capybara.data.ResultCode; +import org.apache.hive.test.capybara.data.Row; +import org.apache.hive.test.capybara.iface.TestTable; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import org.apache.hadoop.hive.metastore.api.FieldSchema; +import org.apache.hive.test.capybara.iface.DataGenerator; +import org.junit.Assert; +import org.junit.BeforeClass; +import org.junit.Test; + +import java.util.Arrays; +import java.util.Iterator; +import java.util.List; + +public class TestDerbyStore { + static final private Logger LOG = LoggerFactory.getLogger(TestDerbyStore.class.getName()); + static private DerbyStore derby; + + @BeforeClass + public static void setup() { + derby = new DerbyStore(); + } + + @Test + public void allTypes() throws Exception { + TestTable table = TestTable.getBuilder("derbyAllTypes") + .addCol("c1", "bigint") + .addCol("c2", "int") + .addCol("c3", "smallint") + .addCol("c4", "tinyint") + .addCol("c5", "float") + .addCol("c6", "double") + .addCol("c7", "decimal(19,2)") + .addCol("c8", "date") + .addCol("c9", "timestamp") + .addCol("c10", "varchar(32)") + .addCol("c11", "char(32)") + .addCol("c12", "string") + .addCol("c13", "boolean") + // Binary doesn't work on derby at the moment + .build(); + derby.dropTable(table); + derby.createTable(table); + + DataGenerator gen = new RandomDataGenerator(1); + derby.loadData(table, gen.generateData(table)); + } + + @Test + public void derby() throws Exception { + TestTable table = TestTable.getBuilder("foo") + .addCol("c1", "int") + .addCol("c2", "varchar(25)") + .build(); + + derby.dropTable(table); + derby.createTable(table); + + List rows = Arrays.asList("1,fred", "2,bob"); + StaticDataGenerator gen = new StaticDataGenerator(rows, ","); + DataSet data = gen.generateData(table); + + derby.loadData(table, data); + + // Re-create and if it returns true re-load the table to make sure that piece works correctly. + if (derby.createTable(table)) { + Assert.fail(); + } + + FetchResult fetch = derby.fetchData("select c1 from foo"); + Assert.assertEquals(ResultCode.SUCCESS, fetch.rc); + + Iterator iter = fetch.data.iterator(); + Assert.assertTrue(iter.hasNext()); + Assert.assertEquals(1, iter.next().get(0).asInt()); + Assert.assertTrue(iter.hasNext()); + Assert.assertEquals(2, iter.next().get(0).asInt()); + Assert.assertFalse(iter.hasNext()); + } + + @Test + public void sameNameDifferentDbs() throws Exception { + TestTable otherTable = null; + boolean createdSchema = false; + try { + List cols = Arrays.asList( + new FieldSchema("c1", "int", ""), + new FieldSchema("c2", "varchar(25)", "") + ); + TestTable defaultTable = TestTable.getBuilder("tind").setCols(cols).build(); + derby.dropTable(defaultTable); + derby.createTable(defaultTable); + + List rows = Arrays.asList("1,fred", "2,bob"); + StaticDataGenerator gen = new StaticDataGenerator(rows, ","); + DataSet data = gen.generateData(defaultTable); + + derby.loadData(defaultTable, data); + + derby.fetchData("create schema testschema"); + createdSchema = true; + otherTable = TestTable.getBuilder("tind").setDbName("testschema").setCols(cols).build(); + rows = Arrays.asList("3,mary", "4,elizabeth"); + gen = new StaticDataGenerator(rows, ","); + data = gen.generateData(otherTable); + derby.dropTable(otherTable); + derby.createTable(otherTable); + derby.loadData(otherTable, data); + + FetchResult fetch = derby.fetchData("select c1 from tind"); + Assert.assertEquals(ResultCode.SUCCESS, fetch.rc); + + Iterator iter = fetch.data.iterator(); + Assert.assertTrue(iter.hasNext()); + Assert.assertEquals(1, iter.next().get(0).asInt()); + Assert.assertTrue(iter.hasNext()); + Assert.assertEquals(2, iter.next().get(0).asInt()); + Assert.assertFalse(iter.hasNext()); + + fetch = derby.fetchData("select c1 from testschema.tind"); + Assert.assertEquals(ResultCode.SUCCESS, fetch.rc); + + iter = fetch.data.iterator(); + Assert.assertTrue(iter.hasNext()); + Assert.assertEquals(3, iter.next().get(0).asInt()); + Assert.assertTrue(iter.hasNext()); + Assert.assertEquals(4, iter.next().get(0).asInt()); + Assert.assertFalse(iter.hasNext()); + } finally { + if (otherTable != null) derby.dropTable(otherTable); + if (createdSchema) derby.fetchData("drop schema testschema restrict"); + } + } + + @Test + public void createTable() throws Exception { + List cols = Arrays.asList( + new FieldSchema("c1", "int", ""), + new FieldSchema("c2", "varchar(25)", "") + ); + derby.createTable(TestTable.getBuilder("foodle").setCols(cols).build()); + + // Do it twice so we can see that we handle it properly if it already exists. + derby.createTable(TestTable.getBuilder("foodle").setCols(cols).build()); + + } + + @Test + public void createPartitionedTable() throws Exception { + TestTable table = TestTable.getBuilder("part_voo") + .addCol("c1", "int") + .addCol("c2", "varchar(25)") + .addPartCol("p", "int") + .build(); + derby.createTable(table); + + List rows = Arrays.asList("1,fred,3", "2,bob,3"); + StaticDataGenerator gen = new StaticDataGenerator(rows, ","); + DataSet data = gen.generateData(table); + + derby.loadData(table, data); + } + + @Test + public void failureOk() throws Exception { + FetchResult fetch = derby.fetchData("drop table if exists fred"); + Assert.assertEquals(ResultCode.SUCCESS, fetch.rc); + + fetch = derby.fetchData("create table fred (a int)"); + Assert.assertEquals(ResultCode.SUCCESS, fetch.rc); + + fetch = derby.fetchData("create table if not exists fred (a int)"); + Assert.assertEquals(ResultCode.SUCCESS, fetch.rc); + } +} diff --git itests/capybara/src/test/java/org/apache/hive/test/capybara/infra/TestDerbyTranslator.java itests/capybara/src/test/java/org/apache/hive/test/capybara/infra/TestDerbyTranslator.java new file mode 100644 index 0000000..12d34c2 --- /dev/null +++ itests/capybara/src/test/java/org/apache/hive/test/capybara/infra/TestDerbyTranslator.java @@ -0,0 +1,204 @@ +/** + * 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.hive.test.capybara.infra; + +import org.junit.Assert; +import org.junit.Before; +import org.junit.Ignore; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.ExpectedException; + +public class TestDerbyTranslator { + private DerbyStore store; + private SQLTranslator translator; + + @Rule + public ExpectedException thrown = ExpectedException.none(); + + @Before + public void getTranslator() { + store = new DerbyStore(); + translator = store.getTranslator(); + } + + @Test + public void createDatabase() throws Exception { + Assert.assertEquals("create schema add_part_test_db", + translator.translate("CREATE DATABASE add_part_test_db")); + Assert.assertFalse(translator.isFailureOk()); + Assert.assertEquals("create schema newdb", + translator.translate("create database newDB location \"/tmp/\"")); + Assert.assertEquals("create schema dummydb", + translator.translate("create database if not exists dummydb")); + Assert.assertEquals("create schema test_db", + translator.translate("CREATE DATABASE IF NOT EXISTS test_db COMMENT 'Hive test database'")); + Assert.assertTrue(translator.isFailureOk()); + Assert.assertEquals("create schema test_db", + translator.translate("CREATE DATABASE test_db COMMENT 'Hive test database'")); + Assert.assertEquals("create schema db2", + translator.translate("create database db2 with dbproperties (\n" + + "'mapred.jobtracker.url'='http://my.jobtracker.com:53000',\n" + + "'hive.warehouse.dir' = '/user/hive/warehouse',\n" + + "'mapred.scratch.dir' = 'hdfs://tmp.dfs.com:50029/tmp')")); + Assert.assertEquals("create schema test_db", + translator.translate("create database test_db with dbproperties ('key1' = 'value1', 'key2' = 'value2')")); + Assert.assertEquals("create schema jsondb1", + translator.translate("CREATE DATABASE IF NOT EXISTS jsondb1 COMMENT 'Test database' " + + "LOCATION '${hiveconf:hive.metastore.warehouse.dir}/jsondb1' WITH DBPROPERTIES ('id' = 'jsondb1')")); + Assert.assertEquals("create schema some_database", + translator.translate("CREATE DATABASE some_database comment 'for show create db test' WITH DBPROPERTIES ('somekey'='somevalue')")); + Assert.assertEquals("create schema \"name with a space\"", + translator.translate("CREATE DATABASE `name with a space` location somewhere")); + + Assert.assertEquals("create schema add_part_test_db", + translator.translate("CREATE SCHEMA add_part_test_db")); + Assert.assertEquals("create schema newdb", + translator.translate("create schema newDB location \"/tmp/\"")); + Assert.assertEquals("create schema dummydb", + translator.translate("create schema if not exists dummydb")); + Assert.assertEquals("create schema test_db", + translator.translate("CREATE SCHEMA IF NOT EXISTS test_db COMMENT 'Hive test database'")); + Assert.assertEquals("create schema test_db", + translator.translate("CREATE SCHEMA test_db COMMENT 'Hive test database'")); + Assert.assertEquals("create schema db2", + translator.translate("create schema db2 with dbproperties (\n" + + "'mapred.jobtracker.url'='http://my.jobtracker.com:53000'\n" + + "'hive.warehouse.dir' = '/user/hive/warehouse'\n" + + "'mapred.scratch.dir' = 'hdfs://tmp.dfs.com:50029/tmp')")); + Assert.assertEquals("create schema test_db", + translator.translate("create schema test_db with dbproperties ('key1' = 'value1', 'key2' " + + "= 'value2')")); + Assert.assertEquals("create schema jsondb1", + translator.translate("CREATE SCHEMA IF NOT EXISTS jsondb1 COMMENT 'Test database' " + + "LOCATION '${hiveconf:hive.metastore.warehouse.dir}/jsondb1' WITH DBPROPERTIES ('id' = 'jsondb1')")); + Assert.assertEquals("create schema some_database", + translator.translate("CREATE SCHEMA some_database comment 'for show create db test' WITH " + + "DBPROPERTIES ('somekey'='somevalue')")); + Assert.assertEquals("create schema \"name with a space\"", + translator.translate("CREATE SCHEMA `name with a space`")); + } + + @Test + public void dropDatabase() throws Exception { + Assert.assertEquals("drop schema add_part_test_db restrict", + translator.translate("DROP DATABASE add_part_test_db")); + Assert.assertEquals("drop schema statsdb1 restrict", + translator.translate("drop database if exists statsdb1")); + Assert.assertEquals("drop schema to_drop_db1 restrict", + translator.translate("DROP DATABASE to_drop_db1 CASCADE")); + Assert.assertEquals("drop schema non_exists_db3 restrict", + translator.translate("DROP DATABASE IF EXISTS non_exists_db3 RESTRICT")); + Assert.assertEquals("drop schema to_drop_db4 restrict", + translator.translate("DROP DATABASE to_drop_db4 RESTRICT")); + + Assert.assertEquals("drop schema add_part_test_db restrict", + translator.translate("DROP SCHEMA add_part_test_db")); + Assert.assertEquals("drop schema statsdb1 restrict", + translator.translate("drop schema if exists statsdb1")); + Assert.assertEquals("drop schema to_drop_db1 restrict", + translator.translate("DROP SCHEMA to_drop_db1 CASCADE")); + Assert.assertEquals("drop schema non_exists_db3 restrict", + translator.translate("DROP SCHEMA IF EXISTS non_exists_db3 RESTRICT")); + Assert.assertEquals("drop schema to_drop_db4 restrict", + translator.translate("DROP SCHEMA to_drop_db4 RESTRICT")); + } + + @Test + public void createTableLike() throws Exception { + Assert.assertEquals("create table alter3_like as select * from alter3", + translator.translate("create table alter3_like like alter3")); + Assert.assertEquals("create table emp_orc as select * from emp_staging", + translator.translate("create table if not exists emp_orc like emp_staging")); + Assert.assertTrue(translator.isFailureOk()); + Assert.assertEquals("create table source.srcpart as select * from default.srcpart", + translator.translate("create table source.srcpart like default.srcpart;")); + } + + @Test + public void createTableWithCols() throws Exception { + Assert.assertEquals("create table acidjoin1 (name varchar(50), age int)", + translator.translate("create table acidjoin1(name varchar(50), age int) clustered by " + + "(age) into 2 buckets stored as orc TBLPROPERTIES (\"transactional\"=\"true\")")); + Assert.assertEquals("create table alter1 (a int, b int)", + translator.translate("create table alter1(a int, b int)")); + Assert.assertEquals("create table alter2 (a int, b int)", + translator.translate("create table alter2(a int, b int) partitioned by (insertdate string)")); + Assert.assertEquals("create table alter3_src ( col1 varchar(255) )", + translator.translate("create table alter3_src ( col1 string ) stored as textfile ")); + Assert.assertEquals("create table alter3 ( col1 varchar(255) )", + translator.translate("create table alter3 ( col1 string ) partitioned by (pcol1 string , " + + "pcol2 string) stored as sequencefile")); + Assert.assertEquals("create table ac.alter_char_1 (key varchar(255), value varchar(255))", + translator.translate("create table ac.alter_char_1 (key string, value string)")); + Assert.assertEquals("create table tst1 (key varchar(255), value varchar(255))", + translator.translate("create table tst1(key string, value string) partitioned by (ds " + + "string) clustered by (key) into 10 buckets")); + Assert.assertEquals("create table over1k ( t smallint, si smallint, i int, b bigint, f float, " + + "d double, bo boolean, s varchar(255), ts timestamp, dec decimal(4,2), bin " + + "blob)", + translator.translate("create table over1k( t tinyint, si smallint, i int, b bigint, f " + + "float, d double, bo boolean, s string, ts timestamp, dec decimal(4,2), bin binary) " + + "ROW FORMAT DELIMITED FIELDS TERMINATED BY '|' STORED AS TEXTFILE")); + Assert.assertEquals("create table loc_staging (state varchar(255),locid int,zip bigint,year " + + "int )", + translator.translate("create table if not exists loc_staging (state string,locid int,zip " + + "bigint,year int ) row format delimited fields terminated by '|' stored as textfile")); + Assert.assertEquals("declare global temporary table acid_dtt (a int, b varchar(128))", + translator.translate("create temporary table acid_dtt(a int, b varchar(128)) clustered by" + + " (a) into 2 buckets stored as orc TBLPROPERTIES ('transactional'='true')")); + Assert.assertEquals("create table roottable (key varchar(255))", + translator.translate("create external table roottable (key string) row format delimited " + + "fields terminated by '\\t' stored as textfile")); + } + + @Test + public void dropTable() throws Exception { + Assert.assertEquals("drop table t", translator.translate("drop table t")); + Assert.assertEquals("drop table t", translator.translate("drop table if exists t")); + Assert.assertTrue(translator.isFailureOk()); + Assert.assertEquals("drop table db.t", translator.translate("drop table db.t")); + Assert.assertEquals("drop table t", translator.translate("drop table t purge")); + } + + @Test + public void alterTable() throws Exception { + thrown.expect(TranslationException.class); + thrown.expectMessage("Could not translate alter table rename, Hive SQL:"); + translator.translate("alter table tab1 rename to tab2"); + } + + @Test + public void selectLimit() throws Exception { + Assert.assertEquals("select key from src_autho_test order by key", + translator.translate("select key from src_autho_test order by key limit 20")); + Assert.assertEquals(20, store.getLimit()); + } + + @Ignore + public void constantCast() throws Exception { + Assert.assertEquals("select dateval - '1999-06-07' from interval_arithmetic_1", + translator.translate("select dateval - date '1999-06-07' from interval_arithmetic_1")); + Assert.assertEquals("select dateval - '1999-06-07' from interval_arithmetic_1", + translator.translate("select dateval - date '1999-6-7' from interval_arithmetic_1")); + Assert.assertEquals("select '1999-01-01 01:00:00' from interval_arithmetic_1", + translator.translate("select timestamp '1999-01-01 01:00:00' from interval_arithmetic_1")); + Assert.assertEquals("select '1999-01-01 01:00:00' from interval_arithmetic_1", + translator.translate("select timestamp '1999-1-1 01:00:00' from interval_arithmetic_1")); + } +} diff --git itests/capybara/src/test/java/org/apache/hive/test/capybara/infra/TestHiveStore.java itests/capybara/src/test/java/org/apache/hive/test/capybara/infra/TestHiveStore.java new file mode 100644 index 0000000..67f0e5f --- /dev/null +++ itests/capybara/src/test/java/org/apache/hive/test/capybara/infra/TestHiveStore.java @@ -0,0 +1,232 @@ +/** + * 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.hive.test.capybara.infra; + +import org.apache.commons.lang.StringUtils; +import org.apache.hadoop.hive.conf.HiveConf; +import org.apache.hadoop.hive.metastore.api.FieldSchema; +import org.apache.hive.test.capybara.data.DataSet; +import org.apache.hive.test.capybara.data.FetchResult; +import org.apache.hive.test.capybara.data.ResultCode; +import org.apache.hive.test.capybara.data.Row; +import org.apache.hive.test.capybara.iface.ClusterManager; +import org.apache.hive.test.capybara.iface.TestTable; +import org.junit.AfterClass; +import org.junit.Assert; +import org.junit.BeforeClass; +import org.junit.Test; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.util.Arrays; +import java.util.Iterator; +import java.util.List; + +public class TestHiveStore { + + static final private Logger LOG = LoggerFactory.getLogger(TestHiveStore.class.getName()); + + private static HiveConf conf; + private static ClusterManager mgr; + private static HiveStore hive; + + @BeforeClass + public static void setup() throws IOException { + conf = new HiveConf(); + conf.setVar(HiveConf.ConfVars.DYNAMICPARTITIONINGMODE, "nonstrict"); + conf.setVar(HiveConf.ConfVars.HIVEMAPREDMODE, "nonstrict"); + TestConf.setEngine(TestConf.ENGINE_UNSPECIFIED); + mgr = new MiniClusterManager(); + mgr.setConf(conf); + mgr.setup(); + hive = mgr.getHive(); + TestManager.getTestManager().setClusterManager(mgr); + TestManager.getTestManager().setConf(conf); + + } + + @AfterClass + public static void tearDown() { + mgr.tearDown(); + } + + @Test + public void hive() throws Exception { + // Load some data, then read it back. + FetchResult fetch = hive.fetchData("create table foo (c1 int, c2 varchar(32))"); + Assert.assertEquals(ResultCode.SUCCESS, fetch.rc); + + TestTable table = TestTable.fromHiveMetastore("default", "foo"); + + List rows = Arrays.asList("1,fred", "2,bob"); + StaticDataGenerator gen = new StaticDataGenerator(rows, ","); + DataSet data = gen.generateData(table); + + hive.loadData(table, data); + + fetch = hive.fetchData("select c1 from foo"); + Assert.assertEquals(ResultCode.SUCCESS, fetch.rc); + + fetch.data.setSchema(Arrays.asList(new FieldSchema("c1", "int", ""))); + Iterator iter = fetch.data.iterator(); + Assert.assertTrue(iter.hasNext()); + Row row = iter.next(); + Assert.assertEquals(1, row.get(0).asInt()); + Assert.assertTrue(iter.hasNext()); + + row = iter.next(); + Assert.assertEquals(2, row.get(0).asInt()); + Assert.assertFalse(iter.hasNext()); + } + + @Test + public void sameNameDifferentDbs() throws Exception { + boolean createdSchema = false; + try { + List cols = Arrays.asList( + new FieldSchema("c1", "int", ""), + new FieldSchema("c2", "varchar(25)", "") + ); + TestTable defaultTable = TestTable.getBuilder("tind").setCols(cols).build(); + hive.dropTable(defaultTable); + Assert.assertTrue(hive.createTable(defaultTable)); + + List rows = Arrays.asList("1,fred", "2,bob"); + StaticDataGenerator gen = new StaticDataGenerator(rows, ","); + DataSet data = gen.generateData(defaultTable); + + hive.loadData(defaultTable, data); + + hive.fetchData("create database testschema"); + createdSchema = true; + TestTable otherTable = TestTable.getBuilder("tind").setDbName("testschema").setCols(cols).build(); + rows = Arrays.asList("3,mary", "4,elizabeth"); + gen = new StaticDataGenerator(rows, ","); + data = gen.generateData(otherTable); + hive.dropTable(otherTable); + Assert.assertTrue(hive.createTable(otherTable)); + hive.loadData(otherTable, data); + + FetchResult fetch = hive.fetchData("select c1 from tind"); + Assert.assertEquals(ResultCode.SUCCESS, fetch.rc); + + fetch.data.setSchema(Arrays.asList(new FieldSchema("c1", "int", ""))); + Iterator iter = fetch.data.iterator(); + Assert.assertTrue(iter.hasNext()); + Row row = iter.next(); + Assert.assertEquals(1, row.get(0).asInt()); + Assert.assertTrue(iter.hasNext()); + + row = iter.next(); + Assert.assertEquals(2, row.get(0).asInt()); + Assert.assertFalse(iter.hasNext()); + + fetch = hive.fetchData("select c1 from testschema.tind"); + Assert.assertEquals(ResultCode.SUCCESS, fetch.rc); + + + fetch.data.setSchema(Arrays.asList(new FieldSchema("c1", "int", ""))); + iter = fetch.data.iterator(); + Assert.assertTrue(iter.hasNext()); + row = iter.next(); + Assert.assertEquals(3, row.get(0).asInt()); + Assert.assertTrue(iter.hasNext()); + + row = iter.next(); + Assert.assertEquals(4, row.get(0).asInt()); + Assert.assertFalse(iter.hasNext()); + } finally { + if (createdSchema) hive.fetchData("drop database testschema cascade"); + } + } + + @Test + public void hiveWithCreateTable() throws Exception { + // Load some data, then read it back. + + TestTable table = TestTable.getBuilder("foozle") + .addCol("c1", "int") + .addCol("c2", "varchar(25)") + .build(); + hive.dropTable(table); + hive.createTable(table); + + // Re-create and if it returns true re-load the table to make sure that piece works correctly. + if (hive.createTable(table)) { + Assert.fail(); + } + + List rows = Arrays.asList("1,fred", "2,bob"); + StaticDataGenerator gen = new StaticDataGenerator(rows, ","); + DataSet data = gen.generateData(table); + + hive.loadData(table, data); + + FetchResult fetch = hive.fetchData("select c1 from foozle"); + Assert.assertEquals(ResultCode.SUCCESS, fetch.rc); + + fetch.data.setSchema(Arrays.asList(new FieldSchema("c1", "int", ""))); + Iterator iter = fetch.data.iterator(); + Assert.assertTrue(iter.hasNext()); + + Row row = iter.next(); + Assert.assertEquals(1, row.get(0).asInt()); + Assert.assertTrue(iter.hasNext()); + row = iter.next(); + Assert.assertEquals(2, row.get(0).asInt()); + Assert.assertFalse(iter.hasNext()); + } + + @Test + public void createTable() throws Exception { + hive.setConf(conf); + List cols = Arrays.asList( + new FieldSchema("c1", "int", ""), + new FieldSchema("c2", "varchar(25)", "") + ); + hive.createTable(TestTable.getBuilder("foodle").setCols(cols).build()); + + // Make sure we drop and re-create the table as necessary + hive.createTable(TestTable.getBuilder("foodle").setCols(cols).build()); + } + + @Test + public void createPartitionedTable() throws Exception { + hive.setConf(conf); + TestTable table = TestTable.getBuilder("foo_part") + .addCol("c1", "int") + .addCol("c2", "varchar(25)") + .addPartCol("pcol", "string") + .build(); + hive.dropTable(table); + hive.createTable(table); + + List rows = Arrays.asList("1,fred,3", "2,bob,3"); + StaticDataGenerator gen = new StaticDataGenerator(rows, ","); + DataSet data = gen.generateData(table); + + hive.loadData(table, data); + + FetchResult fetch = hive.fetchData("select count(*) from foo_part"); + Assert.assertEquals(ResultCode.SUCCESS, fetch.rc); + fetch.data.setSchema(Arrays.asList(new FieldSchema("c0", "bigint", ""))); + Iterator output = fetch.data.stringIterator(",", "", "\""); + LOG.debug("Query result: " + StringUtils.join(output, "\n")); + } +} diff --git itests/capybara/src/test/java/org/apache/hive/test/capybara/infra/TestIntegrationRunner.java itests/capybara/src/test/java/org/apache/hive/test/capybara/infra/TestIntegrationRunner.java new file mode 100644 index 0000000..57a43e5 --- /dev/null +++ itests/capybara/src/test/java/org/apache/hive/test/capybara/infra/TestIntegrationRunner.java @@ -0,0 +1,77 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hive.test.capybara.infra; + +import org.apache.hadoop.hive.conf.HiveConf; +import org.apache.hive.test.capybara.IntegrationTest; +import org.apache.hive.test.capybara.annotations.AcidOn; +import org.apache.hive.test.capybara.annotations.NoCli; +import org.apache.hive.test.capybara.annotations.NoJdbc; +import org.apache.hive.test.capybara.annotations.NoOrc; +import org.apache.hive.test.capybara.annotations.NoParquet; +import org.apache.hive.test.capybara.annotations.NoRcFile; +import org.apache.hive.test.capybara.annotations.NoSpark; +import org.apache.hive.test.capybara.annotations.NoTextFile; +import org.apache.hive.test.capybara.annotations.NoTez; +import org.junit.Assert; +import org.junit.Test; + +import java.io.IOException; +import java.sql.SQLException; + +@NoSpark // Make sure no tests run when spark is set +public class TestIntegrationRunner extends IntegrationTest { + + @NoCli @Test public void testNoCli() { + Assert.assertNotEquals("cli", TestConf.access()); + } + + @NoJdbc @Test public void testNoJdbc() { + Assert.assertNotEquals("jdbc", TestConf.access()); + } + + @NoOrc @Test public void testNoOrc() { + Assert.assertNotEquals("orc", TestConf.fileFormat()); + } + + @NoParquet @Test public void testNoParquet() { + Assert.assertNotEquals("parquet", TestConf.fileFormat()); + } + + @NoRcFile @Test public void testNoRcFile() { + Assert.assertNotEquals("rcfile", TestConf.fileFormat()); + } + + @NoTextFile @Test public void testTextFile() { + Assert.assertNotEquals("text", TestConf.fileFormat()); + } + + @NoTez @Test public void testTez() { + Assert.assertNotEquals("tez", TestConf.engine()); + } + + @AcidOn @Test public void testAcid() throws IOException, SQLException { + Assert.assertTrue(getCurrentConf().getBoolVar(HiveConf.ConfVars.HIVE_SUPPORT_CONCURRENCY)); + } + + @Test public void testNoSpark() { + Assert.assertNotEquals("spark", TestConf.engine()); + Assert.assertFalse(getCurrentConf().getBoolVar(HiveConf.ConfVars.HIVE_SUPPORT_CONCURRENCY)); + } + +} diff --git itests/capybara/src/test/java/org/apache/hive/test/capybara/infra/TestMiniHS2HiveStore.java itests/capybara/src/test/java/org/apache/hive/test/capybara/infra/TestMiniHS2HiveStore.java new file mode 100644 index 0000000..a6f256e --- /dev/null +++ itests/capybara/src/test/java/org/apache/hive/test/capybara/infra/TestMiniHS2HiveStore.java @@ -0,0 +1,95 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + *

+ * http://www.apache.org/licenses/LICENSE-2.0 + *

+ * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hive.test.capybara.infra; + +import org.apache.hadoop.hive.conf.HiveConf; +import org.apache.hadoop.hive.metastore.api.FieldSchema; +import org.apache.hive.test.capybara.data.DataSet; +import org.apache.hive.test.capybara.data.FetchResult; +import org.apache.hive.test.capybara.data.ResultCode; +import org.apache.hive.test.capybara.data.Row; +import org.apache.hive.test.capybara.iface.ClusterManager; +import org.apache.hive.test.capybara.iface.TestTable; +import org.junit.AfterClass; +import org.junit.Assert; +import org.junit.BeforeClass; +import org.junit.Test; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.util.Arrays; +import java.util.Iterator; +import java.util.List; + +public class TestMiniHS2HiveStore { + static final private Logger LOG = LoggerFactory.getLogger(TestHiveStore.class.getName()); + + private static HiveConf conf; + private static ClusterManager mgr; + private static HiveStore hive; + + @BeforeClass + public static void setup() throws IOException { + conf = new HiveConf(); + conf.setVar(HiveConf.ConfVars.DYNAMICPARTITIONINGMODE, "nonstrict"); + //conf.setBoolVar(HiveConf.ConfVars.HIVE_SUPPORT_CONCURRENCY, false); + + TestManager.getTestManager().setConf(conf); + TestConf.setEngine(TestConf.ENGINE_UNSPECIFIED); + TestConf.setAccess(TestConf.ACCESS_JDBC); + mgr = TestManager.getTestManager().getClusterManager(); + mgr.setup(); + mgr.setConfVar(HiveConf.ConfVars.HIVE_SUPPORT_CONCURRENCY.varname, "false"); + hive = mgr.getHive(); + } + + @AfterClass + public static void tearDown() { + mgr.tearDown(); + } + + @Test + public void hive() throws Exception { + // Load some data, then read it back. + FetchResult fetch = hive.fetchData("create table foo (c1 int, c2 varchar(32))"); + Assert.assertEquals(ResultCode.SUCCESS, fetch.rc); + + TestTable table = TestTable.fromHiveMetastore("default", "foo"); + + List rows = Arrays.asList("1,fred", "2,bob"); + StaticDataGenerator gen = new StaticDataGenerator(rows, ","); + DataSet data = gen.generateData(table); + + hive.loadData(table, data); + + fetch = hive.fetchData("select c1 from foo"); + Assert.assertEquals(ResultCode.SUCCESS, fetch.rc); + + fetch.data.setSchema(Arrays.asList(new FieldSchema("c1", "int", ""))); + Iterator iter = fetch.data.iterator(); + Assert.assertTrue(iter.hasNext()); + Row row = iter.next(); + Assert.assertEquals(1, row.get(0).asInt()); + Assert.assertTrue(iter.hasNext()); + + row = iter.next(); + Assert.assertEquals(2, row.get(0).asInt()); + Assert.assertFalse(iter.hasNext()); + } +} diff --git itests/capybara/src/test/java/org/apache/hive/test/capybara/infra/TestNonSortingComparator.java itests/capybara/src/test/java/org/apache/hive/test/capybara/infra/TestNonSortingComparator.java new file mode 100644 index 0000000..813ed65 --- /dev/null +++ itests/capybara/src/test/java/org/apache/hive/test/capybara/infra/TestNonSortingComparator.java @@ -0,0 +1,261 @@ +/** + * 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.hive.test.capybara.infra; + +import org.apache.hive.test.capybara.data.DataSet; +import org.apache.hive.test.capybara.iface.ResultComparator; +import org.apache.hive.test.capybara.iface.TestTable; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import org.apache.hadoop.hive.metastore.api.FieldSchema; +import org.apache.hive.test.capybara.iface.DataGenerator; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; + +import java.io.IOException; +import java.sql.SQLException; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; + +public class TestNonSortingComparator { + + static final private Logger LOG = LoggerFactory.getLogger(TestNonSortingComparator.class.getName()); + + private TestTable table; + private List rows; + private List cols; + + @Before + public void createTable() { + final String tableName = "alltypes"; + + + cols = Arrays.asList( + new FieldSchema("col_bi", "bigint", ""), + new FieldSchema("col_i", "int", ""), + new FieldSchema("col_si", "smallint", ""), + new FieldSchema("col_ti", "tinyint", ""), + new FieldSchema("col_bin", "binary", ""), + new FieldSchema("col_bool", "boolean", ""), + new FieldSchema("col_ch", "char(8)", ""), + new FieldSchema("col_vc", "varchar(89)", ""), + new FieldSchema("col_str", "string", ""), + new FieldSchema("col_date", "date", ""), + new FieldSchema("col_dec", "decimal(10,2)", ""), + new FieldSchema("col_fl", "float", ""), + new FieldSchema("col_dbl", "double", ""), + new FieldSchema("col_tm", "timestamp", "") + ); + + table = TestTable.getBuilder(tableName).setCols(cols).build(); + + rows = new ArrayList<>(); + rows.add("-6022141300000000000,-299792458,-1432,-7,def,false,joe,mary had a little " + + "lamb,her fleece was white as snow,2015-08-04,371.89,-1.234,-6.0221413E-23,NULL"); + rows.add("6,2,1432,7,abc,true,NULL,mary had a little lamb,her" + + " fleece was white as snow,2015-08-04,371.89,1.234,6.0221413E23,2015-08-04 17:16:32"); + } + + @Test + public void allGood() throws SQLException, IOException { + DataGenerator gen1 = new StaticDataGenerator(rows, ","); + DataSet hive = gen1.generateData(table); + DataGenerator gen2 = new StaticDataGenerator(rows, ","); + DataSet bench = gen2.generateData(table); + ResultComparator comparator = new NonSortingComparator(); + comparator.compare(hive, bench); + + } + + @Test + public void hiveMoreRows() throws SQLException, IOException { + DataGenerator gen1 = new StaticDataGenerator(rows, ","); + DataSet hive = gen1.generateData(table); + DataGenerator gen2 = new StaticDataGenerator(rows.subList(0, 1), ","); + DataSet bench = gen2.generateData(table); + ResultComparator comparator = new NonSortingComparator(); + try { + comparator.compare(hive, bench); + Assert.fail(); + } catch (AssertionError e) { + Assert.assertEquals("Benchmark ran out of rows at 1 but hive still has rows", e.getMessage()); + } + } + + @Test + public void benchMoreRows() throws SQLException, IOException { + DataGenerator gen1 = new StaticDataGenerator(rows.subList(0, 1), ","); + DataSet hive = gen1.generateData(table); + DataGenerator gen2 = new StaticDataGenerator(rows, ","); + DataSet bench = gen2.generateData(table); + ResultComparator comparator = new NonSortingComparator(); + try { + comparator.compare(hive, bench); + Assert.fail(); + } catch (AssertionError e) { + Assert.assertEquals("Hive ran out of rows at 1 but benchmark still has rows", e.getMessage()); + } + } + + @Test + public void diffNumberColumns() throws Exception { + + TestTable diffTable = TestTable.getBuilder("difftable").addCol("col_bi", "bigint").build(); + + List diffRows = new ArrayList<>(); + diffRows.add("6"); + diffRows.add("7"); + DataGenerator gen1 = new StaticDataGenerator(rows, ","); + DataSet hive = gen1.generateData(table); + DataGenerator gen2 = new StaticDataGenerator(diffRows, ","); + DataSet bench = gen2.generateData(diffTable); + ResultComparator comparator = new NonSortingComparator(); + try { + comparator.compare(hive, bench); + Assert.fail(); + } catch (AssertionError e) { + Assert.assertEquals("Different number of columns expected:<1> but was:<14>", e.getMessage()); + } + } + + @Test + public void uncompatibleDataTypes() throws Exception { + + TestTable diffTable = TestTable.getBuilder("difftable") + .addCol("col_bi", "bigint") + .addCol("col_i", "string") + .addCol("col_si", "smallint") + .addCol("col_ti", "tinyint") + .addCol("col_bin", "binary") + .addCol("col_bool", "boolean") + .addCol("col_ch", "char(8)") + .addCol("col_vc", "varchar(89)") + .addCol("col_str", "string") + .addCol("col_date", "date") + .addCol("col_dec", "decimal(10,2)") + .addCol("col_fl", "float") + .addCol("col_dbl", "double") + .addCol("col_tm", "timestamp") + .build(); + + List diffRows = new ArrayList<>(); + diffRows.add("6,2,1432,7,abc,true,NULL,mary had a little lamb,her" + + " fleece was white as snow,2015-08-04,371.89,1.234,6.0221413E23,2015-08-04 17:16:32"); + diffRows.add("-6022141300000000000,-299792458,-1432,-7,def,false,joe,mary had a little " + + "lamb,her fleece was white as snow,2015-08-04,371.89,-1.234,-6.0221413E-23,NULL"); + DataGenerator gen1 = new StaticDataGenerator(rows, ","); + DataSet hive = gen1.generateData(table); + DataGenerator gen2 = new StaticDataGenerator(diffRows, ","); + DataSet bench = gen2.generateData(diffTable); + ResultComparator comparator = new NonSortingComparator(); + try { + comparator.compare(hive, bench); + Assert.fail(); + } catch (AssertionError e) { + Assert.assertEquals("Found discrepency in metadata at column 1", e.getMessage()); + } + } + + @Test + public void diffNull() throws Exception { + List diffRows = new ArrayList<>(); + diffRows.add("6,2,1432,7,abc,true,NULL,mary had a little lamb,her" + + " fleece was white as snow,2015-08-04,371.89,1.234,6.0221413E23,2015-08-04 17:16:32"); + diffRows.add("NULL,-299792458,-1432,-7,def,false,joe,mary had a little " + + "lamb,her fleece was white as snow,2015-08-04,371.89,-1.234,-6.0221413E-23,NULL"); + DataGenerator gen1 = new StaticDataGenerator(rows, ","); + DataSet hive = gen1.generateData(table); + DataGenerator gen2 = new StaticDataGenerator(diffRows, ","); + DataSet bench = gen2.generateData(table); + ResultComparator comparator = new NonSortingComparator(); + try { + comparator.compare(hive, bench); + Assert.fail(); + } catch (AssertionError e) { + Assert.assertEquals("Mismatch at row 1 hive row is <-6022141300000000000,-299792458,-1432," + + "-7,dec,false,joe,mary had a little lamb,her fleece was white as snow,2015-08-04," + + "371.89,-1.234,-6.0221413E-23,NULL> bench row is ", e.getMessage()); + } + } + + @Test + public void diff() throws Exception { + List diffRows = new ArrayList<>(); + diffRows.add("6,2,1432,7,abc,true,NULL,mary had a little lamb,her" + + " fleece was white as snow,2015-08-04,371.89,1.234,6.0221413E23,2015-08-04 17:16:32"); + diffRows.add("-6022141300000000000,-299792458,-1432,-7,def,false,joe,mary had a little " + + "lamb,her fleece was white as snow,2015-08-04,371.89,1.234,-6.0221413E-23,NULL"); + DataGenerator gen1 = new StaticDataGenerator(rows, ","); + DataSet hive = gen1.generateData(table); + DataGenerator gen2 = new StaticDataGenerator(diffRows, ","); + DataSet bench = gen2.generateData(table); + ResultComparator comparator = new NonSortingComparator(); + try { + comparator.compare(hive, bench); + Assert.fail(); + } catch (AssertionError e) { + Assert.assertEquals("Mismatch at row 1 hive row is <-6022141300000000000," + + "-299792458,-1432,-7,dec,false,joe,mary had a little lamb,her fleece was white as snow," + + "2015-08-04,371" + + ".89,-1.234,-6.0221413E-23,NULL> bench row is <-6022141300000000000,-299792458,-1432," + + "-7,dec,false,joe,mary had a little lamb,her fleece was white as snow,2015-08-04,371" + + ".89,1.234,-6.0221413E-23,NULL>", e.getMessage()); + } + } + + @Test + public void cli() throws Exception { + DataGenerator gen1 = new StaticDataGenerator(rows, ","); + DataSet bench = gen1.generateData(table); + + List cliRows = new ArrayList<>(); + cliRows.add("-6022141300000000000,-299792458,-1432,-7,def,false,joe,mary had a little " + + "lamb,her fleece was white as snow,2015-08-04,371.89,-1.234,-6" + + ".0221413E-23,NULL"); + cliRows.add("6,2,1432,7,abc,true,NULL,mary had a little " + + "lamb,her fleece was white as snow,2015-08-04,371.89,1.234,6" + + ".0221413E23,2015-08-04 17:16:32"); + + DataSet hive = new StringDataSet(cols, cliRows, ",", "NULL"); + ResultComparator comparator = new NonSortingComparator(); + comparator.compare(hive, bench); + } + + @Test + public void setSchema() throws Exception { + DataGenerator gen1 = new StaticDataGenerator(rows, ","); + DataSet bench = gen1.generateData(table); + + List cliRows = new ArrayList<>(); + cliRows.add("-6022141300000000000,-299792458,-1432,-7,def,false,joe,mary had a little " + + "lamb,her fleece was white as snow,2015-08-04,371.89,-1.234,-6" + + ".0221413E-23,NULL"); + cliRows.add("6,2,1432,7,abc,true,NULL,mary had a little " + + "lamb,her fleece was white as snow,2015-08-04,371.89,1.234,6" + + ".0221413E23,2015-08-04 17:16:32"); + + DataSet hive = new StringDataSet(cliRows, ",", "NULL"); + hive.setSchema(cols); + ResultComparator comparator = new NonSortingComparator(); + comparator.compare(hive, bench); + } +} diff --git itests/capybara/src/test/java/org/apache/hive/test/capybara/infra/TestPostgresTranslator.java itests/capybara/src/test/java/org/apache/hive/test/capybara/infra/TestPostgresTranslator.java new file mode 100644 index 0000000..c00c711 --- /dev/null +++ itests/capybara/src/test/java/org/apache/hive/test/capybara/infra/TestPostgresTranslator.java @@ -0,0 +1,475 @@ +/** + * 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.hive.test.capybara.infra; + +import org.junit.Assert; +import org.junit.Before; +import org.junit.Ignore; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.ExpectedException; + +public class TestPostgresTranslator { + private SQLTranslator translator; + + @Rule + public ExpectedException thrown = ExpectedException.none(); + + @Before + public void getTranslator() { + translator = new PostgresStore().getTranslator(); + } + + @Test + public void unrecognized() throws Exception { + thrown.expect(TranslationException.class); + translator.translate("bla bla bla"); + } + + @Test + public void createDatabase() throws Exception { + Assert.assertEquals("create schema add_part_test_db", + translator.translate("CREATE DATABASE add_part_test_db")); + Assert.assertFalse(translator.isFailureOk()); + Assert.assertEquals("create schema newdb", + translator.translate("create database newDB location \"/tmp/\"")); + Assert.assertEquals("create schema if not exists dummydb", + translator.translate("create database if not exists dummydb")); + Assert.assertEquals("create schema if not exists test_db", + translator.translate("CREATE DATABASE IF NOT EXISTS test_db COMMENT 'Hive test database'")); + Assert.assertFalse(translator.isFailureOk()); + Assert.assertEquals("create schema test_db", + translator.translate("CREATE DATABASE test_db COMMENT 'Hive test database'")); + Assert.assertEquals("create schema db2", + translator.translate("create database db2 with dbproperties (\n" + + "'mapred.jobtracker.url'='http://my.jobtracker.com:53000',\n" + + "'hive.warehouse.dir' = '/user/hive/warehouse',\n" + + "'mapred.scratch.dir' = 'hdfs://tmp.dfs.com:50029/tmp')")); + Assert.assertEquals("create schema test_db", + translator.translate( + "create database test_db with dbproperties ('key1' = 'value1', 'key2' = 'value2')")); + Assert.assertEquals("create schema if not exists jsondb1", + translator.translate("CREATE DATABASE IF NOT EXISTS jsondb1 COMMENT 'Test database' " + + "LOCATION '${hiveconf:hive.metastore.warehouse.dir}/jsondb1' WITH DBPROPERTIES ('id' = 'jsondb1')")); + Assert.assertEquals("create schema some_database", + translator.translate("CREATE DATABASE some_database comment 'for show create db test' WITH DBPROPERTIES ('somekey'='somevalue')")); + Assert.assertEquals("create schema \"name with a space\"", + translator.translate("CREATE DATABASE `name with a space` location somewhere")); + + Assert.assertEquals("create schema add_part_test_db", + translator.translate("CREATE SCHEMA add_part_test_db")); + Assert.assertEquals("create schema newdb", + translator.translate("create schema newDB location \"/tmp/\"")); + Assert.assertEquals("create schema if not exists dummydb", + translator.translate("create schema if not exists dummydb")); + Assert.assertEquals("create schema if not exists test_db", + translator.translate("CREATE SCHEMA IF NOT EXISTS test_db COMMENT 'Hive test database'")); + Assert.assertEquals("create schema test_db", + translator.translate("CREATE SCHEMA test_db COMMENT 'Hive test database'")); + Assert.assertEquals("create schema db2", + translator.translate("create schema db2 with dbproperties (\n" + + "'mapred.jobtracker.url'='http://my.jobtracker.com:53000'\n" + + "'hive.warehouse.dir' = '/user/hive/warehouse'\n" + + "'mapred.scratch.dir' = 'hdfs://tmp.dfs.com:50029/tmp')")); + Assert.assertEquals("create schema test_db", + translator.translate("create schema test_db with dbproperties ('key1' = 'value1', 'key2' " + + "= 'value2')")); + Assert.assertEquals("create schema if not exists jsondb1", + translator.translate("CREATE SCHEMA IF NOT EXISTS jsondb1 COMMENT 'Test database' " + + "LOCATION '${hiveconf:hive.metastore.warehouse.dir}/jsondb1' WITH DBPROPERTIES ('id' = 'jsondb1')")); + Assert.assertEquals("create schema some_database", + translator.translate("CREATE SCHEMA some_database comment 'for show create db test' WITH " + + "DBPROPERTIES ('somekey'='somevalue')")); + Assert.assertEquals("create schema \"name with a space\"", + translator.translate("CREATE SCHEMA `name with a space`")); + } + + @Test + public void dropDatabase() throws Exception { + Assert.assertEquals("drop schema add_part_test_db", + translator.translate("DROP DATABASE add_part_test_db")); + Assert.assertEquals("drop schema if exists statsdb1", + translator.translate("drop database if exists statsdb1")); + Assert.assertEquals("drop schema to_drop_db1 cascade", + translator.translate("DROP DATABASE to_drop_db1 CASCADE")); + Assert.assertEquals("drop schema if exists non_exists_db3 restrict", + translator.translate("DROP DATABASE IF EXISTS non_exists_db3 RESTRICT")); + Assert.assertEquals("drop schema to_drop_db4 restrict", + translator.translate("DROP DATABASE to_drop_db4 RESTRICT")); + + Assert.assertEquals("drop schema add_part_test_db", + translator.translate("DROP SCHEMA add_part_test_db")); + Assert.assertEquals("drop schema if exists statsdb1", + translator.translate("drop schema if exists statsdb1")); + Assert.assertEquals("drop schema to_drop_db1 cascade", + translator.translate("DROP SCHEMA to_drop_db1 CASCADE")); + Assert.assertEquals("drop schema if exists non_exists_db3 restrict", + translator.translate("DROP SCHEMA IF EXISTS non_exists_db3 RESTRICT")); + Assert.assertEquals("drop schema to_drop_db4 restrict", + translator.translate("DROP SCHEMA to_drop_db4 RESTRICT")); + } + + @Test + public void createTableLike() throws Exception { + Assert.assertEquals("create table alter3_like like alter3", + translator.translate("create table alter3_like like alter3")); + Assert.assertEquals("create table if not exists emp_orc like emp_staging", + translator.translate("create table if not exists emp_orc like emp_staging")); + Assert.assertEquals("create table source.srcpart like default.srcpart", + translator.translate("create table source.srcpart like default.srcpart;")); + Assert.assertFalse(translator.isFailureOk()); + } + + @Test + public void createTableAs() throws Exception { + Assert.assertEquals("create table src_stat as select * from src1", + translator.translate("create table src_stat as select * from src1")); + Assert.assertEquals("create table dest_grouped_old1 as select 1+1, " + + "2+2 as zz, src.key, src.value, count(src.value), count(src.value)" + + ", count(src.value), sum(value) from src group by src.key", + translator.translate("create table dest_grouped_old1 as select 1+1, " + + "2+2 as zz, src.key, src.value, count(src.value), count(src.value)" + + ", count(src.value), SUM(value) from src group by src.key")); + } + + @Test + public void createTableWithCols() throws Exception { + Assert.assertEquals("create table acidjoin1 (name varchar(50), age int)", + translator.translate("create table acidjoin1(name varchar(50), age int) clustered by " + + "(age) into 2 buckets stored as orc TBLPROPERTIES (\"transactional\"=\"true\")")); + Assert.assertEquals("create table alter1 (a int, b int)", + translator.translate("create table alter1(a int, b int)")); + Assert.assertEquals("create table alter2 (a int, b int)", + translator.translate("create table alter2(a int, b int) partitioned by (insertdate string)")); + Assert.assertEquals("create table alter3_src ( col1 varchar(255) )", + translator.translate("create table alter3_src ( col1 string ) stored as textfile ")); + Assert.assertEquals("create table alter3 ( col1 varchar(255) )", + translator.translate("create table alter3 ( col1 string ) partitioned by (pcol1 string , " + + "pcol2 string) stored as sequencefile")); + Assert.assertEquals("create table ac.alter_char_1 (key varchar(255), value varchar(255))", + translator.translate("create table ac.alter_char_1 (key string, value string)")); + Assert.assertEquals("create table tst1 (key varchar(255), value varchar(255))", + translator.translate("create table tst1(key string, value string) partitioned by (ds " + + "string) clustered by (key) into 10 buckets")); + Assert.assertEquals("create table over1k ( t smallint, si smallint, i int, b bigint, f real, " + + "d double precision, bo boolean, s varchar(255), ts timestamp, dec decimal(4,2), bin " + + "blob)", + translator.translate("create table over1k( t tinyint, si smallint, i int, b bigint, f " + + "float, d double, bo boolean, s string, ts timestamp, dec decimal(4,2), bin binary) " + + "ROW FORMAT DELIMITED FIELDS TERMINATED BY '|' STORED AS TEXTFILE")); + Assert.assertEquals("create table if not exists loc_staging (state varchar(255),locid int,zip" + + " bigint,year int )", + translator.translate("create table if not exists loc_staging (state string,locid int,zip " + + "bigint,year int ) row format delimited fields terminated by '|' stored as textfile")); + Assert.assertEquals("create temporary table acid_dtt (a int, b varchar(128))", + translator.translate("create temporary table acid_dtt(a int, b varchar(128)) clustered by" + + " (a) into 2 buckets stored as orc TBLPROPERTIES ('transactional'='true')")); + Assert.assertEquals("create table roottable (key varchar(255))", + translator.translate("create external table roottable (key string) row format delimited " + + "fields terminated by '\\t' stored as textfile")); + } + + @Test + public void dropTable() throws Exception { + Assert.assertEquals("drop table t", translator.translate("drop table t")); + Assert.assertEquals("drop table if exists t", translator.translate("drop table if exists t")); + Assert.assertFalse(translator.isFailureOk()); + Assert.assertEquals("drop table db.t", translator.translate("drop table db.t")); + Assert.assertEquals("drop table t", translator.translate("drop table t purge")); + } + + @Test + public void alterTable() throws Exception { + Assert.assertEquals("alter table tab1 rename to tab2", translator.translate("alter table tab1 rename to tab2")); + Assert.assertEquals("", translator.translate("alter table test set fileformat orc")); + Assert.assertEquals("", + translator.translate("alter table tst1 clustered by (key) into 8 buckets")); + Assert.assertEquals("", translator.translate("alter table fact_daily skewed by (key, value) on (('484','val_484'),('238','val_238')) stored as DIRECTORIES")); + Assert.assertEquals("", translator.translate("alter table skew_test.original3 not skewed")); + Assert.assertEquals("", translator.translate("alter table stored_as_dirs_multiple not stored as DIRECTORIES")); + Assert.assertEquals("", translator.translate("alter table T1 add partition (ds = 'today')")); + Assert.assertEquals("", translator.translate("alter table temp add if not exists partition (p ='p1')")); + + } + + @Test + public void selectSimple() throws Exception { + Assert.assertEquals("select * from add_part_test", + translator.translate("select * from add_part_test")); + Assert.assertEquals("select key, value from dest1", + translator.translate("select key, value from dest1")); + Assert.assertEquals("select count(key) from src", + translator.translate("select count(key) from src")); + Assert.assertEquals("select count(key), sum(key) from src", + translator.translate("select count(key), sum(key) from src")); + Assert.assertEquals("select sum(sin(key)), sum(cos(value)) from src_rc_concatenate_test", + translator.translate("select sum(sin(key)), sum(cos(value)) from src_rc_concatenate_test")); + Assert.assertEquals("select cast(key as int) / cast(key as varchar(255)) from src", + translator.translate("select cast(key as int) / cast(key as string) from src")); + Assert.assertEquals("select 1", translator.translate("select 1")); + Assert.assertEquals("select distinct l_partkey as p_partkey from lineitem", + translator.translate("select distinct l_partkey as p_partkey from Lineitem")); + Assert.assertEquals("select all l_partkey as p_partkey from lineitem", + translator.translate("select all l_partkey as p_partkey from Lineitem")); + } + + @Ignore + public void selectInterval() throws Exception { + thrown.expect(TranslationException.class); + thrown.expectMessage("Could not translate interval, Hive SQL:"); + translator.translate( + "select interval '10-11' year to month, interval '10' year, interval '11' month from src limit 1"); + } + + @Ignore + public void selectConstantCasts() throws Exception { + Assert.assertEquals("select dateval - date '1999-06-07' from interval_arithmetic_1", + translator.translate("select dateval - date '1999-06-07' from interval_arithmetic_1")); + Assert.assertEquals("select dateval - date '1999-06-07' from interval_arithmetic_1", + translator.translate("select dateval - date '1999-6-7' from interval_arithmetic_1")); + Assert.assertEquals("select timestamp '1999-01-01 01:00:00' from interval_arithmetic_1", + translator.translate("select timestamp '1999-01-01 01:00:00' from interval_arithmetic_1")); + Assert.assertEquals("select timestamp '1999-01-01 01:00:00' from interval_arithmetic_1", + translator.translate("select timestamp '1999-1-1 01:00:00' from interval_arithmetic_1")); + Assert.assertEquals("select 101, -101, 100, -100, 100.00 from t", + translator.translate("select 101Y, -101S, 100, -100L, 100.00BD from T")); + + } + + @Test + public void selectJoin() throws Exception { + Assert.assertEquals("select s.name, count(distinct registration) from studenttab10k s join " + + "votertab10k v on (s.name = v.name) group by s.name", + translator.translate("select s.name, count(distinct registration) from studenttab10k s " + + "join votertab10k v on (s.name = v.name) group by s.name")); + Assert.assertEquals("select count(*) from bucket_small a join bucket_big b on a.key = b.key", + translator.translate("select /*+ mapjoin(a) */ count(*) FROM bucket_small a JOIN bucket_big b ON a.key = b.key")); + Assert.assertEquals("select count(*) from tbl1 a left outer join tbl2 b on a.key = b.key", + translator.translate("select count(*) FROM tbl1 a LEFT OUTER JOIN tbl2 b ON a.key = b.key")); + + } + + @Test + public void selectFromSubquery() throws Exception { + Assert.assertEquals("select a, b from t", translator.translate("select a, b from default.t")); + Assert.assertEquals("select count(*) from (select a.key as key, a.value as val1, b.value " + + "as val2 from tbl1 a join tbl2 b on a.key = b.key) subq1", + translator.translate("select count(*) from ( select a.key as key, a.value as val1, b" + + ".value as val2 from tbl1 a join tbl2 b on a.key = b.key ) subq1")); + Assert.assertEquals("select count(*) from (select key, count(*) from (select a.key " + + "as key, a.value as val1, b.value as val2 from tbl1 a join tbl2 b on a.key = b.key) " + + "subq1 group by key) subq2", + translator.translate("select count(*) from ( select key, count(*) from ( select a" + + ".key as key, a.value as val1, b.value as val2 from tbl1 a join tbl2 b on a.key = b" + + ".key ) subq1 group by key ) subq2")); + Assert.assertEquals("select sum(subq.key) from (select a.key, a.value from " + + "src a where a.key > 10) subq join src tab on (subq.key = tab.key and subq.key > 20 and " + + "subq.value = tab.value) where tab.value < 200", + translator.translate("SELECT sum(subq.key) FROM (select a.key, a.value " + + "from src a where a.key > 10 ) subq JOIN src tab ON (subq.key = tab.key and subq.key " + + "> 20 and subq.value = tab.value) where tab.value < 200")); + Assert.assertEquals("select s1.k, count(*) from (select k from t) s1 join (select k from u) " + + "on s1.k = s2.k s2 group by s1.k", + translator.translate("select s1.k, count(*) from ( select k from t ) s1 join ( select k " + + "from u ) on s1.k = s2.k s2 group by s1.k")); + Assert.assertEquals("select src1.key, src1.cnt1, src2.cnt1 from (select key, count(*) as " + + "cnt1 from (select a.key as key, a.value as val1, b.value as val2 from tbl1 a join" + + " tbl2 b on a.key = b.key) subq1 group by key) src1 join (select key, count(*) as " + + "cnt1 from (select a.key as key, a.value as val1, b.value as val2 from tbl1 a join" + + " tbl2 b on a.key = b.key) subq2 group by key) src2 on src1.key = src2.key", + translator.translate("select src1.key, src1.cnt1, src2.cnt1 from ( select key, count(*)" + + " as cnt1 from ( select a.key as key, a.value as val1, b.value as val2 from " + + "tbl1 a join tbl2 b on a.key = b.key ) subq1 group by key ) src1 join ( select " + + "key, count(*) as cnt1 from ( select a.key as key, a.value as val1, b.value as " + + "val2 from tbl1 a join tbl2 b on a.key = b.key ) subq2 group by key ) src2 on src1" + + ".key = src2.key")); + } + + @Test + public void selectWhere() throws Exception { + Assert.assertEquals("select * from alter5 where dt='a'", + translator.translate("select * from alter5 where dt='a'")); + Assert.assertEquals("select hr, c1, length(c1) from alter_char2 where hr = 1", + translator.translate("select hr, c1, length(c1) from alter_char2 where hr = 1")); + Assert.assertEquals("select key, value, count(*) from src_cbo b where b.key in (select key " + + "from src_cbo where src_cbo.key > '8') group by key, value order by key", + translator.translate("select key, value, count(*) from src_cbo b where b.key in ( select " + + "key from src_cbo where src_cbo.key > '8' ) group by key, value order by key")); + } + + @Test + public void selectGroupBy() throws Exception { + Assert.assertEquals("select c1, count(*) from tmp1 group by c1", + translator.translate("select c1, count(*) from tmp1 group by c1")); + Assert.assertEquals("select k, count(*) from b group by k having count(*) > 100", + translator.translate("select k, count(*) from b group by k having count(*) > 100")); + Assert.assertEquals("select * from src_cbo b group by key, value having not exists (select a" + + ".key from src_cbo a where b.value = a.value and a.key = b.key and a.value > 'val_12')", + translator.translate("select * from src_cbo b group by key, value having not exists ( " + + "select a.key from src_cbo a where b.value = a.value and a.key = b.key and a" + + ".value > 'val_12' )")); + Assert.assertEquals("select key, value, count(*) from src_cbo b where b.key in (select key " + + "from src_cbo where src_cbo.key > '8') group by key, value having count(*) in (select" + + " count(*) from src_cbo s1 where s1.key > '9' group by s1.key) order by key", + translator.translate("select key, value, count(*) from src_cbo b where b.key in (select " + + "key from src_cbo where src_cbo.key > '8') group by key, value having count(*) in " + + "(select count(*) from src_cbo s1 where s1.key > '9' group by s1.key ) order by key")); + + } + + @Test + public void selectOrderBy() throws Exception { + Assert.assertEquals("select a, b from acid_vectorized order by a, b", + translator.translate("select a, b from acid_vectorized order by a, b")); + Assert.assertEquals("select c1, count(*) from tmp1 group by c1 order by c1", + translator.translate("select c1, count(*) from tmp1 group by c1 order by c1")); + } + + @Test + public void selectLimit() throws Exception { + Assert.assertEquals("select key from src_autho_test order by key limit 20", + translator.translate("select key from src_autho_test order by key limit 20")); + } + + @Test + public void selectUnion() throws Exception { + Assert.assertEquals("select key, value from u1 union all select key, value from u2", + translator.translate("select key, value from u1 union all select key, value FROM u2")); + Assert.assertEquals("select key, value from u1 union distinct select key, value from u2", + translator.translate("select key, value from u1 union distinct select key, value FROM u2")); + Assert.assertEquals("select key, value from u1 union all select key, value from u2 union all select key as key, value from u", + translator.translate("select key, value from u1 union all select key, value from u2 union all select key as key, value FROM u")); + Assert.assertEquals("select key from src1 union select key2 from src2 order by key", + translator.translate("select key from src1 union select key2 from src2 order BY key")); + Assert.assertEquals("select key from src1 union select key2 from src2 order by key limit 5", + translator.translate("select key from src1 union select key2 from src2 order BY key limit 5")); + } + + @Test + public void insert() throws Exception { + Assert.assertEquals( + "insert into acidjoin1 values ('aaa', 35), ('bbb', 32), ('ccc', 32), ('ddd', 35), ('eee', 32)", + translator.translate( + "insert into table acidjoin1 values ('aaa', 35), ('bbb', 32), ('ccc', 32), ('ddd', 35), ('eee', 32)")); + Assert.assertEquals( + "insert into acid_vectorized select cint, cstring1 from alltypesorc where cint is not null order by cint limit 10", + translator.translate( + "insert into table acid_vectorized select cint, cstring1 from alltypesorc where cint is not null order by cint limit 10")); + Assert.assertEquals( + "insert into ac.alter_char_1 select key, value from src order by key limit 5", + translator.translate( + "insert overwrite table ac.alter_char_1 select key, value from src order by key limit 5")); + Assert.assertEquals("insert into acid values(\"foo\", \"bar\")", + translator.translate( + "insert into table acid partition(ds='2008-04-08') values(\"foo\", \"bar\")")); + Assert.assertEquals("insert into acid select key,value,ds from srcpart", + translator.translate( + "insert into table acid partition(ds) select key,value,ds from srcpart")); + Assert.assertEquals("insert into tab_part select key,value from srcbucket_mapjoin_part", + translator.translate( + "insert overwrite table tab_part partition (ds='2008-04-08') select key,value from srcbucket_mapjoin_part")); + } + + @Test + public void update() throws Exception { + Assert.assertEquals("update t_auth_up set j = 0 where i > 0", + translator.translate("update t_auth_up set j = 0 where i > 0")); + Assert.assertEquals("update acid set value = 'bar'", + translator.translate("update acid set value = 'bar'")); + } + + @Test + public void delete() throws Exception { + Assert.assertEquals("delete from acid_iud", + translator.translate("delete from acid_iud")); + Assert.assertEquals("delete from acid where key = 'foo' and ds='2008-04-08'", + translator.translate("delete from acid where key = 'foo' and ds='2008-04-08'")); + } + + @Test + public void nullTranslator() throws Exception { + Assert.assertEquals("", translator.translate("show tables")); + Assert.assertEquals("", translator.translate("describe t")); + Assert.assertEquals("", translator.translate("explain select * from t")); + Assert.assertEquals("", translator.translate("analyze table src_rc_merge_test_stat compute statistics")); + Assert.assertEquals("", translator.translate("grant select on table src_auth_tmp to user hive_test_user")); + Assert.assertEquals("", translator.translate("revoke select on table src_autho_test from user hive_test_user")); + Assert.assertEquals("", translator.translate("create index t1_index on table t1(a) as 'COMPACT' WITH DEFERRED REBUILD")); + Assert.assertEquals("", translator.translate("alter index t1_index on t1 rebuild")); + Assert.assertEquals("", translator.translate("drop index src_index_2 on src")); + Assert.assertEquals("", translator.translate("create role role1")); + Assert.assertEquals("", translator.translate("drop role sRc_roLE")); + Assert.assertEquals("", translator.translate("set role ADMIN")); + Assert.assertEquals("", translator.translate("alter database db_alter_onr set owner user user1")); + Assert.assertEquals("", translator.translate("alter schema db_alter_onr set owner user user1")); + } + + @Test + public void createFunction() throws Exception { + thrown.expect(TranslationException.class); + thrown.expectMessage("Could not translate create function, Hive SQL:"); + translator.translate( + "create function lookup as 'org.apache.hadoop.hive.ql.udf.UDFFileLookup' using file 'hdfs:///tmp/udf_using/sales.txt'"); + } + + @Test + public void createTemporaryFunction() throws Exception { + thrown.expect(TranslationException.class); + thrown.expectMessage("Could not translate create function, Hive SQL:"); + translator.translate( + "create temporary function udtfCount2 as 'org.apache.hadoop.hive.contrib.udtf.example.GenericUDTFCount2'"); + } + + @Test + public void reloadFunction() throws Exception { + thrown.expect(TranslationException.class); + thrown.expectMessage("Could not translate reload function, Hive SQL:"); + translator.translate("reload function"); + } + + @Test + public void dropFunction() throws Exception { + thrown.expect(TranslationException.class); + thrown.expectMessage("Could not translate drop function, Hive SQL:"); + translator.translate("drop function perm_fn"); + } + + @Test + public void dropTemporaryFunction() throws Exception { + thrown.expect(TranslationException.class); + thrown.expectMessage("Could not translate drop function, Hive SQL:"); + translator.translate("drop temporary function matchpathtest"); + } + + @Test + public void quoting() throws Exception { + Assert.assertEquals("select * from t", + translator.translate("select * from t")); + Assert.assertEquals("select 'select from where' from t", + translator.translate("select 'select from where' from T")); + Assert.assertEquals("select \"select from where\" from \"table with a space\"", + translator.translate("select \"select from where\" from `table with a space`")); + Assert.assertEquals("select 'escaped ''quote' from t", + translator.translate("select 'escaped ''quote' from t")); + Assert.assertEquals("select 'ends on quote'", + translator.translate("select 'ends on quote'")); + Assert.assertEquals("select 'ends on escaped quote'''", + translator.translate("select 'ends on escaped quote'''")); + } + +} diff --git itests/capybara/src/test/java/org/apache/hive/test/capybara/infra/TestRandomDataGenerator.java itests/capybara/src/test/java/org/apache/hive/test/capybara/infra/TestRandomDataGenerator.java new file mode 100644 index 0000000..2b76351 --- /dev/null +++ itests/capybara/src/test/java/org/apache/hive/test/capybara/infra/TestRandomDataGenerator.java @@ -0,0 +1,360 @@ +/** + * 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.hive.test.capybara.infra; + +import org.apache.hive.test.capybara.data.Column; +import org.apache.hive.test.capybara.data.DataSet; +import org.apache.hive.test.capybara.data.Row; +import org.apache.hive.test.capybara.iface.TestTable; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import org.apache.hadoop.hive.metastore.api.FieldSchema; +import org.apache.hive.test.capybara.iface.DataGenerator; +import org.junit.Assert; +import org.junit.Test; + +import java.math.BigDecimal; +import java.sql.Date; +import java.sql.Timestamp; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.HashSet; +import java.util.Iterator; +import java.util.List; +import java.util.Set; +import java.util.SortedSet; +import java.util.TreeSet; + +public class TestRandomDataGenerator { + static final private Logger LOG = LoggerFactory.getLogger(TestRandomDataGenerator.class.getName()); + + @Test + public void nonPartitionedAllTypes() { + List cols = new ArrayList<>(); + cols.add(new FieldSchema("col1", "bigint", "")); + cols.add(new FieldSchema("col2", "int", "")); + cols.add(new FieldSchema("col3", "smallint", "")); + cols.add(new FieldSchema("col4", "tinyint", "")); + cols.add(new FieldSchema("col5", "float", "")); + cols.add(new FieldSchema("col6", "double", "")); + cols.add(new FieldSchema("col7", "decimal(12,4)", "")); + cols.add(new FieldSchema("col8", "date", "")); + cols.add(new FieldSchema("col9", "timestamp", "")); + cols.add(new FieldSchema("col10", "varchar(32)", "")); + cols.add(new FieldSchema("col11", "char(9)", "")); + cols.add(new FieldSchema("col12", "string", "")); + cols.add(new FieldSchema("col13", "boolean", "")); + cols.add(new FieldSchema("col14", "binary", "")); + TestTable table = TestTable.getBuilder("t1").setCols(cols).build(); + + RandomDataGenerator rand = new RandomDataGenerator(1); + + DataSet data = rand.generateData(table, 100); + + int rowCnt = 0; + int[] nullsSeen = new int[cols.size()]; + Arrays.fill(nullsSeen, 0); + Iterator strIter = data.stringIterator(",", "NULL", ""); + for (Row row : data) { + Assert.assertTrue(strIter.hasNext()); + LOG.debug("Row is " + strIter.next()); + // All the column sizes combined + rowCnt++; + Assert.assertEquals(cols.size(), row.size()); + + // Check each of the columns to make sure we got something valid. Many of tests are + // bogus; they are there to make sure the compiler doesn't outsmart us and optimize out the + // line. + for (int i = 0; i < row.size(); i++) { + if (row.get(i).isNull()) { + nullsSeen[i]++; + } else { + switch (i) { + case 0: Assert.assertTrue(row.get(i).asLong() >= Long.MIN_VALUE); break; + + case 1: Assert.assertTrue(row.get(i).asInt() >= Integer.MIN_VALUE); break; + + case 2: Assert.assertTrue(row.get(i).asShort() >= Short.MIN_VALUE); break; + + case 3: Assert.assertTrue(row.get(i).asByte() >= Byte.MIN_VALUE); break; + + case 4: + // For strange reasons I can't fathom the float value >= min_value fails + Assert.assertTrue(row.get(i).asFloat() != Float.NaN); break; + + case 5: Assert.assertTrue(row.get(i).asDouble() != Double.NaN); break; + + case 6: + BigDecimal bd = row.get(i).asBigDecimal(); + // Precision might be as low as scale, since BigDecimal.precision returns real + // precision for this instance. + Assert.assertTrue(bd.precision() >= 4 && bd.precision() < 13); + Assert.assertEquals(4, bd.scale()); + break; + + case 7: + Assert.assertTrue(row.get(i).asDate().after(new Date(-100000000000000000L))); break; + + case 8: + Assert.assertTrue(row.get(i).asTimestamp().after(new Timestamp(-10000000000000L))); + break; + + case 9: Assert.assertTrue(row.get(i).asString().length() <= 32); break; + + case 10: Assert.assertTrue(row.get(i).asString().length() <= 9); break; + + case 11: Assert.assertTrue(row.get(i).asString().length() <= 20); break; + + case 12: + Assert.assertTrue(row.get(i).asBoolean() || !row.get(i).asBoolean()); + break; + + case 13: + Assert.assertTrue(row.get(i).asBytes().length <= 100); break; + + default: throw new RuntimeException("Too many columns"); + + } + } + } + } + Assert.assertFalse(strIter.hasNext()); + + long totalSize = data.lengthInBytes(); + Assert.assertTrue("Expected totalSize > 102400, but was " + totalSize, totalSize >= 102400); + // For all rows we should have around 1% nulls + for (int i = 0; i < cols.size(); i++) { + LOG.debug("For column " + i + " nulls seen is " + nullsSeen[i] + " rowCnt is " + rowCnt); + Assert.assertTrue(nullsSeen[i] > rowCnt * 0.001 && nullsSeen[i] < rowCnt * 0.019); + } + } + + @Test + public void differentNumNulls() { + List cols = new ArrayList<>(); + cols.add(new FieldSchema("col1", "bigint", "")); + cols.add(new FieldSchema("col2", "int", "")); + TestTable table = TestTable.getBuilder("t2").setCols(cols).build(); + + RandomDataGenerator rand = new RandomDataGenerator(2); + + DataSet data = rand.generateData(table, 100, new double[]{0.05, 0.02}); + + int rowCnt = 0; + int[] nullsSeen = new int[cols.size()]; + Arrays.fill(nullsSeen, 0); + Iterator strIter = data.stringIterator(",", "NULL", ""); + for (Row row : data) { + LOG.debug("Row is " + strIter.next()); + rowCnt++; + Assert.assertEquals(cols.size(), row.size()); + + for (int i = 0; i < row.size(); i++) { + if (row.get(i).isNull()) { + nullsSeen[i]++; + } + } + } + + for (int i = 0; i < cols.size(); i++) { + LOG.debug("For column " + i + " nulls seen is " + nullsSeen[i] + " rowCnt is " + rowCnt); + } + Assert.assertTrue(nullsSeen[0] > rowCnt * 0.04 && nullsSeen[0] < rowCnt * 0.06); + Assert.assertTrue(nullsSeen[1] > rowCnt * 0.01 && nullsSeen[1] < rowCnt * 0.03); + } + + @Test + public void partitions() { + List cols = new ArrayList<>(); + cols.add(new FieldSchema("col1", "varchar(3)", "")); + cols.add(new FieldSchema("col2", "date", "")); + List partCols = new ArrayList<>(); + partCols.add(new FieldSchema("p1", "smallint", "")); + TestTable table = TestTable.getBuilder("t3") + .setCols(cols) + .setPartCols(partCols) + .setNumParts(5) + .build(); + + RandomDataGenerator rand = new RandomDataGenerator(3); + + DataSet data = rand.generateData(table, 100); + + Set partValsSeen = new HashSet<>(); + Iterator strIter = data.stringIterator(",", "NULL", ""); + for (Row row : data) { + LOG.debug("Row is " + strIter.next()); + Assert.assertEquals(cols.size() + partCols.size(), row.size()); + + partValsSeen.add(row.get(cols.size())); + } + + Assert.assertEquals(5, partValsSeen.size()); + } + + @Test + public void presetPartitions() { + List cols = new ArrayList<>(); + cols.add(new FieldSchema("col1", "varchar(3)", "")); + cols.add(new FieldSchema("col2", "date", "")); + List partCols = new ArrayList<>(); + partCols.add(new FieldSchema("p1", "smallint", "")); + DataGenerator partValGenerator = new StaticDataGenerator(Arrays.asList("1", "2", "3"), ","); + TestTable table = TestTable.getBuilder("t3") + .setCols(cols) + .setPartCols(partCols) + .setPartValsGenerator(partValGenerator) + .build(); + + RandomDataGenerator rand = new RandomDataGenerator(3); + + DataSet data = rand.generateData(table, 100); + + SortedSet partValsSeen = new TreeSet<>(); + Iterator strIter = data.stringIterator(",", "NULL", ""); + for (Row row : data) { + LOG.debug("Row is " + strIter.next()); + Assert.assertEquals(cols.size() + partCols.size(), row.size()); + + partValsSeen.add(row.get(cols.size())); + } + + Assert.assertEquals(3, partValsSeen.size()); + Iterator iter = partValsSeen.iterator(); + short nextPartValExpected = 1; + while (iter.hasNext()) { + Assert.assertEquals(nextPartValExpected++, iter.next().asShort()); + } + } + + @Test + public void autoPartitionCount() { + List cols = new ArrayList<>(); + cols.add(new FieldSchema("col1", "varchar(3)", "")); + cols.add(new FieldSchema("col2", "date", "")); + List partCols = new ArrayList<>(); + partCols.add(new FieldSchema("p1", "date", "")); + TestTable table = TestTable.getBuilder("t3") + .setCols(cols) + .setPartCols(partCols) + .build(); + + RandomDataGenerator rand = new RandomDataGenerator(4); + + DataSet data = rand.generateData(table, 100); + + Set partValsSeen = new HashSet<>(); + Iterator strIter = data.stringIterator(",", "NULL", ""); + for (Row row : data) { + LOG.debug("Row is " + strIter.next()); + Assert.assertEquals(cols.size() + partCols.size(), row.size()); + + partValsSeen.add(row.get(cols.size())); + } + + Assert.assertEquals(3, partValsSeen.size()); + } + + @Test + public void primaryKey() { + List cols = new ArrayList<>(); + cols.add(new FieldSchema("col1", "varchar(3)", "")); + cols.add(new FieldSchema("col2", "date", "")); + List partCols = new ArrayList<>(); + partCols.add(new FieldSchema("p1", "smallint", "")); + TestTable table = TestTable.getBuilder("t4") + .setCols(cols) + .setPartCols(partCols) + .setPrimaryKey(new TestTable.PrimaryKey(0)) + .build(); + + RandomDataGenerator rand = new RandomDataGenerator(4); + + DataSet data = rand.generateData(table, 100); + + int rowCnt = 0; + Set pkSeen = new HashSet<>(); + Iterator strIter = data.stringIterator(",", "NULL", ""); + for (Row row : data) { + rowCnt++; + LOG.debug("Row is " + strIter.next()); + Assert.assertEquals(cols.size() + partCols.size(), row.size()); + + Assert.assertTrue(pkSeen.add(row.get(0))); + } + Assert.assertEquals(rowCnt, pkSeen.size()); + } + + @Test + public void sequence() { + List cols = new ArrayList<>(); + cols.add(new FieldSchema("col1", "bigint", "")); + cols.add(new FieldSchema("col2", "date", "")); + List partCols = new ArrayList<>(); + partCols.add(new FieldSchema("p1", "smallint", "")); + TestTable table = TestTable.getBuilder("t4") + .setCols(cols) + .setPartCols(partCols) + .setPrimaryKey(new TestTable.Sequence(0)) + .build(); + + RandomDataGenerator rand = new RandomDataGenerator(4); + + DataSet data = rand.generateData(table, 100); + + long nextSequence = 1; + Iterator strIter = data.stringIterator(",", "NULL", ""); + for (Row row : data) { + LOG.debug("sequence row is " + strIter.next()); + Assert.assertEquals(cols.size() + partCols.size(), row.size()); + + Assert.assertEquals(nextSequence++, row.get(0).asLong()); + } + } + + @Test + public void foreignKey() { + TestTable dim = TestTable.getBuilder("dim") + .addCol("col1", "bigint") + .addCol("col2", "date") + .setPrimaryKey(new TestTable.Sequence(0)) + .build(); + + RandomDataGenerator rand = new RandomDataGenerator(4); + + DataSet dimData = rand.generateData(dim, 5); + + int dimRowCnt = 0; + for (Row dimRow : dimData) dimRowCnt++; + LOG.debug("Number of dimension rows " + dimRowCnt); + + TestTable fact = TestTable.getBuilder("fact") + .addCol("pk", "int") + .addCol("fk", "bigint") + .addForeignKey(new TestTable.ForeignKey(dimData, 0, 1)) + .build(); + + DataSet factData = rand.generateData(fact, 100); + + Iterator strIter = factData.stringIterator(",", "NULL", ""); + for (Row row : factData) { + LOG.debug("Row is " + strIter.next()); + Assert.assertTrue(row.get(1).asLong() <= dimRowCnt); + } + } +} diff --git itests/capybara/src/test/java/org/apache/hive/test/capybara/infra/TestSortingComparator.java itests/capybara/src/test/java/org/apache/hive/test/capybara/infra/TestSortingComparator.java new file mode 100644 index 0000000..62285d0 --- /dev/null +++ itests/capybara/src/test/java/org/apache/hive/test/capybara/infra/TestSortingComparator.java @@ -0,0 +1,216 @@ +/** + * 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.hive.test.capybara.infra; + +import org.apache.hadoop.hive.metastore.api.FieldSchema; +import org.apache.hive.test.capybara.data.DataSet; +import org.apache.hive.test.capybara.iface.DataGenerator; +import org.apache.hive.test.capybara.iface.ResultComparator; +import org.apache.hive.test.capybara.iface.TestTable; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; + +import java.io.IOException; +import java.sql.SQLException; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; +import java.util.ListIterator; +import java.util.Random; + +public class TestSortingComparator { + + private TestTable table; + private List rows; + private List cols; + + @Before + public void createTable() { + final String tableName = "alltypes"; + + + cols = Arrays.asList( + new FieldSchema("col_bi", "bigint", ""), + new FieldSchema("col_i", "int", "") + ); + table = TestTable.getBuilder(tableName).setCols(cols).build(); + + rows = new ArrayList<>(); + rows.add("6,2"); + rows.add("6,3"); + rows.add("-6,37"); + } + + @Test + public void allGood() throws SQLException, IOException { + DataGenerator gen1 = new StaticDataGenerator(rows, ","); + DataSet hive = gen1.generateData(table); + DataGenerator gen2 = new StaticDataGenerator(rows, ","); + DataSet bench = gen2.generateData(table); + ResultComparator comparator = new SortingComparator(); + comparator.compare(hive, bench); + } + + @Test + public void allGoodOrderDifferent() throws SQLException, IOException { + DataGenerator gen1 = new StaticDataGenerator(rows, ","); + DataSet hive = gen1.generateData(table); + List differentOrder = new ArrayList<>(); + differentOrder.add(rows.get(2)); + differentOrder.add(rows.get(1)); + differentOrder.add(rows.get(0)); + DataGenerator gen2 = new StaticDataGenerator(differentOrder, ","); + DataSet bench = gen2.generateData(table); + ResultComparator comparator = new SortingComparator(); + comparator.compare(hive, bench); + } + + @Test + public void diffNumberColumns() throws Exception { + + TestTable diffTable = TestTable.getBuilder("difftable") + .addCol("col_bi", "bigint") + .build(); + + List diffRows = new ArrayList<>(); + diffRows.add("6"); + diffRows.add("7"); + diffRows.add("6"); + DataGenerator gen1 = new StaticDataGenerator(rows, ","); + DataSet hive = gen1.generateData(table); + DataGenerator gen2 = new StaticDataGenerator(diffRows, ","); + DataSet bench = gen2.generateData(diffTable); + ResultComparator comparator = new SortingComparator(); + boolean sawError = false; + try { + comparator.compare(hive, bench); + } catch (AssertionError e) { + Assert.assertEquals("Different number of columns expected:<1> but was:<2>", e.getMessage()); + sawError = true; + } + Assert.assertTrue(sawError); + } + + @Test + public void uncompatibleDataTypes() throws Exception { + + TestTable diffTable = TestTable.getBuilder("difftable") + .addCol("col_bi", "bigint") + .addCol("col_i", "string") + .build(); + + List diffRows = new ArrayList<>(); + diffRows.add("6,mary had a little lamb"); + diffRows.add("-6022141300000000000,def"); + DataGenerator gen1 = new StaticDataGenerator(rows, ","); + DataSet hive = gen1.generateData(table); + DataGenerator gen2 = new StaticDataGenerator(diffRows, ","); + DataSet bench = gen2.generateData(diffTable); + ResultComparator comparator = new SortingComparator(); + boolean sawError = false; + try { + comparator.compare(hive, bench); + } catch (AssertionError e) { + Assert.assertEquals("Found discrepency in metadata at column 1", e.getMessage()); + sawError = true; + } + Assert.assertTrue(sawError); + } + + @Test + public void diffNull() throws Exception { + List diffRows = new ArrayList<>(); + diffRows.add("NULL,2"); + diffRows.add("-6,37"); + diffRows.add("6,3"); + DataGenerator gen1 = new StaticDataGenerator(rows, ","); + DataSet hive = gen1.generateData(table); + DataGenerator gen2 = new StaticDataGenerator(diffRows, ","); + DataSet bench = gen2.generateData(table); + ResultComparator comparator = new SortingComparator(); + boolean sawError = false; + try { + comparator.compare(hive, bench); + } catch (AssertionError e) { + Assert.assertEquals("Mismatch at row 1 hive row is <-6,37> bench row is ", + e.getMessage()); + sawError = true; + } + Assert.assertTrue(sawError); + } + + @Test + public void diff() throws Exception { + List diffRows = new ArrayList<>(); + diffRows.add("-6,37"); + diffRows.add("6,3"); + diffRows.add("6,3"); + DataGenerator gen1 = new StaticDataGenerator(rows, ","); + DataSet hive = gen1.generateData(table); + DataGenerator gen2 = new StaticDataGenerator(diffRows, ","); + DataSet bench = gen2.generateData(table); + ResultComparator comparator = new SortingComparator(); + boolean sawError = false; + try { + comparator.compare(hive, bench); + } catch (AssertionError e) { + Assert.assertEquals("Mismatch at row 2 hive row is <6,2> bench row is <6,3>", + e.getMessage()); + sawError = true; + } + Assert.assertTrue(sawError); + } + + @Test + public void big() throws Exception { + List spillRows = new ArrayList<>(); + Random rand = new Random(1); + for (int i = 0; i < 937; i++) { + spillRows.add(Long.toString(rand.nextInt(10)) + "," + Long.toString(rand.nextInt(10))); + } + List reverseRows = new ArrayList<>(); + ListIterator iter = spillRows.listIterator(spillRows.size()); + while (iter.hasPrevious()) reverseRows.add(iter.previous()); + + assert spillRows.size() == reverseRows.size(); + + DataGenerator gen1 = new StaticDataGenerator(spillRows, ","); + DataSet hive = gen1.generateData(table); + DataGenerator gen2 = new StaticDataGenerator(reverseRows, ","); + DataSet bench = gen2.generateData(table); + SortingComparator comparator = new SortingComparator(); + comparator.compare(hive, bench); + } + + @Test + public void setSchema() throws Exception { + DataGenerator gen1 = new StaticDataGenerator(rows, ","); + DataSet bench = gen1.generateData(table); + + List cliRows = new ArrayList<>(); + cliRows.add("-6,37"); + cliRows.add("6,2"); + cliRows.add("6,3"); + + DataSet hive = new StringDataSet(cliRows, ",", "NULL"); + hive.setSchema(cols); + ResultComparator comparator = new SortingComparator(); + comparator.compare(hive, bench); + } +} diff --git itests/capybara/src/test/java/org/apache/hive/test/capybara/infra/TestSpillingGeneratedDataSet.java itests/capybara/src/test/java/org/apache/hive/test/capybara/infra/TestSpillingGeneratedDataSet.java new file mode 100644 index 0000000..66ea4d5 --- /dev/null +++ itests/capybara/src/test/java/org/apache/hive/test/capybara/infra/TestSpillingGeneratedDataSet.java @@ -0,0 +1,45 @@ +/** + * 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.hive.test.capybara.infra; + +import org.apache.hive.test.capybara.iface.DataGenerator; +import org.apache.hive.test.capybara.IntegrationTest; +import org.apache.hive.test.capybara.iface.TestTable; +import org.junit.Test; + +public class TestSpillingGeneratedDataSet extends IntegrationTest { + + @Test + public void spill() throws Exception { + + System.setProperty(TestConf.SPILL_SIZE_PROPERTY, Integer.toString(1024 * 128)); + TestTable table = TestTable.getBuilder("spill") + .addCol("c1", "varchar(120)") + .addCol("c2", "int") + .addCol("c3", "decimal(12,2)") + .build(); + + DataGenerator generator = new RandomDataGenerator(23); + table.create(); + table.populate(generator, 500, null); + + runQuery("select count(*) from spill"); + compare(); + } + +} diff --git itests/capybara/src/test/java/org/apache/hive/test/capybara/infra/TestStaticDataGenerator.java itests/capybara/src/test/java/org/apache/hive/test/capybara/infra/TestStaticDataGenerator.java new file mode 100644 index 0000000..b7a8a4a --- /dev/null +++ itests/capybara/src/test/java/org/apache/hive/test/capybara/infra/TestStaticDataGenerator.java @@ -0,0 +1,128 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hive.test.capybara.infra; + +import org.apache.commons.codec.binary.Base64; +import org.apache.hadoop.hive.metastore.api.FieldSchema; +import org.apache.hive.test.capybara.data.DataSet; +import org.apache.hive.test.capybara.data.Row; +import org.apache.hive.test.capybara.iface.DataGenerator; +import org.apache.hive.test.capybara.iface.TestTable; +import org.junit.Assert; +import org.junit.Test; + +import java.math.BigDecimal; +import java.sql.Date; +import java.sql.SQLException; +import java.sql.Timestamp; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Iterator; +import java.util.List; + +public class TestStaticDataGenerator { + + @Test + public void allTypes() throws SQLException { + + final String tableName = "alltypes"; + + + List cols = Arrays.asList( + new FieldSchema("col_bi", "bigint", ""), + new FieldSchema("col_i", "int", ""), + new FieldSchema("col_si", "smallint", ""), + new FieldSchema("col_ti", "tinyint", ""), + new FieldSchema("col_bin", "binary", ""), + new FieldSchema("col_bool", "boolean", ""), + new FieldSchema("col_ch", "char(8)", ""), + new FieldSchema("col_vc", "varchar(89)", ""), + new FieldSchema("col_str", "string", ""), + new FieldSchema("col_date", "date", ""), + new FieldSchema("col_dec", "decimal(10,2)", ""), + new FieldSchema("col_fl", "float", ""), + new FieldSchema("col_dbl", "double", ""), + new FieldSchema("col_tm", "timestamp", "") + ); + + TestTable table = TestTable.getBuilder(tableName).setCols(cols).build(); + + List rows = new ArrayList<>(); + rows.add("6022141300000000000,299792458,1432,7,abc,true,bob,mary had a little lamb,her" + + " fleece was white as snow,2015-08-04,371.89,1.234,6.0221413e+23,2015-08-04 17:16:32"); + rows.add("NULL,NULL,NULL,NULL,NULL,NULL,NULL,NULL,NULL,NULL,NULL,NULL,NULL,NULL"); + DataGenerator gen = new StaticDataGenerator(rows, ","); + DataSet ds = gen.generateData(table); + + Iterator iter = ds.iterator(); + Assert.assertTrue(iter.hasNext()); + Row row = iter.next(); + Assert.assertEquals(6022141300000000000L, row.get(0).asLong()); + Assert.assertEquals(299792458, row.get(1).asInt()); + Assert.assertEquals((short)1432, row.get(2).asShort()); + Assert.assertEquals((byte) 7, row.get(3).asByte()); + // Have to wash this back and forth through the encode/decoder to get the same result. + Assert.assertEquals("abc", Base64.encodeBase64URLSafeString(row.get(4).asBytes())); + Assert.assertEquals(true, row.get(5).asBoolean()); + Assert.assertEquals("bob", row.get(6).asString()); + Assert.assertEquals("mary had a little lamb", row.get(7).asString()); + Assert.assertEquals("her fleece was white as snow", row.get(8).asString()); + Assert.assertEquals(Date.valueOf("2015-08-04"), row.get(9).asDate()); + Assert.assertEquals(new BigDecimal("371.89"), row.get(10).asBigDecimal()); + Assert.assertEquals(1.234f, row.get(11).asFloat(), 0.01); + Assert.assertEquals(6.0221413e+23, row.get(12).asDouble(), 1e+16); + Assert.assertEquals(Timestamp.valueOf("2015-08-04 17:16:32"), row.get(13).asTimestamp()); + + Assert.assertTrue(iter.hasNext()); + row = iter.next(); + for (int i = 0; i < cols.size(); i++) { + Assert.assertTrue(row.get(i).isNull()); + } + Assert.assertFalse(iter.hasNext()); + } + + @Test + public void partitioned() throws SQLException { + + final String tableName = "alltypes"; + + + TestTable table = TestTable.getBuilder(tableName) + .addCol("col_bi", "bigint") + .addPartCol("partcol", "string") + .build(); + + List rows = new ArrayList<>(); + rows.add("6,na"); + rows.add("7,eu"); + DataGenerator gen = new StaticDataGenerator(rows, ","); + DataSet ds = gen.generateData(table); + + Iterator iter = ds.iterator(); + Assert.assertTrue(iter.hasNext()); + Row row = iter.next(); + Assert.assertEquals(6L, row.get(0).asLong()); + Assert.assertEquals("na", row.get(1).asString()); + + Assert.assertTrue(iter.hasNext()); + row = iter.next(); + Assert.assertEquals(7L, row.get(0).asLong()); + Assert.assertEquals("eu", row.get(1).asString()); + Assert.assertFalse(iter.hasNext()); + } +} diff --git itests/capybara/src/test/java/org/apache/hive/test/capybara/infra/TestStatsDataGenerator.java itests/capybara/src/test/java/org/apache/hive/test/capybara/infra/TestStatsDataGenerator.java new file mode 100644 index 0000000..33fa5e2 --- /dev/null +++ itests/capybara/src/test/java/org/apache/hive/test/capybara/infra/TestStatsDataGenerator.java @@ -0,0 +1,463 @@ +/** + * 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.hive.test.capybara.infra; + +import org.apache.hive.test.capybara.data.DataSet; +import org.apache.hive.test.capybara.data.Row; +import org.apache.hive.test.capybara.iface.TestTable; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import org.apache.hadoop.hive.metastore.api.FieldSchema; +import org.junit.Assert; +import org.junit.Ignore; +import org.junit.Test; + +import java.math.BigDecimal; +import java.sql.Date; +import java.sql.Timestamp; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.HashMap; +import java.util.HashSet; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.Set; + +public class TestStatsDataGenerator { + static final private Logger LOG = LoggerFactory.getLogger(TestStatsDataGenerator.class.getName()); + + @Test + public void scale() { + List cols = new ArrayList<>(); + cols.add(new FieldSchema("col1", "bigint", "")); + TestTable table = TestTable.getBuilder("t1").setCols(cols).build(); + Map colStats = new HashMap<>(); + colStats.put("col1", new StatsDataGenerator.ColStats(cols.get(0).getName(), + cols.get(0).getType(), -37, 37, 0, 0, 0, 0, 51, 16)); + StatsDataGenerator.TableStats tableStats = new StatsDataGenerator.TableStats(colStats, 0, + table.getDbName(), table.getTableName(), 1024, 128); + tableStats.scale(1); + Assert.assertEquals(128, tableStats.numRows); + Assert.assertEquals(0, tableStats.numPartitions); + Assert.assertEquals(51, tableStats.colStats.get("col1").distinctCount); + Assert.assertEquals(0.125, tableStats.colStats.get("col1").pctNull, 0.005); + + colStats = new HashMap<>(); + colStats.put("col1", new StatsDataGenerator.ColStats(cols.get(0).getName(), + cols.get(0).getType(), -37, 37, 0, 0, 0, 0, 52, 4)); + tableStats = new StatsDataGenerator.TableStats(colStats, 0, + table.getDbName(), table.getTableName(), 1024, 128); + tableStats.scale(10); + Assert.assertEquals(1280, tableStats.numRows); + Assert.assertEquals(0, tableStats.numPartitions); + Assert.assertEquals(520, tableStats.colStats.get("col1").distinctCount); + Assert.assertEquals(0.03, tableStats.colStats.get("col1").pctNull, 0.005); + + colStats = new HashMap<>(); + colStats.put("col1", new StatsDataGenerator.ColStats(cols.get(0).getName(), + cols.get(0).getType(), -37, 37, 0, 0, 0, 0, 51, 7)); + tableStats = new StatsDataGenerator.TableStats(colStats, 0, + table.getDbName(), table.getTableName(), 4096, 512); + tableStats.scale(1); + Assert.assertEquals(128, tableStats.numRows); + Assert.assertEquals(0, tableStats.numPartitions); + Assert.assertEquals(12, tableStats.colStats.get("col1").distinctCount); + Assert.assertEquals(0.015, tableStats.colStats.get("col1").pctNull, 0.005); + + // Test going from very large data to low scale, make sure we hit minimums. + colStats = new HashMap<>(); + colStats.put("col1", new StatsDataGenerator.ColStats(cols.get(0).getName(), + cols.get(0).getType(), -37, 37, 0, 0, 0, 0, 519, 367)); + tableStats = new StatsDataGenerator.TableStats(colStats, 0, + table.getDbName(), table.getTableName(), 104857600000000L, 1310720L); + tableStats.scale(1); + Assert.assertEquals(100, tableStats.numRows); + Assert.assertEquals(0, tableStats.numPartitions); + Assert.assertEquals(10, tableStats.colStats.get("col1").distinctCount); + Assert.assertEquals(0.0003, tableStats.colStats.get("col1").pctNull, 0.00005); + + // Make sure it works with partitions too + table = TestTable.getBuilder("t1").setCols(cols).addPartCol("partcol", "string").build(); + colStats = new HashMap<>(); + colStats.put("col1", new StatsDataGenerator.ColStats(cols.get(0).getName(), + cols.get(0).getType(), -37, 37, 0, 0, 0, 0, 59, 37)); + tableStats = new StatsDataGenerator.TableStats(colStats, 10, + table.getDbName(), table.getTableName(), 1024, 128); + tableStats.scale(1); + Assert.assertEquals(10, tableStats.numPartitions); + + colStats = new HashMap<>(); + colStats.put("col1", new StatsDataGenerator.ColStats(cols.get(0).getName(), + cols.get(0).getType(), -37, 37, 0, 0, 0, 0, 519, 367)); + tableStats = new StatsDataGenerator.TableStats(colStats, 10, + table.getDbName(), table.getTableName(), 1024, 128); + tableStats.scale(10); + Assert.assertEquals(100, tableStats.numPartitions); + + colStats = new HashMap<>(); + colStats.put("col1", new StatsDataGenerator.ColStats(cols.get(0).getName(), + cols.get(0).getType(), -37, 37, 0, 0, 0, 0, 519, 367)); + tableStats = new StatsDataGenerator.TableStats(colStats, 100, + table.getDbName(), table.getTableName(), 10240, 1280); + tableStats.scale(1); + Assert.assertEquals(10, tableStats.numPartitions); + + colStats = new HashMap<>(); + colStats.put("col1", new StatsDataGenerator.ColStats(cols.get(0).getName(), + cols.get(0).getType(), -37, 37, 0, 0, 0, 0, 519, 367)); + tableStats = new StatsDataGenerator.TableStats(colStats, 10, + table.getDbName(), table.getTableName(), 1048576000000000L, 1310720); + tableStats.scale(1); + Assert.assertEquals(2, tableStats.numPartitions); + } + + @Test + public void nonPartitionedAllTypes() { + List cols = new ArrayList<>(); + cols.add(new FieldSchema("col1", "bigint", "")); + cols.add(new FieldSchema("col2", "int", "")); + cols.add(new FieldSchema("col3", "smallint", "")); + cols.add(new FieldSchema("col4", "tinyint", "")); + cols.add(new FieldSchema("col5", "float", "")); + cols.add(new FieldSchema("col6", "double", "")); + cols.add(new FieldSchema("col7", "decimal(12,2)", "")); + cols.add(new FieldSchema("col8", "date", "")); + cols.add(new FieldSchema("col9", "timestamp", "")); + cols.add(new FieldSchema("col10", "varchar(32)", "")); + cols.add(new FieldSchema("col11", "char(9)", "")); + cols.add(new FieldSchema("col12", "string", "")); + cols.add(new FieldSchema("col13", "boolean", "")); + cols.add(new FieldSchema("col14", "binary", "")); + TestTable table = TestTable.getBuilder("t1").setCols(cols).build(); + + Map colStats = new HashMap<>(); + long[] distinctCounts = new long[cols.size()]; + long[] nullCounts = new long[cols.size()]; + + long minLong = -37L; + long maxLong = 82387563739L; + distinctCounts[0] = 317; + nullCounts[0] = 53; + colStats.put("col1", new StatsDataGenerator.ColStats(cols.get(0).getName(), + cols.get(0).getType(), minLong, maxLong, 0, 0, 0, 0, distinctCounts[0], nullCounts[0])); + + int minInt = -98; + int maxInt = 7619234; + distinctCounts[1] = 127; + nullCounts[1] = 47; + colStats.put("col2", new StatsDataGenerator.ColStats(cols.get(1).getName(), + cols.get(1).getType(), minInt, maxInt, 0, 0, 0, 0, distinctCounts[1], nullCounts[1])); + + + short minShort = -988; + short maxShort = 7134; + distinctCounts[2] = 227; + nullCounts[2] = 171; + colStats.put("col3", new StatsDataGenerator.ColStats(cols.get(2).getName(), + cols.get(2).getType(), minShort, maxShort, 0, 0, 0, 0, distinctCounts[2], nullCounts[2])); + + byte minByte = -88; + byte maxByte = 34; + distinctCounts[3] = 17; + nullCounts[3] = 61; + colStats.put("col4", new StatsDataGenerator.ColStats(cols.get(3).getName(), + cols.get(3).getType(), minByte, maxByte, 0, 0, 0, 0, distinctCounts[3], nullCounts[3])); + + float minFloat = -9.098e8f; + float maxFloat = 3.876363e4f; + distinctCounts[4] = 135; + nullCounts[4] = 198; + colStats.put("col5", new StatsDataGenerator.ColStats(cols.get(4).getName(), + cols.get(4).getType(), minFloat, maxFloat, 0, 0, 0, 0, distinctCounts[4], nullCounts[4])); + + double minDouble = -9.098e18; + double maxDouble = 3.876363e14; + distinctCounts[5] = 620; + nullCounts[5] = 98; + colStats.put("col6", new StatsDataGenerator.ColStats(cols.get(5).getName(), + cols.get(5).getType(), minDouble, maxDouble, 0, 0, 0, 0, distinctCounts[5], nullCounts[5])); + + BigDecimal minDecimal = new BigDecimal("-1098098.23"); + BigDecimal maxDecimal = new BigDecimal("2983928293.12"); + distinctCounts[6] = 352; + nullCounts[6] = 0; + colStats.put("col7", new StatsDataGenerator.ColStats(cols.get(6).getName(), + cols.get(6).getType(), minDecimal, maxDecimal, 0, 0, 0, 0, distinctCounts[6], nullCounts[6])); + + Date minDate = new Date(-1098098); + Date maxDate = new Date(298392893); + distinctCounts[7] = 52; + nullCounts[7] = 1; + colStats.put("col8", new StatsDataGenerator.ColStats(cols.get(7).getName(), + cols.get(7).getType(), minDate, maxDate, 0, 0, 0, 0, distinctCounts[7], nullCounts[7])); + + Timestamp minTimestamp = new Timestamp(-198098); + Timestamp maxTimestamp = new Timestamp(198392893); + distinctCounts[8] = 152; + nullCounts[8] = 10; + colStats.put("col9", new StatsDataGenerator.ColStats(cols.get(8).getName(), + cols.get(8).getType(), minTimestamp, maxTimestamp, 0, 0, 0, 0, distinctCounts[8], nullCounts[8])); + + long maxVarcharLen = 30; + double avgVarcharLen = 25.0; // weight it toward one end to see if our average works. + distinctCounts[9] = 122; + nullCounts[9] = 10; + colStats.put("col10", new StatsDataGenerator.ColStats(cols.get(9).getName(), + cols.get(9).getType(), null, null, avgVarcharLen, maxVarcharLen, 0, 0, distinctCounts[9], + nullCounts[9])); + + long maxCharLen = 7; + double avgCharLen = 2.5; // weight it toward one end to see if our average works. + distinctCounts[10] = 72; + nullCounts[10] = 0; + colStats.put("col11", new StatsDataGenerator.ColStats(cols.get(10).getName(), + cols.get(10).getType(), null, null, avgCharLen, maxCharLen, 0, 0, distinctCounts[10], + nullCounts[10])); + + long maxStringLen = 79; + double avgStringLen = 42.15; + distinctCounts[11] = 172; + nullCounts[11] = 3; + colStats.put("col12", new StatsDataGenerator.ColStats(cols.get(11).getName(), + cols.get(11).getType(), null, null, avgStringLen, maxStringLen, 0, 0, distinctCounts[11], + nullCounts[11])); + + long numTrues = 284; + long numFalses = 400; + nullCounts[12] = 100; + colStats.put("col13", new StatsDataGenerator.ColStats(cols.get(12).getName(), + cols.get(12).getType(), null, null, 0, 0, numFalses, numTrues, 0, nullCounts[12])); + + long maxBinaryLen = 179; + double avgBinaryLen = 12.15; // weight it toward one end to see if our average works. + nullCounts[13] = 3; + colStats.put("col14", new StatsDataGenerator.ColStats(cols.get(13).getName(), + cols.get(13).getType(), null, null, avgBinaryLen, maxBinaryLen, 0, 0, 0, nullCounts[13])); + + // Set the number of rows to match the passed in scale of 100 so our expected counts don't + // change significantly. + StatsDataGenerator.TableStats tableStats = new StatsDataGenerator.TableStats(colStats, 0, + table.getDbName(), table.getTableName(), 102400, 784); + + StatsDataGenerator gen = new StatsDataGenerator(tableStats, 1); + + DataSet data = gen.generateData(table, 100); + + int rowCnt = 0; + int[] nullsSeen = new int[cols.size()]; + Arrays.fill(nullsSeen, 0); + Set distinctLongs = new HashSet<>(); + Set distinctIntegers = new HashSet<>(); + Set distinctShorts = new HashSet<>(); + Set distinctBytes = new HashSet<>(); + Set distinctFloats = new HashSet<>(); + Set distinctDoubles = new HashSet<>(); + Set distinctBigDecimals = new HashSet<>(); + Set distinctDates = new HashSet<>(); + Set distinctTimestamps = new HashSet<>(); + Set distinctVarchars = new HashSet<>(); + Set distinctChars = new HashSet<>(); + Set distinctStrings = new HashSet<>(); + long varcharLen, charLen, strLen, binLen, truesSeen, falsesSeen; + varcharLen = charLen = strLen = binLen = truesSeen = falsesSeen = 0; + + Iterator strIter = data.stringIterator(",", "NULL", ""); + for (Row row : data) { + Assert.assertTrue(strIter.hasNext()); + LOG.debug("Row is " + strIter.next()); + // All the column sizes combined + rowCnt++; + Assert.assertEquals(cols.size(), row.size()); + + for (int i = 0; i < row.size(); i++) { + if (row.get(i).isNull()) { + nullsSeen[i]++; + } else { + switch (i) { + case 0: + Assert.assertTrue("Expected long between " + minLong + " and " + maxLong + " got " + + row.get(i).asLong(), + row.get(i).asLong() >= minLong && row.get(i).asLong() <= maxLong); + distinctLongs.add(row.get(i).asLong()); + break; + + case 1: + Assert.assertTrue(row.get(i).asInt() >= minInt && row.get(i).asInt() <= maxInt); + distinctIntegers.add(row.get(i).asInt()); + break; + + case 2: + Assert.assertTrue(row.get(i).asShort() >= minShort && row.get(i).asShort() <= maxShort); + distinctShorts.add(row.get(i).asShort()); + break; + + case 3: + Assert.assertTrue(row.get(i).asByte() >= minByte && row.get(i).asByte() <= maxByte); + distinctBytes.add(row.get(i).asByte()); + break; + + case 4: + Assert.assertTrue("Expected float between " + minFloat + " and " + maxFloat + " got " + + row.get(i).asFloat(), + row.get(i).asFloat() >= minFloat && row.get(i).asFloat() <= maxFloat); + distinctFloats.add(row.get(i).asFloat()); + break; + + case 5: + Assert.assertTrue(row.get(i).asDouble() >= minDouble && row.get(i).asDouble() <= maxDouble); + distinctDoubles.add(row.get(i).asDouble()); + break; + + case 6: + Assert.assertTrue("Expected big decimal between " + minDecimal + " and " + maxDecimal + + " got " + row.get(i).asBigDecimal(), + row.get(i).asBigDecimal().compareTo(minDecimal) >= 0 && + row.get(i).asBigDecimal().compareTo(maxDecimal) <= 0); + distinctBigDecimals.add(row.get(i).asBigDecimal()); + break; + + case 7: + Assert.assertTrue(row.get(i).asDate().compareTo(minDate) >= 0 && + row.get(i).asDate().compareTo(maxDate) <= 0); + distinctDates.add(row.get(i).asDate()); + break; + + case 8: + Assert.assertTrue(row.get(i).asTimestamp().compareTo(minTimestamp) >= 0 && + row.get(i).asTimestamp().compareTo(maxTimestamp) <= 0); + distinctTimestamps.add(row.get(i).asTimestamp()); + break; + + case 9: + Assert.assertTrue(row.get(i).asString().length() <= maxVarcharLen); + distinctVarchars.add(row.get(i).asString()); + varcharLen += row.get(i).length(); + break; + + case 10: + Assert.assertTrue(row.get(i).asString().length() <= maxCharLen); + distinctChars.add(row.get(i).asString()); + charLen += row.get(i).length(); + break; + + case 11: + Assert.assertTrue(row.get(i).asString().length() <= maxStringLen); + distinctStrings.add(row.get(i).asString()); + strLen += row.get(i).length(); + break; + + case 12: + if (row.get(i).asBoolean()) truesSeen++; + else falsesSeen++; + break; + + case 13: + Assert.assertTrue(row.get(i).asBytes().length <= maxBinaryLen); + binLen += row.get(i).length(); + break; + + default: + throw new RuntimeException("Too many columns"); + + } + } + } + } + Assert.assertFalse(strIter.hasNext()); + + LOG.debug("Total generated rows is " + rowCnt); + + // Check each of the distinct values is near what we expect + long range = (long)(distinctCounts[0] * 0.1); + Assert.assertTrue("Distinct longs observed " + distinctLongs.size() + " expected " + + distinctCounts[0], Math.abs(distinctLongs.size() - distinctCounts[0]) < range); + range = (long)(distinctCounts[1] * 0.1); + Assert.assertTrue("Distinct ints observed " + distinctIntegers.size() + " expected " + + distinctCounts[1], Math.abs(distinctIntegers.size() - distinctCounts[1]) < range); + range = (long)(distinctCounts[2] * 0.1); + Assert.assertTrue("Distinct shorts observed " + distinctShorts.size() + " expected " + + distinctCounts[2], Math.abs(distinctShorts.size() - distinctCounts[2]) < range); + range = (long)(distinctCounts[3] * 0.2); // Seem to have greater margin of error on bytes, + // not sure why. + Assert.assertTrue("Distinct bytes observed " + distinctBytes.size() + " expected " + + distinctCounts[3], Math.abs(distinctBytes.size() - distinctCounts[3]) < range); + range = (long)(distinctCounts[4] * 0.1); + Assert.assertTrue("Distinct floats observed " + distinctFloats.size() + " expected " + + distinctCounts[4], Math.abs(distinctFloats.size() - distinctCounts[4]) < range); + range = (long)(distinctCounts[5] * 0.1); + Assert.assertTrue("Distinct doubles observed " + distinctDoubles.size() + " expected " + + distinctCounts[5], Math.abs(distinctDoubles.size() - distinctCounts[5]) < range); + range = (long)(distinctCounts[6] * 0.1); + Assert.assertTrue("Distinct decimals observed " + distinctBigDecimals.size() + " expected " + + distinctCounts[6], Math.abs(distinctBigDecimals.size() - distinctCounts[6]) < range); + range = (long)(distinctCounts[7] * 0.1); + Assert.assertTrue("Distinct dates observed " + distinctDates.size() + " expected " + + distinctCounts[7], Math.abs(distinctDates.size() - distinctCounts[7]) < range); + range = (long)(distinctCounts[8] * 0.1); + Assert.assertTrue("Distinct timestamps observed " + distinctTimestamps.size() + " expected " + + distinctCounts[8], Math.abs(distinctTimestamps.size() - distinctCounts[8]) < range); + range = (long)(distinctCounts[9] * 0.1); + Assert.assertTrue("Distinct varchars observed " + distinctVarchars.size() + " expected " + + distinctCounts[9], Math.abs(distinctVarchars.size() - distinctCounts[9]) < range); + range = (long)(distinctCounts[10] * 0.1); + Assert.assertTrue("Distinct chars observed " + distinctChars.size() + " expected " + + distinctCounts[10], Math.abs(distinctChars.size() - distinctCounts[10]) < range); + range = (long)(distinctCounts[11] * 0.1); + Assert.assertTrue("Distinct strings observed " + distinctStrings.size() + " expected " + + distinctCounts[11], Math.abs(distinctStrings.size() - distinctCounts[11]) < range); + + // Check average lengths + double dRange = avgVarcharLen * 0.2; + Assert.assertEquals(avgVarcharLen, varcharLen / (double)rowCnt, dRange); + dRange = avgCharLen * 0.25; + Assert.assertEquals(avgCharLen, charLen / (double)rowCnt, dRange); + dRange = avgStringLen * 0.2; + Assert.assertEquals(avgStringLen, strLen / (double)rowCnt, dRange); + dRange = avgBinaryLen * 0.2; + Assert.assertEquals(avgBinaryLen, binLen / (double)rowCnt, dRange); + + long totalSize = data.lengthInBytes(); + LOG.debug("Total size is " + totalSize); + Assert.assertTrue("Expected totalSize > 102400, but was " + totalSize, totalSize >= 102400); + + // Check numTrues and numFalses + range = (long)(numTrues * 0.15); + Assert.assertTrue("Expected truesSeen of " + numTrues + " got " + truesSeen, + Math.abs(truesSeen - numTrues) < range); + range = (long)(numFalses * 0.15); + Assert.assertTrue("Expected falsesSeen of " + numFalses + " got " + falsesSeen, + Math.abs(falsesSeen - numFalses) < range); + + + //long totalSize = data.length(); + Assert.assertTrue("Expected totalSize > 102400, but was " + totalSize, totalSize >= 102400); + for (int i = 0; i < cols.size(); i++) { + range = (long)(nullCounts[i] * 0.2); + Assert.assertTrue("For column " + i + " expected " + nullCounts[i] + " but got " + + nullsSeen[i], Math.abs(nullsSeen[i] - nullCounts[i]) <= range || + Math.abs(nullsSeen[i] - nullCounts[i]) < 10); // cover very small cases + } + } + + @Ignore + public void partitioned() throws Exception { + Assert.fail(); + } +} + diff --git itests/capybara/src/test/java/org/apache/hive/test/capybara/qconverted/TestSkewJoin.java itests/capybara/src/test/java/org/apache/hive/test/capybara/qconverted/TestSkewJoin.java new file mode 100644 index 0000000..a73c723 --- /dev/null +++ itests/capybara/src/test/java/org/apache/hive/test/capybara/qconverted/TestSkewJoin.java @@ -0,0 +1,276 @@ +/** + * 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.hive.test.capybara.qconverted; + +import org.apache.hive.test.capybara.IntegrationTest; +import org.apache.hive.test.capybara.TableTool; +import org.apache.hive.test.capybara.infra.RandomDataGenerator; +import org.apache.hive.test.capybara.iface.TestTable; +import org.junit.Ignore; +import org.junit.Test; + +public class TestSkewJoin extends IntegrationTest { + + @Ignore // This needs some work + public void skewjoin() throws Exception { + set("hive.explain.user", false); + set("hive.optimize.skewjoin", true); + set("hive.skewjoin.key", 2); + + // SORT_QUERY_RESULTS + TestTable t1 = TestTable.getBuilder("SJ1") + .addCol("k", "string") + .addCol("val", "string") + .build(); + t1.create(); + + TestTable t2 = TestTable.getBuilder("SJ2") + .addCol("k", "string") + .addCol("val", "string") + .build(); + t2.create(); + + TestTable t3 = TestTable.getBuilder("SJ3") + .addCol("k", "string") + .addCol("val", "string") + .build(); + t3.create(); + + TestTable t4 = TestTable.getBuilder("SJ4") + .addCol("k", "string") + .addCol("val", "string") + .build(); + t4.create(); + + TestTable dest_j1 = TestTable.getBuilder("dest_j1") + .addCol("k", "string") + .addCol("value", "string") + .build(); + dest_j1.create(); + + RandomDataGenerator generator = new RandomDataGenerator(39); + t1.populate(generator); + t2.populate(generator); + t3.populate(generator); + t4.populate(generator); + TableTool.createCapySrc(); + + runQuery("INSERT OVERWRITE TABLE dest_j1 SELECT src1.k, src2.value FROM capysrc src1 JOIN " + + "capysrc src2 ON (src1.k = src2.k)"); + runQuery("SELECT k, value FROM dest_j1"); + sortAndCompare(); + + runQuery("SELECT /*+ STREAMTABLE(a) */ * FROM SJ1 a JOIN SJ2 b ON a.k = b.k JOIN SJ3 c ON b" + + ".k = c.k JOIN SJ4 d ON c.k = d.k"); + assertEmpty(); + + runQuery("SELECT /*+ STREAMTABLE(a,c) */ * FROM SJ1 a JOIN SJ2 b ON a.k = b.k JOIN SJ3 c ON " + + "b.k = c.k JOIN SJ4 d ON c.k = d.k"); + sortAndCompare(); + + runQuery("SELECT Y.k, Y.value FROM (SELECT * FROM capysrc) x JOIN (SELECT * FROM capysrc) Y" + + " ON (x.k = Y.k)"); + sortAndCompare(); + + runQuery("SELECT Y.k, Y.value FROM (SELECT * FROM capysrc) x JOIN (SELECT * FROM capysrc) Y" + + " ON (x.k = Y.k and substring(x.value, 5)=substring(y.value, 5))"); + sortAndCompare(); + + runQuery("SELECT src1.c1, src2.c4 FROM (SELECT k as c1, value as c2 from capysrc) src1 " + + "JOIN (SELECT k as c3, value as c4 from capysrc) src2 ON src1.c1 = src2.c3 AND src1" + + ".c1 < 100 JOIN (SELECT k as c5, value as c6 from capysrc) src3 ON src1.c1 = src3" + + ".c5 AND src3.c5 < 80"); + sortAndCompare(); + + runQuery("SELECT /*+ mapjoin(v)*/ k.k, v.val FROM SJ1 k LEFT OUTER JOIN SJ1 v ON k.k=v.k;"); + sortAndCompare(); + + runQuery("select /*+ mapjoin(k)*/ k.k, v.val from SJ1 k join SJ1 v on k.k=v.val"); + sortAndCompare(); + + runQuery("select /*+ mapjoin(k)*/ k.k, v.val from SJ1 k join SJ1 v on k.k=v.k"); + sortAndCompare(); + + runQuery("select k.k, v.val from SJ1 k join SJ1 v on k.k=v.k"); + sortAndCompare(); + + runQuery("select count(1) from SJ1 a join SJ1 b on a.k = b.k"); + sortAndCompare(); + + runQuery("SELECT a.k, a.val, c.k FROM SJ1 a LEFT OUTER JOIN SJ2 c ON c.k=a.k "); + sortAndCompare(); + + runQuery("SELECT /*+ STREAMTABLE(a) */ a.k, a.val, c.k FROM SJ1 a RIGHT OUTER JOIN SJ2 c ON" + + " c.k=a.k "); + sortAndCompare(); + + runQuery("SELECT /*+ STREAMTABLE(a) */ a.k, a.val, c.k FROM SJ1 a FULL OUTER JOIN SJ2 c ON " + + "c.k=a.k "); + sortAndCompare(); + + runQuery("SELECT src1.k, src1.val, src2.k FROM SJ1 src1 LEFT OUTER JOIN SJ2 src2 ON src1" + + ".k = src2.k RIGHT OUTER JOIN SJ2 src3 ON src2.k = src3.k"); + sortAndCompare(); + + runQuery("SELECT src1.k, src1.val, src2.k FROM SJ1 src1 JOIN SJ2 src2 ON src1.k+1 = src2" + + ".k JOIN SJ2 src3 ON src2.k = src3.k"); + sortAndCompare(); + + runQuery("select /*+ mapjoin(v)*/ k.k, v.val from SJ1 k left outer join SJ1 v on k.k=v.k"); + sortAndCompare(); + } + + @Test + public void skewjoin_union_remove_1() throws Exception { + set("hive.optimize.skewjoin.compiletime", true); + set("hive.mapred.supports.subdirectories", true); + + set("hive.stats.autogather", false); + set("hive.optimize.union.remove", true); + + set("hive.merge.mapfiles", false); + set("hive.merge.mapredfiles", false); + set("hive.merge.sparkfiles", false); + set("mapred.input.dir.recursive", true); + + // This is to test the union->selectstar->filesink and skewjoin optimization + // Union of 2 map-reduce subqueries is performed for the skew join + // There is no need to write the temporary results of the sub-queries, and then read them + // again to process the union. The union can be removed completely. + // INCLUDE_HADOOP_MAJOR_VERSIONS(0.23) + // Since this test creates sub-directories for the output, it might be easier to run the test + // only on hadoop 23 + RandomDataGenerator generator = new RandomDataGenerator(37); + + runQuery("drop table if exists T1"); + runQuery("CREATE TABLE T1(k STRING, val STRING) SKEWED BY (k) ON ((2))"); + + TestTable t1 = TestTable.fromHiveMetastore("default", "T1"); + t1.populate(generator); + + runQuery("drop table if exists T2"); + runQuery("CREATE TABLE T2(k STRING, val STRING) SKEWED BY (k) ON ((3))"); + + TestTable t2 = TestTable.fromHiveMetastore("default", "T2"); + t2.populate(generator); + + // a simple join query with skew on both the tables on the join key + + set("hive.input.format", "org.apache.hadoop.hive.ql.io.HiveInputFormat"); + + runQuery("SELECT * FROM T1 a JOIN T2 b ON a.k = b.k ORDER BY a.k, b.k, a.val, b.val"); + + // test outer joins also + + runQuery( + "SELECT a.*, b.* FROM T1 a RIGHT OUTER JOIN T2 b ON a.k = b.k ORDER BY a.k, b.k, a.val, b.val"); + + runQuery("drop table if exists DEST1"); + runQuery("create table DEST1(k1 STRING, val1 STRING, k2 STRING, val2 STRING)"); + + runQuery("INSERT INTO TABLE DEST1 SELECT * FROM T1 a JOIN T2 b ON a.k = b.k"); + + runQuery("SELECT * FROM DEST1 ORDER BY k1, k2, val1, val2"); + + runQuery("INSERT OVERWRITE TABLE DEST1 SELECT * FROM T1 a RIGHT OUTER JOIN T2 b ON a.k = b.k"); + + runQuery("SELECT * FROM DEST1 ORDER BY k1, k2, val1, val2"); + } + + @Test + public void skewjoin_union_remove_2() throws Exception { + set("hive.optimize.skewjoin.compiletime", true); + set("hive.mapred.supports.subdirectories", true); + + set("hive.stats.autogather", false); + set("hive.optimize.union.remove", true); + + set("hive.merge.mapfiles", false); + set("hive.merge.mapredfiles", false); + set("hive.merge.sparkfiles", false); + set("mapred.input.dir.recursive", true); + RandomDataGenerator generator = new RandomDataGenerator(38); + + runQuery("drop table if exists T1"); + runQuery("CREATE TABLE T1(k STRING, val STRING) SKEWED BY (k) ON ((2), (8))"); + TestTable t1 = TestTable.fromHiveMetastore("default", "T1"); + t1.populate(generator); + + runQuery("drop table if exists T2"); + runQuery("CREATE TABLE T2(k STRING, val STRING) SKEWED BY (k) ON ((3), (8))"); + TestTable t2 = TestTable.fromHiveMetastore("default", "T2"); + t2.populate(generator); + + runQuery("drop table if exists T3"); + runQuery("CREATE TABLE T3(k STRING, val STRING)"); + TestTable t3 = TestTable.fromHiveMetastore("default", "T3"); + t3.populate(generator); + + // This is to test the union->selectstar->filesink and skewjoin optimization + // Union of 3 map-reduce subqueries is performed for the skew join + // There is no need to write the temporary results of the sub-queries, and then read them + // again to process the union. The union can be removed completely. + // INCLUDE_HADOOP_MAJOR_VERSIONS(0.23) + // Since this test creates sub-directories for the output table, it might be easier + // to run the test only on hadoop 23 + + set("hive.input.format", "org.apache.hadoop.hive.ql.io.HiveInputFormat"); + + runQuery("SELECT a.*, b.*, c.* FROM T1 a JOIN T2 b ON a.k = b.k JOIN T3 c on a.k = c.k " + + "ORDER BY a.k, b.k, c.k, a.val, b.val, c.val"); + } + + /* + TODO: +ql/src/test/queries/clientpositive//skewjoin.q +ql/src/test/queries/clientpositive//skewjoin_mapjoin1.q +ql/src/test/queries/clientpositive//skewjoin_mapjoin10.q +ql/src/test/queries/clientpositive//skewjoin_mapjoin11.q +ql/src/test/queries/clientpositive//skewjoin_mapjoin2.q +ql/src/test/queries/clientpositive//skewjoin_mapjoin3.q +ql/src/test/queries/clientpositive//skewjoin_mapjoin4.q +ql/src/test/queries/clientpositive//skewjoin_mapjoin5.q +ql/src/test/queries/clientpositive//skewjoin_mapjoin6.q +ql/src/test/queries/clientpositive//skewjoin_mapjoin7.q +ql/src/test/queries/clientpositive//skewjoin_mapjoin8.q +ql/src/test/queries/clientpositive//skewjoin_mapjoin9.q +ql/src/test/queries/clientpositive//skewjoin_noskew.q +ql/src/test/queries/clientpositive//skewjoin_onesideskew.q +ql/src/test/queries/clientpositive//skewjoinopt1.q +ql/src/test/queries/clientpositive//skewjoinopt10.q +ql/src/test/queries/clientpositive//skewjoinopt11.q +ql/src/test/queries/clientpositive//skewjoinopt12.q +ql/src/test/queries/clientpositive//skewjoinopt13.q +ql/src/test/queries/clientpositive//skewjoinopt14.q +ql/src/test/queries/clientpositive//skewjoinopt15.q +ql/src/test/queries/clientpositive//skewjoinopt16.q +ql/src/test/queries/clientpositive//skewjoinopt17.q +ql/src/test/queries/clientpositive//skewjoinopt18.q +ql/src/test/queries/clientpositive//skewjoinopt19.q +ql/src/test/queries/clientpositive//skewjoinopt2.q +ql/src/test/queries/clientpositive//skewjoinopt20.q +ql/src/test/queries/clientpositive//skewjoinopt21.q +ql/src/test/queries/clientpositive//skewjoinopt3.q +ql/src/test/queries/clientpositive//skewjoinopt4.q +ql/src/test/queries/clientpositive//skewjoinopt5.q +ql/src/test/queries/clientpositive//skewjoinopt6.q +ql/src/test/queries/clientpositive//skewjoinopt7.q +ql/src/test/queries/clientpositive//skewjoinopt8.q +ql/src/test/queries/clientpositive//skewjoinopt9.q + */ +} diff --git itests/capybara/src/test/java/org/apache/hive/test/capybara/qconverted/TestUnicode.java itests/capybara/src/test/java/org/apache/hive/test/capybara/qconverted/TestUnicode.java new file mode 100644 index 0000000..6c8e1c6 --- /dev/null +++ itests/capybara/src/test/java/org/apache/hive/test/capybara/qconverted/TestUnicode.java @@ -0,0 +1,54 @@ +/** + * 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.hive.test.capybara.qconverted; + +import org.apache.hive.test.capybara.IntegrationTest; +import org.apache.hive.test.capybara.iface.TestTable; +import org.apache.hive.test.capybara.infra.StaticDataGenerator; +import org.junit.Ignore; + +import java.util.Arrays; +import java.util.List; + +public class TestUnicode extends IntegrationTest { + + @Ignore + public void unicodeData() throws Exception { + TestTable uni = TestTable.getBuilder("unitable") + .addCol("a", "varchar(20)") + .addCol("b", "char(25)") + .addCol("c", "string") + .build(); + uni.create(); + + List data = Arrays.asList( + "aáâàäbcçdeéêèf,ghiíìjklmnoóôòpqrs,tuúûùüvwxyz", + "AÁÂÀÄBCÇDEÉÊÈF,GHIÍÌJKLMNOÓÔÒPQRS,TUÚÛÙÜVWXYZ", + "αβγδεζηθ,ικλμνξοπρσ,τυφχψω", + "ΑΒΓΔΕΖΗΘ,ΙΚΛΜΝΞΟΠΡΣ,ΤΥΦΧΨΩ"); + StaticDataGenerator generator = new StaticDataGenerator(data, ","); + uni.populate(generator); + + runQuery("select * from unitable"); + sortAndCompare(); + + // Force a job rather than just a read from HDFS + runQuery("select a, b, c, count(*) from unitable group by a, b, c"); + sortAndCompare(); + } +} diff --git itests/capybara/src/test/resources/tez-container-log4j2.xml itests/capybara/src/test/resources/tez-container-log4j2.xml new file mode 100644 index 0000000..be949dc --- /dev/null +++ itests/capybara/src/test/resources/tez-container-log4j2.xml @@ -0,0 +1,49 @@ + + + + + + + ALL + INFO + CLA + ${sys:yarn.app.container.log.dir} + syslog + + + + + + + + + + + + + + + + + + + + + diff --git itests/hive-unit/src/main/java/org/apache/hive/jdbc/miniHS2/MiniHS2.java itests/hive-unit/src/main/java/org/apache/hive/jdbc/miniHS2/MiniHS2.java index 488ba93..b2e8db5 100644 --- itests/hive-unit/src/main/java/org/apache/hive/jdbc/miniHS2/MiniHS2.java +++ itests/hive-unit/src/main/java/org/apache/hive/jdbc/miniHS2/MiniHS2.java @@ -18,13 +18,7 @@ package org.apache.hive.jdbc.miniHS2; -import java.io.File; -import java.io.IOException; -import java.util.HashMap; -import java.util.Map; -import java.util.concurrent.TimeoutException; -import java.util.concurrent.atomic.AtomicLong; - +import com.google.common.io.Files; import org.apache.commons.io.FileUtils; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; @@ -45,7 +39,12 @@ import org.apache.hive.service.cli.thrift.ThriftHttpCLIService; import org.apache.hive.service.server.HiveServer2; -import com.google.common.io.Files; +import java.io.File; +import java.io.IOException; +import java.util.HashMap; +import java.util.Map; +import java.util.concurrent.TimeoutException; +import java.util.concurrent.atomic.AtomicLong; public class MiniHS2 extends AbstractHiveService { public static final String HS2_BINARY_MODE = "binary"; @@ -68,7 +67,8 @@ public enum MiniClusterType { MR, TEZ, - DFS_ONLY; + DFS_ONLY, + BRING_YOUR_OWN; // In this case a mini-cluster already exists, use it with this } public static class Builder { @@ -125,7 +125,7 @@ public MiniHS2 build() throws Exception { hiveConf.setVar(ConfVars.HIVE_SERVER2_TRANSPORT_MODE, HS2_BINARY_MODE); } return new MiniHS2(hiveConf, miniClusterType, useMiniKdc, serverPrincipal, serverKeytab, - isMetastoreRemote); + isMetastoreRemote, null); } } @@ -162,7 +162,8 @@ public boolean isUseMiniKdc() { } private MiniHS2(HiveConf hiveConf, MiniClusterType miniClusterType, boolean useMiniKdc, - String serverPrincipal, String serverKeytab, boolean isMetastoreRemote) throws Exception { + String serverPrincipal, String serverKeytab, boolean isMetastoreRemote, + FileSystem externalFs) throws Exception { super(hiveConf, "localhost", MetaStoreUtils.findFreePort(), MetaStoreUtils.findFreePort()); this.miniClusterType = miniClusterType; this.useMiniKdc = useMiniKdc; @@ -172,8 +173,21 @@ private MiniHS2(HiveConf hiveConf, MiniClusterType miniClusterType, boolean useM localFS = FileSystem.getLocal(hiveConf); FileSystem fs; - if (miniClusterType != MiniClusterType.DFS_ONLY) { - // Initialize dfs + switch (miniClusterType) { + case BRING_YOUR_OWN: + fs = externalFs; + baseDfsDir = new Path(new Path(fs.getUri()), "/base"); + break; + + case DFS_ONLY: + // This is DFS only mode, just initialize the dfs root directory. + fs = FileSystem.getLocal(hiveConf); + baseDfsDir = new Path("file://"+ baseDir.toURI().getPath()); + break; + + case TEZ: + case MR: + // Initialize dfs dfs = ShimLoader.getHadoopShims().getMiniDfs(hiveConf, 4, true, null); fs = dfs.getFileSystem(); String uriString = WindowsPathUtil.getHdfsUriString(fs.getUri().toString()); @@ -192,11 +206,12 @@ private MiniHS2(HiveConf hiveConf, MiniClusterType miniClusterType, boolean useM // store the config in system properties mr.setupConfiguration(getHiveConf()); baseDfsDir = new Path(new Path(fs.getUri()), "/base"); - } else { - // This is DFS only mode, just initialize the dfs root directory. - fs = FileSystem.getLocal(hiveConf); - baseDfsDir = new Path("file://"+ baseDir.toURI().getPath()); + break; + + default: + throw new RuntimeException("Unknown miniClusterType " + miniClusterType.name()); } + if (useMiniKdc) { hiveConf.setVar(ConfVars.HIVE_SERVER2_KERBEROS_PRINCIPAL, serverPrincipal); hiveConf.setVar(ConfVars.HIVE_SERVER2_KERBEROS_KEYTAB, serverKeytab); @@ -236,7 +251,19 @@ public MiniHS2(HiveConf hiveConf) throws Exception { } public MiniHS2(HiveConf hiveConf, MiniClusterType clusterType) throws Exception { - this(hiveConf, clusterType, false, null, null, false); + this(hiveConf, clusterType, false, null, null, false, null); + } + + /** + * Build a minicluster with an existing set of mini-clusters, rather than relying on this one + * to build everything itself + * @param hiveConf hive configuration file + * @param external external existing file system. + * @throws Exception + */ + public MiniHS2(HiveConf hiveConf, FileSystem external) throws Exception { + this(hiveConf, MiniClusterType.BRING_YOUR_OWN, false, null, null, false, external); + // I'm living dangerously here by not setting mr or dfs, but I don't want to use the shims } public void start(Map confOverlay) throws Exception { diff --git itests/pom.xml itests/pom.xml index 5d8249f..cd6a12a 100644 --- itests/pom.xml +++ itests/pom.xml @@ -42,6 +42,7 @@ hive-jmh hive-unit-hadoop2 hive-minikdc + capybara