From 9ddb82aadfc2fd32bbb10650392b2df8ad0f5dfa Mon Sep 17 00:00:00 2001 From: Laszlo Bodor Date: Mon, 14 Apr 2025 14:00:26 +0200 Subject: [PATCH 1/2] HIVE-28904: Remove Arrow from Hive --- .../org/apache/hadoop/hive/conf/HiveConf.java | 15 - .../TestReExecuteKilledTezAMQueryPlugin.java | 18 +- .../hive/jdbc/BaseJdbcWithMiniLlap.java | 2 - .../hive/jdbc/TestJdbcWithMiniLlapArrow.java | 544 --------- .../hive/jdbc/TestJdbcWithMiniLlapRow.java | 2 - .../jdbc/TestJdbcWithMiniLlapVectorArrow.java | 440 ------- .../TestJdbcWithMiniLlapVectorArrowBatch.java | 489 -------- ...apExtClientWithCloudDeploymentConfigs.java | 139 --- ...MiniLlapVectorArrowWithLlapIODisabled.java | 199 --- .../hive/jdbc/TestNewGetSplitsFormat.java | 127 -- .../TestNewGetSplitsFormatReturnPath.java | 52 - .../hive/llap/LlapArrowBatchRecordReader.java | 91 -- .../hive/llap/LlapArrowRowInputFormat.java | 65 - .../hive/llap/LlapArrowRowRecordReader.java | 113 -- .../hadoop/hive/llap/LlapBaseInputFormat.java | 50 +- .../hadoop/hive/llap/LlapRowInputFormat.java | 2 +- llap-server/pom.xml | 15 - .../cli/service/AsyncTaskCopyLocalJars.java | 6 +- pom.xml | 2 - ql/pom.xml | 15 - .../hive/llap/LlapArrowRecordWriter.java | 92 -- .../hive/llap/LlapOutputFormatService.java | 11 +- .../hive/llap/WritableByteChannelAdapter.java | 135 --- .../hadoop/hive/ql/exec/FileSinkOperator.java | 10 - .../filesink/VectorFileSinkArrowOperator.java | 180 --- .../ql/io/arrow/ArrowColumnarBatchSerDe.java | 261 ---- .../ql/io/arrow/ArrowWrapperWritable.java | 80 -- .../hadoop/hive/ql/io/arrow/Deserializer.java | 453 ------- .../ql/io/arrow/RootAllocatorFactory.java | 53 - .../hadoop/hive/ql/io/arrow/Serializer.java | 1006 ---------------- .../ql/optimizer/physical/Vectorizer.java | 63 +- .../hive/ql/parse/SemanticAnalyzer.java | 8 +- .../hive/llap/TestLlapOutputFormat.java | 1 - .../io/arrow/TestArrowColumnarBatchSerDe.java | 1070 ----------------- .../hive/ql/io/arrow/TestSerializer.java | 101 -- serde/pom.xml | 11 - 36 files changed, 29 insertions(+), 5892 deletions(-) delete mode 100644 itests/hive-unit/src/test/java/org/apache/hive/jdbc/TestJdbcWithMiniLlapArrow.java delete mode 100644 itests/hive-unit/src/test/java/org/apache/hive/jdbc/TestJdbcWithMiniLlapVectorArrow.java delete mode 100644 itests/hive-unit/src/test/java/org/apache/hive/jdbc/TestJdbcWithMiniLlapVectorArrowBatch.java delete mode 100644 itests/hive-unit/src/test/java/org/apache/hive/jdbc/TestLlapExtClientWithCloudDeploymentConfigs.java delete mode 100644 itests/hive-unit/src/test/java/org/apache/hive/jdbc/TestMiniLlapVectorArrowWithLlapIODisabled.java delete mode 100644 itests/hive-unit/src/test/java/org/apache/hive/jdbc/TestNewGetSplitsFormat.java delete mode 100644 itests/hive-unit/src/test/java/org/apache/hive/jdbc/TestNewGetSplitsFormatReturnPath.java delete mode 100644 llap-ext-client/src/java/org/apache/hadoop/hive/llap/LlapArrowBatchRecordReader.java delete mode 100644 llap-ext-client/src/java/org/apache/hadoop/hive/llap/LlapArrowRowInputFormat.java delete mode 100644 llap-ext-client/src/java/org/apache/hadoop/hive/llap/LlapArrowRowRecordReader.java delete mode 100644 ql/src/java/org/apache/hadoop/hive/llap/LlapArrowRecordWriter.java delete mode 100644 ql/src/java/org/apache/hadoop/hive/llap/WritableByteChannelAdapter.java delete mode 100644 ql/src/java/org/apache/hadoop/hive/ql/exec/vector/filesink/VectorFileSinkArrowOperator.java delete mode 100644 ql/src/java/org/apache/hadoop/hive/ql/io/arrow/ArrowColumnarBatchSerDe.java delete mode 100644 ql/src/java/org/apache/hadoop/hive/ql/io/arrow/ArrowWrapperWritable.java delete mode 100644 ql/src/java/org/apache/hadoop/hive/ql/io/arrow/Deserializer.java delete mode 100644 ql/src/java/org/apache/hadoop/hive/ql/io/arrow/RootAllocatorFactory.java delete mode 100644 ql/src/java/org/apache/hadoop/hive/ql/io/arrow/Serializer.java delete mode 100644 ql/src/test/org/apache/hadoop/hive/ql/io/arrow/TestArrowColumnarBatchSerDe.java delete mode 100644 ql/src/test/org/apache/hadoop/hive/ql/io/arrow/TestSerializer.java diff --git a/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java b/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java index 4da48659e289..8951b59f17e8 100644 --- a/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java +++ b/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java @@ -456,7 +456,6 @@ private static void populateLlapDaemonVarsSet(Set llapDaemonVarsSetLocal llapDaemonVarsSetLocal.add(ConfVars.LLAP_VALIDATE_ACLS.varname); llapDaemonVarsSetLocal.add(ConfVars.LLAP_DAEMON_LOGGER.varname); llapDaemonVarsSetLocal.add(ConfVars.LLAP_DAEMON_AM_USE_FQDN.varname); - llapDaemonVarsSetLocal.add(ConfVars.LLAP_OUTPUT_FORMAT_ARROW.varname); llapDaemonVarsSetLocal.add(ConfVars.LLAP_IO_PATH_CACHE_SIZE.varname); } @@ -3488,13 +3487,6 @@ public static enum ConfVars { "If true, don't decode Acid metadata columns from storage unless" + " they are needed."), - // For Arrow SerDe - HIVE_ARROW_ROOT_ALLOCATOR_LIMIT("hive.arrow.root.allocator.limit", Long.MAX_VALUE, - "Arrow root allocator memory size limitation in bytes."), - HIVE_ARROW_BATCH_ALLOCATOR_LIMIT("hive.arrow.batch.allocator.limit", 10_000_000_000L, - "Max bytes per arrow batch. This is a threshold, the memory is not pre-allocated."), - HIVE_ARROW_BATCH_SIZE("hive.arrow.batch.size", 1000, "The number of rows sent in one Arrow batch."), - // For Druid storage handler HIVE_DRUID_INDEXING_GRANULARITY("hive.druid.indexer.segments.granularity", "DAY", new PatternSet("YEAR", "MONTH", "WEEK", "DAY", "HOUR", "MINUTE", "SECOND"), @@ -4773,11 +4765,6 @@ public static enum ConfVars { "internal use only. When false, don't suppress fatal exceptions like\n" + "NullPointerException, etc so the query will fail and assure it will be noticed", true), - HIVE_VECTORIZATION_FILESINK_ARROW_NATIVE_ENABLED( - "hive.vectorized.execution.filesink.arrow.native.enabled", false, - "This flag should be set to true to enable the native vectorization\n" + - "of queries using the Arrow SerDe and FileSink.\n" + - "The default value is false."), HIVE_TYPE_CHECK_ON_INSERT("hive.typecheck.on.insert", true, "This property has been extended to control " + "whether to check, convert, and normalize partition value to conform to its column type in " + "partition operations including but not limited to insert, such as alter, describe etc."), @@ -5563,8 +5550,6 @@ public static enum ConfVars { Constants.LLAP_LOGGER_NAME_RFA, Constants.LLAP_LOGGER_NAME_CONSOLE), "logger used for llap-daemons."), - LLAP_OUTPUT_FORMAT_ARROW("hive.llap.output.format.arrow", true, - "Whether LLapOutputFormatService should output arrow batches"), LLAP_COLLECT_LOCK_METRICS("hive.llap.lockmetrics.collect", false, "Whether lock metrics (wait times, counts) are collected for LLAP " + "related locks"), diff --git a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/reexec/TestReExecuteKilledTezAMQueryPlugin.java b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/reexec/TestReExecuteKilledTezAMQueryPlugin.java index 35b98fa08b23..2dfe5069f993 100644 --- a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/reexec/TestReExecuteKilledTezAMQueryPlugin.java +++ b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/reexec/TestReExecuteKilledTezAMQueryPlugin.java @@ -20,11 +20,11 @@ import org.apache.hadoop.fs.Path; import org.apache.hadoop.hive.conf.HiveConf; +import org.apache.hadoop.hive.ql.exec.UDF; import org.apache.hadoop.yarn.api.records.ApplicationReport; import org.apache.hadoop.yarn.api.records.YarnApplicationState; import org.apache.hadoop.yarn.client.api.YarnClient; import org.apache.hive.jdbc.HiveStatement; -import org.apache.hive.jdbc.TestJdbcWithMiniLlapArrow; import org.apache.hive.jdbc.miniHS2.MiniHS2; import org.junit.*; import org.slf4j.Logger; @@ -83,7 +83,7 @@ public static void beforeTest() throws Exception { Statement stmt = conDefault.createStatement(); String tblName = testDbName + "." + tableName; Path dataFilePath = new Path(dataFileDir, "kv1.txt"); - String udfName = TestJdbcWithMiniLlapArrow.SleepMsUDF.class.getName(); + String udfName = SleepMsUDF.class.getName(); stmt.execute("drop database if exists " + testDbName + " cascade"); stmt.execute("create database " + testDbName); stmt.execute("set role admin"); @@ -105,6 +105,20 @@ public static void afterTest() { } } + /** + * SleepMsUDF + */ + public static class SleepMsUDF extends UDF { + public Integer evaluate(int value, int ms) { + try { + Thread.sleep(ms); + } catch (InterruptedException e) { + // No-op + } + return value; + } + } + public static Connection getConnection(String jdbcURL, String user, String pwd) throws SQLException { Connection conn = DriverManager.getConnection(jdbcURL, user, pwd); conn.createStatement().execute("set hive.support.concurrency = false"); diff --git a/itests/hive-unit/src/test/java/org/apache/hive/jdbc/BaseJdbcWithMiniLlap.java b/itests/hive-unit/src/test/java/org/apache/hive/jdbc/BaseJdbcWithMiniLlap.java index 5859f639242e..12b1e6b14313 100644 --- a/itests/hive-unit/src/test/java/org/apache/hive/jdbc/BaseJdbcWithMiniLlap.java +++ b/itests/hive-unit/src/test/java/org/apache/hive/jdbc/BaseJdbcWithMiniLlap.java @@ -716,8 +716,6 @@ protected int processQuery(String currentDatabase, String query, int numSplits, rowProcessor.process(row); ++rowCount; } - //In arrow-mode this will throw exception unless all buffers have been released - //See org.apache.hadoop.hive.llap.LlapArrowBatchRecordReader reader.close(); } LlapBaseInputFormat.close(handleId); diff --git a/itests/hive-unit/src/test/java/org/apache/hive/jdbc/TestJdbcWithMiniLlapArrow.java b/itests/hive-unit/src/test/java/org/apache/hive/jdbc/TestJdbcWithMiniLlapArrow.java deleted file mode 100644 index 209c42b52e07..000000000000 --- a/itests/hive-unit/src/test/java/org/apache/hive/jdbc/TestJdbcWithMiniLlapArrow.java +++ /dev/null @@ -1,544 +0,0 @@ -/* - * 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.jdbc; - -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertArrayEquals; -import java.math.BigDecimal; - -import com.google.common.collect.Iterables; -import org.apache.hadoop.hive.common.type.Date; -import org.apache.hadoop.hive.common.type.Timestamp; - -import java.util.Iterator; -import java.util.List; -import java.util.Map; -import org.apache.hadoop.hive.llap.FieldDesc; -import org.apache.hadoop.hive.llap.LlapBaseInputFormat; -import org.apache.hadoop.hive.llap.Row; -import org.apache.hadoop.hive.ql.ddl.process.kill.KillQueriesOperation; -import org.apache.hadoop.io.NullWritable; -import org.apache.hadoop.mapred.InputSplit; -import org.apache.hadoop.mapred.JobConf; -import org.junit.BeforeClass; -import org.apache.hadoop.hive.conf.HiveConf; -import org.apache.hadoop.hive.conf.HiveConf.ConfVars; -import org.junit.AfterClass; -import org.junit.Ignore; -import org.junit.Test; -import java.sql.SQLException; -import java.sql.Statement; -import java.sql.Connection; -import java.util.concurrent.Callable; -import java.util.stream.IntStream; - -import org.apache.hadoop.hive.ql.exec.UDF; -import org.apache.hadoop.mapred.InputFormat; -import org.apache.hadoop.hive.llap.LlapArrowRowInputFormat; -import org.apache.hive.jdbc.miniHS2.MiniHS2; -import static org.junit.Assert.assertNotNull; -import static org.junit.Assert.assertNull; -import static org.junit.Assert.assertTrue; -import org.apache.hadoop.fs.Path; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -/** - * TestJdbcWithMiniLlap for Arrow format - */ -@Ignore("unstable HIVE-23549") -public class TestJdbcWithMiniLlapArrow extends BaseJdbcWithMiniLlap { - - protected static final Logger LOG = LoggerFactory.getLogger(TestJdbcWithMiniLlapArrow.class); - - private static MiniHS2 miniHS2 = null; - private static final String tableName = "testJdbcMinihs2Tbl"; - private static String dataFileDir; - private static final String testDbName = "testJdbcMinihs2"; - private static final String tag = "mytag"; - - private static class ExceptionHolder { - Throwable throwable; - } - - @BeforeClass - public static void beforeTest() throws Exception { - HiveConf conf = defaultConf(); - conf.setBoolVar(ConfVars.LLAP_OUTPUT_FORMAT_ARROW, true); - conf.setVar(ConfVars.HIVE_AUTHENTICATOR_MANAGER, "org.apache.hadoop.hive.ql.security" + - ".SessionStateUserAuthenticator"); - conf.setVar(ConfVars.USERS_IN_ADMIN_ROLE, System.getProperty("user.name")); - conf.setBoolVar(ConfVars.HIVE_AUTHORIZATION_ENABLED, true); - conf.setVar(ConfVars.HIVE_AUTHORIZATION_SQL_STD_AUTH_CONFIG_WHITELIST_APPEND, ConfVars.HIVE_SUPPORT_CONCURRENCY - .varname + "|" + ConfVars.HIVE_SERVER2_ENABLE_DOAS.varname); - MiniHS2.cleanupLocalDir(); - miniHS2 = BaseJdbcWithMiniLlap.beforeTest(conf); - dataFileDir = conf.get("test.data.files").replace('\\', '/').replace("c:", ""); - - Connection conDefault = BaseJdbcWithMiniLlap.getConnection(miniHS2.getJdbcURL(), - System.getProperty("user.name"), "bar"); - Statement stmt = conDefault.createStatement(); - String tblName = testDbName + "." + tableName; - Path dataFilePath = new Path(dataFileDir, "kv1.txt"); - String udfName = SleepMsUDF.class.getName(); - stmt.execute("drop database if exists " + testDbName + " cascade"); - stmt.execute("create database " + testDbName); - stmt.execute("set role admin"); - stmt.execute("dfs -put " + dataFilePath.toString() + " " + "kv1.txt"); - stmt.execute("use " + testDbName); - stmt.execute("create table " + tblName + " (int_col int, value string) "); - stmt.execute("load data inpath 'kv1.txt' into table " + tblName); - stmt.execute("create function sleepMsUDF as '" + udfName + "'"); - stmt.execute("grant select on table " + tblName + " to role public"); - - stmt.close(); - conDefault.close(); - } - - @AfterClass - public static void afterTest() { - if (miniHS2 != null && miniHS2.isStarted()) { - miniHS2.stop(); - } - } - - @Override - protected InputFormat getInputFormat() { - //For unit testing, no harm in hard-coding allocator ceiling to LONG.MAX_VALUE - return new LlapArrowRowInputFormat(Long.MAX_VALUE); - } - - @Test - @Override - public void testDataTypes() throws Exception { - createDataTypesTable("datatypes"); - RowCollector2 rowCollector = new RowCollector2(); - String query = "select * from datatypes"; - int rowCount = processQuery(query, 1, rowCollector); - assertEquals(3, rowCount); - - // Verify schema - String[][] colNameTypes = new String[][] { - {"datatypes.c1", "int"}, - {"datatypes.c2", "boolean"}, - {"datatypes.c3", "double"}, - {"datatypes.c4", "string"}, - {"datatypes.c5", "array"}, - {"datatypes.c6", "map"}, - {"datatypes.c7", "map"}, - {"datatypes.c8", "struct"}, - {"datatypes.c9", "tinyint"}, - {"datatypes.c10", "smallint"}, - {"datatypes.c11", "float"}, - {"datatypes.c12", "bigint"}, - {"datatypes.c13", "array>"}, - {"datatypes.c14", "map>"}, - {"datatypes.c15", "struct>"}, - {"datatypes.c16", "array,n:int>>"}, - {"datatypes.c17", "timestamp"}, - {"datatypes.c18", "decimal(16,7)"}, - {"datatypes.c19", "binary"}, - {"datatypes.c20", "date"}, - {"datatypes.c21", "varchar(20)"}, - {"datatypes.c22", "char(15)"}, - {"datatypes.c23", "binary"}, - }; - FieldDesc fieldDesc; - assertEquals(23, rowCollector.numColumns); - for (int idx = 0; idx < rowCollector.numColumns; ++idx) { - fieldDesc = rowCollector.schema.getColumns().get(idx); - assertEquals("ColName idx=" + idx, colNameTypes[idx][0], fieldDesc.getName()); - assertEquals("ColType idx=" + idx, colNameTypes[idx][1], fieldDesc.getTypeInfo().getTypeName()); - } - - // First row is all nulls - Object[] rowValues = rowCollector.rows.get(0); - for (int idx = 0; idx < rowCollector.numColumns; ++idx) { - assertEquals("idx=" + idx, null, rowValues[idx]); - } - - // Second Row - rowValues = rowCollector.rows.get(1); - assertEquals(Integer.valueOf(-1), rowValues[0]); - assertEquals(Boolean.FALSE, rowValues[1]); - assertEquals(Double.valueOf(-1.1d), rowValues[2]); - assertEquals("", rowValues[3]); - - List c5Value = (List) rowValues[4]; - assertEquals(0, c5Value.size()); - - Map c6Value = (Map) rowValues[5]; - assertEquals(1, c6Value.size()); - assertEquals(null, c6Value.get(1)); - - Map c7Value = (Map) rowValues[6]; - assertEquals(1, c7Value.size()); - assertEquals("b", c7Value.get("a")); - - List c8Value = (List) rowValues[7]; - assertEquals(null, c8Value.get(0)); - assertEquals(null, c8Value.get(1)); - assertEquals(null, c8Value.get(2)); - - assertEquals(Byte.valueOf((byte) -1), rowValues[8]); - assertEquals(Short.valueOf((short) -1), rowValues[9]); - assertEquals(Float.valueOf(-1.0f), rowValues[10]); - assertEquals(Long.valueOf(-1l), rowValues[11]); - - List c13Value = (List) rowValues[12]; - assertEquals(0, c13Value.size()); - - Map c14Value = (Map) rowValues[13]; - assertEquals(1, c14Value.size()); - Map mapVal = (Map) c14Value.get(Integer.valueOf(1)); - assertEquals(1, mapVal.size()); - assertEquals(100, mapVal.get(Integer.valueOf(10))); - - List c15Value = (List) rowValues[14]; - assertEquals(null, c15Value.get(0)); - assertEquals(null, c15Value.get(1)); - - List c16Value = (List) rowValues[15]; - assertEquals(0, c16Value.size()); - - assertEquals(null, rowValues[16]); - assertEquals(null, rowValues[17]); - assertEquals(null, rowValues[18]); - assertEquals(null, rowValues[19]); - assertEquals(null, rowValues[20]); - assertEquals(null, rowValues[21]); - assertEquals(null, rowValues[22]); - - // Third row - rowValues = rowCollector.rows.get(2); - assertEquals(Integer.valueOf(1), rowValues[0]); - assertEquals(Boolean.TRUE, rowValues[1]); - assertEquals(Double.valueOf(1.1d), rowValues[2]); - assertEquals("1", rowValues[3]); - - c5Value = (List) rowValues[4]; - assertEquals(2, c5Value.size()); - assertEquals(Integer.valueOf(1), c5Value.get(0)); - assertEquals(Integer.valueOf(2), c5Value.get(1)); - - c6Value = (Map) rowValues[5]; - assertEquals(2, c6Value.size()); - assertEquals("x", c6Value.get(Integer.valueOf(1))); - assertEquals("y", c6Value.get(Integer.valueOf(2))); - - c7Value = (Map) rowValues[6]; - assertEquals(2, c7Value.size()); - assertEquals("v", c7Value.get("k")); - assertEquals("c", c7Value.get("b")); - - c8Value = (List) rowValues[7]; - assertEquals("a", c8Value.get(0)); - assertEquals(Integer.valueOf(9), c8Value.get(1)); - assertEquals(Double.valueOf(2.2d), c8Value.get(2)); - - assertEquals(Byte.valueOf((byte) 1), rowValues[8]); - assertEquals(Short.valueOf((short) 1), rowValues[9]); - assertEquals(Float.valueOf(1.0f), rowValues[10]); - assertEquals(Long.valueOf(1l), rowValues[11]); - - c13Value = (List) rowValues[12]; - assertEquals(2, c13Value.size()); - List listVal = (List) c13Value.get(0); - assertEquals("a", listVal.get(0)); - assertEquals("b", listVal.get(1)); - listVal = (List) c13Value.get(1); - assertEquals("c", listVal.get(0)); - assertEquals("d", listVal.get(1)); - - c14Value = (Map) rowValues[13]; - assertEquals(2, c14Value.size()); - mapVal = (Map) c14Value.get(Integer.valueOf(1)); - assertEquals(2, mapVal.size()); - assertEquals(Integer.valueOf(12), mapVal.get(Integer.valueOf(11))); - assertEquals(Integer.valueOf(14), mapVal.get(Integer.valueOf(13))); - mapVal = (Map) c14Value.get(Integer.valueOf(2)); - assertEquals(1, mapVal.size()); - assertEquals(Integer.valueOf(22), mapVal.get(Integer.valueOf(21))); - - c15Value = (List) rowValues[14]; - assertEquals(Integer.valueOf(1), c15Value.get(0)); - listVal = (List) c15Value.get(1); - assertEquals(2, listVal.size()); - assertEquals(Integer.valueOf(2), listVal.get(0)); - assertEquals("x", listVal.get(1)); - - c16Value = (List) rowValues[15]; - assertEquals(2, c16Value.size()); - listVal = (List) c16Value.get(0); - assertEquals(2, listVal.size()); - mapVal = (Map) listVal.get(0); - assertEquals(0, mapVal.size()); - assertEquals(Integer.valueOf(1), listVal.get(1)); - listVal = (List) c16Value.get(1); - mapVal = (Map) listVal.get(0); - assertEquals(2, mapVal.size()); - assertEquals("b", mapVal.get("a")); - assertEquals("d", mapVal.get("c")); - assertEquals(Integer.valueOf(2), listVal.get(1)); - - assertEquals(Timestamp.valueOf("2012-04-22 09:00:00.123456"), rowValues[16]); - assertEquals(new BigDecimal("123456789.123456"), rowValues[17]); - assertArrayEquals("abcd".getBytes("UTF-8"), (byte[]) rowValues[18]); - assertEquals(Date.valueOf("2013-01-01"), rowValues[19]); - assertEquals("abc123", rowValues[20]); - assertEquals("abc123 ", rowValues[21]); - assertArrayEquals("X'01FF'".getBytes("UTF-8"), (byte[]) rowValues[22]); - } - - /** - * SleepMsUDF - */ - public static class SleepMsUDF extends UDF { - public Integer evaluate(int value, int ms) { - try { - Thread.sleep(ms); - } catch (InterruptedException e) { - // No-op - } - return value; - } - } - - /** - * Test CLI kill command of a query that is running. - * We spawn 2 threads - one running the query and - * the other attempting to cancel. - * We're using a dummy udf to simulate a query, - * that runs for a sufficiently long time. - * @throws Exception - */ - private void testKillQueryInternal(String user, String killUser, boolean useTag, final - ExceptionHolder stmtHolder, final ExceptionHolder tKillHolder) throws Exception { - Connection con1 = BaseJdbcWithMiniLlap.getConnection(miniHS2.getJdbcURL(testDbName), - user, "bar"); - Connection con2 = BaseJdbcWithMiniLlap.getConnection(miniHS2.getJdbcURL(testDbName), - killUser, "bar"); - - final Statement stmt2 = con2.createStatement(); - final HiveStatement stmt = (HiveStatement)con1.createStatement(); - final StringBuffer stmtQueryId = new StringBuffer(); - - // Thread executing the query - Thread tExecute = new Thread(new Runnable() { - @Override - public void run() { - try { - System.out.println("Executing query: "); - stmt.execute("set hive.llap.execution.mode = none"); - - if (useTag) { - stmt.execute("set hive.query.tag = " + tag); - } - // The test table has 500 rows, so total query time should be ~ 500*500ms - stmt.executeAsync("select sleepMsUDF(t1.int_col, 100), t1.int_col, t2.int_col " + - "from " + tableName + " t1 join " + tableName + " t2 on t1.int_col = t2.int_col"); - stmtQueryId.append(stmt.getQueryId()); - stmt.getUpdateCount(); - } catch (SQLException e) { - stmtHolder.throwable = e; - } - } - }); - - tExecute.start(); - - // wait for other thread to create the stmt handle - int count = 0; - while (++count <= 10) { - try { - tKillHolder.throwable = null; - Thread.sleep(2000); - String queryId; - if (useTag) { - queryId = tag; - } else { - if (stmtQueryId.length() != 0) { - queryId = stmtQueryId.toString(); - } else { - continue; - } - } - System.out.println("Killing query: " + queryId); - if (killUser.equals(System.getProperty("user.name"))) { - stmt2.execute("set role admin"); - } - stmt2.execute("kill query '" + queryId + "'"); - stmt2.close(); - break; - } catch (SQLException e) { - LOG.warn("Exception when kill query", e); - tKillHolder.throwable = e; - } - } - - tExecute.join(); - try { - stmt.close(); - con1.close(); - con2.close(); - } catch (Exception e) { - // ignore error - LOG.warn("Exception when close stmt and con", e); - } - } - - @Test - @Override - public void testKillQuery() throws Exception { - testKillQueryById(); - testKillQueryByTagNegative(); - testKillQueryByTagAdmin(); - testKillQueryByTagOwner(); - } - - @Test - public void testKillQueryById() throws Exception { - ExceptionHolder tExecuteHolder = new ExceptionHolder(); - ExceptionHolder tKillHolder = new ExceptionHolder(); - testKillQueryInternal(System.getProperty("user.name"), System.getProperty("user.name"), false, - tExecuteHolder, tKillHolder); - assertNotNull("tExecute", tExecuteHolder.throwable); - assertEquals(HiveStatement.QUERY_CANCELLED_MESSAGE + " "+ KillQueriesOperation.KILL_QUERY_MESSAGE, - tExecuteHolder.throwable.getMessage()); - assertNull("tCancel", tKillHolder.throwable); - } - - @Test - public void testKillQueryByTagNegative() throws Exception { - ExceptionHolder tExecuteHolder = new ExceptionHolder(); - ExceptionHolder tKillHolder = new ExceptionHolder(); - testKillQueryInternal("user1", "user2", true, tExecuteHolder, tKillHolder); - assertNull("tExecute", tExecuteHolder.throwable); - assertNotNull("tCancel", tKillHolder.throwable); - assertTrue(tKillHolder.throwable.getMessage(), tKillHolder.throwable.getMessage().contains("No privilege")); - } - - @Test - public void testKillQueryByTagAdmin() throws Exception { - ExceptionHolder tExecuteHolder = new ExceptionHolder(); - ExceptionHolder tKillHolder = new ExceptionHolder(); - testKillQueryInternal("user1", System.getProperty("user.name"), true, tExecuteHolder, tKillHolder); - assertNotNull("tExecute", tExecuteHolder.throwable); - assertEquals(HiveStatement.QUERY_CANCELLED_MESSAGE + " "+ KillQueriesOperation.KILL_QUERY_MESSAGE, - tExecuteHolder.throwable.getMessage()); - assertNull("tCancel", tKillHolder.throwable); - } - - @Test - public void testKillQueryByTagOwner() throws Exception { - ExceptionHolder tExecuteHolder = new ExceptionHolder(); - ExceptionHolder tKillHolder = new ExceptionHolder(); - testKillQueryInternal("user1", "user1", true, tExecuteHolder, tKillHolder); - assertNotNull("tExecute", tExecuteHolder.throwable); - assertEquals(HiveStatement.QUERY_CANCELLED_MESSAGE + " "+ KillQueriesOperation.KILL_QUERY_MESSAGE, - tExecuteHolder.throwable.getMessage()); - assertNull("tCancel", tKillHolder.throwable); - } - - @Test - public void testConcurrentAddAndCloseAndCloseAllConnections() throws Exception { - createTestTable("testtab1"); - - String url = miniHS2.getJdbcURL(); - String user = System.getProperty("user.name"); - String pwd = user; - - InputFormat inputFormat = getInputFormat(); - - // Get splits - JobConf job = new JobConf(conf); - job.set(LlapBaseInputFormat.URL_KEY, url); - job.set(LlapBaseInputFormat.USER_KEY, user); - job.set(LlapBaseInputFormat.PWD_KEY, pwd); - job.set(LlapBaseInputFormat.QUERY_KEY, "select * from testtab1"); - - final String[] handleIds = IntStream.range(0, 20).boxed().map(i -> "handleId-" + i).toArray(String[]::new); - - final ExceptionHolder exceptionHolder = new ExceptionHolder(); - - // addConnThread thread will keep adding connections - // closeConnThread thread tries close connection(s) associated to handleIds, one at a time - // closeAllConnThread thread tries to close All at once. - - final int numIterations = 100; - final Iterator addConnIterator = Iterables.cycle(handleIds).iterator(); - Thread addConnThread = new Thread(() -> executeNTimes(() -> { - String handleId = addConnIterator.next(); - job.set(LlapBaseInputFormat.HANDLE_ID, handleId); - InputSplit[] splits = inputFormat.getSplits(job, 1); - assertTrue(splits.length > 0); - return null; - }, numIterations, 1, exceptionHolder)); - - final Iterator removeConnIterator = Iterables.cycle(handleIds).iterator(); - Thread closeConnThread = new Thread(() -> executeNTimes(() -> { - String handleId = removeConnIterator.next(); - LlapBaseInputFormat.close(handleId); - return null; - }, numIterations, 2, exceptionHolder)); - - Thread closeAllConnThread = new Thread(() -> executeNTimes(() -> { - LlapBaseInputFormat.closeAll(); - return null; - }, numIterations, 5, exceptionHolder)); - - addConnThread.start(); - closeConnThread.start(); - closeAllConnThread.start(); - - closeAllConnThread.join(); - closeConnThread.join(); - addConnThread.join(); - - Throwable throwable = exceptionHolder.throwable; - assertNull("Something went wrong while testAddCloseCloseAllConnections" + throwable, throwable); - - } - - @Override - @Ignore - public void testMultipleBatchesOfComplexTypes() { - // ToDo: FixMe - } - - private void executeNTimes(Callable action, int noOfTimes, long intervalMillis, ExceptionHolder exceptionHolder) { - for (int i = 0; i < noOfTimes; i++) { - try { - action.call(); - Thread.sleep(intervalMillis); - } catch (Exception e) { - // populate first exception only - if (exceptionHolder.throwable == null) { - synchronized (this) { - if (exceptionHolder.throwable == null) { - exceptionHolder.throwable = e; - } - } - } - } - } - } - -} - diff --git a/itests/hive-unit/src/test/java/org/apache/hive/jdbc/TestJdbcWithMiniLlapRow.java b/itests/hive-unit/src/test/java/org/apache/hive/jdbc/TestJdbcWithMiniLlapRow.java index 2c2836e83e1f..0718239d84ad 100644 --- a/itests/hive-unit/src/test/java/org/apache/hive/jdbc/TestJdbcWithMiniLlapRow.java +++ b/itests/hive-unit/src/test/java/org/apache/hive/jdbc/TestJdbcWithMiniLlapRow.java @@ -26,7 +26,6 @@ import org.junit.After; import org.apache.hadoop.mapred.InputFormat; import org.apache.hadoop.hive.conf.HiveConf; -import org.apache.hadoop.hive.conf.HiveConf.ConfVars; import org.junit.Ignore; /** @@ -38,7 +37,6 @@ public class TestJdbcWithMiniLlapRow extends BaseJdbcWithMiniLlap { @BeforeClass public static void beforeTest() throws Exception { HiveConf conf = defaultConf(); - conf.setBoolVar(ConfVars.LLAP_OUTPUT_FORMAT_ARROW, false); BaseJdbcWithMiniLlap.beforeTest(conf); } diff --git a/itests/hive-unit/src/test/java/org/apache/hive/jdbc/TestJdbcWithMiniLlapVectorArrow.java b/itests/hive-unit/src/test/java/org/apache/hive/jdbc/TestJdbcWithMiniLlapVectorArrow.java deleted file mode 100644 index f7cb4062c487..000000000000 --- a/itests/hive-unit/src/test/java/org/apache/hive/jdbc/TestJdbcWithMiniLlapVectorArrow.java +++ /dev/null @@ -1,440 +0,0 @@ -/* - * 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.jdbc; - -import static java.util.Arrays.asList; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertArrayEquals; - -import java.math.BigDecimal; - -import com.google.common.collect.ImmutableMap; -import org.apache.hadoop.hive.common.type.Date; -import org.apache.hadoop.hive.common.type.Timestamp; - -import java.sql.Statement; -import java.util.ArrayList; -import java.util.List; -import java.util.Map; -import org.apache.hadoop.hive.llap.FieldDesc; -import org.apache.hadoop.hive.llap.Row; -import org.apache.hadoop.io.NullWritable; -import org.junit.BeforeClass; -import org.apache.hadoop.hive.conf.HiveConf; -import org.apache.hadoop.hive.conf.HiveConf.ConfVars; - -import org.apache.hadoop.mapred.InputFormat; -import org.apache.hadoop.hive.llap.LlapArrowRowInputFormat; -import org.junit.Ignore; -import org.junit.Test; - -/** - * TestJdbcWithMiniLlap for Arrow format with vectorized output sink - */ -@Ignore("unstable HIVE-23549") -public class TestJdbcWithMiniLlapVectorArrow extends BaseJdbcWithMiniLlap { - - - @BeforeClass - public static void beforeTest() throws Exception { - HiveConf conf = defaultConf(); - conf.setBoolVar(ConfVars.LLAP_OUTPUT_FORMAT_ARROW, true); - conf.setBoolVar(ConfVars.HIVE_VECTORIZATION_FILESINK_ARROW_NATIVE_ENABLED, true); - BaseJdbcWithMiniLlap.beforeTest(conf); - } - - @Override - protected InputFormat getInputFormat() { - //For unit testing, no harm in hard-coding allocator ceiling to LONG.MAX_VALUE - return new LlapArrowRowInputFormat(Long.MAX_VALUE); - } - - @Override - public void testDataTypes() throws Exception { - createDataTypesTable("datatypes"); - RowCollector2 rowCollector = new RowCollector2(); - String query = "select * from datatypes"; - int rowCount = processQuery(query, 1, rowCollector); - assertEquals(3, rowCount); - - // Verify schema - String[][] colNameTypes = new String[][] { - {"datatypes.c1", "int"}, - {"datatypes.c2", "boolean"}, - {"datatypes.c3", "double"}, - {"datatypes.c4", "string"}, - {"datatypes.c5", "array"}, - {"datatypes.c6", "map"}, - {"datatypes.c7", "map"}, - {"datatypes.c8", "struct"}, - {"datatypes.c9", "tinyint"}, - {"datatypes.c10", "smallint"}, - {"datatypes.c11", "float"}, - {"datatypes.c12", "bigint"}, - {"datatypes.c13", "array>"}, - {"datatypes.c14", "map>"}, - {"datatypes.c15", "struct>"}, - {"datatypes.c16", "array,n:int>>"}, - {"datatypes.c17", "timestamp"}, - {"datatypes.c18", "decimal(16,7)"}, - {"datatypes.c19", "binary"}, - {"datatypes.c20", "date"}, - {"datatypes.c21", "varchar(20)"}, - {"datatypes.c22", "char(15)"}, - {"datatypes.c23", "binary"}, - }; - FieldDesc fieldDesc; - assertEquals(23, rowCollector.numColumns); - for (int idx = 0; idx < rowCollector.numColumns; ++idx) { - fieldDesc = rowCollector.schema.getColumns().get(idx); - assertEquals("ColName idx=" + idx, colNameTypes[idx][0], fieldDesc.getName()); - assertEquals("ColType idx=" + idx, colNameTypes[idx][1], fieldDesc.getTypeInfo().getTypeName()); - } - - // First row is all nulls - Object[] rowValues = rowCollector.rows.get(0); - for (int idx = 0; idx < rowCollector.numColumns; ++idx) { - assertEquals("idx=" + idx, null, rowValues[idx]); - } - - // Second Row - rowValues = rowCollector.rows.get(1); - assertEquals(Integer.valueOf(-1), rowValues[0]); - assertEquals(Boolean.FALSE, rowValues[1]); - assertEquals(Double.valueOf(-1.1d), rowValues[2]); - assertEquals("", rowValues[3]); - - List c5Value = (List) rowValues[4]; - assertEquals(0, c5Value.size()); - - Map c6Value = (Map) rowValues[5]; - assertEquals(1, c6Value.size()); - assertEquals(null, c6Value.get(1)); - - Map c7Value = (Map) rowValues[6]; - assertEquals(1, c7Value.size()); - assertEquals("b", c7Value.get("a")); - - List c8Value = (List) rowValues[7]; - assertEquals(null, c8Value.get(0)); - assertEquals(null, c8Value.get(1)); - assertEquals(null, c8Value.get(2)); - - assertEquals(Byte.valueOf((byte) -1), rowValues[8]); - assertEquals(Short.valueOf((short) -1), rowValues[9]); - assertEquals(Float.valueOf(-1.0f), rowValues[10]); - assertEquals(Long.valueOf(-1l), rowValues[11]); - - List c13Value = (List) rowValues[12]; - assertEquals(0, c13Value.size()); - - Map c14Value = (Map) rowValues[13]; - assertEquals(1, c14Value.size()); - Map mapVal = (Map) c14Value.get(Integer.valueOf(1)); - assertEquals(1, mapVal.size()); - assertEquals(100, mapVal.get(Integer.valueOf(10))); - - List c15Value = (List) rowValues[14]; - assertEquals(null, c15Value.get(0)); - assertEquals(null, c15Value.get(1)); - - List c16Value = (List) rowValues[15]; - assertEquals(0, c16Value.size()); - - assertEquals(null, rowValues[16]); - assertEquals(null, rowValues[17]); - assertEquals(null, rowValues[18]); - assertEquals(null, rowValues[19]); - assertEquals(null, rowValues[20]); - assertEquals(null, rowValues[21]); - assertEquals(null, rowValues[22]); - - // Third row - rowValues = rowCollector.rows.get(2); - assertEquals(Integer.valueOf(1), rowValues[0]); - assertEquals(Boolean.TRUE, rowValues[1]); - assertEquals(Double.valueOf(1.1d), rowValues[2]); - assertEquals("1", rowValues[3]); - - c5Value = (List) rowValues[4]; - assertEquals(2, c5Value.size()); - assertEquals(Integer.valueOf(1), c5Value.get(0)); - assertEquals(Integer.valueOf(2), c5Value.get(1)); - - c6Value = (Map) rowValues[5]; - assertEquals(2, c6Value.size()); - assertEquals("x", c6Value.get(Integer.valueOf(1))); - assertEquals("y", c6Value.get(Integer.valueOf(2))); - - c7Value = (Map) rowValues[6]; - assertEquals(2, c7Value.size()); - assertEquals("v", c7Value.get("k")); - assertEquals("c", c7Value.get("b")); - - c8Value = (List) rowValues[7]; - assertEquals("a", c8Value.get(0)); - assertEquals(Integer.valueOf(9), c8Value.get(1)); - assertEquals(Double.valueOf(2.2d), c8Value.get(2)); - - assertEquals(Byte.valueOf((byte) 1), rowValues[8]); - assertEquals(Short.valueOf((short) 1), rowValues[9]); - assertEquals(Float.valueOf(1.0f), rowValues[10]); - assertEquals(Long.valueOf(1l), rowValues[11]); - - c13Value = (List) rowValues[12]; - assertEquals(2, c13Value.size()); - List listVal = (List) c13Value.get(0); - assertEquals("a", listVal.get(0)); - assertEquals("b", listVal.get(1)); - listVal = (List) c13Value.get(1); - assertEquals("c", listVal.get(0)); - assertEquals("d", listVal.get(1)); - - c14Value = (Map) rowValues[13]; - assertEquals(2, c14Value.size()); - mapVal = (Map) c14Value.get(Integer.valueOf(1)); - assertEquals(2, mapVal.size()); - assertEquals(Integer.valueOf(12), mapVal.get(Integer.valueOf(11))); - assertEquals(Integer.valueOf(14), mapVal.get(Integer.valueOf(13))); - mapVal = (Map) c14Value.get(Integer.valueOf(2)); - assertEquals(1, mapVal.size()); - assertEquals(Integer.valueOf(22), mapVal.get(Integer.valueOf(21))); - - c15Value = (List) rowValues[14]; - assertEquals(Integer.valueOf(1), c15Value.get(0)); - listVal = (List) c15Value.get(1); - assertEquals(2, listVal.size()); - assertEquals(Integer.valueOf(2), listVal.get(0)); - assertEquals("x", listVal.get(1)); - - c16Value = (List) rowValues[15]; - assertEquals(2, c16Value.size()); - listVal = (List) c16Value.get(0); - assertEquals(2, listVal.size()); - mapVal = (Map) listVal.get(0); - assertEquals(0, mapVal.size()); - assertEquals(Integer.valueOf(1), listVal.get(1)); - listVal = (List) c16Value.get(1); - mapVal = (Map) listVal.get(0); - assertEquals(2, mapVal.size()); - assertEquals("b", mapVal.get("a")); - assertEquals("d", mapVal.get("c")); - assertEquals(Integer.valueOf(2), listVal.get(1)); - - assertEquals(Timestamp.valueOf("2012-04-22 09:00:00.123456"), rowValues[16]); - assertEquals(new BigDecimal("123456789.123456"), rowValues[17]); - assertArrayEquals("abcd".getBytes("UTF-8"), (byte[]) rowValues[18]); - assertEquals(Date.valueOf("2013-01-01"), rowValues[19]); - assertEquals("abc123", rowValues[20]); - assertEquals("abc123 ", rowValues[21]); - - assertArrayEquals("X'01FF'".getBytes("UTF-8"), (byte[]) rowValues[22]); - } - - - // ToDo: Fix me - @Ignore - @Test - public void testTypesNestedInListWithLimitAndFilters() throws Exception { - try (Statement statement = hs2Conn.createStatement()) { - statement.execute("CREATE TABLE complex_tbl(c1 array, " + - "c2 array>, " + - "c3 array>>, " + - "c4 int) STORED AS ORC"); - - statement.executeUpdate("INSERT INTO complex_tbl VALUES " + - "(" + - "ARRAY('a1', 'a2', 'a3', null), " + - "ARRAY(NAMED_STRUCT('f1','a1', 'f2','a2'), NAMED_STRUCT('f1','a3', 'f2','a4')), " + - "ARRAY((ARRAY(NAMED_STRUCT('f1','a1', 'f2','a2'), NAMED_STRUCT('f1','a3', 'f2','a4')))), " + - "1), " + - "(" + - "ARRAY('b1'), " + - "ARRAY(NAMED_STRUCT('f1','b1', 'f2','b2'), NAMED_STRUCT('f1','b3', 'f2','b4')), " + - "ARRAY((ARRAY(NAMED_STRUCT('f1','b1', 'f2','b2'), NAMED_STRUCT('f1','b3', 'f2','b4'))), " + - "(ARRAY(NAMED_STRUCT('f1','b5', 'f2','b6'), NAMED_STRUCT('f1','b7', 'f2','b8')))), " + - "2), " + - "(" + - "ARRAY('c1', 'c2'), ARRAY(NAMED_STRUCT('f1','c1', 'f2','c2'), NAMED_STRUCT('f1','c3', 'f2','c4'), " + - "NAMED_STRUCT('f1','c5', 'f2','c6')), ARRAY((ARRAY(NAMED_STRUCT('f1','c1', 'f2','c2'), " + - "NAMED_STRUCT('f1','c3', 'f2','c4'))), (ARRAY(NAMED_STRUCT('f1','c5', 'f2','c6'), " + - "NAMED_STRUCT('f1','c7', 'f2','c8'))), (ARRAY(NAMED_STRUCT('f1','c9', 'f2','c10'), " + - "NAMED_STRUCT('f1','c11', 'f2','c12')))), " + - "3), " + - "(" + - "ARRAY(null), " + - "ARRAY(NAMED_STRUCT('f1','d1', 'f2','d2'), NAMED_STRUCT('f1','d3', 'f2','d4'), " + - "NAMED_STRUCT('f1','d5', 'f2','d6'), NAMED_STRUCT('f1','d7', 'f2','d8')), " + - "ARRAY((ARRAY(NAMED_STRUCT('f1','d1', 'f2', 'd2')))), " + - "4)"); - - } - - List expected = new ArrayList<>(); - expected.add(new Object[]{ - asList("a1", "a2", "a3", null), - asList(asList("a1", "a2"), asList("a3", "a4")), - asList(asList(asList("a1", "a2"), asList("a3", "a4"))), - 1 - }); - expected.add(new Object[]{ - asList("b1"), - asList(asList("b1", "b2"), asList("b3", "b4")), - asList(asList(asList("b1", "b2"), asList("b3", "b4")), asList(asList("b5", "b6"), asList("b7", "b8"))), - 2 - }); - expected.add(new Object[]{ - asList("c1", "c2"), - asList(asList("c1", "c2"), asList("c3", "c4"), asList("c5", "c6")), - asList(asList(asList("c1", "c2"), asList("c3", "c4")), asList(asList("c5", "c6"), asList("c7", "c8")), - asList(asList("c9", "c10"), asList("c11", "c12"))), - 3 - }); - List nullList = new ArrayList<>(); - nullList.add(null); - expected.add(new Object[]{ - nullList, - asList(asList("d1", "d2"), asList("d3", "d4"), asList("d5", "d6"), asList("d7", "d8")), - asList(asList(asList("d1", "d2"))), - 4 - }); - - // test without limit and filters (i.e VectorizedRowBatch#selectedInUse=false) - RowCollector2 rowCollector = new RowCollector2(); - String query = "select * from complex_tbl"; - processQuery(query, 1, rowCollector); - verifyResult(rowCollector.rows, expected.get(0), - expected.get(1), - expected.get(2), - expected.get(3)); - - // test with filter - rowCollector = new RowCollector2(); - query = "select * from complex_tbl where c4 > 1 "; - processQuery(query, 1, rowCollector); - verifyResult(rowCollector.rows, expected.get(1), expected.get(2), expected.get(3)); - - // test with limit - rowCollector = new RowCollector2(); - query = "select * from complex_tbl limit 3"; - processQuery(query, 1, rowCollector); - verifyResult(rowCollector.rows, expected.get(0), expected.get(1), expected.get(2)); - - // test with filters and limit - rowCollector = new RowCollector2(); - query = "select * from complex_tbl where c4 > 1 limit 2"; - processQuery(query, 1, rowCollector); - verifyResult(rowCollector.rows, expected.get(1), expected.get(2)); - - } - - // ToDo: Fix me - @Ignore - @Test - public void testTypesNestedInMapWithLimitAndFilters() throws Exception { - try (Statement statement = hs2Conn.createStatement()) { - statement.execute("CREATE TABLE complex_tbl2(c1 map," + - " c2 map>, " + - " c3 map>, c4 int) STORED AS ORC"); - - statement.executeUpdate("INSERT INTO complex_tbl2 VALUES " + - "(MAP(1, 'a1'), MAP(1, ARRAY('a1', 'a2')), MAP(1, NAMED_STRUCT('f1','a1', 'f2','a2')), " + - "1), " + - "(MAP(1, 'b1',2, 'b2'), MAP(1, ARRAY('b1', 'b2'), 2, ARRAY('b3') ), " + - "MAP(1, NAMED_STRUCT('f1','b1', 'f2','b2')), " + - "2), " + - "(MAP(1, 'c1',2, 'c2'), MAP(1, ARRAY('c1', 'c2'), 2, ARRAY('c3') ), " + - "MAP(1, NAMED_STRUCT('f1','c1', 'f2','c2'), 2, NAMED_STRUCT('f1', 'c3', 'f2', 'c4') ), " + - "3)"); - - } - - List expected = new ArrayList<>(); - expected.add(new Object[]{ - ImmutableMap.of(1, "a1"), - ImmutableMap.of(1, asList("a1", "a2")), - ImmutableMap.of(1, asList("a1", "a2")), - 1, - }); - expected.add(new Object[]{ - ImmutableMap.of(1, "b1", 2, "b2"), - ImmutableMap.of(1, asList("b1", "b2"), 2, asList("b3")), - ImmutableMap.of(1, asList("b1", "b2")), - 2, - }); - expected.add(new Object[]{ - ImmutableMap.of(1, "c1", 2, "c2"), - ImmutableMap.of(1, asList("c1", "c2"), 2, asList("c3")), - ImmutableMap.of(1, asList("c1", "c2"), 2, asList("c3", "c4")), - 3, - }); - - - // test without limit and filters (i.e. VectorizedRowBatch#selectedInUse=false) - RowCollector2 rowCollector = new RowCollector2(); - String query = "select * from complex_tbl2"; - processQuery(query, 1, rowCollector); - verifyResult(rowCollector.rows, expected.get(0), expected.get(1), expected.get(2)); - - // test with filter - rowCollector = new RowCollector2(); - query = "select * from complex_tbl2 where c4 > 1 "; - processQuery(query, 1, rowCollector); - verifyResult(rowCollector.rows, expected.get(1), expected.get(2)); - - // test with limit - rowCollector = new RowCollector2(); - query = "select * from complex_tbl2 limit 2"; - processQuery(query, 1, rowCollector); - verifyResult(rowCollector.rows, expected.get(0), expected.get(1)); - - // test with filters and limit - rowCollector = new RowCollector2(); - query = "select * from complex_tbl2 where c4 > 1 limit 1"; - processQuery(query, 1, rowCollector); - verifyResult(rowCollector.rows, expected.get(1)); - - } - - private void verifyResult(List actual, Object[]... expected) { - assertEquals(expected.length, actual.size()); - for (int i = 0; i < expected.length; i++) { - assertArrayEquals(expected[i], actual.get(i)); - } - } - - @Override - @Ignore - public void testMultipleBatchesOfComplexTypes() { - // ToDo: FixMe - } - - @Override - @Ignore - public void testComplexQuery() { - // ToDo: FixMe - } - - @Override - @Ignore - public void testLlapInputFormatEndToEnd() { - // ToDo: FixMe - } -} - diff --git a/itests/hive-unit/src/test/java/org/apache/hive/jdbc/TestJdbcWithMiniLlapVectorArrowBatch.java b/itests/hive-unit/src/test/java/org/apache/hive/jdbc/TestJdbcWithMiniLlapVectorArrowBatch.java deleted file mode 100644 index 9ed40d702670..000000000000 --- a/itests/hive-unit/src/test/java/org/apache/hive/jdbc/TestJdbcWithMiniLlapVectorArrowBatch.java +++ /dev/null @@ -1,489 +0,0 @@ -/* - * 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.jdbc; - -import com.google.common.collect.ImmutableMap; -import com.google.common.collect.Lists; -import org.apache.arrow.memory.BufferAllocator; -import org.apache.arrow.vector.FieldVector; -import org.apache.commons.collections4.MultiSet; -import org.apache.commons.collections4.multiset.HashMultiSet; -import org.apache.hadoop.hive.common.type.CalendarUtils; -import org.apache.hadoop.hive.conf.HiveConf; -import org.apache.hadoop.hive.conf.HiveConf.ConfVars; -import org.apache.hadoop.hive.llap.LlapArrowRowInputFormat; -import org.apache.hadoop.hive.llap.LlapBaseInputFormat; -import org.apache.hadoop.hive.llap.Row; -import org.apache.hadoop.hive.ql.io.arrow.ArrowWrapperWritable; -import org.apache.hadoop.hive.ql.io.arrow.RootAllocatorFactory; -import org.apache.hadoop.io.NullWritable; -import org.apache.hadoop.mapred.InputFormat; -import org.apache.hadoop.mapred.InputSplit; -import org.apache.hadoop.mapred.JobConf; -import org.apache.hadoop.mapred.RecordReader; -import org.junit.BeforeClass; -import org.junit.Ignore; -import org.junit.Test; - -import java.sql.SQLException; -import java.sql.Statement; -import java.util.Arrays; -import java.util.ArrayList; -import java.util.Map; -import java.util.List; -import java.util.UUID; -import java.util.stream.Collectors; - -import static org.apache.hadoop.hive.conf.HiveConf.ConfVars.HIVE_AVRO_PROLEPTIC_GREGORIAN_DEFAULT; -import static org.apache.hadoop.hive.conf.HiveConf.ConfVars.HIVE_PARQUET_DATE_PROLEPTIC_GREGORIAN_DEFAULT; -import static org.apache.hadoop.hive.conf.HiveConf.ConfVars.LLAP_EXTERNAL_CLIENT_USE_HYBRID_CALENDAR; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertTrue; - -/** - * TestJdbcWithMiniLlap for Arrow format - uses batch record reader. - * We can obtain arrow batches and compare the results. - * - */ -public class TestJdbcWithMiniLlapVectorArrowBatch extends BaseJdbcWithMiniLlap { - - private final MultiSet> legacyDateExpectedOutput = getLegacyDateExpectedOutput(); - private final MultiSet> hybridMixedDateExpectedOutput = getHybridMixedDateExpectedOutput(); - - private final MultiSet> legacyTimestampExpectedOutput = getLegacyTimestampExpectedOutput(); - private final MultiSet> convertedLegacyTimestampExpectedOutput = - getConvertedLegacyTimestampExpectedOutput(); - private final MultiSet> hybridMixedTimestampExpectedOutput = getHybridMixedTimestampExpectedOutput(); - - @BeforeClass public static void beforeTest() throws Exception { - HiveConf conf = defaultConf(); - conf.setBoolVar(ConfVars.LLAP_OUTPUT_FORMAT_ARROW, true); - conf.setBoolVar(ConfVars.HIVE_VECTORIZATION_FILESINK_ARROW_NATIVE_ENABLED, true); - BaseJdbcWithMiniLlap.beforeTest(conf); - } - - @Override protected InputFormat getInputFormat() { - return new LlapArrowRowInputFormat(Long.MAX_VALUE); - } - - private MultiSet> getHybridMixedDateExpectedOutput() { - MultiSet> expected = new HashMultiSet<>(); - expected.add(Lists.newArrayList("2012-02-21")); - expected.add(Lists.newArrayList("2014-02-11")); - expected.add(Lists.newArrayList("1947-02-11")); - expected.add(Lists.newArrayList("8200-02-11")); - expected.add(Lists.newArrayList("1012-02-21")); - expected.add(Lists.newArrayList("1014-02-11")); - expected.add(Lists.newArrayList("0947-02-11")); - expected.add(Lists.newArrayList("0200-02-11")); - return expected; - } - - private MultiSet> getLegacyDateExpectedOutput() { - MultiSet> expected = new HashMultiSet<>(); - expected.add(Lists.newArrayList("2012-02-21")); - expected.add(Lists.newArrayList("2014-02-11")); - expected.add(Lists.newArrayList("1947-02-11")); - expected.add(Lists.newArrayList("8200-02-11")); - expected.add(Lists.newArrayList("1012-02-27")); - expected.add(Lists.newArrayList("1014-02-17")); - expected.add(Lists.newArrayList("0947-02-16")); - expected.add(Lists.newArrayList("0200-02-10")); - return expected; - } - - private MultiSet> getHybridMixedTimestampExpectedOutput() { - MultiSet> expected = new HashMultiSet<>(); - expected.add(Lists.newArrayList("2012-02-21 07:08:09.123")); - expected.add(Lists.newArrayList("2014-02-11 07:08:09.123")); - expected.add(Lists.newArrayList("1947-02-11 07:08:09.123")); - expected.add(Lists.newArrayList("8200-02-11 07:08:09.123")); - expected.add(Lists.newArrayList("1012-02-21 07:08:09.123")); - expected.add(Lists.newArrayList("1014-02-11 07:08:09.123")); - expected.add(Lists.newArrayList("0947-02-11 07:08:09.123")); - expected.add(Lists.newArrayList("0200-02-11 07:08:09.123")); - return expected; - } - - private MultiSet> getLegacyTimestampExpectedOutput() { - MultiSet> expected = new HashMultiSet<>(); - expected.add(Lists.newArrayList("2012-02-21 07:08:09.123")); - expected.add(Lists.newArrayList("2014-02-11 07:08:09.123")); - expected.add(Lists.newArrayList("1947-02-11 07:08:09.123")); - expected.add(Lists.newArrayList("8200-02-11 07:08:09.123")); - expected.add(Lists.newArrayList("1012-02-21 07:08:09.123")); - expected.add(Lists.newArrayList("1014-02-11 07:08:09.123")); - expected.add(Lists.newArrayList("0947-02-11 07:08:09.123")); - expected.add(Lists.newArrayList("0200-02-11 07:08:09.123")); - return expected; - } - - private MultiSet> getConvertedLegacyTimestampExpectedOutput() { - MultiSet> expected = new HashMultiSet<>(); - expected.add(Lists.newArrayList("2012-02-21 07:08:09.123")); - expected.add(Lists.newArrayList("2014-02-11 07:08:09.123")); - expected.add(Lists.newArrayList("1947-02-11 07:08:09.123")); - expected.add(Lists.newArrayList("8200-02-11 07:08:09.123")); - expected.add(Lists.newArrayList("1012-02-27 07:08:09.123")); - expected.add(Lists.newArrayList("1014-02-17 07:08:09.123")); - expected.add(Lists.newArrayList("0947-02-16 07:08:09.123")); - expected.add(Lists.newArrayList("0200-02-10 07:08:09.123")); - return expected; - } - - // test newly inserted orc records which have calendar info in orc footer. - // similar to ql/src/test/queries/clientpositive/orc_hybrid_mixed_date.q - @Test public void testOrcHybridMixedDates() throws Exception { - - final String tableName = "testOrcHybridMixedDates"; - executeSQL("create table " + tableName + " (d date) stored as orc"); - executeSQL("INSERT INTO " + tableName + " VALUES " + "('2012-02-21'), " + "('2014-02-11'), " + "('1947-02-11'), " - + "('8200-02-11'), " + "('1012-02-21'), " + "('1014-02-11'), " + "('0947-02-11'), " + "('0200-02-11')"); - - final String query = "select * from " + tableName; - - testDateQueries(query, "orc.proleptic.gregorian.default", hybridMixedDateExpectedOutput, - hybridMixedDateExpectedOutput); - } - - // test with legacy orc files - // similar to ql/src/test/queries/clientpositive/orc_legacy_mixed_date.q - @Test public void testOrcLegacyMixedDates() throws Exception { - - final String tableName = "testOrcLegacyMixedDates"; - executeSQL("create table " + tableName + " (d date) stored as orc"); - executeSQL("load data local inpath '../../data/files/orc_legacy_mixed_dates.orc' into table " + tableName); - - final String query = "select * from " + tableName; - - // ORC properties (here orc.proleptic.gregorian.default) are not propogated to LLAP as of now - // and hence the expected output hybridMixedDateExpectedOutput otherwise it should be hybridMixedDateExpectedOutput ideally - testDateQueries(query, "orc.proleptic.gregorian.default", hybridMixedDateExpectedOutput, - hybridMixedDateExpectedOutput); - } - - // test newly inserted orc records which have calendar info in orc footer. - // similar to ql/src/test/queries/clientpositive/orc_hybrid_mixed_timestamp.q - @Test public void testOrcHybridMixedTimestamps() throws Exception { - - final String tableName = "testOrcHybridMixedTimestamps"; - executeSQL("create table " + tableName + " (d timestamp) stored as orc"); - executeSQL("INSERT INTO " + tableName + " VALUES ('2012-02-21 07:08:09.123')," + "('2014-02-11 07:08:09.123')," - + "('1947-02-11 07:08:09.123')," + "('8200-02-11 07:08:09.123')," + "('1012-02-21 07:08:09.123')," - + "('1014-02-11 07:08:09.123')," + "('0947-02-11 07:08:09.123')," + "('0200-02-11 07:08:09.123')"); - - final String query = "select * from " + tableName; - - testTimestampQueries(query, "orc.proleptic.gregorian.default", hybridMixedTimestampExpectedOutput, - hybridMixedTimestampExpectedOutput); - } - - // test with legacy parquet files - // similar to ql/src/test/queries/clientpositive/orc_legacy_mixed_timestamp.q - @Test public void testOrcLegacyMixedTimestamps() throws Exception { - - final String tableName = "testOrcLegacyMixedTimestamps"; - executeSQL("create table " + tableName + " (ts timestamp) stored as orc"); - executeSQL("load data local inpath '../../data/files/orc_legacy_mixed_timestamps.orc' into table " + tableName); - - final String query = "select * from " + tableName; - - // ORC properties (here orc.proleptic.gregorian.default) are not propogated to LLAP as of now - testTimestampQueries(query, "orc.proleptic.gregorian.default", legacyTimestampExpectedOutput, - legacyTimestampExpectedOutput); - } - - // test with new parquet files - // similar to ql/src/test/queries/clientpositive/parquet_hybrid_mixed_date.q - @Test public void testParquetHybridMixedDates() throws Exception { - - final String tableName = "testParquetHybrcidMixedDates"; - executeSQL("create table " + tableName + " (d date) stored as parquet"); - executeSQL("INSERT INTO " + tableName + " VALUES " + "('2012-02-21'), " + "('2014-02-11'), " + "('1947-02-11'), " - + "('8200-02-11'), " + "('1012-02-21'), " + "('1014-02-11'), " + "('0947-02-11'), " + "('0200-02-11')"); - - final String query = "select * from " + tableName; - - testDateQueries(query, HIVE_PARQUET_DATE_PROLEPTIC_GREGORIAN_DEFAULT.toString(), hybridMixedDateExpectedOutput, - hybridMixedDateExpectedOutput); - } - - // test with legacy parquet files - // similar to ql/src/test/queries/clientpositive/parquet_legacy_mixed_date.q - @Test public void testParquetLegacyMixedDates() throws Exception { - - final String tableName = "testParquetLegacyMixedDates"; - - executeSQL("create table " + tableName + " (d date) stored as parquet"); - executeSQL("load data local inpath '../../data/files/parquet_legacy_mixed_dates.parq' into table " + tableName); - - final String query = "select * from " + tableName; - - testDateQueries(query, HIVE_PARQUET_DATE_PROLEPTIC_GREGORIAN_DEFAULT.toString(), hybridMixedDateExpectedOutput, - legacyDateExpectedOutput); - } - - // test newly inserted parquet records. - // similar to ql/src/test/queries/clientpositive/parquet_hybrid_mixed_timestamp.q - @Test public void testParquetHybridMixedTimestamps() throws Exception { - - final String tableName = "testParquetHybridMixedTimestamps"; - executeSQL("create table " + tableName + " (ts timestamp) stored as parquet"); - executeSQL("INSERT INTO " + tableName + " VALUES ('2012-02-21 07:08:09.123')," + "('2014-02-11 07:08:09.123')," - + "('1947-02-11 07:08:09.123')," + "('8200-02-11 07:08:09.123')," + "('1012-02-21 07:08:09.123')," - + "('1014-02-11 07:08:09.123')," + "('0947-02-11 07:08:09.123')," + "('0200-02-11 07:08:09.123')"); - - final String query = "select * from " + tableName; - - testTimestampQueries(query, HIVE_PARQUET_DATE_PROLEPTIC_GREGORIAN_DEFAULT.toString(), - hybridMixedTimestampExpectedOutput, hybridMixedTimestampExpectedOutput); - } - - // test with legacy parquet files - // similar to ql/src/test/queries/clientpositive/parquet_legacy_mixed_timestamp.q - @Test public void testParquetLegacyMixedTimestamps() throws Exception { - - final String tableName = "testParquetLegacyMixedTimestamps"; - executeSQL("create table " + tableName + " (d timestamp) stored as parquet"); - executeSQL( - "load data local inpath '../../data/files/parquet_legacy_mixed_timestamps.parq' into table " + tableName); - - final String query = "select * from " + tableName; - - testTimestampQueries(query, HIVE_PARQUET_DATE_PROLEPTIC_GREGORIAN_DEFAULT.toString(), - hybridMixedTimestampExpectedOutput, hybridMixedTimestampExpectedOutput); - } - - // test with new avro files - // similar to ql/src/test/queries/clientpositive/avro_hybrid_mixed_date.q - @Test public void testAvroHybridMixedDates() throws Exception { - - final String tableName = "testAvroHybridMixedDates"; - executeSQL("create table " + tableName + " (d date) stored as avro"); - executeSQL("INSERT INTO " + tableName + " VALUES " + "('2012-02-21'), " + "('2014-02-11'), " + "('1947-02-11'), " - + "('8200-02-11'), " + "('1012-02-21'), " + "('1014-02-11'), " + "('0947-02-11'), " + "('0200-02-11')"); - - final String query = "select * from " + tableName; - - testDateQueries(query, HIVE_AVRO_PROLEPTIC_GREGORIAN_DEFAULT.toString(), hybridMixedDateExpectedOutput, - hybridMixedDateExpectedOutput); - } - - // test with legacy parquet files - // similar to ql/src/test/queries/clientpositive/avro_legacy_mixed_date.q - @Test public void testAvroLegacyMixedDates() throws Exception { - - final String tableName = "testAvroLegacyMixedDates"; - - executeSQL("create table " + tableName + " (d date) ROW FORMAT DELIMITED FIELDS TERMINATED BY '|' " - + "COLLECTION ITEMS TERMINATED BY ',' MAP KEYS TERMINATED BY ':' stored as avro"); - executeSQL("load data local inpath '../../data/files/avro_legacy_mixed_dates.avro' into table " + tableName); - - final String query = "select * from " + tableName; - - testDateQueries(query, HIVE_AVRO_PROLEPTIC_GREGORIAN_DEFAULT.toString(), hybridMixedDateExpectedOutput, - legacyDateExpectedOutput); - } - - // test newly inserted avro records - // similar to ql/src/test/queries/clientpositive/avro_hybrid_mixed_timestamp.q - @Test public void testAvroHybridMixedTimestamps() throws Exception { - - final String tableName = "testAvroHybridMixedTimestamps"; - executeSQL("create table " + tableName + " (d timestamp) ROW FORMAT DELIMITED FIELDS TERMINATED BY '|' " - + "COLLECTION ITEMS TERMINATED BY ',' MAP KEYS TERMINATED BY ':' stored as avro"); - executeSQL("INSERT INTO " + tableName + " VALUES ('2012-02-21 07:08:09.123')," + "('2014-02-11 07:08:09.123')," - + "('1947-02-11 07:08:09.123')," + "('8200-02-11 07:08:09.123')," + "('1012-02-21 07:08:09.123')," - + "('1014-02-11 07:08:09.123')," + "('0947-02-11 07:08:09.123')," + "('0200-02-11 07:08:09.123')"); - - final String query = "select * from " + tableName; - - testTimestampQueries(query, HIVE_AVRO_PROLEPTIC_GREGORIAN_DEFAULT.toString(), hybridMixedTimestampExpectedOutput, - hybridMixedTimestampExpectedOutput); - } - - // test with legacy avro files - // similar to ql/src/test/queries/clientpositive/avro_legacy_mixed_timestamp.q - @Test public void testAvroLegacyMixedTimestamps() throws Exception { - - final String tableName = "testAvroLegacyMixedTimestamps"; - executeSQL("create table " + tableName + "(d timestamp) ROW FORMAT DELIMITED FIELDS TERMINATED BY '|'" - + "COLLECTION ITEMS TERMINATED BY ',' MAP KEYS TERMINATED BY ':' stored as avro"); - executeSQL("load data local inpath '../../data/files/avro_legacy_mixed_timestamps.avro' into table " + tableName); - - final String query = "select * from " + tableName; - - testTimestampQueries(query, HIVE_AVRO_PROLEPTIC_GREGORIAN_DEFAULT.toString(), hybridMixedTimestampExpectedOutput, - convertedLegacyTimestampExpectedOutput); - } - - private void testDateQueries(final String query, final String fileReaderLevelCalendarProp, - MultiSet> newHybridExpected, MultiSet> legacyExpected) throws Exception { - - MultiSet> llapResultTransformed = transformLlapDateResultSet(runQueryUsingLlapArrowBatchReader(query, - ImmutableMap.of(LLAP_EXTERNAL_CLIENT_USE_HYBRID_CALENDAR.toString(), "true")), false); - assertEquals(newHybridExpected, llapResultTransformed); - - // test hybrid output with fileReaderLevelCalendarProp - llapResultTransformed = transformLlapDateResultSet(runQueryUsingLlapArrowBatchReader(query, ImmutableMap - .of(LLAP_EXTERNAL_CLIENT_USE_HYBRID_CALENDAR.toString(), "true", fileReaderLevelCalendarProp, "true")), false); - assertEquals(legacyExpected, llapResultTransformed); - } - - private void testTimestampQueries(final String query, final String fileReaderLevelCalendarProp, - MultiSet> newHybridExpected, MultiSet> legacyExpected) throws Exception { - - MultiSet> llapResultTransformed = transformLlapTimestampResultSet( - runQueryUsingLlapArrowBatchReader(query, - ImmutableMap.of(LLAP_EXTERNAL_CLIENT_USE_HYBRID_CALENDAR.toString(), "true")), false); - assertEquals(newHybridExpected, llapResultTransformed); - - // test hybrid output with fileReaderLevelCalendarProp - llapResultTransformed = transformLlapTimestampResultSet(runQueryUsingLlapArrowBatchReader(query, ImmutableMap - .of(LLAP_EXTERNAL_CLIENT_USE_HYBRID_CALENDAR.toString(), "true", fileReaderLevelCalendarProp, "true")), false); - assertEquals(legacyExpected, llapResultTransformed); - } - - private MultiSet> transformLlapDateResultSet(MultiSet> llapResult, - final boolean useProleptic) { - MultiSet> llapResultTransformed = new HashMultiSet<>(); - for (List list : llapResult) { - llapResultTransformed.add( - list.stream().map(ele -> CalendarUtils.formatDate((int) ele, useProleptic)).collect(Collectors.toList())); - } - return llapResultTransformed; - } - - private MultiSet> transformLlapTimestampResultSet(MultiSet> llapResult, - final boolean useProleptic) { - MultiSet> llapResultTransformed = new HashMultiSet<>(); - for (List list : llapResult) { - llapResultTransformed.add(list.stream().map(ele -> CalendarUtils.formatTimestamp((long) ele / 1000, useProleptic)) - .collect(Collectors.toList())); - } - return llapResultTransformed; - } - - private void executeSQL(String query, String... moreQueries) throws SQLException { - try (Statement stmt = hs2Conn.createStatement()) { - stmt.execute(query); - if (moreQueries != null) { - for (String q : moreQueries) { - stmt.execute(q); - } - } - } - } - - private MultiSet> runQueryUsingLlapArrowBatchReader(String query, Map extraHiveConfs) - throws Exception { - String url = miniHS2.getJdbcURL(); - - if (extraHiveConfs != null) { - url = url + "?" + extraHiveConfs.entrySet().stream().map(e -> e.getKey() + "=" + e.getValue()) - .collect(Collectors.joining(";")); - } - - String user = System.getProperty("user.name"); - String pwd = user; - String handleId = UUID.randomUUID().toString(); - - // Get splits - JobConf job = new JobConf(conf); - job.set(LlapBaseInputFormat.URL_KEY, url); - job.set(LlapBaseInputFormat.USER_KEY, user); - job.set(LlapBaseInputFormat.PWD_KEY, pwd); - job.set(LlapBaseInputFormat.QUERY_KEY, query); - job.set(LlapBaseInputFormat.HANDLE_ID, handleId); - job.set(LlapBaseInputFormat.USE_NEW_SPLIT_FORMAT, "false"); - - BufferAllocator allocator = RootAllocatorFactory.INSTANCE.getOrCreateRootAllocator(Long.MAX_VALUE) - .newChildAllocator(UUID.randomUUID().toString(), 0, Long.MAX_VALUE); - - LlapBaseInputFormat llapBaseInputFormat = new LlapBaseInputFormat(true, allocator); - InputSplit[] splits = llapBaseInputFormat.getSplits(job, 1); - - assertTrue(splits.length > 0); - - MultiSet> queryResult = new HashMultiSet<>(); - for (InputSplit split : splits) { - System.out.println("Processing split " + Arrays.toString(split.getLocations())); - RecordReader reader = llapBaseInputFormat.getRecordReader(split, job, null); - ArrowWrapperWritable wrapperWritable = new ArrowWrapperWritable(); - while (reader.next(NullWritable.get(), wrapperWritable)) { - queryResult.addAll(collectResultFromArrowVector(wrapperWritable)); - } - reader.close(); - } - LlapBaseInputFormat.close(handleId); - return queryResult; - } - - private MultiSet> collectResultFromArrowVector(ArrowWrapperWritable wrapperWritable) { - List fieldVectors = wrapperWritable.getVectorSchemaRoot().getFieldVectors(); - MultiSet> result = new HashMultiSet<>(); - int valueCount = fieldVectors.get(0).getValueCount(); - for (int recordIndex = 0; recordIndex < valueCount; recordIndex++) { - List row = new ArrayList<>(); - for (FieldVector fieldVector : fieldVectors) { - row.add(fieldVector.getObject(recordIndex)); - } - result.add(row); - } - return result; - } - - @Override public void testLlapInputFormatEndToEnd() throws Exception { - // to be implemented for this reader - } - - @Override public void testNonAsciiStrings() throws Exception { - // to be implemented for this reader - } - - @Override public void testEscapedStrings() throws Exception { - // to be implemented for this reader - } - - @Override public void testDataTypes() throws Exception { - // to be implemented for this reader - } - - @Override public void testComplexQuery() throws Exception { - // to be implemented for this reader - } - - @Override public void testKillQuery() throws Exception { - // to be implemented for this reader - } - - @Override - @Ignore - public void testMultipleBatchesOfComplexTypes() { - // ToDo: FixMe - } - - @Override - @Ignore - public void testLlapInputFormatEndToEndWithMultipleBatches() { - // ToDo: FixMe - } - -} - diff --git a/itests/hive-unit/src/test/java/org/apache/hive/jdbc/TestLlapExtClientWithCloudDeploymentConfigs.java b/itests/hive-unit/src/test/java/org/apache/hive/jdbc/TestLlapExtClientWithCloudDeploymentConfigs.java deleted file mode 100644 index 2b247c0bbf2f..000000000000 --- a/itests/hive-unit/src/test/java/org/apache/hive/jdbc/TestLlapExtClientWithCloudDeploymentConfigs.java +++ /dev/null @@ -1,139 +0,0 @@ -/* - * 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.jdbc; - -import org.apache.hadoop.hive.conf.HiveConf; -import org.apache.hadoop.hive.llap.LlapArrowRowInputFormat; -import org.apache.hadoop.hive.llap.LlapBaseInputFormat; -import org.apache.hadoop.hive.llap.LlapInputSplit; -import org.apache.hadoop.hive.llap.Row; -import org.apache.hadoop.io.NullWritable; -import org.apache.hadoop.mapred.InputFormat; -import org.apache.hadoop.mapred.InputSplit; -import org.apache.hadoop.mapred.JobConf; -import org.apache.hadoop.mapred.RecordReader; -import org.junit.BeforeClass; -import org.junit.Ignore; - -import java.net.InetAddress; -import java.util.ArrayList; -import java.util.List; -import java.util.UUID; - -/** - * TestLlapExtClientWithCloudDeploymentConfigs - */ -@org.junit.Ignore("flaky; see: HIVE-24364") -public class TestLlapExtClientWithCloudDeploymentConfigs extends BaseJdbcWithMiniLlap { - - @BeforeClass - public static void beforeTest() throws Exception { - System.setProperty("PUBLIC_HOSTNAME", InetAddress.getLocalHost().getHostAddress()); - - HiveConf conf = defaultConf(); - conf.set("minillap.usePortsFromConf", "true"); - - // enable setup for cloud based deployment - conf.setBoolVar(HiveConf.ConfVars.LLAP_EXTERNAL_CLIENT_CLOUD_DEPLOYMENT_SETUP_ENABLED, true); - conf.setVar(HiveConf.ConfVars.LLAP_EXTERNAL_CLIENT_CLOUD_JWT_SHARED_SECRET, - "Three may keep a secret, if two of them are dead -- Benjamin Franklin"); - - conf.setBoolVar(HiveConf.ConfVars.LLAP_OUTPUT_FORMAT_ARROW, true); - conf.setBoolVar(HiveConf.ConfVars.HIVE_VECTORIZATION_FILESINK_ARROW_NATIVE_ENABLED, true); - BaseJdbcWithMiniLlap.beforeTest(conf); - - } - - @Override protected InputFormat getInputFormat() { - //For unit testing, no harm in hard-coding allocator ceiling to LONG.MAX_VALUE - return new LlapArrowRowInputFormat(Long.MAX_VALUE); - } - - @Override public void testDataTypes() throws Exception { - TestJdbcWithMiniLlapVectorArrow testJdbcWithMiniLlapVectorArrow = new TestJdbcWithMiniLlapVectorArrow(); - testJdbcWithMiniLlapVectorArrow.testDataTypes(); - } - - @Override - @Ignore - public void testMultipleBatchesOfComplexTypes() { - // TODO: something else has broken parent test, need to check - } - - @Override protected int processQuery(String currentDatabase, String query, int numSplits, RowProcessor rowProcessor) - throws Exception { - String url = miniHS2.getJdbcURL(); - String user = System.getProperty("user.name"); - String pwd = user; - String handleId = UUID.randomUUID().toString(); - - InputFormat inputFormat = getInputFormat(); - - // Get splits - JobConf job = new JobConf(conf); - job.set(LlapBaseInputFormat.URL_KEY, url); - job.set(LlapBaseInputFormat.USER_KEY, user); - job.set(LlapBaseInputFormat.PWD_KEY, pwd); - job.set(LlapBaseInputFormat.QUERY_KEY, query); - job.set(LlapBaseInputFormat.HANDLE_ID, handleId); - job.set(LlapBaseInputFormat.USE_NEW_SPLIT_FORMAT, "true"); - if (currentDatabase != null) { - job.set(LlapBaseInputFormat.DB_KEY, currentDatabase); - } - - InputSplit[] splits = inputFormat.getSplits(job, numSplits); - - if (splits.length <= 1) { - return 0; - } - - - // populate actual splits with schema and planBytes[] - LlapInputSplit schemaSplit = (LlapInputSplit) splits[0]; - LlapInputSplit planSplit = (LlapInputSplit) splits[1]; - - List actualSplits = new ArrayList<>(); - - for (int i = 2; i < splits.length; i++) { - LlapInputSplit actualSplit = (LlapInputSplit) splits[i]; - actualSplit.setSchema(schemaSplit.getSchema()); - actualSplit.setPlanBytes(planSplit.getPlanBytes()); - actualSplits.add(actualSplit); - } - - // Fetch rows from splits - int rowCount = 0; - for (InputSplit split : actualSplits) { - System.out.println("Processing split " + split.getLocations()); - RecordReader reader = inputFormat.getRecordReader(split, job, null); - Row row = reader.createValue(); - while (reader.next(NullWritable.get(), row)) { - rowProcessor.process(row); - ++rowCount; - } - //In arrow-mode this will throw exception unless all buffers have been released - //See org.apache.hadoop.hive.llap.LlapArrowBatchRecordReader - reader.close(); - } - LlapBaseInputFormat.close(handleId); - - return rowCount; - } - -} diff --git a/itests/hive-unit/src/test/java/org/apache/hive/jdbc/TestMiniLlapVectorArrowWithLlapIODisabled.java b/itests/hive-unit/src/test/java/org/apache/hive/jdbc/TestMiniLlapVectorArrowWithLlapIODisabled.java deleted file mode 100644 index eaa77007aa7e..000000000000 --- a/itests/hive-unit/src/test/java/org/apache/hive/jdbc/TestMiniLlapVectorArrowWithLlapIODisabled.java +++ /dev/null @@ -1,199 +0,0 @@ -/* - * 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.jdbc; - -import org.apache.hadoop.hive.conf.HiveConf; -import org.apache.hadoop.hive.conf.HiveConf.ConfVars; -import org.apache.hadoop.hive.llap.FieldDesc; -import org.apache.hadoop.hive.llap.LlapArrowRowInputFormat; -import org.apache.hadoop.hive.llap.Row; -import org.apache.hadoop.io.NullWritable; -import org.apache.hadoop.mapred.InputFormat; -import org.junit.BeforeClass; -import org.junit.Ignore; -import org.junit.Test; - -import java.util.List; -import java.util.Map; - -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertNull; - -/** - * TestMiniLlapVectorArrowWithLlapIODisabled - turns off llap io while testing LLAP external client flow. - * The aim of turning off LLAP IO is - - * when we create table through this test, LLAP caches them and returns the same - * when we do a read query, due to this we miss some code paths which may have been hit otherwise. - */ -public class TestMiniLlapVectorArrowWithLlapIODisabled extends BaseJdbcWithMiniLlap { - - @BeforeClass - public static void beforeTest() throws Exception { - HiveConf conf = defaultConf(); - conf.setBoolVar(ConfVars.LLAP_OUTPUT_FORMAT_ARROW, true); - conf.setBoolVar(ConfVars.HIVE_VECTORIZATION_FILESINK_ARROW_NATIVE_ENABLED, true); - conf.set(ConfVars.LLAP_IO_ENABLED.varname, "false"); - BaseJdbcWithMiniLlap.beforeTest(conf); - } - - @Override - protected InputFormat getInputFormat() { - //For unit testing, no harm in hard-coding allocator ceiling to LONG.MAX_VALUE - return new LlapArrowRowInputFormat(Long.MAX_VALUE); - } - - @Test - public void testNullsInStructFields() throws Exception { - createDataTypesTable("datatypes"); - RowCollector2 rowCollector = new RowCollector2(); - // c1 int - // c8 struct - // c15 struct> - // c16 array,n:int>> - String query = "select c1, c8, c15, c16 from datatypes"; - int rowCount = processQuery(query, 1, rowCollector); - assertEquals(4, rowCollector.numColumns); - assertEquals(3, rowCount); - - FieldDesc fieldDesc = rowCollector.schema.getColumns().get(0); - assertEquals("c1", fieldDesc.getName()); - assertEquals("int", fieldDesc.getTypeInfo().getTypeName()); - - fieldDesc = rowCollector.schema.getColumns().get(1); - assertEquals("c8", fieldDesc.getName()); - assertEquals("struct", fieldDesc.getTypeInfo().getTypeName()); - - fieldDesc = rowCollector.schema.getColumns().get(2); - assertEquals("c15", fieldDesc.getName()); - assertEquals("struct>", fieldDesc.getTypeInfo().getTypeName()); - - fieldDesc = rowCollector.schema.getColumns().get(3); - assertEquals("c16", fieldDesc.getName()); - assertEquals("array,n:int>>", fieldDesc.getTypeInfo().getTypeName()); - - // First row is all nulls - Object[] rowValues = rowCollector.rows.get(0); - for (int idx = 0; idx < rowCollector.numColumns; ++idx) { - assertNull("idx=" + idx, rowValues[idx]); - } - - // Second Row - rowValues = rowCollector.rows.get(1); - assertEquals(-1, rowValues[0]); - - List c8Value = (List) rowValues[1]; - assertNull(c8Value.get(0)); - assertNull(c8Value.get(1)); - assertNull(c8Value.get(2)); - - List c15Value = (List) rowValues[2]; - assertNull(c15Value.get(0)); - assertNull(c15Value.get(1)); - - List c16Value = (List) rowValues[3]; - assertEquals(0, c16Value.size()); - - // Third row - rowValues = rowCollector.rows.get(2); - assertEquals(1, rowValues[0]); - - c8Value = (List) rowValues[1]; - assertEquals("a", c8Value.get(0)); - assertEquals(9, c8Value.get(1)); - assertEquals(2.2d, c8Value.get(2)); - - - c15Value = (List) rowValues[2]; - assertEquals(1, c15Value.get(0)); - List listVal = (List) c15Value.get(1); - assertEquals(2, listVal.size()); - assertEquals(2, listVal.get(0)); - assertEquals("x", listVal.get(1)); - - c16Value = (List) rowValues[3]; - assertEquals(2, c16Value.size()); - listVal = (List) c16Value.get(0); - assertEquals(2, listVal.size()); - Map mapVal = (Map) listVal.get(0); - assertEquals(0, mapVal.size()); - assertEquals(1, listVal.get(1)); - listVal = (List) c16Value.get(1); - mapVal = (Map) listVal.get(0); - assertEquals(2, mapVal.size()); - assertEquals("b", mapVal.get("a")); - assertEquals("d", mapVal.get("c")); - assertEquals(2, listVal.get(1)); - } - - @Override - @Ignore - public void testDataTypes() throws Exception { - // To be implemented - } - - @Override - @Ignore - public void testLlapInputFormatEndToEnd() throws Exception { - // To be implemented - } - - @Override - @Ignore - public void testMultipleBatchesOfComplexTypes() throws Exception { - // To be implemented - } - - @Override - @Ignore - public void testLlapInputFormatEndToEndWithMultipleBatches() throws Exception { - // To be implemented - } - - @Override - @Ignore - public void testInvalidReferenceCountScenario() throws Exception { - // To be implemented - } - - @Override - @Ignore - public void testNonAsciiStrings() throws Exception { - // To be implemented - } - - @Override - @Ignore - public void testEscapedStrings() throws Exception { - // To be implemented - } - - - @Override - @Ignore - public void testComplexQuery() throws Exception { - // To be implemented - } - - @Override - @Ignore - public void testKillQuery() throws Exception { - // To be implemented - } -} - diff --git a/itests/hive-unit/src/test/java/org/apache/hive/jdbc/TestNewGetSplitsFormat.java b/itests/hive-unit/src/test/java/org/apache/hive/jdbc/TestNewGetSplitsFormat.java deleted file mode 100644 index b275982cdddf..000000000000 --- a/itests/hive-unit/src/test/java/org/apache/hive/jdbc/TestNewGetSplitsFormat.java +++ /dev/null @@ -1,127 +0,0 @@ -/* - * 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.jdbc; - -import org.apache.hadoop.hive.conf.HiveConf; -import org.apache.hadoop.hive.llap.LlapArrowRowInputFormat; -import org.apache.hadoop.hive.llap.LlapBaseInputFormat; -import org.apache.hadoop.hive.llap.LlapInputSplit; -import org.apache.hadoop.hive.llap.Row; -import org.apache.hadoop.io.NullWritable; -import org.apache.hadoop.mapred.InputFormat; -import org.apache.hadoop.mapred.InputSplit; -import org.apache.hadoop.mapred.JobConf; -import org.apache.hadoop.mapred.RecordReader; -import org.junit.BeforeClass; -import org.junit.Ignore; - -import java.util.ArrayList; -import java.util.List; -import java.util.UUID; - -/** - * TestNewGetSplitsFormat. - */ -@Ignore("test unstable HIVE-23524") -public class TestNewGetSplitsFormat extends BaseJdbcWithMiniLlap { - - @BeforeClass public static void beforeTest() throws Exception { - HiveConf conf = defaultConf(); - conf.setBoolVar(HiveConf.ConfVars.LLAP_OUTPUT_FORMAT_ARROW, true); - conf.setBoolVar(HiveConf.ConfVars.HIVE_VECTORIZATION_FILESINK_ARROW_NATIVE_ENABLED, true); - BaseJdbcWithMiniLlap.beforeTest(conf); - } - - @Override protected InputFormat getInputFormat() { - //For unit testing, no harm in hard-coding allocator ceiling to LONG.MAX_VALUE - return new LlapArrowRowInputFormat(Long.MAX_VALUE); - } - - @Override public void testDataTypes() throws Exception { - TestJdbcWithMiniLlapVectorArrow testJdbcWithMiniLlapVectorArrow = new TestJdbcWithMiniLlapVectorArrow(); - testJdbcWithMiniLlapVectorArrow.testDataTypes(); - } - - @Override - @Ignore - public void testMultipleBatchesOfComplexTypes() { - // ToDo: FixMe - } - - @Override protected int processQuery(String currentDatabase, String query, int numSplits, RowProcessor rowProcessor) - throws Exception { - String url = miniHS2.getJdbcURL(); - String user = System.getProperty("user.name"); - String pwd = user; - String handleId = UUID.randomUUID().toString(); - - InputFormat inputFormat = getInputFormat(); - - // Get splits - JobConf job = new JobConf(conf); - job.set(LlapBaseInputFormat.URL_KEY, url); - job.set(LlapBaseInputFormat.USER_KEY, user); - job.set(LlapBaseInputFormat.PWD_KEY, pwd); - job.set(LlapBaseInputFormat.QUERY_KEY, query); - job.set(LlapBaseInputFormat.HANDLE_ID, handleId); - job.set(LlapBaseInputFormat.USE_NEW_SPLIT_FORMAT, "true"); - if (currentDatabase != null) { - job.set(LlapBaseInputFormat.DB_KEY, currentDatabase); - } - - InputSplit[] splits = inputFormat.getSplits(job, numSplits); - - if (splits.length <= 1) { - return 0; - } - - - // populate actual splits with schema and planBytes[] - LlapInputSplit schemaSplit = (LlapInputSplit) splits[0]; - LlapInputSplit planSplit = (LlapInputSplit) splits[1]; - - List actualSplits = new ArrayList<>(); - - for (int i = 2; i < splits.length; i++) { - LlapInputSplit actualSplit = (LlapInputSplit) splits[i]; - actualSplit.setSchema(schemaSplit.getSchema()); - actualSplit.setPlanBytes(planSplit.getPlanBytes()); - actualSplits.add(actualSplit); - } - - // Fetch rows from splits - int rowCount = 0; - for (InputSplit split : actualSplits) { - System.out.println("Processing split " + split.getLocations()); - RecordReader reader = inputFormat.getRecordReader(split, job, null); - Row row = reader.createValue(); - while (reader.next(NullWritable.get(), row)) { - rowProcessor.process(row); - ++rowCount; - } - //In arrow-mode this will throw exception unless all buffers have been released - //See org.apache.hadoop.hive.llap.LlapArrowBatchRecordReader - reader.close(); - } - LlapBaseInputFormat.close(handleId); - - return rowCount; - } - -} diff --git a/itests/hive-unit/src/test/java/org/apache/hive/jdbc/TestNewGetSplitsFormatReturnPath.java b/itests/hive-unit/src/test/java/org/apache/hive/jdbc/TestNewGetSplitsFormatReturnPath.java deleted file mode 100644 index 398362a17514..000000000000 --- a/itests/hive-unit/src/test/java/org/apache/hive/jdbc/TestNewGetSplitsFormatReturnPath.java +++ /dev/null @@ -1,52 +0,0 @@ -/* - * 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.jdbc; - -import org.apache.hadoop.hive.conf.HiveConf; -import org.junit.BeforeClass; -import org.junit.Ignore; -import org.junit.Test; - -/** - * TestNewGetSplitsFormatReturnPath. - */ -@Ignore("flaky HIVE-23524") -public class TestNewGetSplitsFormatReturnPath extends TestNewGetSplitsFormat { - - @BeforeClass public static void beforeTest() throws Exception { - HiveConf conf = defaultConf(); - conf.setBoolVar(HiveConf.ConfVars.LLAP_OUTPUT_FORMAT_ARROW, true); - conf.setBoolVar(HiveConf.ConfVars.HIVE_VECTORIZATION_FILESINK_ARROW_NATIVE_ENABLED, true); - conf.setBoolVar(HiveConf.ConfVars.HIVE_CBO_RETPATH_HIVEOP, true); - BaseJdbcWithMiniLlap.beforeTest(conf); - } - - @Override - @Ignore - @Test - public void testMultipleBatchesOfComplexTypes() { - // ToDo: FixMe - } - - @Override - @Ignore("HIVE-23524 flaky") - @Test - public void testLlapInputFormatEndToEndWithMultipleBatches() { - } -} diff --git a/llap-ext-client/src/java/org/apache/hadoop/hive/llap/LlapArrowBatchRecordReader.java b/llap-ext-client/src/java/org/apache/hadoop/hive/llap/LlapArrowBatchRecordReader.java deleted file mode 100644 index 1ac7637db3d0..000000000000 --- a/llap-ext-client/src/java/org/apache/hadoop/hive/llap/LlapArrowBatchRecordReader.java +++ /dev/null @@ -1,91 +0,0 @@ -/* - * 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.hadoop.hive.llap; - -import com.google.common.base.Preconditions; -import org.apache.arrow.memory.BufferAllocator; -import org.apache.arrow.vector.VectorSchemaRoot; -import org.apache.arrow.vector.ipc.ArrowStreamReader; -import org.apache.hadoop.hive.ql.io.arrow.ArrowWrapperWritable; -import org.apache.hadoop.hive.ql.io.arrow.RootAllocatorFactory; -import org.apache.hadoop.io.NullWritable; -import org.apache.hadoop.mapred.JobConf; -import java.io.Closeable; -import java.io.IOException; -import java.io.InputStream; -import java.net.Socket; - -/* - * Read from Arrow stream batch-by-batch - */ -public class LlapArrowBatchRecordReader extends LlapBaseRecordReader { - - private BufferAllocator allocator; - private ArrowStreamReader arrowStreamReader; - - //Allows client to provide and manage their own arrow BufferAllocator - public LlapArrowBatchRecordReader(InputStream in, Schema schema, Class clazz, - JobConf job, Closeable client, Socket socket, BufferAllocator allocator) throws IOException { - super(in, schema, clazz, job, client, socket); - this.allocator = allocator; - this.arrowStreamReader = new ArrowStreamReader(socket.getInputStream(), allocator); - } - - //Use the global arrow BufferAllocator - public LlapArrowBatchRecordReader(InputStream in, Schema schema, Class clazz, - JobConf job, Closeable client, Socket socket, long arrowAllocatorLimit) throws IOException { - this(in, schema, clazz, job, client, socket, - RootAllocatorFactory.INSTANCE.getOrCreateRootAllocator(arrowAllocatorLimit)); - } - - @Override - public boolean next(NullWritable key, ArrowWrapperWritable value) throws IOException { - try { - // Need a way to know what thread to interrupt, since this is a blocking thread. - setReaderThread(Thread.currentThread()); - - boolean hasInput = arrowStreamReader.loadNextBatch(); - if (hasInput) { - VectorSchemaRoot vectorSchemaRoot = arrowStreamReader.getVectorSchemaRoot(); - //There must be at least one column vector - Preconditions.checkState(vectorSchemaRoot.getFieldVectors().size() > 0); - // We should continue even if FieldVectors are empty. The next read might have the - // data. We should stop only when loadNextBatch returns false. - value.setVectorSchemaRoot(arrowStreamReader.getVectorSchemaRoot()); - return true; - } else { - processReaderEvent(); - return false; - } - } catch (IOException io) { - failOnInterruption(io); - return false; - } - } - - @Override - public void close() throws IOException { - arrowStreamReader.close(); - //allocator.close() will throw exception unless all buffers have been released - //See org.apache.arrow.memory.BaseAllocator.close() - allocator.close(); - } - -} - diff --git a/llap-ext-client/src/java/org/apache/hadoop/hive/llap/LlapArrowRowInputFormat.java b/llap-ext-client/src/java/org/apache/hadoop/hive/llap/LlapArrowRowInputFormat.java deleted file mode 100644 index 46566be332e7..000000000000 --- a/llap-ext-client/src/java/org/apache/hadoop/hive/llap/LlapArrowRowInputFormat.java +++ /dev/null @@ -1,65 +0,0 @@ -/* - * 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.hadoop.hive.llap; - -import org.apache.hadoop.io.NullWritable; -import org.apache.hadoop.mapred.InputFormat; -import org.apache.hadoop.mapred.InputSplit; -import org.apache.hadoop.mapred.JobConf; -import org.apache.hadoop.mapred.RecordReader; -import org.apache.hadoop.mapred.Reporter; -import java.io.IOException; -import org.apache.arrow.memory.BufferAllocator; -import org.apache.hadoop.hive.ql.io.arrow.RootAllocatorFactory; -import java.util.UUID; - -/* - * Adapts an Arrow batch reader to a row reader - * Only used for testing - */ -public class LlapArrowRowInputFormat implements InputFormat { - - private LlapBaseInputFormat baseInputFormat; - - public LlapArrowRowInputFormat(long arrowAllocatorLimit) { - BufferAllocator allocator = RootAllocatorFactory.INSTANCE.getOrCreateRootAllocator(arrowAllocatorLimit).newChildAllocator( - //allocator name, use UUID for testing - UUID.randomUUID().toString(), - //No use for reservation, allocators claim memory from the same pool, - //but allocate/releases are tracked per-allocator - 0, - //Limit passed in by client - arrowAllocatorLimit); - baseInputFormat = new LlapBaseInputFormat(true, allocator); - } - - @Override - public InputSplit[] getSplits(JobConf job, int numSplits) throws IOException { - return baseInputFormat.getSplits(job, numSplits); - } - - @Override - public RecordReader getRecordReader(InputSplit split, JobConf job, Reporter reporter) - throws IOException { - LlapInputSplit llapSplit = (LlapInputSplit) split; - LlapArrowBatchRecordReader reader = - (LlapArrowBatchRecordReader) baseInputFormat.getRecordReader(llapSplit, job, reporter); - return new LlapArrowRowRecordReader(job, reader.getSchema(), reader); - } -} diff --git a/llap-ext-client/src/java/org/apache/hadoop/hive/llap/LlapArrowRowRecordReader.java b/llap-ext-client/src/java/org/apache/hadoop/hive/llap/LlapArrowRowRecordReader.java deleted file mode 100644 index 24a82c7f323a..000000000000 --- a/llap-ext-client/src/java/org/apache/hadoop/hive/llap/LlapArrowRowRecordReader.java +++ /dev/null @@ -1,113 +0,0 @@ -/* - * 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.hadoop.hive.llap; - -import com.google.common.base.Preconditions; -import org.apache.arrow.vector.FieldVector; -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.hive.ql.io.arrow.ArrowColumnarBatchSerDe; -import org.apache.hadoop.hive.ql.io.arrow.ArrowWrapperWritable; -import org.apache.hadoop.hive.serde2.AbstractSerDe; -import org.apache.hadoop.hive.serde2.SerDeException; -import org.apache.hadoop.hive.serde2.objectinspector.StructObjectInspector; -import org.apache.hadoop.io.NullWritable; -import org.apache.hadoop.io.Writable; -import org.apache.hadoop.mapred.RecordReader; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import java.io.IOException; -import java.util.List; - -/** - * Buffers a batch for reading one row at a time. - */ -public class LlapArrowRowRecordReader extends LlapRowRecordReader { - - private static final Logger LOG = LoggerFactory.getLogger(LlapArrowRowRecordReader.class); - private int rowIndex = 0; - private int batchSize = 0; - - //Buffer one batch at a time, for row retrieval - private Object[][] currentBatch; - - public LlapArrowRowRecordReader(Configuration conf, Schema schema, - RecordReader reader) throws IOException { - super(conf, schema, reader); - } - - @Override - public boolean next(NullWritable key, Row value) throws IOException { - Preconditions.checkArgument(value != null); - boolean hasNext = false; - ArrowWrapperWritable batchData = (ArrowWrapperWritable) data; - if((batchSize == 0) || (rowIndex == batchSize)) { - //This is either the first batch or we've used up the current batch buffer - batchSize = 0; - rowIndex = 0; - - // since HIVE-22856, a zero length batch doesn't mean that we won't have any more batches - // we can have more batches with data even after after a zero length batch - // we should keep trying until we get a batch with some data or reader.next() returns false - while (batchSize == 0 && (hasNext = reader.next(key, data))) { - List vectors = batchData.getVectorSchemaRoot().getFieldVectors(); - //hasNext implies there is some column in the batch - Preconditions.checkState(vectors.size() > 0); - //All the vectors have the same length, - //we can get the number of rows from the first vector - batchSize = vectors.get(0).getValueCount(); - } - - if (hasNext) { - //There is another batch to buffer - try { - ArrowWrapperWritable wrapper = new ArrowWrapperWritable(batchData.getVectorSchemaRoot()); - currentBatch = (Object[][]) serde.deserialize(wrapper); - StructObjectInspector rowOI = (StructObjectInspector) serde.getObjectInspector(); - setRowFromStruct(value, currentBatch[rowIndex], rowOI); - } catch (Exception e) { - LOG.error("Failed to fetch Arrow batch", e); - throw new RuntimeException(e); - } - } - //There were no more batches AND - //this is either the first batch or we've used up the current batch buffer. - //goto return false - } else if(rowIndex < batchSize) { - //Take a row from the current buffered batch - hasNext = true; - StructObjectInspector rowOI = null; - try { - rowOI = (StructObjectInspector) serde.getObjectInspector(); - } catch (SerDeException e) { - throw new RuntimeException(e); - } - setRowFromStruct(value, currentBatch[rowIndex], rowOI); - } - //Always inc the batch buffer index - //If we return false, it is just a noop - rowIndex++; - return hasNext; - } - - protected AbstractSerDe createSerDe() throws SerDeException { - return new ArrowColumnarBatchSerDe(); - } - -} diff --git a/llap-ext-client/src/java/org/apache/hadoop/hive/llap/LlapBaseInputFormat.java b/llap-ext-client/src/java/org/apache/hadoop/hive/llap/LlapBaseInputFormat.java index 7b04e2c4552f..9e88f3c0d100 100644 --- a/llap-ext-client/src/java/org/apache/hadoop/hive/llap/LlapBaseInputFormat.java +++ b/llap-ext-client/src/java/org/apache/hadoop/hive/llap/LlapBaseInputFormat.java @@ -51,7 +51,6 @@ import org.apache.hadoop.hive.llap.registry.LlapServiceInstance; import org.apache.hadoop.hive.llap.security.LlapTokenIdentifier; import org.apache.hadoop.hive.llap.tez.Converters; -import org.apache.hadoop.hive.ql.io.arrow.ArrowWrapperWritable; import org.apache.hadoop.io.BytesWritable; import org.apache.hadoop.io.DataInputBuffer; import org.apache.hadoop.io.DataOutputBuffer; @@ -62,7 +61,6 @@ import org.apache.hadoop.mapred.JobConf; import org.apache.hadoop.mapred.RecordReader; import org.apache.hadoop.mapred.Reporter; -import org.apache.arrow.memory.BufferAllocator; import org.apache.hadoop.mapreduce.MRJobConfig; import org.apache.hadoop.mapreduce.TaskAttemptID; import org.apache.hadoop.security.Credentials; @@ -102,9 +100,6 @@ public class LlapBaseInputFormat> private String user; // "hive", private String pwd; // "" private String query; - private boolean useArrow; - private long arrowAllocatorLimit; - private BufferAllocator allocator; private final Random rand = new Random(); public static final String URL_KEY = "llap.if.hs2.connection"; @@ -119,29 +114,6 @@ public class LlapBaseInputFormat> public static final String SPLIT_QUERY = "select get_llap_splits(\"%s\",%d)"; - public LlapBaseInputFormat(String url, String user, String pwd, String query) { - this.url = url; - this.user = user; - this.pwd = pwd; - this.query = query; - } - - //Exposed only for testing, clients should use LlapBaseInputFormat(boolean, BufferAllocator instead) - public LlapBaseInputFormat(boolean useArrow, long arrowAllocatorLimit) { - this.useArrow = useArrow; - this.arrowAllocatorLimit = arrowAllocatorLimit; - } - - public LlapBaseInputFormat(boolean useArrow, BufferAllocator allocator) { - this.useArrow = useArrow; - this.allocator = allocator; - } - - public LlapBaseInputFormat() { - this.useArrow = false; - } - - @SuppressWarnings("unchecked") @Override public RecordReader getRecordReader(InputSplit split, JobConf job, Reporter reporter) throws IOException { @@ -219,26 +191,8 @@ public RecordReader getRecordReader(InputSplit split, JobConf j LOG.info("Registered id: " + fragmentId); - @SuppressWarnings("rawtypes") - LlapBaseRecordReader recordReader; - if(useArrow) { - if(allocator != null) { - //Client provided their own allocator - recordReader = new LlapArrowBatchRecordReader( - socket.getInputStream(), llapSplit.getSchema(), - ArrowWrapperWritable.class, job, llapClient, socket, - allocator); - } else { - //Client did not provide their own allocator, use constructor for global allocator - recordReader = new LlapArrowBatchRecordReader( - socket.getInputStream(), llapSplit.getSchema(), - ArrowWrapperWritable.class, job, llapClient, socket, - arrowAllocatorLimit); - } - } else { - recordReader = new LlapBaseRecordReader(socket.getInputStream(), - llapSplit.getSchema(), BytesWritable.class, job, llapClient, (java.io.Closeable)socket); - } + LlapBaseRecordReader recordReader = new LlapBaseRecordReader(socket.getInputStream(), + llapSplit.getSchema(), BytesWritable.class, job, llapClient, socket); umbilicalResponder.setRecordReader(recordReader); return recordReader; } diff --git a/llap-ext-client/src/java/org/apache/hadoop/hive/llap/LlapRowInputFormat.java b/llap-ext-client/src/java/org/apache/hadoop/hive/llap/LlapRowInputFormat.java index aee55024409e..63714d1b62cc 100644 --- a/llap-ext-client/src/java/org/apache/hadoop/hive/llap/LlapRowInputFormat.java +++ b/llap-ext-client/src/java/org/apache/hadoop/hive/llap/LlapRowInputFormat.java @@ -38,7 +38,7 @@ public class LlapRowInputFormat implements InputFormat { - private LlapBaseInputFormat baseInputFormat = new LlapBaseInputFormat(); + private final LlapBaseInputFormat baseInputFormat = new LlapBaseInputFormat<>(); @Override public InputSplit[] getSplits(JobConf job, int numSplits) throws IOException { diff --git a/llap-server/pom.xml b/llap-server/pom.xml index 6eaf85827fe0..7dfb4da06982 100644 --- a/llap-server/pom.xml +++ b/llap-server/pom.xml @@ -251,21 +251,6 @@ disruptor ${disruptor.version} - - org.apache.arrow - arrow-memory-netty - ${arrow.version} - - - io.netty - netty-buffer - - - io.netty - netty-common - - - org.apache.hive diff --git a/llap-server/src/java/org/apache/hadoop/hive/llap/cli/service/AsyncTaskCopyLocalJars.java b/llap-server/src/java/org/apache/hadoop/hive/llap/cli/service/AsyncTaskCopyLocalJars.java index 01ebff2e934d..db69117dc6e3 100644 --- a/llap-server/src/java/org/apache/hadoop/hive/llap/cli/service/AsyncTaskCopyLocalJars.java +++ b/llap-server/src/java/org/apache/hadoop/hive/llap/cli/service/AsyncTaskCopyLocalJars.java @@ -63,12 +63,8 @@ public Void call() throws Exception { // log4j-1.2-API needed for NDC org.apache.log4j.config.Log4j1ConfigurationFactory.class, io.netty.util.NetUtil.class, // netty4 - io.netty.handler.codec.http.HttpObjectAggregator.class, // - org.apache.arrow.vector.types.pojo.ArrowType.class, //arrow-vector - org.apache.arrow.memory.RootAllocator.class, //arrow-memory - org.apache.arrow.memory.netty.NettyAllocationManager.class, //arrow-memory-netty + io.netty.handler.codec.http.HttpObjectAggregator.class, io.netty.handler.codec.http.HttpObjectAggregator.class, // netty-all - org.apache.arrow.flatbuf.Schema.class, //arrow-format com.google.flatbuffers.Table.class, //flatbuffers com.carrotsearch.hppc.ByteArrayDeque.class, //hppc io.jsonwebtoken.security.Keys.class, //jjwt-api diff --git a/pom.xml b/pom.xml index 9f7476736a72..cbf8f7aaca75 100644 --- a/pom.xml +++ b/pom.xml @@ -109,8 +109,6 @@ 4.9.3 2.0.0-M24 - - 16.0.0 1.12.0 1.11.4 1.78 diff --git a/ql/pom.xml b/ql/pom.xml index 102f9cb409b2..b667168a3b92 100644 --- a/ql/pom.xml +++ b/ql/pom.xml @@ -544,21 +544,6 @@ hive-standalone-metastore-server ${standalone-metastore.version} - - org.apache.arrow - arrow-memory-netty - ${arrow.version} - - - io.netty - netty-buffer - - - io.netty - netty-common - - - org.apache.hive diff --git a/ql/src/java/org/apache/hadoop/hive/llap/LlapArrowRecordWriter.java b/ql/src/java/org/apache/hadoop/hive/llap/LlapArrowRecordWriter.java deleted file mode 100644 index c0b20115fe07..000000000000 --- a/ql/src/java/org/apache/hadoop/hive/llap/LlapArrowRecordWriter.java +++ /dev/null @@ -1,92 +0,0 @@ -/* - * 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.hadoop.hive.llap; - -import java.io.IOException; - -import org.apache.arrow.memory.BufferAllocator; -import org.apache.arrow.vector.VectorSchemaRoot; -import org.apache.arrow.vector.complex.NonNullableStructVector; -import org.apache.arrow.vector.ipc.ArrowStreamWriter; -import org.apache.hadoop.hive.ql.io.arrow.ArrowWrapperWritable; -import org.apache.hadoop.io.Writable; -import org.apache.hadoop.mapred.RecordWriter; -import org.apache.hadoop.mapred.Reporter; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -/** - * Writes Arrow batches to an {@link org.apache.arrow.vector.ipc.ArrowStreamWriter}. - * The byte stream will be formatted according to the Arrow Streaming format. - * Because ArrowStreamWriter is bound to a {@link org.apache.arrow.vector.VectorSchemaRoot} - * when it is created, - * calls to the {@link #write(Writable, Writable)} method only serve as a signal that - * a new batch has been loaded to the associated VectorSchemaRoot. - * Payload data for writing is indirectly made available by reference: - * ArrowStreamWriter -> VectorSchemaRoot -> List<FieldVector> - * i.e. both they key and value are ignored once a reference to the VectorSchemaRoot - * is obtained. - */ -public class LlapArrowRecordWriter - implements RecordWriter { - public static final Logger LOG = LoggerFactory.getLogger(LlapArrowRecordWriter.class); - - ArrowStreamWriter arrowStreamWriter; - VectorSchemaRoot vectorSchemaRoot; - WritableByteChannelAdapter out; - BufferAllocator allocator; - NonNullableStructVector rootVector; - - public LlapArrowRecordWriter(WritableByteChannelAdapter out) { - this.out = out; - } - - @Override - public void close(Reporter reporter) throws IOException { - try { - arrowStreamWriter.close(); - } finally { - rootVector.close(); - //bytesLeaked should always be 0 - long bytesLeaked = allocator.getAllocatedMemory(); - if(bytesLeaked != 0) { - LOG.error("Arrow memory leaked bytes: {}", bytesLeaked); - throw new IllegalStateException("Arrow memory leaked bytes:" + bytesLeaked); - } - allocator.close(); - } - } - - @Override - public void write(K key, V value) throws IOException { - ArrowWrapperWritable arrowWrapperWritable = (ArrowWrapperWritable) value; - if (arrowStreamWriter == null) { - vectorSchemaRoot = arrowWrapperWritable.getVectorSchemaRoot(); - arrowStreamWriter = new ArrowStreamWriter(vectorSchemaRoot, null, out); - allocator = arrowWrapperWritable.getAllocator(); - this.out.setAllocator(allocator); - rootVector = arrowWrapperWritable.getRootVector(); - } else { - // We need to set the row count for the current vector - // since root is reused by the stream writer. - vectorSchemaRoot.setRowCount(rootVector.getValueCount()); - } - arrowStreamWriter.writeBatch(); - } -} diff --git a/ql/src/java/org/apache/hadoop/hive/llap/LlapOutputFormatService.java b/ql/src/java/org/apache/hadoop/hive/llap/LlapOutputFormatService.java index c71c637c7186..200d573051ee 100644 --- a/ql/src/java/org/apache/hadoop/hive/llap/LlapOutputFormatService.java +++ b/ql/src/java/org/apache/hadoop/hive/llap/LlapOutputFormatService.java @@ -198,16 +198,9 @@ private void registerReader(ChannelHandlerContext ctx, String id, byte[] tokenBy LOG.debug("registering socket for: " + id); int maxPendingWrites = HiveConf.getIntVar(conf, HiveConf.ConfVars.LLAP_DAEMON_OUTPUT_SERVICE_MAX_PENDING_WRITES); - boolean useArrow = HiveConf.getBoolVar(conf, HiveConf.ConfVars.LLAP_OUTPUT_FORMAT_ARROW); - @SuppressWarnings("rawtypes") - RecordWriter writer = null; - if(useArrow) { - writer = new LlapArrowRecordWriter(new WritableByteChannelAdapter(ctx, maxPendingWrites, id)); - } else { - writer = new LlapRecordWriter(id, + RecordWriter writer = new LlapRecordWriter(id, new ChunkedOutputStream( - new ChannelOutputStream(ctx, id, sendBufferSize, maxPendingWrites), sendBufferSize, id)); - } + new ChannelOutputStream(ctx, id, sendBufferSize, maxPendingWrites), sendBufferSize, id)); boolean isFailed = true; synchronized (lock) { if (!writers.containsKey(id)) { diff --git a/ql/src/java/org/apache/hadoop/hive/llap/WritableByteChannelAdapter.java b/ql/src/java/org/apache/hadoop/hive/llap/WritableByteChannelAdapter.java deleted file mode 100644 index 0028c2503f77..000000000000 --- a/ql/src/java/org/apache/hadoop/hive/llap/WritableByteChannelAdapter.java +++ /dev/null @@ -1,135 +0,0 @@ -/* - * 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.hadoop.hive.llap; - -import io.netty.buffer.ByteBuf; -import io.netty.channel.ChannelHandlerContext; - -import java.io.IOException; -import java.nio.ByteBuffer; -import java.nio.channels.WritableByteChannel; -import java.util.concurrent.Semaphore; -import org.apache.arrow.memory.patch.ArrowByteBufAllocator; -import org.apache.arrow.memory.BufferAllocator; - -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; -import io.netty.channel.ChannelFuture; -import io.netty.channel.ChannelFutureListener; - -/** - * Provides an adapter between {@link java.nio.channels.WritableByteChannel} - * and {@link io.netty.channel.ChannelHandlerContext}. - * Additionally provides a form of flow-control by limiting the number of - * queued async writes. - */ -public class WritableByteChannelAdapter implements WritableByteChannel { - - private static final Logger LOG = LoggerFactory.getLogger(WritableByteChannelAdapter.class); - private ChannelHandlerContext chc; - private final int maxPendingWrites; - // This semaphore provides two functions: - // 1. Forces a cap on the number of outstanding async writes to channel - // 2. Ensures that channel isn't closed if there are any outstanding async writes - private final Semaphore writeResources; - private boolean closed = false; - private final String id; - private BufferAllocator allocator; - - private ChannelFutureListener writeListener = new ChannelFutureListener() { - @Override - public void operationComplete(ChannelFuture future) { - //Async write completed - //Up the semaphore - writeResources.release(); - - if (future.isCancelled()) { - LOG.error("Write cancelled on ID " + id); - } else if (!future.isSuccess()) { - LOG.error("Write error on ID " + id, future.cause()); - } - } - }; - - private ChannelFutureListener closeListener = new ChannelFutureListener() { - @Override - public void operationComplete(ChannelFuture future) { - if (future.isCancelled()) { - LOG.error("Close cancelled on ID " + id); - } else if (!future.isSuccess()) { - LOG.error("Close failed on ID " + id, future.cause()); - } - } - }; - - public WritableByteChannelAdapter(ChannelHandlerContext chc, int maxPendingWrites, String id) { - this.chc = chc; - this.maxPendingWrites = maxPendingWrites; - this.writeResources = new Semaphore(maxPendingWrites); - this.id = id; - } - - public void setAllocator(BufferAllocator allocator) { - this.allocator = allocator; - } - - @Override - public int write(ByteBuffer src) throws IOException { - int size = src.remaining(); - //Down the semaphore or block until available - takeWriteResources(1); - ArrowByteBufAllocator abba = new ArrowByteBufAllocator(allocator); - ByteBuf buf = abba.buffer(size); - buf.writeBytes(src); - chc.writeAndFlush(buf).addListener(writeListener); - return size; - } - - @Override - public boolean isOpen() { - return chc.channel().isOpen(); - } - - @Override - public void close() throws IOException { - if (closed) { - throw new IOException("Already closed: " + id); - } - - closed = true; - //Block until all semaphore resources are released - //by outstanding async writes - takeWriteResources(maxPendingWrites); - - try { - chc.close().addListener(closeListener); - } finally { - chc = null; - closed = true; - } - } - - private void takeWriteResources(int numResources) throws IOException { - try { - writeResources.acquire(numResources); - } catch (InterruptedException ie) { - throw new IOException("Interrupted while waiting for write resources for " + id); - } - } -} diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/FileSinkOperator.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/FileSinkOperator.java index e69d1fa0e633..44144aae8fb8 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/exec/FileSinkOperator.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/FileSinkOperator.java @@ -65,7 +65,6 @@ import org.apache.hadoop.hive.ql.io.RecordUpdater; import org.apache.hadoop.hive.ql.io.StatsProvidingRecordWriter; import org.apache.hadoop.hive.ql.io.StreamingOutputFormat; -import org.apache.hadoop.hive.ql.io.arrow.ArrowWrapperWritable; import org.apache.hadoop.hive.ql.io.orc.OrcRecordUpdater; import org.apache.hadoop.hive.ql.metadata.HiveException; import org.apache.hadoop.hive.ql.metadata.HiveFatalException; @@ -1500,15 +1499,6 @@ public void closeOp(boolean abort) throws HiveException { if (null != fpaths) { rowOutWriters = fpaths.outWriters; rowOutWriters[0].write(recordValue); - } else if(recordValue instanceof ArrowWrapperWritable) { - //Because LLAP arrow output depends on the ThriftJDBCBinarySerDe code path - //this is required for 0 row outputs - //i.e. we need to write a 0 size batch to signal EOS to the consumer - for (FSPaths fsPaths : valToPaths.values()) { - for(RecordWriter writer : fsPaths.outWriters) { - writer.write(recordValue); - } - } } } catch (SerDeException | IOException e) { throw new HiveException(e); diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/filesink/VectorFileSinkArrowOperator.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/filesink/VectorFileSinkArrowOperator.java deleted file mode 100644 index 1603703ec7e2..000000000000 --- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/filesink/VectorFileSinkArrowOperator.java +++ /dev/null @@ -1,180 +0,0 @@ -/* - * 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.hadoop.hive.ql.exec.vector.filesink; - -import java.io.Serializable; - -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.hive.ql.CompilationOpContext; -import org.apache.hadoop.hive.ql.exec.FileSinkOperator; -import org.apache.hadoop.hive.ql.exec.TerminalOperator; -import org.apache.hadoop.hive.ql.exec.vector.VectorizationContext; -import org.apache.hadoop.hive.ql.exec.vector.VectorizationOperator; -import org.apache.hadoop.hive.ql.exec.vector.VectorizedBatchUtil; -import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch; -import org.apache.hadoop.hive.ql.metadata.HiveException; -import org.apache.hadoop.hive.ql.plan.FileSinkDesc; -import org.apache.hadoop.hive.ql.plan.OperatorDesc; -import org.apache.hadoop.hive.ql.plan.ReduceSinkDesc; -import org.apache.hadoop.hive.ql.plan.VectorDesc; -import org.apache.hadoop.hive.ql.plan.VectorFileSinkDesc; -import org.apache.hadoop.hive.ql.plan.api.OperatorType; -import org.apache.hadoop.hive.ql.io.arrow.ArrowWrapperWritable; -import org.apache.hadoop.mapred.RecordWriter; -import org.apache.hadoop.hive.llap.LlapOutputFormatService; -import org.apache.hadoop.hive.ql.exec.ColumnInfo; -import org.apache.hadoop.hive.serde2.objectinspector.StructObjectInspector; -import org.apache.hadoop.hive.serde2.objectinspector.StructField; -import org.apache.hadoop.hive.serde2.typeinfo.TypeInfo; -import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoUtils; -import java.util.List; -import java.util.ArrayList; -import org.apache.hadoop.hive.ql.io.arrow.Serializer; -import static org.apache.hadoop.hive.llap.LlapOutputFormat.LLAP_OF_ID_KEY; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import com.google.common.annotations.VisibleForTesting; -import org.apache.logging.log4j.core.layout.AbstractStringLayout; - -/** - * Native Vectorized File Sink operator implementation for Arrow. - * Assumes output to LlapOutputFormatService - **/ -public class VectorFileSinkArrowOperator extends TerminalOperator - implements Serializable, VectorizationOperator { - - private static final long serialVersionUID = 1L; - - private VectorizationContext vContext; - private VectorFileSinkDesc vectorDesc; - public static final Logger LOG = LoggerFactory.getLogger(VectorFileSinkArrowOperator.class.getName()); - - // The above members are initialized by the constructor and must not be - // transient. - //--------------------------------------------------------------------------- - - private transient Serializer converter; - private transient RecordWriter recordWriter; - private transient boolean wroteData; - private transient String attemptId; - - public VectorFileSinkArrowOperator(CompilationOpContext ctx, OperatorDesc conf, - VectorizationContext vContext, VectorDesc vectorDesc) { - this(ctx); - this.conf = (FileSinkDesc) conf; - this.vContext = vContext; - this.vectorDesc = (VectorFileSinkDesc) vectorDesc; - } - - /** Kryo ctor. */ - @VisibleForTesting - public VectorFileSinkArrowOperator() { - super(); - } - - public VectorFileSinkArrowOperator(CompilationOpContext ctx) { - super(ctx); - } - - @Override - public VectorizationContext getInputVectorizationContext() { - return vContext; - } - - @Override - protected void initializeOp(Configuration hconf) throws HiveException { - super.initializeOp(hconf); - //attemptId identifies a RecordWriter initialized by LlapOutputFormatService - this.attemptId = hconf.get(LLAP_OF_ID_KEY); - try { - //Initialize column names and types - List typeInfos = new ArrayList<>(); - List fieldNames = new ArrayList<>(); - StructObjectInspector schema = (StructObjectInspector) inputObjInspectors[0]; - for(int i = 0; i < schema.getAllStructFieldRefs().size(); i++) { - StructField structField = schema.getAllStructFieldRefs().get(i); - fieldNames.add(structField.getFieldName()); - TypeInfo typeInfo = TypeInfoUtils.getTypeInfoFromObjectInspector(structField.getFieldObjectInspector()); - typeInfos.add(typeInfo); - } - //Initialize an Arrow serializer - converter = new Serializer(hconf, attemptId, typeInfos, fieldNames); - } catch (Exception e) { - LOG.error("Unable to initialize VectorFileSinkArrowOperator"); - throw new RuntimeException(e); - } - } - - @Override - public void process(Object data, int tag) throws HiveException { - //ArrowStreamReader expects at least the schema metadata, if this op writes no data, - //we need to send the schema to close the stream gracefully - VectorizedRowBatch batch = (VectorizedRowBatch) data; - try { - if(recordWriter == null) { - recordWriter = LlapOutputFormatService.get().getWriter(this.attemptId); - } - //Convert the VectorizedRowBatch to a handle for the Arrow batch - ArrowWrapperWritable writable = converter.serializeBatch(batch, true); - //Pass the handle to the LlapOutputFormatService recordWriter - recordWriter.write(null, writable); - this.wroteData = true; - } catch(Exception e) { - LOG.error("Failed to convert VectorizedRowBatch to Arrow batch"); - throw new RuntimeException(e); - } - } - - @Override - protected void closeOp(boolean abort) throws HiveException { - try { - if(!wroteData) { - //Send a schema only batch to signal EOS with no data written - ArrowWrapperWritable writable = converter.emptyBatch(); - if(recordWriter == null) { - recordWriter = LlapOutputFormatService.get().getWriter(this.attemptId); - } - recordWriter.write(null, writable); - } - } catch(Exception e) { - LOG.error("Failed to write Arrow stream schema"); - throw new RuntimeException(e); - } finally { - try { - //Close the recordWriter with null Reporter - recordWriter.close(null); - } catch(Exception e) { - LOG.error("Failed to close Arrow stream"); - throw new RuntimeException(e); - } - } - } - - @Override - public VectorDesc getVectorDesc() { - return vectorDesc; - } - - @Override - public OperatorType getType() { - return OperatorType.FILESINK; - } -} - diff --git a/ql/src/java/org/apache/hadoop/hive/ql/io/arrow/ArrowColumnarBatchSerDe.java b/ql/src/java/org/apache/hadoop/hive/ql/io/arrow/ArrowColumnarBatchSerDe.java deleted file mode 100644 index ceb794f56197..000000000000 --- a/ql/src/java/org/apache/hadoop/hive/ql/io/arrow/ArrowColumnarBatchSerDe.java +++ /dev/null @@ -1,261 +0,0 @@ -/* - * 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.hadoop.hive.ql.io.arrow; - -import com.google.common.annotations.VisibleForTesting; -import com.google.common.collect.Lists; -import org.apache.arrow.memory.BufferAllocator; -import org.apache.arrow.vector.complex.impl.UnionListWriter; -import org.apache.arrow.vector.complex.writer.BaseWriter; -import org.apache.arrow.vector.types.TimeUnit; -import org.apache.arrow.vector.types.Types.MinorType; -import org.apache.arrow.vector.types.pojo.ArrowType; -import org.apache.arrow.vector.types.pojo.Field; -import org.apache.arrow.vector.types.pojo.FieldType; -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.hive.ql.exec.vector.ColumnVector; -import org.apache.hadoop.hive.ql.exec.vector.ListColumnVector; -import org.apache.hadoop.hive.ql.exec.vector.MapColumnVector; -import org.apache.hadoop.hive.ql.exec.vector.StructColumnVector; -import org.apache.hadoop.hive.ql.exec.vector.VectorAssignRow; -import org.apache.hadoop.hive.serde2.AbstractSerDe; -import org.apache.hadoop.hive.serde2.SerDeException; -import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector; -import org.apache.hadoop.hive.serde2.objectinspector.StructObjectInspector; -import org.apache.hadoop.hive.serde2.typeinfo.DecimalTypeInfo; -import org.apache.hadoop.hive.serde2.typeinfo.ListTypeInfo; -import org.apache.hadoop.hive.serde2.typeinfo.MapTypeInfo; -import org.apache.hadoop.hive.serde2.typeinfo.PrimitiveTypeInfo; -import org.apache.hadoop.hive.serde2.typeinfo.StructTypeInfo; -import org.apache.hadoop.hive.serde2.typeinfo.TimestampLocalTZTypeInfo; -import org.apache.hadoop.hive.serde2.typeinfo.TypeInfo; -import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoFactory; -import org.apache.hadoop.hive.serde2.typeinfo.UnionTypeInfo; -import org.apache.hadoop.io.Writable; - -import java.io.DataInput; -import java.io.DataOutput; -import java.util.ArrayList; -import java.util.List; -import java.util.Properties; - -import static org.apache.hadoop.hive.serde2.typeinfo.TypeInfoUtils.getStandardWritableObjectInspectorFromTypeInfo; - -/** - * ArrowColumnarBatchSerDe converts Apache Hive rows to Apache Arrow columns. Its serialized - * class is {@link ArrowWrapperWritable}, which doesn't support {@link - * Writable#readFields(DataInput)} and {@link Writable#write(DataOutput)}. - * - * Followings are known issues of current implementation. - * - * A list column cannot have a decimal column. {@link UnionListWriter} doesn't have an - * implementation for {@link BaseWriter.ListWriter#decimal()}. - * - * A union column can have only one of string, char, varchar fields at a same time. Apache Arrow - * doesn't have string and char, so {@link ArrowColumnarBatchSerDe} uses varchar to simulate - * string and char. They will be considered as a same data type in - * {@link org.apache.arrow.vector.complex.UnionVector}. - * - * Timestamp with local timezone is not supported. {@link VectorAssignRow} doesn't support it. - */ -public class ArrowColumnarBatchSerDe extends AbstractSerDe { - - private static final String DEFAULT_ARROW_FIELD_NAME = "[DEFAULT]"; - - static final int MILLIS_PER_SECOND = 1_000; - static final int MICROS_PER_SECOND = 1_000_000; - static final int NS_PER_SECOND = 1_000_000_000; - - static final int NS_PER_MILLIS = NS_PER_SECOND / MILLIS_PER_SECOND; - static final int NS_PER_MICROS = NS_PER_SECOND / MICROS_PER_SECOND; - static final int MICROS_PER_MILLIS = MICROS_PER_SECOND / MILLIS_PER_SECOND; - static final int SECOND_PER_DAY = 24 * 60 * 60; - - BufferAllocator rootAllocator; - StructTypeInfo rowTypeInfo; - StructObjectInspector rowObjectInspector; - - @VisibleForTesting - Serializer serializer; - private Deserializer deserializer; - - @Override - public void initialize(Configuration configuration, Properties tableProperties, Properties partitionProperties) - throws SerDeException { - super.initialize(configuration, tableProperties, partitionProperties); - - rowTypeInfo = (StructTypeInfo) TypeInfoFactory.getStructTypeInfo(getColumnNames(), getColumnTypes()); - rowObjectInspector = (StructObjectInspector) getStandardWritableObjectInspectorFromTypeInfo(rowTypeInfo); - - final List fields = new ArrayList<>(); - final int size = getColumnNames().size(); - for (int i = 0; i < size; i++) { - fields.add(toField(getColumnNames().get(i), getColumnTypes().get(i))); - } - - } - - private static Field toField(String name, TypeInfo typeInfo) { - switch (typeInfo.getCategory()) { - case PRIMITIVE: - final PrimitiveTypeInfo primitiveTypeInfo = (PrimitiveTypeInfo) typeInfo; - switch (primitiveTypeInfo.getPrimitiveCategory()) { - case BOOLEAN: - return Field.nullable(name, MinorType.BIT.getType()); - case BYTE: - return Field.nullable(name, MinorType.TINYINT.getType()); - case SHORT: - return Field.nullable(name, MinorType.SMALLINT.getType()); - case INT: - return Field.nullable(name, MinorType.INT.getType()); - case LONG: - return Field.nullable(name, MinorType.BIGINT.getType()); - case FLOAT: - return Field.nullable(name, MinorType.FLOAT4.getType()); - case DOUBLE: - return Field.nullable(name, MinorType.FLOAT8.getType()); - case STRING: - case VARCHAR: - case CHAR: - return Field.nullable(name, MinorType.VARCHAR.getType()); - case DATE: - return Field.nullable(name, MinorType.DATEDAY.getType()); - case TIMESTAMP: - return Field.nullable(name, MinorType.TIMESTAMPMILLI.getType()); - case TIMESTAMPLOCALTZ: - final TimestampLocalTZTypeInfo timestampLocalTZTypeInfo = - (TimestampLocalTZTypeInfo) typeInfo; - final String timeZone = timestampLocalTZTypeInfo.getTimeZone().toString(); - return Field.nullable(name, new ArrowType.Timestamp(TimeUnit.MILLISECOND, timeZone)); - case BINARY: - return Field.nullable(name, MinorType.VARBINARY.getType()); - case DECIMAL: - final DecimalTypeInfo decimalTypeInfo = (DecimalTypeInfo) typeInfo; - final int precision = decimalTypeInfo.precision(); - final int scale = decimalTypeInfo.scale(); - return Field.nullable(name, new ArrowType.Decimal(precision, scale)); - case INTERVAL_YEAR_MONTH: - return Field.nullable(name, MinorType.INTERVALYEAR.getType()); - case INTERVAL_DAY_TIME: - return Field.nullable(name, MinorType.INTERVALDAY.getType()); - default: - throw new IllegalArgumentException(); - } - case LIST: - final ListTypeInfo listTypeInfo = (ListTypeInfo) typeInfo; - final TypeInfo elementTypeInfo = listTypeInfo.getListElementTypeInfo(); - return new Field(name, FieldType.nullable(MinorType.LIST.getType()), - Lists.newArrayList(toField(DEFAULT_ARROW_FIELD_NAME, elementTypeInfo))); - case STRUCT: - final StructTypeInfo structTypeInfo = (StructTypeInfo) typeInfo; - final List fieldTypeInfos = structTypeInfo.getAllStructFieldTypeInfos(); - final List fieldNames = structTypeInfo.getAllStructFieldNames(); - final List structFields = Lists.newArrayList(); - final int structSize = fieldNames.size(); - for (int i = 0; i < structSize; i++) { - structFields.add(toField(fieldNames.get(i), fieldTypeInfos.get(i))); - } - return new Field(name, FieldType.nullable(MinorType.STRUCT.getType()), structFields); - case UNION: - final UnionTypeInfo unionTypeInfo = (UnionTypeInfo) typeInfo; - final List objectTypeInfos = unionTypeInfo.getAllUnionObjectTypeInfos(); - final List unionFields = Lists.newArrayList(); - final int unionSize = unionFields.size(); - for (int i = 0; i < unionSize; i++) { - unionFields.add(toField(DEFAULT_ARROW_FIELD_NAME, objectTypeInfos.get(i))); - } - return new Field(name, FieldType.nullable(MinorType.UNION.getType()), unionFields); - case MAP: - final MapTypeInfo mapTypeInfo = (MapTypeInfo) typeInfo; - final TypeInfo keyTypeInfo = mapTypeInfo.getMapKeyTypeInfo(); - final TypeInfo valueTypeInfo = mapTypeInfo.getMapValueTypeInfo(); - - final List mapFields = Lists.newArrayList(); - mapFields.add(toField(name+"_keys", keyTypeInfo)); - mapFields.add(toField(name+"_values", valueTypeInfo)); - - FieldType struct = new FieldType(false, new ArrowType.Struct(), null); - List childrenOfList = Lists.newArrayList(new Field(name, struct, mapFields)); - return new Field(name, FieldType.nullable(MinorType.LIST.getType()), childrenOfList); - default: - throw new IllegalArgumentException(); - } - } - - static ListTypeInfo toStructListTypeInfo(MapTypeInfo mapTypeInfo) { - final StructTypeInfo structTypeInfo = new StructTypeInfo(); - structTypeInfo.setAllStructFieldNames(Lists.newArrayList("key", "value")); - structTypeInfo.setAllStructFieldTypeInfos(Lists.newArrayList( - mapTypeInfo.getMapKeyTypeInfo(), mapTypeInfo.getMapValueTypeInfo())); - final ListTypeInfo structListTypeInfo = new ListTypeInfo(); - structListTypeInfo.setListElementTypeInfo(structTypeInfo); - return structListTypeInfo; - } - - static ListColumnVector toStructListVector(MapColumnVector mapVector) { - final StructColumnVector structVector; - final ListColumnVector structListVector; - structVector = new StructColumnVector(mapVector.childCount); - structVector.fields = new ColumnVector[] {mapVector.keys, mapVector.values}; - structListVector = new ListColumnVector(mapVector.childCount, null); - structListVector.child = structVector; - structListVector.childCount = mapVector.childCount; - structListVector.isRepeating = mapVector.isRepeating; - structListVector.noNulls = mapVector.noNulls; - System.arraycopy(mapVector.isNull, 0, structListVector.isNull, 0, mapVector.childCount); - System.arraycopy(mapVector.offsets, 0, structListVector.offsets, 0, mapVector.childCount); - System.arraycopy(mapVector.lengths, 0, structListVector.lengths, 0, mapVector.childCount); - return structListVector; - } - - @Override - public Class getSerializedClass() { - return ArrowWrapperWritable.class; - } - - @Override - public ArrowWrapperWritable serialize(Object obj, ObjectInspector objInspector) { - if(serializer == null) { - try { - rootAllocator = RootAllocatorFactory.INSTANCE.getRootAllocator(configuration.get()); - serializer = new Serializer(this); - } catch(Exception e) { - throw new RuntimeException("Unable to initialize serializer for ArrowColumnarBatchSerDe", e); - } - } - return serializer.serialize(obj, objInspector); - } - - @Override - public Object deserialize(Writable writable) { - if(deserializer == null) { - try { - rootAllocator = RootAllocatorFactory.INSTANCE.getRootAllocator(configuration.get()); - deserializer = new Deserializer(this); - } catch(Exception e) { - throw new RuntimeException(e); - } - } - return deserializer.deserialize(writable); - } - - @Override - public ObjectInspector getObjectInspector() { - return rowObjectInspector; - } -} diff --git a/ql/src/java/org/apache/hadoop/hive/ql/io/arrow/ArrowWrapperWritable.java b/ql/src/java/org/apache/hadoop/hive/ql/io/arrow/ArrowWrapperWritable.java deleted file mode 100644 index 53bee6b823f3..000000000000 --- a/ql/src/java/org/apache/hadoop/hive/ql/io/arrow/ArrowWrapperWritable.java +++ /dev/null @@ -1,80 +0,0 @@ -/* - * 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.hadoop.hive.ql.io.arrow; - -import org.apache.arrow.memory.BufferAllocator; -import org.apache.arrow.vector.VectorSchemaRoot; -import org.apache.hadoop.io.WritableComparable; -import org.apache.arrow.vector.complex.NonNullableStructVector; - -import java.io.DataInput; -import java.io.DataOutput; -import java.io.IOException; - -public class ArrowWrapperWritable implements WritableComparable { - private VectorSchemaRoot vectorSchemaRoot; - private BufferAllocator allocator; - private NonNullableStructVector rootVector; - - public ArrowWrapperWritable(VectorSchemaRoot vectorSchemaRoot) { - this.vectorSchemaRoot = vectorSchemaRoot; - } - - public ArrowWrapperWritable(VectorSchemaRoot vectorSchemaRoot, BufferAllocator allocator, NonNullableStructVector rootVector) { - this.vectorSchemaRoot = vectorSchemaRoot; - this.allocator = allocator; - this.rootVector = rootVector; - } - - public ArrowWrapperWritable() {} - - public VectorSchemaRoot getVectorSchemaRoot() { - return vectorSchemaRoot; - } - - public void setVectorSchemaRoot(VectorSchemaRoot vectorSchemaRoot) { - this.vectorSchemaRoot = vectorSchemaRoot; - } - - public BufferAllocator getAllocator() { - return allocator; - } - - public NonNullableStructVector getRootVector() { - return rootVector; - } - - @Override - public void write(DataOutput dataOutput) throws IOException { - throw new UnsupportedOperationException(); - } - - @Override - public void readFields(DataInput dataInput) throws IOException { - throw new UnsupportedOperationException(); - } - - @Override public int compareTo(Object o) { - return 0; - } - - @Override public boolean equals(Object o) { - return true; - } -} diff --git a/ql/src/java/org/apache/hadoop/hive/ql/io/arrow/Deserializer.java b/ql/src/java/org/apache/hadoop/hive/ql/io/arrow/Deserializer.java deleted file mode 100644 index 85b4ec62d97a..000000000000 --- a/ql/src/java/org/apache/hadoop/hive/ql/io/arrow/Deserializer.java +++ /dev/null @@ -1,453 +0,0 @@ -/* - * 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.hadoop.hive.ql.io.arrow; - -import org.apache.arrow.memory.ArrowBuf; -import org.apache.arrow.vector.BigIntVector; -import org.apache.arrow.vector.BitVector; -import org.apache.arrow.vector.DateDayVector; -import org.apache.arrow.vector.DecimalVector; -import org.apache.arrow.vector.FieldVector; -import org.apache.arrow.vector.Float4Vector; -import org.apache.arrow.vector.Float8Vector; -import org.apache.arrow.vector.IntVector; -import org.apache.arrow.vector.IntervalDayVector; -import org.apache.arrow.vector.IntervalYearVector; -import org.apache.arrow.vector.SmallIntVector; -import org.apache.arrow.vector.TimeStampVector; -import org.apache.arrow.vector.TinyIntVector; -import org.apache.arrow.vector.VarBinaryVector; -import org.apache.arrow.vector.VarCharVector; -import org.apache.arrow.vector.VectorSchemaRoot; -import org.apache.arrow.vector.holders.NullableIntervalDayHolder; -import org.apache.arrow.vector.types.Types; -import org.apache.hadoop.hive.common.type.HiveDecimal; -import org.apache.hadoop.hive.common.type.HiveIntervalDayTime; -import org.apache.hadoop.hive.ql.exec.vector.BytesColumnVector; -import org.apache.hadoop.hive.ql.exec.vector.ColumnVector; -import org.apache.hadoop.hive.ql.exec.vector.DecimalColumnVector; -import org.apache.hadoop.hive.ql.exec.vector.DoubleColumnVector; -import org.apache.hadoop.hive.ql.exec.vector.IntervalDayTimeColumnVector; -import org.apache.hadoop.hive.ql.exec.vector.ListColumnVector; -import org.apache.hadoop.hive.ql.exec.vector.LongColumnVector; -import org.apache.hadoop.hive.ql.exec.vector.MapColumnVector; -import org.apache.hadoop.hive.ql.exec.vector.StructColumnVector; -import org.apache.hadoop.hive.ql.exec.vector.TimestampColumnVector; -import org.apache.hadoop.hive.ql.exec.vector.UnionColumnVector; -import org.apache.hadoop.hive.ql.exec.vector.VectorExtractRow; -import org.apache.hadoop.hive.ql.exec.vector.VectorizedBatchUtil; -import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch; -import org.apache.hadoop.hive.ql.metadata.HiveException; -import org.apache.hadoop.hive.serde2.SerDeException; -import org.apache.hadoop.hive.serde2.typeinfo.ListTypeInfo; -import org.apache.hadoop.hive.serde2.typeinfo.MapTypeInfo; -import org.apache.hadoop.hive.serde2.typeinfo.StructTypeInfo; -import org.apache.hadoop.hive.serde2.typeinfo.TypeInfo; -import org.apache.hadoop.hive.serde2.typeinfo.UnionTypeInfo; -import org.apache.hadoop.io.Writable; - -import java.util.List; - -import static org.apache.hadoop.hive.ql.exec.vector.VectorizedBatchUtil.createColumnVector; -import static org.apache.hadoop.hive.ql.io.arrow.ArrowColumnarBatchSerDe.MICROS_PER_SECOND; -import static org.apache.hadoop.hive.ql.io.arrow.ArrowColumnarBatchSerDe.MILLIS_PER_SECOND; -import static org.apache.hadoop.hive.ql.io.arrow.ArrowColumnarBatchSerDe.NS_PER_MICROS; -import static org.apache.hadoop.hive.ql.io.arrow.ArrowColumnarBatchSerDe.NS_PER_MILLIS; -import static org.apache.hadoop.hive.ql.io.arrow.ArrowColumnarBatchSerDe.NS_PER_SECOND; -import static org.apache.hadoop.hive.ql.io.arrow.ArrowColumnarBatchSerDe.SECOND_PER_DAY; -import static org.apache.hadoop.hive.ql.io.arrow.ArrowColumnarBatchSerDe.toStructListTypeInfo; -import static org.apache.hadoop.hive.ql.io.arrow.ArrowColumnarBatchSerDe.toStructListVector; - -class Deserializer { - private final ArrowColumnarBatchSerDe serDe; - private final VectorExtractRow vectorExtractRow; - private final VectorizedRowBatch vectorizedRowBatch; - private Object[][] rows; - - Deserializer(ArrowColumnarBatchSerDe serDe) throws SerDeException { - this.serDe = serDe; - vectorExtractRow = new VectorExtractRow(); - final List fieldTypeInfoList = serDe.rowTypeInfo.getAllStructFieldTypeInfos(); - final int fieldCount = fieldTypeInfoList.size(); - final TypeInfo[] typeInfos = fieldTypeInfoList.toArray(new TypeInfo[fieldCount]); - try { - vectorExtractRow.init(typeInfos); - } catch (HiveException e) { - throw new SerDeException(e); - } - - vectorizedRowBatch = new VectorizedRowBatch(fieldCount); - for (int fieldIndex = 0; fieldIndex < fieldCount; fieldIndex++) { - final ColumnVector columnVector = createColumnVector(typeInfos[fieldIndex]); - columnVector.init(); - vectorizedRowBatch.cols[fieldIndex] = columnVector; - } - } - - public Object deserialize(Writable writable) { - final ArrowWrapperWritable arrowWrapperWritable = (ArrowWrapperWritable) writable; - final VectorSchemaRoot vectorSchemaRoot = arrowWrapperWritable.getVectorSchemaRoot(); - final List fieldVectors = vectorSchemaRoot.getFieldVectors(); - final int fieldCount = fieldVectors.size(); - final int rowCount = vectorSchemaRoot.getFieldVectors().get(0).getValueCount(); - vectorizedRowBatch.ensureSize(rowCount); - - if (rows == null || rows.length < rowCount ) { - rows = new Object[rowCount][]; - for (int rowIndex = 0; rowIndex < rowCount; rowIndex++) { - rows[rowIndex] = new Object[fieldCount]; - } - } - - for (int fieldIndex = 0; fieldIndex < fieldCount; fieldIndex++) { - final FieldVector fieldVector = fieldVectors.get(fieldIndex); - final int projectedCol = vectorizedRowBatch.projectedColumns[fieldIndex]; - final ColumnVector columnVector = vectorizedRowBatch.cols[projectedCol]; - final TypeInfo typeInfo = serDe.rowTypeInfo.getAllStructFieldTypeInfos().get(fieldIndex); - read(fieldVector, columnVector, typeInfo); - } - for (int rowIndex = 0; rowIndex < rowCount; rowIndex++) { - vectorExtractRow.extractRow(vectorizedRowBatch, rowIndex, rows[rowIndex]); - } - vectorizedRowBatch.reset(); - return rows; - } - - private void read(FieldVector arrowVector, ColumnVector hiveVector, TypeInfo typeInfo) { - // make sure that hiveVector is as big as arrowVector - final int size = arrowVector.getValueCount(); - hiveVector.ensureSize(size, false); - - switch (typeInfo.getCategory()) { - case PRIMITIVE: - readPrimitive(arrowVector, hiveVector); - break; - case LIST: - readList(arrowVector, (ListColumnVector) hiveVector, (ListTypeInfo) typeInfo); - break; - case MAP: - readMap(arrowVector, (MapColumnVector) hiveVector, (MapTypeInfo) typeInfo); - break; - case STRUCT: - readStruct(arrowVector, (StructColumnVector) hiveVector, (StructTypeInfo) typeInfo); - break; - case UNION: - readUnion(arrowVector, (UnionColumnVector) hiveVector, (UnionTypeInfo) typeInfo); - break; - default: - throw new IllegalArgumentException(); - } - } - - private void readPrimitive(FieldVector arrowVector, ColumnVector hiveVector) { - final Types.MinorType minorType = arrowVector.getMinorType(); - - final int size = arrowVector.getValueCount(); - - switch (minorType) { - case BIT: - { - for (int i = 0; i < size; i++) { - if (arrowVector.isNull(i)) { - VectorizedBatchUtil.setNullColIsNullValue(hiveVector, i); - } else { - hiveVector.isNull[i] = false; - ((LongColumnVector) hiveVector).vector[i] = ((BitVector) arrowVector).get(i); - } - } - } - break; - case TINYINT: - { - for (int i = 0; i < size; i++) { - if (arrowVector.isNull(i)) { - VectorizedBatchUtil.setNullColIsNullValue(hiveVector, i); - } else { - hiveVector.isNull[i] = false; - ((LongColumnVector) hiveVector).vector[i] = ((TinyIntVector) arrowVector).get(i); - } - } - } - break; - case SMALLINT: - { - for (int i = 0; i < size; i++) { - if (arrowVector.isNull(i)) { - VectorizedBatchUtil.setNullColIsNullValue(hiveVector, i); - } else { - hiveVector.isNull[i] = false; - ((LongColumnVector) hiveVector).vector[i] = ((SmallIntVector) arrowVector).get(i); - } - } - } - break; - case INT: - { - for (int i = 0; i < size; i++) { - if (arrowVector.isNull(i)) { - VectorizedBatchUtil.setNullColIsNullValue(hiveVector, i); - } else { - hiveVector.isNull[i] = false; - ((LongColumnVector) hiveVector).vector[i] = ((IntVector) arrowVector).get(i); - } - } - } - break; - case BIGINT: - { - for (int i = 0; i < size; i++) { - if (arrowVector.isNull(i)) { - VectorizedBatchUtil.setNullColIsNullValue(hiveVector, i); - } else { - hiveVector.isNull[i] = false; - ((LongColumnVector) hiveVector).vector[i] = ((BigIntVector) arrowVector).get(i); - } - } - } - break; - case FLOAT4: - { - for (int i = 0; i < size; i++) { - if (arrowVector.isNull(i)) { - VectorizedBatchUtil.setNullColIsNullValue(hiveVector, i); - } else { - hiveVector.isNull[i] = false; - ((DoubleColumnVector) hiveVector).vector[i] = ((Float4Vector) arrowVector).get(i); - } - } - } - break; - case FLOAT8: - { - for (int i = 0; i < size; i++) { - if (arrowVector.isNull(i)) { - VectorizedBatchUtil.setNullColIsNullValue(hiveVector, i); - } else { - hiveVector.isNull[i] = false; - ((DoubleColumnVector) hiveVector).vector[i] = ((Float8Vector) arrowVector).get(i); - } - } - } - break; - case VARCHAR: - { - for (int i = 0; i < size; i++) { - if (arrowVector.isNull(i)) { - VectorizedBatchUtil.setNullColIsNullValue(hiveVector, i); - } else { - hiveVector.isNull[i] = false; - ((BytesColumnVector) hiveVector).setVal(i, ((VarCharVector) arrowVector).get(i)); - } - } - } - break; - case DATEDAY: - { - for (int i = 0; i < size; i++) { - if (arrowVector.isNull(i)) { - VectorizedBatchUtil.setNullColIsNullValue(hiveVector, i); - } else { - hiveVector.isNull[i] = false; - ((LongColumnVector) hiveVector).vector[i] = ((DateDayVector) arrowVector).get(i); - } - } - } - break; - case TIMESTAMPMILLI: - case TIMESTAMPMILLITZ: - case TIMESTAMPMICRO: - case TIMESTAMPMICROTZ: - case TIMESTAMPNANO: - case TIMESTAMPNANOTZ: - { - for (int i = 0; i < size; i++) { - if (arrowVector.isNull(i)) { - VectorizedBatchUtil.setNullColIsNullValue(hiveVector, i); - } else { - hiveVector.isNull[i] = false; - - // Time = second + sub-second - final long time = ((TimeStampVector) arrowVector).get(i); - long second; - int subSecondInNanos; - switch (minorType) { - case TIMESTAMPMILLI: - case TIMESTAMPMILLITZ: - { - subSecondInNanos = (int) ((time % MILLIS_PER_SECOND) * NS_PER_MILLIS); - second = time / MILLIS_PER_SECOND; - } - break; - case TIMESTAMPMICROTZ: - case TIMESTAMPMICRO: - { - subSecondInNanos = (int) ((time % MICROS_PER_SECOND) * NS_PER_MICROS); - second = time / MICROS_PER_SECOND; - } - break; - case TIMESTAMPNANOTZ: - case TIMESTAMPNANO: - { - subSecondInNanos = (int) (time % NS_PER_SECOND); - second = time / NS_PER_SECOND; - } - break; - default: - throw new IllegalArgumentException(); - } - - final TimestampColumnVector timestampColumnVector = (TimestampColumnVector) hiveVector; - // A nanosecond value should not be negative - if (subSecondInNanos < 0) { - - // So add one second to the negative nanosecond value to make it positive - subSecondInNanos += NS_PER_SECOND; - - // Subtract one second from the second value because we added one second - second -= 1; - } - timestampColumnVector.time[i] = second * MILLIS_PER_SECOND; - timestampColumnVector.nanos[i] = subSecondInNanos; - } - } - } - break; - case VARBINARY: - { - for (int i = 0; i < size; i++) { - if (arrowVector.isNull(i)) { - VectorizedBatchUtil.setNullColIsNullValue(hiveVector, i); - } else { - hiveVector.isNull[i] = false; - ((BytesColumnVector) hiveVector).setVal(i, ((VarBinaryVector) arrowVector).get(i)); - } - } - } - break; - case DECIMAL: - { - for (int i = 0; i < size; i++) { - if (arrowVector.isNull(i)) { - VectorizedBatchUtil.setNullColIsNullValue(hiveVector, i); - } else { - hiveVector.isNull[i] = false; - ((DecimalColumnVector) hiveVector).set(i, - HiveDecimal.create(((DecimalVector) arrowVector).getObject(i))); - } - } - } - break; - case INTERVALYEAR: - { - for (int i = 0; i < size; i++) { - if (arrowVector.isNull(i)) { - VectorizedBatchUtil.setNullColIsNullValue(hiveVector, i); - } else { - hiveVector.isNull[i] = false; - ((LongColumnVector) hiveVector).vector[i] = ((IntervalYearVector) arrowVector).get(i); - } - } - } - break; - case INTERVALDAY: - { - final IntervalDayVector intervalDayVector = (IntervalDayVector) arrowVector; - final NullableIntervalDayHolder intervalDayHolder = new NullableIntervalDayHolder(); - final HiveIntervalDayTime intervalDayTime = new HiveIntervalDayTime(); - for (int i = 0; i < size; i++) { - if (arrowVector.isNull(i)) { - VectorizedBatchUtil.setNullColIsNullValue(hiveVector, i); - } else { - hiveVector.isNull[i] = false; - intervalDayVector.get(i, intervalDayHolder); - final long seconds = intervalDayHolder.days * SECOND_PER_DAY + - intervalDayHolder.milliseconds / MILLIS_PER_SECOND; - final int nanos = (intervalDayHolder.milliseconds % 1_000) * NS_PER_MILLIS; - intervalDayTime.set(seconds, nanos); - ((IntervalDayTimeColumnVector) hiveVector).set(i, intervalDayTime); - } - } - } - break; - default: - throw new IllegalArgumentException(); - } - } - - private void readList(FieldVector arrowVector, ListColumnVector hiveVector, ListTypeInfo typeInfo) { - final int size = arrowVector.getValueCount(); - hiveVector.ensureSize(size, false); - final ArrowBuf offsets = arrowVector.getOffsetBuffer(); - final int OFFSET_WIDTH = 4; - - read(arrowVector.getChildrenFromFields().get(0), - hiveVector.child, - typeInfo.getListElementTypeInfo()); - - for (int i = 0; i < size; i++) { - if (arrowVector.isNull(i)) { - VectorizedBatchUtil.setNullColIsNullValue(hiveVector, i); - } else { - hiveVector.isNull[i] = false; - final int offset = offsets.getInt(i * OFFSET_WIDTH); - hiveVector.offsets[i] = offset; - hiveVector.lengths[i] = offsets.getInt((i + 1) * OFFSET_WIDTH) - offset; - } - } - } - - private void readMap(FieldVector arrowVector, MapColumnVector hiveVector, MapTypeInfo typeInfo) { - final int size = arrowVector.getValueCount(); - hiveVector.ensureSize(size, false); - final ListTypeInfo mapStructListTypeInfo = toStructListTypeInfo(typeInfo); - final ListColumnVector mapStructListVector = toStructListVector(hiveVector); - final StructColumnVector mapStructVector = (StructColumnVector) mapStructListVector.child; - - read(arrowVector, mapStructListVector, mapStructListTypeInfo); - - hiveVector.isRepeating = mapStructListVector.isRepeating; - hiveVector.childCount = mapStructListVector.childCount; - hiveVector.noNulls = mapStructListVector.noNulls; - hiveVector.keys = mapStructVector.fields[0]; - hiveVector.values = mapStructVector.fields[1]; - System.arraycopy(mapStructListVector.offsets, 0, hiveVector.offsets, 0, size); - System.arraycopy(mapStructListVector.lengths, 0, hiveVector.lengths, 0, size); - System.arraycopy(mapStructListVector.isNull, 0, hiveVector.isNull, 0, size); - } - - private void readStruct(FieldVector arrowVector, StructColumnVector hiveVector, StructTypeInfo typeInfo) { - final int size = arrowVector.getValueCount(); - hiveVector.ensureSize(size, false); - final List fieldTypeInfos = typeInfo.getAllStructFieldTypeInfos(); - final int fieldSize = arrowVector.getChildrenFromFields().size(); - for (int i = 0; i < fieldSize; i++) { - read(arrowVector.getChildrenFromFields().get(i), hiveVector.fields[i], fieldTypeInfos.get(i)); - } - - for (int i = 0; i < size; i++) { - if (arrowVector.isNull(i)) { - VectorizedBatchUtil.setNullColIsNullValue(hiveVector, i); - } else { - hiveVector.isNull[i] = false; - } - } - } - - private void readUnion(FieldVector arrowVector, UnionColumnVector hiveVector, UnionTypeInfo typeInfo) { - } -} diff --git a/ql/src/java/org/apache/hadoop/hive/ql/io/arrow/RootAllocatorFactory.java b/ql/src/java/org/apache/hadoop/hive/ql/io/arrow/RootAllocatorFactory.java deleted file mode 100644 index 7aa732bd5c5e..000000000000 --- a/ql/src/java/org/apache/hadoop/hive/ql/io/arrow/RootAllocatorFactory.java +++ /dev/null @@ -1,53 +0,0 @@ -/* - * 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.hadoop.hive.ql.io.arrow; - -import org.apache.arrow.memory.RootAllocator; -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.hive.conf.HiveConf; - -import static org.apache.hadoop.hive.conf.HiveConf.ConfVars.HIVE_ARROW_ROOT_ALLOCATOR_LIMIT; - -/** - * Thread-safe singleton factory for RootAllocator - */ -public enum RootAllocatorFactory { - INSTANCE; - - private RootAllocator rootAllocator; - - RootAllocatorFactory() { - } - - public synchronized RootAllocator getRootAllocator(Configuration conf) { - if (rootAllocator == null) { - final long limit = HiveConf.getLongVar(conf, HIVE_ARROW_ROOT_ALLOCATOR_LIMIT); - rootAllocator = new RootAllocator(limit); - } - return rootAllocator; - } - - //arrowAllocatorLimit is ignored if an allocator was previously created - public synchronized RootAllocator getOrCreateRootAllocator(long arrowAllocatorLimit) { - if (rootAllocator == null) { - rootAllocator = new RootAllocator(arrowAllocatorLimit); - } - return rootAllocator; - } -} diff --git a/ql/src/java/org/apache/hadoop/hive/ql/io/arrow/Serializer.java b/ql/src/java/org/apache/hadoop/hive/ql/io/arrow/Serializer.java deleted file mode 100644 index 62c9fbe303ba..000000000000 --- a/ql/src/java/org/apache/hadoop/hive/ql/io/arrow/Serializer.java +++ /dev/null @@ -1,1006 +0,0 @@ -/* - * 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.hadoop.hive.ql.io.arrow; - -import com.google.common.annotations.VisibleForTesting; -import org.apache.arrow.memory.ArrowBuf; -import org.apache.arrow.vector.BigIntVector; -import org.apache.arrow.vector.BitVector; -import org.apache.arrow.vector.BitVectorHelper; -import org.apache.arrow.vector.DateDayVector; -import org.apache.arrow.vector.DecimalVector; -import org.apache.arrow.vector.FieldVector; -import org.apache.arrow.vector.Float4Vector; -import org.apache.arrow.vector.Float8Vector; -import org.apache.arrow.vector.IntVector; -import org.apache.arrow.vector.IntervalDayVector; -import org.apache.arrow.vector.IntervalYearVector; -import org.apache.arrow.vector.SmallIntVector; -import org.apache.arrow.vector.TimeStampMicroTZVector; -import org.apache.arrow.vector.TinyIntVector; -import org.apache.arrow.vector.VarBinaryVector; -import org.apache.arrow.vector.VarCharVector; -import org.apache.arrow.vector.VectorSchemaRoot; -import org.apache.arrow.vector.complex.ListVector; -import org.apache.arrow.vector.complex.NonNullableStructVector; -import org.apache.arrow.vector.complex.StructVector; -import org.apache.arrow.vector.holders.DecimalHolder; -import org.apache.arrow.vector.types.TimeUnit; -import org.apache.arrow.vector.types.Types; -import org.apache.arrow.vector.types.pojo.ArrowType; -import org.apache.arrow.vector.types.pojo.FieldType; -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.hive.conf.HiveConf; -import org.apache.hadoop.hive.ql.exec.vector.BytesColumnVector; -import org.apache.hadoop.hive.ql.exec.vector.ColumnVector; -import org.apache.hadoop.hive.ql.exec.vector.DateColumnVector; -import org.apache.hadoop.hive.ql.exec.vector.DecimalColumnVector; -import org.apache.hadoop.hive.ql.exec.vector.Decimal64ColumnVector; -import org.apache.hadoop.hive.ql.exec.vector.DoubleColumnVector; -import org.apache.hadoop.hive.ql.exec.vector.IntervalDayTimeColumnVector; -import org.apache.hadoop.hive.ql.exec.vector.ListColumnVector; -import org.apache.hadoop.hive.ql.exec.vector.LongColumnVector; -import org.apache.hadoop.hive.ql.exec.vector.MapColumnVector; -import org.apache.hadoop.hive.ql.exec.vector.MultiValuedColumnVector; -import org.apache.hadoop.hive.ql.exec.vector.StructColumnVector; -import org.apache.hadoop.hive.ql.exec.vector.TimestampColumnVector; -import org.apache.hadoop.hive.ql.exec.vector.UnionColumnVector; -import org.apache.hadoop.hive.ql.exec.vector.VectorAssignRow; -import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch; -import org.apache.hadoop.hive.ql.exec.vector.expressions.StringExpr; -import org.apache.hadoop.hive.ql.metadata.HiveException; -import org.apache.hadoop.hive.serde2.SerDeException; -import org.apache.hadoop.hive.serde2.io.HiveDecimalWritable; -import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector; -import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorUtils; -import org.apache.hadoop.hive.serde2.objectinspector.PrimitiveObjectInspector; -import org.apache.hadoop.hive.serde2.objectinspector.StructObjectInspector; -import org.apache.hadoop.hive.serde2.typeinfo.CharTypeInfo; -import org.apache.hadoop.hive.serde2.typeinfo.DecimalTypeInfo; -import org.apache.hadoop.hive.serde2.typeinfo.ListTypeInfo; -import org.apache.hadoop.hive.serde2.typeinfo.MapTypeInfo; -import org.apache.hadoop.hive.serde2.typeinfo.PrimitiveTypeInfo; -import org.apache.hadoop.hive.serde2.typeinfo.StructTypeInfo; -import org.apache.hadoop.hive.serde2.typeinfo.TypeInfo; -import org.apache.hadoop.hive.serde2.typeinfo.UnionTypeInfo; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; -import org.apache.arrow.memory.BufferAllocator; - -import java.math.BigDecimal; -import java.math.BigInteger; -import java.util.ArrayList; -import java.util.List; - -import static org.apache.hadoop.hive.conf.HiveConf.ConfVars.HIVE_ARROW_BATCH_SIZE; -import static org.apache.hadoop.hive.conf.HiveConf.ConfVars.HIVE_ARROW_BATCH_ALLOCATOR_LIMIT; -import static org.apache.hadoop.hive.conf.HiveConf.ConfVars.LLAP_EXTERNAL_CLIENT_USE_HYBRID_CALENDAR; -import static org.apache.hadoop.hive.ql.exec.vector.VectorizedBatchUtil.createColumnVector; -import static org.apache.hadoop.hive.ql.io.arrow.ArrowColumnarBatchSerDe.MICROS_PER_MILLIS; -import static org.apache.hadoop.hive.ql.io.arrow.ArrowColumnarBatchSerDe.MILLIS_PER_SECOND; -import static org.apache.hadoop.hive.ql.io.arrow.ArrowColumnarBatchSerDe.NS_PER_MICROS; -import static org.apache.hadoop.hive.ql.io.arrow.ArrowColumnarBatchSerDe.NS_PER_MILLIS; -import static org.apache.hadoop.hive.ql.io.arrow.ArrowColumnarBatchSerDe.SECOND_PER_DAY; -import static org.apache.hadoop.hive.ql.io.arrow.ArrowColumnarBatchSerDe.toStructListTypeInfo; -import static org.apache.hadoop.hive.ql.io.arrow.ArrowColumnarBatchSerDe.toStructListVector; -import static org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorUtils.ObjectInspectorCopyOption.WRITABLE; -import static org.apache.hadoop.hive.serde2.typeinfo.TypeInfoUtils.getTypeInfoFromObjectInspector; - -public class Serializer { - private static final Logger LOG = LoggerFactory.getLogger(Serializer.class); - - private final int MAX_BUFFERED_ROWS; - private final static byte[] EMPTY_BYTES = new byte[0]; - - // Hive columns - @VisibleForTesting - final VectorizedRowBatch vectorizedRowBatch; - private final VectorAssignRow vectorAssignRow; - private int batchSize; - private BufferAllocator allocator; - private List fieldTypeInfos; - private List fieldNames; - private int fieldSize; - - private boolean useHybridCalendar; - private final StructVector rootVector; - private final DecimalHolder decimalHolder = new DecimalHolder(); - - //Constructor for non-serde serialization - public Serializer(Configuration conf, String attemptId, List typeInfos, List fieldNames) { - this.fieldTypeInfos = typeInfos; - this.fieldNames = fieldNames; - long childAllocatorLimit = HiveConf.getLongVar(conf, HIVE_ARROW_BATCH_ALLOCATOR_LIMIT); - this.useHybridCalendar = HiveConf.getBoolVar(conf, LLAP_EXTERNAL_CLIENT_USE_HYBRID_CALENDAR); - //Use per-task allocator for accounting only, no need to reserve per-task memory - long childAllocatorReservation = 0L; - //Break out accounting of direct memory per-task, so we can check no memory is leaked when task is completed - allocator = RootAllocatorFactory.INSTANCE.getRootAllocator(conf).newChildAllocator( - attemptId, - childAllocatorReservation, - childAllocatorLimit); - rootVector = StructVector.empty(null, allocator); - //These last fields are unused in non-serde usage - vectorizedRowBatch = null; - vectorAssignRow = null; - MAX_BUFFERED_ROWS = 0; - } - - Serializer(ArrowColumnarBatchSerDe serDe) throws SerDeException { - Configuration serdeConf = serDe.getConfiguration().get(); - MAX_BUFFERED_ROWS = HiveConf.getIntVar(serdeConf, HIVE_ARROW_BATCH_SIZE); - long childAllocatorLimit = HiveConf.getLongVar(serdeConf, HIVE_ARROW_BATCH_ALLOCATOR_LIMIT); - this.useHybridCalendar = HiveConf.getBoolVar(serdeConf, LLAP_EXTERNAL_CLIENT_USE_HYBRID_CALENDAR); - LOG.info("ArrowColumnarBatchSerDe max number of buffered columns: " + MAX_BUFFERED_ROWS); - String childAllocatorName = Thread.currentThread().getName(); - //Use per-task allocator for accounting only, no need to reserve per-task memory - long childAllocatorReservation = 0L; - //Break out accounting of direct memory per-task, so we can check no memory is leaked when task is completed - allocator = serDe.rootAllocator.newChildAllocator( - childAllocatorName, - childAllocatorReservation, - childAllocatorLimit); - - // Schema - StructTypeInfo structTypeInfo = (StructTypeInfo) getTypeInfoFromObjectInspector(serDe.rowObjectInspector); - fieldTypeInfos = structTypeInfo.getAllStructFieldTypeInfos(); - fieldNames = structTypeInfo.getAllStructFieldNames(); - fieldSize = fieldTypeInfos.size(); - // Init Arrow stuffs - rootVector = StructVector.empty(null, allocator); - - // Init Hive stuffs - vectorizedRowBatch = new VectorizedRowBatch(fieldSize); - for (int fieldIndex = 0; fieldIndex < fieldSize; fieldIndex++) { - final ColumnVector columnVector = createColumnVector(fieldTypeInfos.get(fieldIndex)); - vectorizedRowBatch.cols[fieldIndex] = columnVector; - columnVector.init(); - } - vectorizedRowBatch.ensureSize(MAX_BUFFERED_ROWS); - vectorAssignRow = new VectorAssignRow(); - try { - vectorAssignRow.init(serDe.rowObjectInspector); - } catch (HiveException e) { - throw new SerDeException(e); - } - } - - //Construct an emptyBatch which contains schema-only info - public ArrowWrapperWritable emptyBatch() { - return serializeBatch(new VectorizedRowBatch(fieldTypeInfos.size()), false); - } - - //Used for both: - //1. VectorizedRowBatch constructed by batching rows - //2. VectorizedRowBatch provided from upstream (isNative) - public ArrowWrapperWritable serializeBatch(VectorizedRowBatch vectorizedRowBatch, boolean isNative) { - rootVector.setValueCount(0); - - for (int fieldIndex = 0; fieldIndex < vectorizedRowBatch.projectionSize; fieldIndex++) { - final int projectedColumn = vectorizedRowBatch.projectedColumns[fieldIndex]; - final ColumnVector hiveVector = vectorizedRowBatch.cols[projectedColumn]; - final TypeInfo fieldTypeInfo = fieldTypeInfos.get(fieldIndex); - final String fieldName = fieldNames.get(fieldIndex); - final FieldType fieldType = toFieldType(fieldTypeInfo); - //Reuse existing FieldVector buffers - //since we always call setValue or setNull for each row - boolean fieldExists = false; - if(rootVector.getChild(fieldName) != null) { - fieldExists = true; - } - final FieldVector arrowVector = rootVector.addOrGet(fieldName, fieldType, FieldVector.class); - if(fieldExists) { - arrowVector.setValueCount(isNative ? vectorizedRowBatch.size : batchSize); - } else { - arrowVector.setInitialCapacity(isNative ? vectorizedRowBatch.size : batchSize); - arrowVector.allocateNew(); - } - write(arrowVector, hiveVector, fieldTypeInfo, isNative ? vectorizedRowBatch.size : batchSize, vectorizedRowBatch, isNative); - } - if(!isNative) { - //Only mutate batches that are constructed by this serde - vectorizedRowBatch.reset(); - rootVector.setValueCount(batchSize); - } else { - rootVector.setValueCount(vectorizedRowBatch.size); - } - - batchSize = 0; - VectorSchemaRoot vectorSchemaRoot = new VectorSchemaRoot(rootVector); - return new ArrowWrapperWritable(vectorSchemaRoot, allocator, rootVector); - } - - private static FieldType toFieldType(TypeInfo typeInfo) { - return new FieldType(true, toArrowType(typeInfo), null); - } - - private static ArrowType toArrowType(TypeInfo typeInfo) { - switch (typeInfo.getCategory()) { - case PRIMITIVE: - switch (((PrimitiveTypeInfo) typeInfo).getPrimitiveCategory()) { - case BOOLEAN: - return Types.MinorType.BIT.getType(); - case BYTE: - return Types.MinorType.TINYINT.getType(); - case SHORT: - return Types.MinorType.SMALLINT.getType(); - case INT: - return Types.MinorType.INT.getType(); - case LONG: - return Types.MinorType.BIGINT.getType(); - case FLOAT: - return Types.MinorType.FLOAT4.getType(); - case DOUBLE: - return Types.MinorType.FLOAT8.getType(); - case STRING: - case VARCHAR: - case CHAR: - return Types.MinorType.VARCHAR.getType(); - case DATE: - return Types.MinorType.DATEDAY.getType(); - case TIMESTAMP: - // HIVE-19853: Prefer timestamp in microsecond with time zone because Spark supports it - return new ArrowType.Timestamp(TimeUnit.MICROSECOND, "UTC"); - case BINARY: - return Types.MinorType.VARBINARY.getType(); - case DECIMAL: - final DecimalTypeInfo decimalTypeInfo = (DecimalTypeInfo) typeInfo; - return new ArrowType.Decimal(decimalTypeInfo.precision(), decimalTypeInfo.scale()); - case INTERVAL_YEAR_MONTH: - return Types.MinorType.INTERVALYEAR.getType(); - case INTERVAL_DAY_TIME: - return Types.MinorType.INTERVALDAY.getType(); - case VOID: - case TIMESTAMPLOCALTZ: - case UNKNOWN: - default: - throw new IllegalArgumentException(); - } - case LIST: - return ArrowType.List.INSTANCE; - case STRUCT: - return ArrowType.Struct.INSTANCE; - case MAP: - return new ArrowType.Map(false); - case UNION: - default: - throw new IllegalArgumentException(); - } - } - - private void write(FieldVector arrowVector, ColumnVector hiveVector, TypeInfo typeInfo, int size, - VectorizedRowBatch vectorizedRowBatch, boolean isNative) { - switch (typeInfo.getCategory()) { - case PRIMITIVE: - writePrimitive(arrowVector, hiveVector, typeInfo, size, vectorizedRowBatch, isNative); - break; - case LIST: - // the flag 'isMapDataType'=false, for all the list types except for the case when map is converted - // as a list of structs. - writeList((ListVector) arrowVector, (ListColumnVector) hiveVector, (ListTypeInfo) typeInfo, size, vectorizedRowBatch, isNative, false); - break; - case STRUCT: - // the flag 'isMapDataType'=false, for all the struct types except for the case when map is converted - // as a list of structs. - writeStruct((NonNullableStructVector) arrowVector, (StructColumnVector) hiveVector, (StructTypeInfo) typeInfo, size, vectorizedRowBatch, isNative, false); - break; - case UNION: - writeUnion(arrowVector, hiveVector, typeInfo, size, vectorizedRowBatch, isNative); - break; - case MAP: - writeMap((ListVector) arrowVector, (MapColumnVector) hiveVector, (MapTypeInfo) typeInfo, size, vectorizedRowBatch, isNative); - break; - default: - throw new IllegalArgumentException(); - } - } - - private void writeMap(ListVector arrowVector, MapColumnVector hiveVector, MapTypeInfo typeInfo, - int size, VectorizedRowBatch vectorizedRowBatch, boolean isNative) { - final ListTypeInfo structListTypeInfo = toStructListTypeInfo(typeInfo); - final ListColumnVector structListVector = hiveVector == null ? null : toStructListVector(hiveVector); - - // Map is converted as a list of structs and thus we call the writeList() method with the flag 'isMapDataType'=true - writeList(arrowVector, structListVector, structListTypeInfo, size, vectorizedRowBatch, isNative, true); - - for (int rowIndex = 0; rowIndex < size; rowIndex++) { - int selectedIndex = rowIndex; - if (vectorizedRowBatch.selectedInUse) { - selectedIndex = vectorizedRowBatch.selected[rowIndex]; - } - if (hiveVector.isNull[selectedIndex]) { - BitVectorHelper.setValidityBit(arrowVector.getValidityBuffer(), rowIndex, 0); - } else { - BitVectorHelper.setBit(arrowVector.getValidityBuffer(), rowIndex); - } - } - } - - private void writeUnion(FieldVector arrowVector, ColumnVector hiveVector, TypeInfo typeInfo, - int size, VectorizedRowBatch vectorizedRowBatch, boolean isNative) { - final UnionTypeInfo unionTypeInfo = (UnionTypeInfo) typeInfo; - final List objectTypeInfos = unionTypeInfo.getAllUnionObjectTypeInfos(); - final UnionColumnVector hiveUnionVector = (UnionColumnVector) hiveVector; - final ColumnVector[] hiveObjectVectors = hiveUnionVector.fields; - - final int tag = hiveUnionVector.tags[0]; - final ColumnVector hiveObjectVector = hiveObjectVectors[tag]; - final TypeInfo objectTypeInfo = objectTypeInfos.get(tag); - - write(arrowVector, hiveObjectVector, objectTypeInfo, size, vectorizedRowBatch, isNative); - } - - private void writeStruct(NonNullableStructVector arrowVector, StructColumnVector hiveVector, - StructTypeInfo typeInfo, int size, VectorizedRowBatch vectorizedRowBatch, boolean isNative, boolean isMapDataType) { - final List fieldNames = typeInfo.getAllStructFieldNames(); - final List fieldTypeInfos = typeInfo.getAllStructFieldTypeInfos(); - final ColumnVector[] hiveFieldVectors = hiveVector == null ? null : hiveVector.fields; - final int fieldSize = fieldTypeInfos.size(); - // This is to handle following scenario - - // if any struct value itself is NULL, we get structVector.isNull[i]=true - // but we don't get the same for it's child fields which later causes exceptions while setting to arrow vectors - // see - https://issues.apache.org/jira/browse/HIVE-25243 - if (hiveVector != null && hiveFieldVectors != null) { - for (int i = 0; i < size; i++) { - if (hiveVector.isNull[i]) { - for (ColumnVector fieldVector : hiveFieldVectors) { - fieldVector.isNull[i] = true; - fieldVector.noNulls = false; - } - } - } - } - - for (int fieldIndex = 0; fieldIndex < fieldSize; fieldIndex++) { - final TypeInfo fieldTypeInfo = fieldTypeInfos.get(fieldIndex); - final ColumnVector hiveFieldVector = hiveVector == null ? null : hiveFieldVectors[fieldIndex]; - final String fieldName = fieldNames.get(fieldIndex); - - // If the call is coming from writeMap(), then the structs within the list type should be non-nullable. - FieldType elementFieldType = (isMapDataType) ? (new FieldType(false, toArrowType(fieldTypeInfo), null)) - : (toFieldType(fieldTypeInfos.get(fieldIndex))); - final FieldVector arrowFieldVector = arrowVector.addOrGet(fieldName, elementFieldType, FieldVector.class); - - arrowFieldVector.setInitialCapacity(size); - arrowFieldVector.allocateNew(); - write(arrowFieldVector, hiveFieldVector, fieldTypeInfo, size, vectorizedRowBatch, isNative); - } - - for (int rowIndex = 0; rowIndex < size; rowIndex++) { - if (hiveVector == null || hiveVector.isNull[rowIndex]) { - BitVectorHelper.setValidityBit(arrowVector.getValidityBuffer(), rowIndex, 0); - } else { - BitVectorHelper.setBit(arrowVector.getValidityBuffer(), rowIndex); - } - } - } - - // selected[] points to the valid/filtered/selected records at row level. - // for MultiValuedColumnVector such as ListColumnVector one record of vector points to multiple nested records. - // In child vectors we get these records in exploded manner i.e. the number of records in child vectors can have size more - // than actual the VectorizedRowBatch, consequently selected[] also needs to be readjusted. - // This method creates a shallow copy of VectorizedRowBatch with corrected size and selected[] - - private static VectorizedRowBatch correctSelectedAndSize(VectorizedRowBatch sourceVrb, - MultiValuedColumnVector multiValuedColumnVector) { - - VectorizedRowBatch vrb = new VectorizedRowBatch(sourceVrb.numCols, sourceVrb.size); - vrb.cols = sourceVrb.cols; - vrb.endOfFile = sourceVrb.endOfFile; - vrb.projectedColumns = sourceVrb.projectedColumns; - vrb.projectionSize = sourceVrb.projectionSize; - vrb.selectedInUse = sourceVrb.selectedInUse; - vrb.setPartitionInfo(sourceVrb.getDataColumnCount(), sourceVrb.getPartitionColumnCount()); - - int correctedSize = 0; - final int[] srcVrbSelected = sourceVrb.selected; - for (int i = 0; i < sourceVrb.size; i++) { - correctedSize += multiValuedColumnVector.lengths[srcVrbSelected[i]]; - } - - int newIndex = 0; - final int[] selectedOffsetsCorrected = new int[correctedSize]; - for (int i = 0; i < sourceVrb.size; i++) { - long elementIndex = multiValuedColumnVector.offsets[srcVrbSelected[i]]; - long elementSize = multiValuedColumnVector.lengths[srcVrbSelected[i]]; - for (int j = 0; j < elementSize; j++) { - selectedOffsetsCorrected[newIndex++] = (int) (elementIndex + j); - } - } - vrb.selected = selectedOffsetsCorrected; - vrb.size = correctedSize; - return vrb; - } - - private void writeList(ListVector arrowVector, ListColumnVector hiveVector, ListTypeInfo typeInfo, int size, - VectorizedRowBatch vectorizedRowBatch, boolean isNative, boolean isMapDataType) { - final int OFFSET_WIDTH = 4; - final TypeInfo elementTypeInfo = typeInfo.getListElementTypeInfo(); - final ColumnVector hiveElementVector = hiveVector == null ? null : hiveVector.child; - - // If the call is coming from writeMap(), then the List type should be non-nullable. - FieldType elementFieldType = (isMapDataType) ? (new FieldType(false, toArrowType(elementTypeInfo), null)) - : (toFieldType(elementTypeInfo)); - - final FieldVector arrowElementVector = - (FieldVector) arrowVector.addOrGetVector(elementFieldType).getVector(); - - VectorizedRowBatch correctedVrb = vectorizedRowBatch; - int correctedSize = hiveVector == null ? 0 : hiveVector.childCount; - if (vectorizedRowBatch.selectedInUse) { - correctedVrb = correctSelectedAndSize(vectorizedRowBatch, hiveVector); - correctedSize = correctedVrb.size; - } - arrowElementVector.setInitialCapacity(correctedSize); - arrowElementVector.allocateNew(); - - // If the flag 'isMapDataType' is set to True, it means that the call is coming from writeMap() and it has to call - // writeStruct() with the same flag value, as the map is converted as a list of structs. - if (isMapDataType) { - writeStruct((NonNullableStructVector) arrowElementVector, (StructColumnVector) hiveElementVector, (StructTypeInfo) elementTypeInfo, correctedSize, correctedVrb, isNative, isMapDataType); - } else { - write(arrowElementVector, hiveElementVector, elementTypeInfo, correctedSize, correctedVrb, isNative); - } - - int nextOffset = 0; - - for (int rowIndex = 0; rowIndex < size; rowIndex++) { - int selectedIndex = rowIndex; - if (vectorizedRowBatch.selectedInUse) { - selectedIndex = vectorizedRowBatch.selected[rowIndex]; - } - if (hiveVector == null || hiveVector.isNull[selectedIndex]) { - arrowVector.getOffsetBuffer().setInt(rowIndex * OFFSET_WIDTH, nextOffset); - } else { - arrowVector.getOffsetBuffer().setInt(rowIndex * OFFSET_WIDTH, nextOffset); - nextOffset += (int) hiveVector.lengths[selectedIndex]; - arrowVector.setNotNull(rowIndex); - } - } - arrowVector.getOffsetBuffer().setInt(size * OFFSET_WIDTH, nextOffset); - } - - //Handle cases for both internally constructed - //and externally provided (isNative) VectorRowBatch - private void writePrimitive(FieldVector arrowVector, ColumnVector hiveVector, TypeInfo typeInfo, int size, - VectorizedRowBatch vectorizedRowBatch, boolean isNative) { - final PrimitiveObjectInspector.PrimitiveCategory primitiveCategory = - ((PrimitiveTypeInfo) typeInfo).getPrimitiveCategory(); - switch (primitiveCategory) { - case BOOLEAN: - { - if(isNative) { - writeGeneric(arrowVector, hiveVector, size, vectorizedRowBatch.selectedInUse, vectorizedRowBatch.selected, boolNullSetter, boolValueSetter, typeInfo); - return; - } - final BitVector bitVector = (BitVector) arrowVector; - for (int i = 0; i < size; i++) { - if (hiveVector.isNull[i]) { - boolNullSetter.accept(i, arrowVector, hiveVector); - } else { - boolValueSetter.accept(i, i, arrowVector, hiveVector, typeInfo); - } - } - } - break; - case BYTE: - { - if(isNative) { - writeGeneric(arrowVector, hiveVector, size, vectorizedRowBatch.selectedInUse, vectorizedRowBatch.selected, byteNullSetter, byteValueSetter, typeInfo); - return; - } - final TinyIntVector tinyIntVector = (TinyIntVector) arrowVector; - for (int i = 0; i < size; i++) { - if (hiveVector.isNull[i]) { - byteNullSetter.accept(i, arrowVector, hiveVector); - } else { - byteValueSetter.accept(i, i, arrowVector, hiveVector, typeInfo); - } - } - } - break; - case SHORT: - { - if(isNative) { - writeGeneric(arrowVector, hiveVector, size, vectorizedRowBatch.selectedInUse, vectorizedRowBatch.selected, shortNullSetter, shortValueSetter, typeInfo); - return; - } - final SmallIntVector smallIntVector = (SmallIntVector) arrowVector; - for (int i = 0; i < size; i++) { - if (hiveVector.isNull[i]) { - shortNullSetter.accept(i, arrowVector, hiveVector); - } else { - shortValueSetter.accept(i, i, arrowVector, hiveVector, typeInfo); - } - } - } - break; - case INT: - { - if(isNative) { - writeGeneric(arrowVector, hiveVector, size, vectorizedRowBatch.selectedInUse, vectorizedRowBatch.selected, intNullSetter, intValueSetter, typeInfo); - return; - } - for (int i = 0; i < size; i++) { - if (hiveVector.isNull[i]) { - intNullSetter.accept(i, arrowVector, hiveVector); - } else { - intValueSetter.accept(i, i, arrowVector, hiveVector, typeInfo); - } - } - } - break; - case LONG: - { - if(isNative) { - writeGeneric(arrowVector, hiveVector, size, vectorizedRowBatch.selectedInUse, vectorizedRowBatch.selected, longNullSetter, longValueSetter, typeInfo); - return; - } - final BigIntVector bigIntVector = (BigIntVector) arrowVector; - for (int i = 0; i < size; i++) { - if (hiveVector.isNull[i]) { - longNullSetter.accept(i, arrowVector, hiveVector); - } else { - longValueSetter.accept(i, i, arrowVector, hiveVector, typeInfo); - } - } - } - break; - case FLOAT: - { - if(isNative) { - writeGeneric(arrowVector, hiveVector, size, vectorizedRowBatch.selectedInUse, vectorizedRowBatch.selected, floatNullSetter, floatValueSetter, typeInfo); - return; - } - for (int i = 0; i < size; i++) { - if (hiveVector.isNull[i]) { - floatNullSetter.accept(i, arrowVector, hiveVector); - } else { - floatValueSetter.accept(i, i, arrowVector, hiveVector, typeInfo); - } - } - } - break; - case DOUBLE: - { - if(isNative) { - writeGeneric(arrowVector, hiveVector, size, vectorizedRowBatch.selectedInUse, vectorizedRowBatch.selected, doubleNullSetter, doubleValueSetter, typeInfo); - return; - } - final Float8Vector float8Vector = (Float8Vector) arrowVector; - for (int i = 0; i < size; i++) { - if (hiveVector.isNull[i]) { - doubleNullSetter.accept(i, arrowVector, hiveVector); - } else { - doubleValueSetter.accept(i, i, arrowVector, hiveVector, typeInfo); - } - } - } - break; - case CHAR: - { - if(isNative) { - writeGeneric(arrowVector, hiveVector, size, vectorizedRowBatch.selectedInUse, vectorizedRowBatch.selected, charNullSetter, charValueSetter, typeInfo); - return; - } - for (int i = 0; i < size; i++) { - if (hiveVector.isNull[i]) { - charNullSetter.accept(i, arrowVector, hiveVector); - } else { - charValueSetter.accept(i, i, arrowVector, hiveVector, typeInfo); - } - } - } - break; - case STRING: - case VARCHAR: - { - if(isNative) { - writeGeneric(arrowVector, hiveVector, size, vectorizedRowBatch.selectedInUse, vectorizedRowBatch.selected, stringNullSetter, stringValueSetter, typeInfo); - return; - } - for (int i = 0; i < size; i++) { - if (hiveVector.isNull[i]) { - stringNullSetter.accept(i, arrowVector, hiveVector); - } else { - stringValueSetter.accept(i, i, arrowVector, hiveVector, typeInfo); - } - } - } - break; - case DATE: - { - // and since hive always provides data in proleptic calendar format - // set the usingProlepticCalendar flag for conversion to hybrid if required in dateValueSetter - ((DateColumnVector) hiveVector).setUsingProlepticCalendar(true); - if(isNative) { - writeGeneric(arrowVector, hiveVector, size, vectorizedRowBatch.selectedInUse, vectorizedRowBatch.selected, dateNullSetter, dateValueSetter, typeInfo); - return; - } - for (int i = 0; i < size; i++) { - if (hiveVector.isNull[i]) { - dateNullSetter.accept(i, arrowVector, hiveVector); - } else { - dateValueSetter.accept(i, i, arrowVector, hiveVector, typeInfo); - } - } - } - break; - case TIMESTAMP: - { - // and since hive always provides data in proleptic calendar format - // set the usingProlepticCalendar flag for conversion to hybrid if required in timestampValueSetter - ((TimestampColumnVector) hiveVector).setUsingProlepticCalendar(true); - if(isNative) { - writeGeneric(arrowVector, hiveVector, size, vectorizedRowBatch.selectedInUse, vectorizedRowBatch.selected, timestampNullSetter, timestampValueSetter, typeInfo); - return; - } - for (int i = 0; i < size; i++) { - if (hiveVector.isNull[i]) { - timestampNullSetter.accept(i, arrowVector, hiveVector); - } else { - timestampValueSetter.accept(i, i, arrowVector, hiveVector, typeInfo); - } - } - } - break; - case BINARY: - { - if(isNative) { - writeGeneric(arrowVector, hiveVector, size, vectorizedRowBatch.selectedInUse, vectorizedRowBatch.selected, binaryNullSetter, binaryValueSetter, typeInfo); - return; - } - for (int i = 0; i < size; i++) { - if (hiveVector.isNull[i]) { - binaryNullSetter.accept(i, arrowVector, hiveVector); - } else { - binaryValueSetter.accept(i, i, arrowVector, hiveVector, typeInfo); - } - } - } - break; - case DECIMAL: - { - if(isNative) { - if(hiveVector instanceof DecimalColumnVector) { - writeGeneric(arrowVector, hiveVector, size, vectorizedRowBatch.selectedInUse, vectorizedRowBatch.selected, decimalNullSetter, decimalValueSetter, typeInfo); - } else { - writeGeneric(arrowVector, hiveVector, size, vectorizedRowBatch.selectedInUse, vectorizedRowBatch.selected, decimalNullSetter, decimal64ValueSetter, typeInfo); - } - return; - } - for (int i = 0; i < size; i++) { - if (hiveVector.isNull[i]) { - decimalNullSetter.accept(i, arrowVector, hiveVector); - } else if(hiveVector instanceof DecimalColumnVector) { - decimalValueSetter.accept(i, i, arrowVector, hiveVector, typeInfo); - } else if(hiveVector instanceof Decimal64ColumnVector) { - decimal64ValueSetter.accept(i, i, arrowVector, hiveVector, typeInfo); - } else { - throw new IllegalArgumentException("Unsupported vector column type: " + hiveVector.getClass().getName()); - } - } - } - break; - case INTERVAL_YEAR_MONTH: - { - if(isNative) { - writeGeneric(arrowVector, hiveVector, size, vectorizedRowBatch.selectedInUse, vectorizedRowBatch.selected, intervalYearMonthNullSetter, intervalYearMonthValueSetter, typeInfo); - return; - } - for (int i = 0; i < size; i++) { - if (hiveVector.isNull[i]) { - intervalYearMonthNullSetter.accept(i, arrowVector, hiveVector); - } else { - intervalYearMonthValueSetter.accept(i, i, arrowVector, hiveVector, typeInfo); - } - } - } - break; - case INTERVAL_DAY_TIME: - { - if(isNative) { - writeGeneric(arrowVector, hiveVector, size, vectorizedRowBatch.selectedInUse, vectorizedRowBatch.selected, intervalDayTimeNullSetter, intervalDayTimeValueSetter, typeInfo); - return; - } - for (int i = 0; i < size; i++) { - if (hiveVector.isNull[i]) { - intervalDayTimeNullSetter.accept(i, arrowVector, hiveVector); - } else { - intervalDayTimeValueSetter.accept(i, i, arrowVector, hiveVector, typeInfo); - } - } - } - break; - case VOID: - case UNKNOWN: - case TIMESTAMPLOCALTZ: - default: - throw new IllegalArgumentException(); - } - } - - ArrowWrapperWritable serialize(Object obj, ObjectInspector objInspector) { - // if row is null, it means there are no more rows (closeOp()). - // another case can be that the buffer is full. - if (obj == null) { - return serializeBatch(vectorizedRowBatch, false); - } - List standardObjects = new ArrayList(); - ObjectInspectorUtils.copyToStandardObject(standardObjects, obj, - ((StructObjectInspector) objInspector), WRITABLE); - - vectorAssignRow.assignRow(vectorizedRowBatch, batchSize, standardObjects, fieldSize); - batchSize++; - if (batchSize == MAX_BUFFERED_ROWS) { - return serializeBatch(vectorizedRowBatch, false); - } - return null; - } - - //Use a provided nullSetter and valueSetter function to populate - //fieldVector from hiveVector - private static void writeGeneric(final FieldVector fieldVector, final ColumnVector hiveVector, final int size, final boolean selectedInUse, final int[] selected, final IntAndVectorsConsumer nullSetter, final IntIntAndVectorsConsumer valueSetter, TypeInfo typeInfo) - { - final boolean[] inputIsNull = hiveVector.isNull; - final int[] sel = selected; - - if (hiveVector.isRepeating) { - if (hiveVector.noNulls || !inputIsNull[0]) { - for(int i = 0; i < size; i++) { - //Fill n rows with value in row 0 - valueSetter.accept(i, 0, fieldVector, hiveVector, typeInfo); - } - } else { - for(int i = 0; i < size; i++) { - //Fill n rows with NULL - nullSetter.accept(i, fieldVector, hiveVector); - } - } - return; - } - - if (hiveVector.noNulls) { - if (selectedInUse) { - for(int logical = 0; logical < size; logical++) { - final int batchIndex = sel[logical]; - //Add row batchIndex - valueSetter.accept(logical, batchIndex, fieldVector, hiveVector, typeInfo); - } - } else { - for(int batchIndex = 0; batchIndex < size; batchIndex++) { - //Add row batchIndex - valueSetter.accept(batchIndex, batchIndex, fieldVector, hiveVector, typeInfo); - } - } - } else { - if (selectedInUse) { - for(int logical = 0; logical < size; logical++) { - final int batchIndex = sel[logical]; - if (inputIsNull[batchIndex]) { - //Add NULL - nullSetter.accept(batchIndex, fieldVector, hiveVector); - } else { - //Add row batchIndex - valueSetter.accept(logical, batchIndex, fieldVector, hiveVector, typeInfo); - } - } - } else { - for(int batchIndex = 0; batchIndex < size; batchIndex++) { - if (inputIsNull[batchIndex]) { - //Add NULL - nullSetter.accept(batchIndex, fieldVector, hiveVector); - } else { - //Add row batchIndex - valueSetter.accept(batchIndex, batchIndex, fieldVector, hiveVector, typeInfo); - } - } - } - } - } - - //nullSetters and valueSetter for each type - - //bool - private static final IntAndVectorsConsumer boolNullSetter = (i, arrowVector, hiveVector) - -> ((BitVector) arrowVector).setNull(i); - private static final IntIntAndVectorsConsumer boolValueSetter = (i, j, arrowVector, hiveVector, typeInfo) - -> ((BitVector) arrowVector).set(i, (int) ((LongColumnVector) hiveVector).vector[j]); - - //byte - private static final IntAndVectorsConsumer byteNullSetter = (i, arrowVector, hiveVector) - -> ((TinyIntVector) arrowVector).setNull(i); - private static final IntIntAndVectorsConsumer byteValueSetter = (i, j, arrowVector, hiveVector, typeInfo) - -> ((TinyIntVector) arrowVector).set(i, (byte) ((LongColumnVector) hiveVector).vector[j]); - - //short - private static final IntAndVectorsConsumer shortNullSetter = (i, arrowVector, hiveVector) - -> ((SmallIntVector) arrowVector).setNull(i); - private static final IntIntAndVectorsConsumer shortValueSetter = (i, j, arrowVector, hiveVector, typeInfo) - -> ((SmallIntVector) arrowVector).set(i, (short) ((LongColumnVector) hiveVector).vector[j]); - - //int - private static final IntAndVectorsConsumer intNullSetter = (i, arrowVector, hiveVector) - -> ((IntVector) arrowVector).setNull(i); - private static final IntIntAndVectorsConsumer intValueSetter = (i, j, arrowVector, hiveVector, typeInfo) - -> ((IntVector) arrowVector).set(i, (int) ((LongColumnVector) hiveVector).vector[j]); - - //long - private static final IntAndVectorsConsumer longNullSetter = (i, arrowVector, hiveVector) - -> ((BigIntVector) arrowVector).setNull(i); - private static final IntIntAndVectorsConsumer longValueSetter = (i, j, arrowVector, hiveVector, typeInfo) - -> ((BigIntVector) arrowVector).set(i, ((LongColumnVector) hiveVector).vector[j]); - - //float - private static final IntAndVectorsConsumer floatNullSetter = (i, arrowVector, hiveVector) - -> ((Float4Vector) arrowVector).setNull(i); - private static final IntIntAndVectorsConsumer floatValueSetter = (i, j, arrowVector, hiveVector, typeInfo) - -> ((Float4Vector) arrowVector).set(i, (float) ((DoubleColumnVector) hiveVector).vector[j]); - - //double - private static final IntAndVectorsConsumer doubleNullSetter = (i, arrowVector, hiveVector) - -> ((Float8Vector) arrowVector).setNull(i); - private static final IntIntAndVectorsConsumer doubleValueSetter = (i, j, arrowVector, hiveVector, typeInfo) - -> ((Float8Vector) arrowVector).set(i, ((DoubleColumnVector) hiveVector).vector[j]); - - //string/varchar - private static final IntAndVectorsConsumer stringNullSetter = (i, arrowVector, hiveVector) - -> ((VarCharVector) arrowVector).setNull(i); - private static final IntIntAndVectorsConsumer stringValueSetter = (i, j, arrowVector, hiveVector, typeInfo) - -> { - BytesColumnVector bytesVector = (BytesColumnVector) hiveVector; - ((VarCharVector) arrowVector).setSafe(i, bytesVector.vector[j], bytesVector.start[j], bytesVector.length[j]); - }; - - //fixed-length CHAR - private static final IntAndVectorsConsumer charNullSetter = (i, arrowVector, hiveVector) - -> ((VarCharVector) arrowVector).setNull(i); - private static final IntIntAndVectorsConsumer charValueSetter = (i, j, arrowVector, hiveVector, typeInfo) - -> { - BytesColumnVector bytesVector = (BytesColumnVector) hiveVector; - VarCharVector varCharVector = (VarCharVector) arrowVector; - byte[] bytes = bytesVector.vector[j]; - int length = bytesVector.length[j]; - int start = bytesVector.start[j]; - - if (bytes == null) { - bytes = EMPTY_BYTES; - start = 0; - length = 0; - } - - final CharTypeInfo charTypeInfo = (CharTypeInfo) typeInfo; - final int paddedLength = charTypeInfo.getLength(); - final byte[] paddedBytes = StringExpr.padRight(bytes, start, length, paddedLength); - varCharVector.setSafe(i, paddedBytes, 0, paddedBytes.length); - }; - - //date - private static final IntAndVectorsConsumer dateNullSetter = (i, arrowVector, hiveVector) - -> ((DateDayVector) arrowVector).setNull(i); - - private final IntIntAndVectorsConsumer dateValueSetter = (i, j, arrowVector, hiveVector, typeInfo) -> { - - DateColumnVector dateColumnVector = (DateColumnVector) hiveVector; - // useHybridCalendar - means the client wants data in hybrid calendar format - if (useHybridCalendar && dateColumnVector.isUsingProlepticCalendar()) { - dateColumnVector.changeCalendar(false, true); - } - - ((DateDayVector) arrowVector).set(i, (int) (dateColumnVector).vector[j]); - }; - - //timestamp - private static final IntAndVectorsConsumer timestampNullSetter = (i, arrowVector, hiveVector) - -> ((TimeStampMicroTZVector) arrowVector).setNull(i); - private final IntIntAndVectorsConsumer timestampValueSetter = (i, j, arrowVector, hiveVector, typeInfo) - -> { - final TimeStampMicroTZVector timeStampMicroTZVector = (TimeStampMicroTZVector) arrowVector; - final TimestampColumnVector timestampColumnVector = (TimestampColumnVector) hiveVector; - - // useHybridCalendar - means the client wants data in hybrid calendar format - if (useHybridCalendar && timestampColumnVector.usingProlepticCalendar()) { - timestampColumnVector.changeCalendar(false , true); - } - - // Time = second + sub-second - final long secondInMillis = timestampColumnVector.getTime(j); - final long nanos = timestampColumnVector.getNanos(j); - // nanos should be subtracted from total time(secondInMillis) to obtain micros - // Divide nanos by 1000_000 to bring it to millisecond precision and then perform the subtraction - final long secondInMicros = (secondInMillis - nanos / NS_PER_MILLIS) * MICROS_PER_MILLIS; - final long subSecondInMicros = nanos / NS_PER_MICROS; - if ((secondInMillis > 0 && secondInMicros < 0) || (secondInMillis < 0 && secondInMicros > 0)) { - // If the timestamp cannot be represented in long microsecond, set it as a null value - timeStampMicroTZVector.setNull(i); - } else { - timeStampMicroTZVector.set(i, secondInMicros + subSecondInMicros); - } - }; - - //binary - private static final IntAndVectorsConsumer binaryNullSetter = (i, arrowVector, hiveVector) - -> ((VarBinaryVector) arrowVector).setNull(i); - private static final IntIntAndVectorsConsumer binaryValueSetter = (i, j, arrowVector, hiveVector, typeInfo) - -> { - BytesColumnVector bytesVector = (BytesColumnVector) hiveVector; - ((VarBinaryVector) arrowVector).setSafe(i, bytesVector.vector[j], bytesVector.start[j], bytesVector.length[j]); - }; - - //decimal and decimal64 - private static final IntAndVectorsConsumer decimalNullSetter = (i, arrowVector, hiveVector) - -> ((DecimalVector) arrowVector).setNull(i); - private final IntIntAndVectorsConsumer decimalValueSetter = (i, j, arrowVector, hiveVector, typeInfo) - -> { - final DecimalVector decimalVector = (DecimalVector) arrowVector; - final int scale = decimalVector.getScale(); - decimalVector.set(i, ((DecimalColumnVector) hiveVector).vector[j].getHiveDecimal().bigDecimalValue().setScale(scale)); - - final HiveDecimalWritable writable = ((DecimalColumnVector) hiveVector).vector[j]; - decimalHolder.precision = writable.precision(); - decimalHolder.scale = scale; - try (ArrowBuf arrowBuf = allocator.buffer(DecimalHolder.WIDTH)) { - decimalHolder.buffer = arrowBuf; - final BigInteger bigInteger = new BigInteger(writable.getInternalStorage()). - multiply(BigInteger.TEN.pow(scale - writable.scale())); - decimalVector.set(i, new BigDecimal(bigInteger, scale)); - } - }; - private static final IntIntAndVectorsConsumer decimal64ValueSetter = (i, j, arrowVector, hiveVector, typeInfo) - -> { - final DecimalVector decimalVector = (DecimalVector) arrowVector; - final int scale = decimalVector.getScale(); - HiveDecimalWritable decimalHolder = new HiveDecimalWritable(); - decimalHolder.setFromLongAndScale(((Decimal64ColumnVector) hiveVector).vector[j], scale); - decimalVector.set(i, decimalHolder.getHiveDecimal().bigDecimalValue().setScale(scale)); - }; - - //interval year - private static final IntAndVectorsConsumer intervalYearMonthNullSetter = (i, arrowVector, hiveVector) - -> ((IntervalYearVector) arrowVector).setNull(i); - private static IntIntAndVectorsConsumer intervalYearMonthValueSetter = (i, j, arrowVector, hiveVector, typeInfo) - -> ((IntervalYearVector) arrowVector).set(i, (int) ((LongColumnVector) hiveVector).vector[j]); - - //interval day - private static final IntAndVectorsConsumer intervalDayTimeNullSetter = (i, arrowVector, hiveVector) - -> ((IntervalDayVector) arrowVector).setNull(i); - private static IntIntAndVectorsConsumer intervalDayTimeValueSetter = (i, j, arrowVector, hiveVector, typeInfo) - -> { - final IntervalDayVector intervalDayVector = (IntervalDayVector) arrowVector; - final IntervalDayTimeColumnVector intervalDayTimeColumnVector = - (IntervalDayTimeColumnVector) hiveVector; - long totalSeconds = intervalDayTimeColumnVector.getTotalSeconds(j); - final long days = totalSeconds / SECOND_PER_DAY; - final long millis = - (totalSeconds - days * SECOND_PER_DAY) * MILLIS_PER_SECOND + - intervalDayTimeColumnVector.getNanos(j) / NS_PER_MILLIS; - intervalDayVector.set(i, (int) days, (int) millis); - }; - - //Used for setting null at arrowVector[i] - private interface IntAndVectorsConsumer { - void accept(int i, FieldVector arrowVector, ColumnVector hiveVector); - } - - //Used to copy value from hiveVector[j] -> arrowVector[i] - //since hiveVector might be referenced through vector.selected - private interface IntIntAndVectorsConsumer { - void accept(int i, int j, FieldVector arrowVector, ColumnVector hiveVector, TypeInfo typeInfo); - } - -} diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/Vectorizer.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/Vectorizer.java index 4eeb19dfe7b1..8a7fc1287375 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/Vectorizer.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/Vectorizer.java @@ -71,7 +71,6 @@ import org.apache.hadoop.hive.ql.exec.persistence.MapJoinKey; import org.apache.hadoop.hive.ql.exec.tez.TezTask; import org.apache.hadoop.hive.ql.exec.vector.VectorExpressionDescriptor; -import org.apache.hadoop.hive.ql.exec.vector.filesink.VectorFileSinkArrowOperator; import org.apache.hadoop.hive.ql.exec.vector.mapjoin.VectorMapJoinInnerBigOnlyLongOperator; import org.apache.hadoop.hive.ql.exec.vector.mapjoin.VectorMapJoinInnerBigOnlyMultiKeyOperator; import org.apache.hadoop.hive.ql.exec.vector.mapjoin.VectorMapJoinInnerBigOnlyStringOperator; @@ -4319,48 +4318,6 @@ private boolean canSpecializeReduceSink(ReduceSinkDesc desc, return true; } - private boolean checkForArrowFileSink(FileSinkDesc fileSinkDesc, - boolean isTez, VectorizationContext vContext, - VectorFileSinkDesc vectorDesc) throws HiveException { - - // Various restrictions. - - boolean isVectorizationFileSinkArrowNativeEnabled = - HiveConf.getBoolVar(hiveConf, - HiveConf.ConfVars.HIVE_VECTORIZATION_FILESINK_ARROW_NATIVE_ENABLED); - - String engine = HiveConf.getVar(hiveConf, HiveConf.ConfVars.HIVE_EXECUTION_ENGINE); - - String serdeClassName = fileSinkDesc.getTableInfo().getSerdeClassName(); - - boolean isOkArrowFileSink = - serdeClassName.equals("org.apache.hadoop.hive.ql.io.arrow.ArrowColumnarBatchSerDe") && - isVectorizationFileSinkArrowNativeEnabled && - engine.equalsIgnoreCase("tez"); - - return isOkArrowFileSink; - } - - private Operator specializeArrowFileSinkOperator( - Operator op, VectorizationContext vContext, FileSinkDesc desc, - VectorFileSinkDesc vectorDesc) throws HiveException { - - Class> opClass = VectorFileSinkArrowOperator.class; - - Operator vectorOp = null; - try { - vectorOp = OperatorFactory.getVectorOperator( - opClass, op.getCompilationOpContext(), op.getConf(), - vContext, vectorDesc); - } catch (Exception e) { - LOG.info("Vectorizer vectorizeOperator file sink class exception " + opClass.getSimpleName() + - " exception " + e); - throw new HiveException(e); - } - - return vectorOp; - } - private boolean usesVectorUDFAdaptor(VectorExpression vecExpr) { if (vecExpr == null) { return false; @@ -5278,7 +5235,7 @@ public Operator validateAndVectorizeOperator(Operator validateAndVectorizeOperator(Operator validateAndVectorizeOperator(Operator sortColumnNames, List distributeColumnNames, diff --git a/ql/src/test/org/apache/hadoop/hive/llap/TestLlapOutputFormat.java b/ql/src/test/org/apache/hadoop/hive/llap/TestLlapOutputFormat.java index f27cdf49696e..13a3070ef629 100644 --- a/ql/src/test/org/apache/hadoop/hive/llap/TestLlapOutputFormat.java +++ b/ql/src/test/org/apache/hadoop/hive/llap/TestLlapOutputFormat.java @@ -54,7 +54,6 @@ public static void setUp() throws Exception { Configuration conf = new Configuration(); // Pick random avail port HiveConf.setIntVar(conf, HiveConf.ConfVars.LLAP_DAEMON_OUTPUT_SERVICE_PORT, 0); - HiveConf.setBoolVar(conf, HiveConf.ConfVars.LLAP_OUTPUT_FORMAT_ARROW, false); LlapOutputFormatService.initializeAndStart(conf, null); service = LlapOutputFormatService.get(); LlapProxy.setDaemon(true); diff --git a/ql/src/test/org/apache/hadoop/hive/ql/io/arrow/TestArrowColumnarBatchSerDe.java b/ql/src/test/org/apache/hadoop/hive/ql/io/arrow/TestArrowColumnarBatchSerDe.java deleted file mode 100644 index a4b296be854f..000000000000 --- a/ql/src/test/org/apache/hadoop/hive/ql/io/arrow/TestArrowColumnarBatchSerDe.java +++ /dev/null @@ -1,1070 +0,0 @@ -/* - * 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.hadoop.hive.ql.io.arrow; - -import static org.apache.hadoop.hive.conf.HiveConf.ConfVars.HIVE_ARROW_BATCH_SIZE; -import com.google.common.base.Joiner; -import com.google.common.collect.Lists; -import com.google.common.collect.Maps; -import org.apache.arrow.vector.VarCharVector; -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.hive.common.type.HiveChar; -import org.apache.hadoop.hive.common.type.HiveDecimal; -import org.apache.hadoop.hive.common.type.HiveIntervalDayTime; -import org.apache.hadoop.hive.common.type.HiveIntervalYearMonth; -import org.apache.hadoop.hive.common.type.HiveVarchar; -import org.apache.hadoop.hive.common.type.Timestamp; -import org.apache.hadoop.hive.conf.HiveConf; -import org.apache.hadoop.hive.serde.serdeConstants; -import org.apache.hadoop.hive.serde2.AbstractSerDe; -import org.apache.hadoop.hive.serde2.SerDeException; -import org.apache.hadoop.hive.serde2.io.ByteWritable; -import org.apache.hadoop.hive.serde2.io.DateWritableV2; -import org.apache.hadoop.hive.serde2.io.DoubleWritable; -import org.apache.hadoop.hive.serde2.io.HiveCharWritable; -import org.apache.hadoop.hive.serde2.io.HiveDecimalWritable; -import org.apache.hadoop.hive.serde2.io.HiveIntervalDayTimeWritable; -import org.apache.hadoop.hive.serde2.io.HiveIntervalYearMonthWritable; -import org.apache.hadoop.hive.serde2.io.HiveVarcharWritable; -import org.apache.hadoop.hive.serde2.io.ShortWritable; -import org.apache.hadoop.hive.serde2.io.TimestampWritableV2; -import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector; -import org.apache.hadoop.hive.serde2.objectinspector.PrimitiveObjectInspector; -import org.apache.hadoop.hive.serde2.objectinspector.StructField; -import org.apache.hadoop.hive.serde2.objectinspector.StructObjectInspector; -import org.apache.hadoop.hive.serde2.typeinfo.TypeInfo; -import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoFactory; -import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoUtils; -import org.apache.hadoop.io.BooleanWritable; -import org.apache.hadoop.io.BytesWritable; -import org.apache.hadoop.io.FloatWritable; -import org.apache.hadoop.io.IntWritable; -import org.apache.hadoop.io.LongWritable; -import org.apache.hadoop.io.Text; -import org.junit.Before; -import org.junit.Test; - -import java.util.Collections; -import java.util.List; -import java.util.Map; -import java.util.Objects; -import java.util.Properties; -import java.util.Random; -import java.util.Set; -import java.util.concurrent.TimeUnit; - -import static org.junit.Assert.assertArrayEquals; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertNull; - -public class TestArrowColumnarBatchSerDe { - private Configuration conf; - - private final static Object[][] INTEGER_ROWS = { - {byteW(0), shortW(0), intW(0), longW(0)}, - {byteW(1), shortW(1), intW(1), longW(1)}, - {byteW(-1), shortW(-1), intW(-1), longW(-1)}, - {byteW(Byte.MIN_VALUE), shortW(Short.MIN_VALUE), intW(Integer.MIN_VALUE), - longW(Long.MIN_VALUE)}, - {byteW(Byte.MAX_VALUE), shortW(Short.MAX_VALUE), intW(Integer.MAX_VALUE), - longW(Long.MAX_VALUE)}, - {null, null, null, null}, - }; - - private final static Object[][] FLOAT_ROWS = { - {floatW(0f), doubleW(0d)}, - {floatW(1f), doubleW(1d)}, - {floatW(-1f), doubleW(-1d)}, - {floatW(Float.MIN_VALUE), doubleW(Double.MIN_VALUE)}, - {floatW(-Float.MIN_VALUE), doubleW(-Double.MIN_VALUE)}, - {floatW(Float.MAX_VALUE), doubleW(Double.MAX_VALUE)}, - {floatW(-Float.MAX_VALUE), doubleW(-Double.MAX_VALUE)}, - {floatW(Float.POSITIVE_INFINITY), doubleW(Double.POSITIVE_INFINITY)}, - {floatW(Float.NEGATIVE_INFINITY), doubleW(Double.NEGATIVE_INFINITY)}, - {null, null}, - }; - - private final static Object[][] STRING_ROWS = { - {text(""), charW("", 10), varcharW("", 10)}, - {text("Hello"), charW("Hello", 10), varcharW("Hello", 10)}, - {text("world!"), charW("world!", 10), varcharW("world!", 10)}, - {text("안녕?"), charW("안녕?", 10), varcharW("안녕?", 10)}, - {null, null, null}, - }; - - private final static long TIME_IN_MILLIS = TimeUnit.DAYS.toMillis(365 + 31 + 3); - private final static long NEGATIVE_TIME_IN_MILLIS = TimeUnit.DAYS.toMillis(-9 * 365 + 31 + 3); - private final static Timestamp TIMESTAMP; - private final static Timestamp NEGATIVE_TIMESTAMP_WITHOUT_NANOS; - - static { - TIMESTAMP = Timestamp.ofEpochMilli(TIME_IN_MILLIS); - NEGATIVE_TIMESTAMP_WITHOUT_NANOS = Timestamp.ofEpochMilli(NEGATIVE_TIME_IN_MILLIS); - } - - private final static Object[][] DTI_ROWS = { - { - new DateWritableV2(DateWritableV2.millisToDays(TIME_IN_MILLIS)), - new TimestampWritableV2(TIMESTAMP), - new HiveIntervalYearMonthWritable(new HiveIntervalYearMonth(1, 2)), - new HiveIntervalDayTimeWritable(new HiveIntervalDayTime(1, 2, 3, 4, 5_000_000)) - }, - { - new DateWritableV2(DateWritableV2.millisToDays(NEGATIVE_TIME_IN_MILLIS)), - new TimestampWritableV2(NEGATIVE_TIMESTAMP_WITHOUT_NANOS), - null, - null - }, - {null, null, null, null}, - }; - - private final static Object[][] DECIMAL_ROWS = { - {decimalW(HiveDecimal.ZERO)}, - {decimalW(HiveDecimal.ONE)}, - {decimalW(HiveDecimal.ONE.negate())}, - {decimalW(HiveDecimal.create("0.000001"))}, - {decimalW(HiveDecimal.create("100000"))}, - {null}, - }; - - private final static Object[][] BOOLEAN_ROWS = { - {new BooleanWritable(true)}, - {new BooleanWritable(false)}, - {null}, - }; - - private final static Object[][] BINARY_ROWS = { - {new BytesWritable("".getBytes())}, - {new BytesWritable("Hello".getBytes())}, - {new BytesWritable("world!".getBytes())}, - {null}, - }; - - @Before - public void setUp() { - conf = new Configuration(); - conf.setInt(HIVE_ARROW_BATCH_SIZE.varname, 1025); - } - - private static ByteWritable byteW(int value) { - return new ByteWritable((byte) value); - } - - private static ShortWritable shortW(int value) { - return new ShortWritable((short) value); - } - - private static IntWritable intW(int value) { - return new IntWritable(value); - } - - private static LongWritable longW(long value) { - return new LongWritable(value); - } - - private static FloatWritable floatW(float value) { - return new FloatWritable(value); - } - - private static DoubleWritable doubleW(double value) { - return new DoubleWritable(value); - } - - private static Text text(String value) { - return new Text(value); - } - - private static HiveCharWritable charW(String value, int length) { - return new HiveCharWritable(new HiveChar(value, length)); - } - - private static HiveVarcharWritable varcharW(String value, int length) { - return new HiveVarcharWritable(new HiveVarchar(value, length)); - } - - private static HiveDecimalWritable decimalW(HiveDecimal value) { - return new HiveDecimalWritable(value); - } - - private void initAndSerializeAndDeserialize(String[][] schema, Object[][] rows) throws SerDeException { - ArrowColumnarBatchSerDe serDe = new ArrowColumnarBatchSerDe(); - StructObjectInspector rowOI = initSerDe(serDe, schema); - serializeAndDeserialize(serDe, rows, rowOI, null); - } - - private void initAndSerializeAndDeserializeSelected(String[][] schema, Object[][] rows, int[] selected) - throws SerDeException { - ArrowColumnarBatchSerDe serDe = new ArrowColumnarBatchSerDe(); - StructObjectInspector rowOI = initSerDe(serDe, schema); - serializeAndDeserialize(serDe, rows, rowOI, selected); - } - - private StructObjectInspector initSerDe(AbstractSerDe serDe, String[][] schema) - throws SerDeException { - List fieldNameList = newArrayList(); - List fieldTypeList = newArrayList(); - List typeInfoList = newArrayList(); - - for (String[] nameAndType : schema) { - String name = nameAndType[0]; - String type = nameAndType[1]; - fieldNameList.add(name); - fieldTypeList.add(type); - typeInfoList.add(TypeInfoUtils.getTypeInfoFromTypeString(type)); - } - - String fieldNames = Joiner.on(',').join(fieldNameList); - String fieldTypes = Joiner.on(',').join(fieldTypeList); - - Properties schemaProperties = new Properties(); - schemaProperties.setProperty(serdeConstants.LIST_COLUMNS, fieldNames); - schemaProperties.setProperty(serdeConstants.LIST_COLUMN_TYPES, fieldTypes); - serDe.initialize(conf, schemaProperties, null); - return (StructObjectInspector) TypeInfoUtils.getStandardWritableObjectInspectorFromTypeInfo( - TypeInfoFactory.getStructTypeInfo(fieldNameList, typeInfoList)); - } - - private void serializeAndDeserialize(ArrowColumnarBatchSerDe serDe, Object[][] rows, - StructObjectInspector rowOI, int[] selectedRows) { - ArrowWrapperWritable serialized = null; - for (Object[] row : rows) { - serialized = serDe.serialize(row, rowOI); - } -// When obj is null serialized is not Null -- is this expected? -// assertTrue(serialized == null); - boolean useNativeSelected = selectedRows != null && selectedRows.length > 0; - - // Pass null to complete a batch - if (serialized == null) { - // Native-selected mode (triggering Serializer.writePrimitive) - if (useNativeSelected) { - serDe.serializer.vectorizedRowBatch.selectedInUse = true; - serDe.serializer.vectorizedRowBatch.size = selectedRows.length; - serDe.serializer.vectorizedRowBatch.selected = selectedRows; - // Call Native serialization directly - serialized = serDe.serializer.serializeBatch(serDe.serializer.vectorizedRowBatch, true); - } else { - // Non-native mode - serialized = serDe.serialize(null, rowOI); - } - } - final Object[][] deserializedRows = (Object[][]) serDe.deserialize(serialized); - - for (int rowIndex = 0; rowIndex < Math.min(deserializedRows.length, rows.length); rowIndex++) { - // expected row is either at rowIndex or selected[rowIndex] - final Object[] row = useNativeSelected ? rows[selectedRows[rowIndex]] : rows[rowIndex]; - final Object[] deserializedRow = deserializedRows[rowIndex]; - assertEquals(row.length, deserializedRow.length); - - final List fields = rowOI.getAllStructFieldRefs(); - for (int fieldIndex = 0; fieldIndex < fields.size(); fieldIndex++) { - final StructField field = fields.get(fieldIndex); - final ObjectInspector fieldObjInspector = field.getFieldObjectInspector(); - switch (fieldObjInspector.getCategory()) { - case PRIMITIVE: - final PrimitiveObjectInspector primitiveObjInspector = - (PrimitiveObjectInspector) fieldObjInspector; - switch (primitiveObjInspector.getPrimitiveCategory()) { - case STRING: - case VARCHAR: - case CHAR: - assertEquals(Objects.toString(row[fieldIndex]), - Objects.toString(deserializedRow[fieldIndex])); - break; - default: - assertEquals(row[fieldIndex], deserializedRow[fieldIndex]); - break; - } - break; - case STRUCT: - final Object[] rowStruct = (Object[]) row[fieldIndex]; - final List deserializedRowStruct = (List) deserializedRow[fieldIndex]; - if (rowStruct == null) { - assertNull(deserializedRowStruct); - } else { - assertArrayEquals(rowStruct, deserializedRowStruct.toArray()); - } - break; - case LIST: - case UNION: - assertEquals(row[fieldIndex], deserializedRow[fieldIndex]); - break; - case MAP: - final Map rowMap = (Map) row[fieldIndex]; - final Map deserializedRowMap = (Map) deserializedRow[fieldIndex]; - if (rowMap == null) { - assertNull(deserializedRowMap); - } else { - final Set rowMapKeySet = rowMap.keySet(); - final Set deserializedRowMapKeySet = deserializedRowMap.keySet(); - assertEquals(rowMapKeySet, deserializedRowMapKeySet); - for (Object key : rowMapKeySet) { - assertEquals(rowMap.get(key), deserializedRowMap.get(key)); - } - } - break; - } - } - } - } - - @Test - public void testComprehensive() throws SerDeException { - String[][] schema = { - {"datatypes.c1", "int"}, - {"datatypes.c2", "boolean"}, - {"datatypes.c3", "double"}, - {"datatypes.c4", "string"}, - {"datatypes.c5", "array"}, - {"datatypes.c6", "map"}, - {"datatypes.c7", "map"}, - {"datatypes.c8", "struct"}, - {"datatypes.c9", "tinyint"}, - {"datatypes.c10", "smallint"}, - {"datatypes.c11", "float"}, - {"datatypes.c12", "bigint"}, - {"datatypes.c13", "array>"}, - {"datatypes.c14", "map>"}, - {"datatypes.c15", "struct>"}, - {"datatypes.c16", "array,n:int>>"}, - {"datatypes.c17", "timestamp"}, - {"datatypes.c18", "decimal(16,7)"}, - {"datatypes.c19", "binary"}, - {"datatypes.c20", "date"}, - {"datatypes.c21", "varchar(20)"}, - {"datatypes.c22", "char(15)"}, - {"datatypes.c23", "binary"}, - }; - - Object[][] comprehensiveRows = { - { - intW(0), // c1:int - new BooleanWritable(false), // c2:boolean - doubleW(0), // c3:double - text("Hello"), // c4:string - newArrayList(intW(0), intW(1), intW(2)), // c5:array - Maps.toMap( - newArrayList(intW(0), intW(1), intW(2)), - input -> text("Number " + input)), // c6:map - Maps.toMap( - newArrayList(text("apple"), text("banana"), text("carrot")), - input -> text(input.toString().toUpperCase())), // c7:map - new Object[] {text("0"), intW(1), doubleW(2)}, // c8:struct - byteW(0), // c9:tinyint - shortW(0), // c10:smallint - floatW(0), // c11:float - longW(0), // c12:bigint - newArrayList( - newArrayList(text("a"), text("b"), text("c")), - newArrayList(text("A"), text("B"), text("C"))), // c13:array> - Maps.toMap( - newArrayList(intW(0), intW(1), intW(2)), - x -> Maps.toMap( - newArrayList(x, intW(x.get() * 2)), - y -> y)), // c14:map> - new Object[] { - intW(0), - newArrayList( - intW(1), - text("Hello"))}, // c15:struct> - Collections.singletonList( - newArrayList( - Maps.toMap( - newArrayList(text("hello")), - input -> text(input.toString().toUpperCase())), - intW(0))), // c16:array,n:int>> - new TimestampWritableV2(TIMESTAMP), // c17:timestamp - decimalW(HiveDecimal.create(0, 0)), // c18:decimal(16,7) - new BytesWritable("Hello".getBytes()), // c19:binary - new DateWritableV2(123), // c20:date - varcharW("x", 20), // c21:varchar(20) - charW("y", 15), // c22:char(15) - new BytesWritable("world!".getBytes()), // c23:binary - }, { - null, null, null, null, null, null, null, null, null, null, // c1-c10 - null, null, null, null, null, null, null, null, null, null, // c11-c20 - null, null, null, // c21-c23 - } - }; - - initAndSerializeAndDeserialize(schema, comprehensiveRows); - } - - private List newArrayList(E ... elements) { - return Lists.newArrayList(elements); - } - - @Test - public void testPrimitiveInteger() throws SerDeException { - String[][] schema = { - {"tinyint1", "tinyint"}, - {"smallint1", "smallint"}, - {"int1", "int"}, - {"bigint1", "bigint"} - }; - - initAndSerializeAndDeserialize(schema, INTEGER_ROWS); - } - - @Test - public void testPrimitiveIntegerSelected() throws SerDeException { - String[][] schema = { - {"tinyint1", "tinyint"}, - {"smallint1", "smallint"}, - {"int1", "int"}, - {"bigint1", "bigint"} - }; - int[] selectedRows = new int[] {0, 3, 5}; - initAndSerializeAndDeserializeSelected(schema, INTEGER_ROWS, selectedRows); - } - - @Test - public void testPrimitiveBigInt10000() throws SerDeException { - String[][] schema = { - {"bigint1", "bigint"} - }; - - final int batchSize = 1000; - final Object[][] integerRows = new Object[batchSize][]; - final ArrowColumnarBatchSerDe serDe = new ArrowColumnarBatchSerDe(); - StructObjectInspector rowOI = initSerDe(serDe, schema); - - for (int j = 0; j < 10; j++) { - for (int i = 0; i < batchSize; i++) { - integerRows[i] = new Object[] {longW(i + j * batchSize)}; - } - - serializeAndDeserialize(serDe, integerRows, rowOI, null); - } - } - - @Test - public void testPrimitiveBigIntRandom() { - try { - String[][] schema = { - {"bigint1", "bigint"} - }; - - final ArrowColumnarBatchSerDe serDe = new ArrowColumnarBatchSerDe(); - StructObjectInspector rowOI = initSerDe(serDe, schema); - - final Random random = new Random(); - for (int j = 0; j < 1000; j++) { - final int batchSize = random.nextInt(1000); - final Object[][] integerRows = new Object[batchSize][]; - for (int i = 0; i < batchSize; i++) { - integerRows[i] = new Object[] {longW(random.nextLong())}; - } - - serializeAndDeserialize(serDe, integerRows, rowOI, null); - } - } catch (Exception e) { - throw new RuntimeException(e); - } - } - - @Test - public void testPrimitiveFloat() throws SerDeException { - String[][] schema = { - {"float1", "float"}, - {"double1", "double"}, - }; - - initAndSerializeAndDeserialize(schema, FLOAT_ROWS); - } - - @Test - public void testPrimitiveFloatSelected() throws SerDeException { - String[][] schema = { - {"float1", "float"}, - {"double1", "double"}, - }; - int[] selectedRows = new int[] {0, 3, 5, 7, 9}; - initAndSerializeAndDeserializeSelected(schema, FLOAT_ROWS, selectedRows); - } - - @Test(expected = AssertionError.class) - public void testPrimitiveFloatNaN() throws SerDeException { - String[][] schema = { - {"float1", "float"}, - }; - - Object[][] rows = {{new FloatWritable(Float.NaN)}}; - - initAndSerializeAndDeserialize(schema, rows); - } - - @Test(expected = AssertionError.class) - public void testPrimitiveDoubleNaN() throws SerDeException { - String[][] schema = { - {"double1", "double"}, - }; - - Object[][] rows = {{new DoubleWritable(Double.NaN)}}; - - initAndSerializeAndDeserialize(schema, rows); - } - - @Test - public void testPrimitiveString() throws SerDeException { - String[][] schema = { - {"string1", "string"}, - {"char1", "char(10)"}, - {"varchar1", "varchar(10)"}, - }; - - initAndSerializeAndDeserialize(schema, STRING_ROWS); - } - - @Test - public void testPrimitiveStringSelected() throws SerDeException { - String[][] schema = { - {"string1", "string"}, - {"char1", "char(10)"}, - {"varchar1", "varchar(10)"}, - }; - int[] selectedRows = new int[] {0, 2, 4}; - initAndSerializeAndDeserializeSelected(schema, STRING_ROWS, selectedRows); - } - - @Test - public void testPrimitiveDTI() throws SerDeException { - String[][] schema = { - {"date1", "date"}, - {"timestamp1", "timestamp"}, - {"interval_year_month1", "interval_year_month"}, - {"interval_day_time1", "interval_day_time"}, - }; - - initAndSerializeAndDeserialize(schema, DTI_ROWS); - } - - @Test - public void testPrimitiveDTISelected() throws SerDeException { - String[][] schema = { - {"date1", "date"}, - {"timestamp1", "timestamp"}, - {"interval_year_month1", "interval_year_month"}, - {"interval_day_time1", "interval_day_time"}, - }; - int[] selectedRows = new int[] {0, 2}; - initAndSerializeAndDeserializeSelected(schema, DTI_ROWS, selectedRows); - } - - @Test - public void testPrimitiveRandomTimestamp() throws SerDeException { - String[][] schema = { - {"timestamp1", "timestamp"}, - }; - - int size = HiveConf.getIntVar(conf, HiveConf.ConfVars.HIVE_ARROW_BATCH_SIZE); - Random rand = new Random(294722773L); - Object[][] rows = new Object[size][]; - for (int i = 0; i < size; i++) { - long millis = ((long) rand.nextInt(Integer.MAX_VALUE)) * 1000; - Timestamp timestamp = Timestamp.ofEpochMilli(rand.nextBoolean() ? millis : -millis); - timestamp.setNanos(rand.nextInt(1000) * 1000); - rows[i] = new Object[] {new TimestampWritableV2(timestamp)}; - } - - initAndSerializeAndDeserialize(schema, rows); - } - - - @Test - public void testTimestampNanosPrecisionUpTo6Digits() throws SerDeException { - String[][] schema = { - {"timestamp1", "timestamp"}, - }; - //Nanos precise upto 6 digits - Object[][] tsRows = new Object[][]{ - {new TimestampWritableV2(Timestamp.valueOf("1800-04-01 09:01:10.123999"))}, - {new TimestampWritableV2(Timestamp.valueOf("2050-04-01 09:01:10.999999"))}, - null - }; - initAndSerializeAndDeserialize(schema, tsRows); - } - - @Test - public void testPositiveNegativeTSWithNanos() throws SerDeException { - String[][] schema = { - {"timestamp1", "timestamp"}, - }; - - Object[][] tsRows = new Object[][]{ - {new TimestampWritableV2(Timestamp.valueOf("1963-04-01 09:01:10.123"))}, - {new TimestampWritableV2(Timestamp.valueOf("1800-04-01 09:01:10.123999"))}, - {new TimestampWritableV2(Timestamp.valueOf("1750-04-01 09:01:10.123999"))}, - {new TimestampWritableV2(Timestamp.valueOf("1700-04-01 09:01:10.999999"))}, - {new TimestampWritableV2(Timestamp.valueOf("2050-04-01 09:01:10.999999"))}, - {new TimestampWritableV2(Timestamp.valueOf("1991-06-05 09:01:10.999999"))}, - {new TimestampWritableV2(Timestamp.valueOf("1992-11-04 09:01:10.999999"))}, - {new TimestampWritableV2(Timestamp.valueOf("1970-01-01 00:00:00"))}, - {new TimestampWritableV2(Timestamp.valueOf("1964-01-01 00:00:04.78"))}, - {new TimestampWritableV2(Timestamp.valueOf("1950-01-01 09:23:03.21"))}, - {new TimestampWritableV2(Timestamp.valueOf("1956-01-01 10:09:03.00"))}, - {new TimestampWritableV2(Timestamp.valueOf("1947-08-27 10:25:36.26"))}, - null - }; - initAndSerializeAndDeserialize(schema, tsRows); - } - - @Test - public void testPositiveNegativeTSWithNanosSelected() throws SerDeException { - String[][] schema = { - {"timestamp1", "timestamp"}, - }; - - Object[][] tsRows = new Object[][]{ - {new TimestampWritableV2(Timestamp.valueOf("1963-04-01 09:01:10.123"))}, - {new TimestampWritableV2(Timestamp.valueOf("1800-04-01 09:01:10.123999"))}, - {new TimestampWritableV2(Timestamp.valueOf("1750-04-01 09:01:10.123999"))}, - {new TimestampWritableV2(Timestamp.valueOf("1700-04-01 09:01:10.999999"))}, - {new TimestampWritableV2(Timestamp.valueOf("2050-04-01 09:01:10.999999"))}, - {new TimestampWritableV2(Timestamp.valueOf("1991-06-05 09:01:10.999999"))}, - {new TimestampWritableV2(Timestamp.valueOf("1992-11-04 09:01:10.999999"))}, - {new TimestampWritableV2(Timestamp.valueOf("1970-01-01 00:00:00"))}, - {new TimestampWritableV2(Timestamp.valueOf("1964-01-01 00:00:04.78"))}, - {new TimestampWritableV2(Timestamp.valueOf("1950-01-01 09:23:03.21"))}, - {new TimestampWritableV2(Timestamp.valueOf("1956-01-01 10:09:03.00"))}, - {new TimestampWritableV2(Timestamp.valueOf("1947-08-27 10:25:36.26"))}, - }; - int[] selectedRows = new int[] {0, 2, 5, 7, 9, 11}; - initAndSerializeAndDeserializeSelected(schema, tsRows, selectedRows); - } - - @Test - public void testPrimitiveDecimal() throws SerDeException { - String[][] schema = { - {"decimal1", "decimal(38,10)"}, - }; - - initAndSerializeAndDeserialize(schema, DECIMAL_ROWS); - } - - @Test public void testNativeDecimalSelected() throws SerDeException { - String[][] schema = { { "decimal1", "decimal(38,10)" }, }; - int[] selectedRows = new int[] { 0, 2, 4 }; - initAndSerializeAndDeserializeSelected(schema, DECIMAL_ROWS, selectedRows); - } - - @Test - public void testRandomPrimitiveDecimal() throws SerDeException { - String[][] schema = { - {"decimal1", "decimal(38,10)"}, - }; - - int size = 1000; - Object[][] randomDecimals = new Object[size][]; - Random random = new Random(); - for (int i = 0; i < size; i++) { - StringBuilder builder = new StringBuilder(); - builder.append(random.nextBoolean() ? '+' : '-'); - for (int j = 0; j < 28 ; j++) { - builder.append(random.nextInt(10)); - } - builder.append('.'); - for (int j = 0; j < 10; j++) { - builder.append(random.nextInt(10)); - } - randomDecimals[i] = new Object[] {decimalW(HiveDecimal.create(builder.toString()))}; - } - - initAndSerializeAndDeserialize(schema, randomDecimals); - } - - @Test - public void testPrimitiveBoolean() throws SerDeException { - String[][] schema = { - {"boolean1", "boolean"}, - }; - - initAndSerializeAndDeserialize(schema, BOOLEAN_ROWS); - } - - @Test - public void testPrimitiveBooleanSelected() throws SerDeException { - String[][] schema = { - {"boolean1", "boolean"}, - }; - int[] selectedRows = new int[] {1, 2}; - initAndSerializeAndDeserializeSelected(schema, BOOLEAN_ROWS, selectedRows); - } - - @Test - public void testPrimitiveBinary() throws SerDeException { - String[][] schema = { - {"binary1", "binary"}, - }; - - initAndSerializeAndDeserialize(schema, BINARY_ROWS); - } - - @Test - public void testPrimitiveBinarySelected() throws SerDeException { - String[][] schema = { - {"binary1", "binary"}, - }; - int[] selectedRows = new int[] {1, 3}; - initAndSerializeAndDeserializeSelected(schema, BINARY_ROWS, selectedRows); - } - - private List[][] toList(Object[][] rows) { - List[][] array = new List[rows.length][]; - for (int rowIndex = 0; rowIndex < rows.length; rowIndex++) { - Object[] row = rows[rowIndex]; - array[rowIndex] = new List[row.length]; - for (int fieldIndex = 0; fieldIndex < row.length; fieldIndex++) { - array[rowIndex][fieldIndex] = newArrayList(row[fieldIndex]); - } - } - return array; - } - - @Test - public void testListSelected() throws SerDeException { - String[][] schema = { - {"tinyint_list", "array"}, - {"smallint_list", "array"}, - {"int_list", "array"}, - {"bigint_list", "array"}, - }; - int[] selectedRows = new int[] {1, 3, 5}; - initAndSerializeAndDeserializeSelected(schema, toList(INTEGER_ROWS), selectedRows); - } - - @Test - public void testListInteger() throws SerDeException { - String[][] schema = { - {"tinyint_list", "array"}, - {"smallint_list", "array"}, - {"int_list", "array"}, - {"bigint_list", "array"}, - }; - - initAndSerializeAndDeserialize(schema, toList(INTEGER_ROWS)); - } - - @Test - public void testListFloat() throws SerDeException { - String[][] schema = { - {"float_list", "array"}, - {"double_list", "array"}, - }; - - initAndSerializeAndDeserialize(schema, toList(FLOAT_ROWS)); - } - - @Test - public void testListString() throws SerDeException { - String[][] schema = { - {"string_list", "array"}, - {"char_list", "array"}, - {"varchar_list", "array"}, - }; - - initAndSerializeAndDeserialize(schema, toList(STRING_ROWS)); - } - - @Test - public void testListDTI() throws SerDeException { - String[][] schema = { - {"date_list", "array"}, - {"timestamp_list", "array"}, - {"interval_year_month_list", "array"}, - {"interval_day_time_list", "array"}, - }; - - initAndSerializeAndDeserialize(schema, toList(DTI_ROWS)); - } - - @Test - public void testListBoolean() throws SerDeException { - String[][] schema = { - {"boolean_list", "array"}, - }; - - initAndSerializeAndDeserialize(schema, toList(BOOLEAN_ROWS)); - } - - @Test - public void testListBinary() throws SerDeException { - String[][] schema = { - {"binary_list", "array"}, - }; - - initAndSerializeAndDeserialize(schema, toList(BINARY_ROWS)); - } - - private Object[][][] toStruct(Object[][] rows) { - Object[][][] struct = new Object[rows.length][][]; - for (int rowIndex = 0; rowIndex < rows.length; rowIndex++) { - Object[] row = rows[rowIndex]; - struct[rowIndex] = new Object[][] {row}; - } - return struct; - } - - @Test - public void testStructSelected() throws SerDeException { - String[][] schema = { - {"int_struct", "struct"}, - }; - int[] selectedRows = new int[] {1, 3, 5}; - initAndSerializeAndDeserializeSelected(schema, toStruct(INTEGER_ROWS), selectedRows); - } - - @Test - public void testStructInteger() throws SerDeException { - String[][] schema = { - {"int_struct", "struct"}, - }; - - initAndSerializeAndDeserialize(schema, toStruct(INTEGER_ROWS)); - } - - @Test - public void testStructFloat() throws SerDeException { - String[][] schema = { - {"float_struct", "struct"}, - }; - - initAndSerializeAndDeserialize(schema, toStruct(FLOAT_ROWS)); - } - - @Test - public void testStructString() throws SerDeException { - String[][] schema = { - {"string_struct", "struct"}, - }; - - initAndSerializeAndDeserialize(schema, toStruct(STRING_ROWS)); - } - - @Test - public void testStructDTI() throws SerDeException { - String[][] schema = { - {"date_struct", "struct"}, - }; - - initAndSerializeAndDeserialize(schema, toStruct(DTI_ROWS)); - } - - @Test - public void testStructDecimal() throws SerDeException { - String[][] schema = { - {"decimal_struct", "struct"}, - }; - - initAndSerializeAndDeserialize(schema, toStruct(DECIMAL_ROWS)); - } - - @Test - public void testStructBoolean() throws SerDeException { - String[][] schema = { - {"boolean_struct", "struct"}, - }; - - initAndSerializeAndDeserialize(schema, toStruct(BOOLEAN_ROWS)); - } - - @Test - public void testStructBinary() throws SerDeException { - String[][] schema = { - {"binary_struct", "struct"}, - }; - - initAndSerializeAndDeserialize(schema, toStruct(BINARY_ROWS)); - } - - private Object[][] toMap(Object[][] rows) { - Map[][] array = new Map[rows.length][]; - for (int rowIndex = 0; rowIndex < rows.length; rowIndex++) { - Object[] row = rows[rowIndex]; - array[rowIndex] = new Map[row.length]; - for (int fieldIndex = 0; fieldIndex < row.length; fieldIndex++) { - Map map = Maps.newHashMap(); - map.put(new Text(String.valueOf(row[fieldIndex])), row[fieldIndex]); - array[rowIndex][fieldIndex] = map; - } - } - return array; - } - - @Test - public void testMapSelected() throws SerDeException { - String[][] schema = { - {"tinyint_map", "map"}, - {"smallint_map", "map"}, - {"int_map", "map"}, - {"bigint_map", "map"}, - }; - int[] selectedRows = new int[] {1, 3, 5}; - initAndSerializeAndDeserializeSelected(schema, toMap(INTEGER_ROWS), selectedRows); - } - - @Test - public void testMapInteger() throws SerDeException { - String[][] schema = { - {"tinyint_map", "map"}, - {"smallint_map", "map"}, - {"int_map", "map"}, - {"bigint_map", "map"}, - }; - - initAndSerializeAndDeserialize(schema, toMap(INTEGER_ROWS)); - } - - @Test - public void testMapFloat() throws SerDeException { - String[][] schema = { - {"float_map", "map"}, - {"double_map", "map"}, - }; - - initAndSerializeAndDeserialize(schema, toMap(FLOAT_ROWS)); - } - - @Test - public void testMapString() throws SerDeException { - String[][] schema = { - {"string_map", "map"}, - {"char_map", "map"}, - {"varchar_map", "map"}, - }; - - initAndSerializeAndDeserialize(schema, toMap(STRING_ROWS)); - } - - @Test - public void testMapDTI() throws SerDeException { - String[][] schema = { - {"date_map", "map"}, - {"timestamp_map", "map"}, - {"interval_year_month_map", "map"}, - {"interval_day_time_map", "map"}, - }; - - initAndSerializeAndDeserialize(schema, toMap(DTI_ROWS)); - } - - @Test - public void testMapBoolean() throws SerDeException { - String[][] schema = { - {"boolean_map", "map"}, - }; - - initAndSerializeAndDeserialize(schema, toMap(BOOLEAN_ROWS)); - } - - @Test - public void testMapBinary() throws SerDeException { - String[][] schema = { - {"binary_map", "map"}, - }; - - initAndSerializeAndDeserialize(schema, toMap(BINARY_ROWS)); - } - - @Test - public void testPrimitiveCharPadding() throws SerDeException { - String[][] schema = { - {"char1", "char(10)"}, - }; - - HiveCharWritable[][] rows = new HiveCharWritable[][] { - {charW("Hello", 10)}, {charW("world!", 10)}}; - ArrowColumnarBatchSerDe serDe = new ArrowColumnarBatchSerDe(); - StructObjectInspector rowOI = initSerDe(serDe, schema); - - ArrowWrapperWritable serialized = null; - for (Object[] row : rows) { - serialized = serDe.serialize(row, rowOI); - } - // Pass null to complete a batch - if (serialized == null) { - serialized = serDe.serialize(null, rowOI); - } - - VarCharVector varCharVector = (VarCharVector) serialized.getVectorSchemaRoot().getFieldVectors().get(0); - for (int i = 0; i < rows.length; i++) { - assertEquals(rows[i][0].getPaddedValue().toString(), new String(varCharVector.get(i))); - } - } - - @Test - public void testMapDecimal() throws SerDeException { - String[][] schema = { - {"decimal_map", "map"}, - }; - - initAndSerializeAndDeserialize(schema, toMap(DECIMAL_ROWS)); - } - - @Test - public void testListDecimal() throws SerDeException { - String[][] schema = { - {"decimal_list", "array"}, - }; - - initAndSerializeAndDeserialize(schema, toList(DECIMAL_ROWS)); - } - - @Test - public void testListBooleanWithMoreThan1024Values() throws SerDeException { - String[][] schema = { - {"boolean_list", "array"}, - }; - - Object[][] rows = new Object[1025][1]; - for (int i = 0; i < 1025; i++) { - rows[i][0] = new BooleanWritable(true); - } - - initAndSerializeAndDeserialize(schema, toList(rows)); - } - - @Test - public void testStructBooleanWithMoreThan1024Values() throws SerDeException { - String[][] schema = { - {"boolean_struct", "struct"}, - }; - - Object[][] rows = new Object[1025][1]; - for (int i = 0; i < 1025; i++) { - rows[i][0] = new BooleanWritable(true); - } - - initAndSerializeAndDeserialize(schema, toStruct(rows)); - } - - @Test - public void testMapIntergerWithMoreThan1024Values() throws SerDeException { - String[][] schema = { - {"int_map", "map"}, - }; - - Object[][] rows = new Object[1025][1]; - for (int i = 0; i < 1025; i++) { - rows[i][0] = intW(i); - } - - initAndSerializeAndDeserialize(schema, toMap(rows)); - } -} diff --git a/ql/src/test/org/apache/hadoop/hive/ql/io/arrow/TestSerializer.java b/ql/src/test/org/apache/hadoop/hive/ql/io/arrow/TestSerializer.java deleted file mode 100644 index f8daada3d0d3..000000000000 --- a/ql/src/test/org/apache/hadoop/hive/ql/io/arrow/TestSerializer.java +++ /dev/null @@ -1,101 +0,0 @@ -/* - * 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.hadoop.hive.ql.io.arrow; - -import org.apache.hadoop.hive.conf.HiveConf; -import org.apache.hadoop.hive.serde2.typeinfo.TypeInfo; -import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoUtils; -import org.junit.Assert; -import org.junit.Test; - -import java.util.Arrays; -import java.util.List; - -public class TestSerializer { - @Test - public void testEmptyList() { - List typeInfos = TypeInfoUtils.getTypeInfosFromTypeString("array"); - List fieldNames = Arrays.asList(new String[]{"a"}); - Serializer converter = new Serializer(new HiveConf(), "attemptId", typeInfos, fieldNames); - ArrowWrapperWritable writable = converter.emptyBatch(); - Assert.assertEquals("Schema>", - writable.getVectorSchemaRoot().getSchema().toString()); - } - - @Test - public void testEmptyStruct() { - List typeInfos = TypeInfoUtils.getTypeInfosFromTypeString("struct"); - List fieldNames = Arrays.asList(new String[] { "a" }); - Serializer converter = new Serializer(new HiveConf(), "attemptId", typeInfos, fieldNames); - ArrowWrapperWritable writable = converter.emptyBatch(); - Assert.assertEquals("Schema>", - writable.getVectorSchemaRoot().getSchema().toString()); - } - - @Test - public void testEmptyMap() { - List typeInfos = TypeInfoUtils.getTypeInfosFromTypeString("map"); - List fieldNames = Arrays.asList(new String[] { "a" }); - Serializer converter = new Serializer(new HiveConf(), "attemptId", typeInfos, fieldNames); - ArrowWrapperWritable writable = converter.emptyBatch(); - Assert.assertEquals("Schema not null>>", - writable.getVectorSchemaRoot().getSchema().toString()); - } - - @Test - public void testEmptyComplexStruct() { - List typeInfos = TypeInfoUtils.getTypeInfosFromTypeString( - "struct,c:map,d:struct,f:map>>"); - List fieldNames = Arrays.asList(new String[] { "a" }); - Serializer converter = new Serializer(new HiveConf(), "attemptId", typeInfos, fieldNames); - ArrowWrapperWritable writable = converter.emptyBatch(); - Assert.assertEquals( - "Schema, c: Map(false) not null>, d: Struct, f: Map(false) not null>>>>", - writable.getVectorSchemaRoot().getSchema().toString()); - } - - @Test - public void testEmptyComplexMap() { - List typeInfos = TypeInfoUtils.getTypeInfosFromTypeString( - "map,struct,c:map>>"); - List fieldNames = Arrays.asList(new String[] { "a" }); - Serializer converter = new Serializer(new HiveConf(), "attemptId", typeInfos, fieldNames); - ArrowWrapperWritable writable = converter.emptyBatch(); - Assert.assertEquals( - "Schema not null, value: Struct, c: Map(false) " + - "not null>> not null> not null>>", - writable.getVectorSchemaRoot().getSchema().toString()); - } - - @Test - public void testEmptyComplexList() { - List typeInfos = TypeInfoUtils.getTypeInfosFromTypeString("struct>," + - "c:array>,d:array,f:map>>>"); - List fieldNames = Arrays.asList(new String[] { "a" }); - Serializer converter = new Serializer(new HiveConf(), "attemptId", typeInfos, fieldNames); - ArrowWrapperWritable writable = converter.emptyBatch(); - Assert.assertEquals( - "Schema>, c: List<$data$: Map(false) not null>>, d: List<$data$: Struct, f: Map(false) not null>>>>>", - writable.getVectorSchemaRoot().getSchema().toString()); - } -} diff --git a/serde/pom.xml b/serde/pom.xml index cf4396a92f05..c135b0459b18 100644 --- a/serde/pom.xml +++ b/serde/pom.xml @@ -64,17 +64,6 @@ commons-codec commons-codec - - org.apache.arrow - arrow-vector - ${arrow.version} - - - io.netty - * - - - com.carrotsearch hppc From 91bbe965bd7e20db0788061de02378923f924cf4 Mon Sep 17 00:00:00 2001 From: Laszlo Bodor Date: Wed, 23 Apr 2025 16:59:42 +0200 Subject: [PATCH 2/2] SuppressWarnings --- .../java/org/apache/hadoop/hive/llap/LlapBaseInputFormat.java | 1 + .../org/apache/hadoop/hive/llap/LlapOutputFormatService.java | 1 + 2 files changed, 2 insertions(+) diff --git a/llap-ext-client/src/java/org/apache/hadoop/hive/llap/LlapBaseInputFormat.java b/llap-ext-client/src/java/org/apache/hadoop/hive/llap/LlapBaseInputFormat.java index 9e88f3c0d100..1e204b0df736 100644 --- a/llap-ext-client/src/java/org/apache/hadoop/hive/llap/LlapBaseInputFormat.java +++ b/llap-ext-client/src/java/org/apache/hadoop/hive/llap/LlapBaseInputFormat.java @@ -191,6 +191,7 @@ public RecordReader getRecordReader(InputSplit split, JobConf j LOG.info("Registered id: " + fragmentId); + @SuppressWarnings("rawtypes") LlapBaseRecordReader recordReader = new LlapBaseRecordReader(socket.getInputStream(), llapSplit.getSchema(), BytesWritable.class, job, llapClient, socket); umbilicalResponder.setRecordReader(recordReader); diff --git a/ql/src/java/org/apache/hadoop/hive/llap/LlapOutputFormatService.java b/ql/src/java/org/apache/hadoop/hive/llap/LlapOutputFormatService.java index 200d573051ee..0afffd10be51 100644 --- a/ql/src/java/org/apache/hadoop/hive/llap/LlapOutputFormatService.java +++ b/ql/src/java/org/apache/hadoop/hive/llap/LlapOutputFormatService.java @@ -198,6 +198,7 @@ private void registerReader(ChannelHandlerContext ctx, String id, byte[] tokenBy LOG.debug("registering socket for: " + id); int maxPendingWrites = HiveConf.getIntVar(conf, HiveConf.ConfVars.LLAP_DAEMON_OUTPUT_SERVICE_MAX_PENDING_WRITES); + @SuppressWarnings("rawtypes") RecordWriter writer = new LlapRecordWriter(id, new ChunkedOutputStream( new ChannelOutputStream(ctx, id, sendBufferSize, maxPendingWrites), sendBufferSize, id));