You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@phoenix.apache.org by td...@apache.org on 2019/03/16 00:10:14 UTC
[phoenix-connectors] branch master updated: Initial commit
This is an automated email from the ASF dual-hosted git repository.
tdsilva pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/phoenix-connectors.git
The following commit(s) were added to refs/heads/master by this push:
new db25488 Initial commit
db25488 is described below
commit db2548879b9041c4522e8a7efe2855fee7744c21
Author: Thomas D'Silva <td...@apache.org>
AuthorDate: Fri Mar 15 17:09:45 2019 -0700
Initial commit
---
phoenix-flume/pom.xml | 199 +++
phoenix-flume/pom.xml~ | 199 +++
.../apache/phoenix/flume/CsvEventSerializerIT.java | 416 +++++++
.../phoenix/flume/JsonEventSerializerIT.java | 541 +++++++++
.../org/apache/phoenix/flume/PhoenixSinkIT.java | 271 +++++
.../phoenix/flume/RegexEventSerializerIT.java | 417 +++++++
.../phoenix/flume/serializer/CustomSerializer.java | 43 +
.../apache/phoenix/flume/sink/NullPhoenixSink.java | 21 +
.../apache/phoenix/flume/DefaultKeyGenerator.java | 69 ++
.../org/apache/phoenix/flume/FlumeConstants.java | 94 ++
.../org/apache/phoenix/flume/KeyGenerator.java | 24 +
.../org/apache/phoenix/flume/SchemaHandler.java | 47 +
.../flume/serializer/BaseEventSerializer.java | 245 ++++
.../flume/serializer/CsvEventSerializer.java | 196 +++
.../phoenix/flume/serializer/EventSerializer.java | 42 +
.../phoenix/flume/serializer/EventSerializers.java | 36 +
.../flume/serializer/JsonEventSerializer.java | 226 ++++
.../flume/serializer/RegexEventSerializer.java | 145 +++
.../org/apache/phoenix/flume/sink/PhoenixSink.java | 212 ++++
phoenix-hive/pom.xml | 192 +++
phoenix-hive/pom.xml~ | 192 +++
.../phoenix/hive/BaseHivePhoenixStoreIT.java | 168 +++
.../org/apache/phoenix/hive/HiveMapReduceIT.java | 41 +
.../apache/phoenix/hive/HivePhoenixStoreIT.java | 341 ++++++
.../java/org/apache/phoenix/hive/HiveTestUtil.java | 1280 ++++++++++++++++++++
.../it/java/org/apache/phoenix/hive/HiveTezIT.java | 33 +
.../org/apache/phoenix/hive/PhoenixMetaHook.java | 229 ++++
.../apache/phoenix/hive/PhoenixRecordUpdater.java | 336 +++++
.../java/org/apache/phoenix/hive/PhoenixRow.java | 64 +
.../org/apache/phoenix/hive/PhoenixRowKey.java | 62 +
.../java/org/apache/phoenix/hive/PhoenixSerDe.java | 152 +++
.../org/apache/phoenix/hive/PhoenixSerializer.java | 173 +++
.../apache/phoenix/hive/PhoenixStorageHandler.java | 276 +++++
.../org/apache/phoenix/hive/PrimaryKeyData.java | 88 ++
.../constants/PhoenixStorageHandlerConstants.java | 108 ++
.../phoenix/hive/mapreduce/PhoenixInputFormat.java | 270 +++++
.../phoenix/hive/mapreduce/PhoenixInputSplit.java | 160 +++
.../hive/mapreduce/PhoenixOutputFormat.java | 112 ++
.../hive/mapreduce/PhoenixRecordReader.java | 217 ++++
.../hive/mapreduce/PhoenixRecordWriter.java | 355 ++++++
.../hive/mapreduce/PhoenixResultWritable.java | 217 ++++
.../AbstractPhoenixObjectInspector.java | 59 +
.../PhoenixBinaryObjectInspector.java | 58 +
.../PhoenixBooleanObjectInspector.java | 55 +
.../PhoenixByteObjectInspector.java | 59 +
.../PhoenixCharObjectInspector.java | 56 +
.../PhoenixDateObjectInspector.java | 63 +
.../PhoenixDecimalObjectInspector.java | 72 ++
.../PhoenixDoubleObjectInspector.java | 59 +
.../PhoenixFloatObjectInspector.java | 60 +
.../objectinspector/PhoenixIntObjectInspector.java | 62 +
.../PhoenixListObjectInspector.java | 105 ++
.../PhoenixLongObjectInspector.java | 56 +
.../PhoenixObjectInspectorFactory.java | 148 +++
.../PhoenixShortObjectInspector.java | 56 +
.../PhoenixStringObjectInspector.java | 72 ++
.../PhoenixTimestampObjectInspector.java | 61 +
.../hive/ppd/PhoenixPredicateDecomposer.java | 95 ++
.../hive/ql/index/IndexPredicateAnalyzer.java | 523 ++++++++
.../hive/ql/index/IndexSearchCondition.java | 143 +++
.../hive/ql/index/PredicateAnalyzerFactory.java | 40 +
.../phoenix/hive/query/PhoenixQueryBuilder.java | 849 +++++++++++++
.../phoenix/hive/util/ColumnMappingUtils.java | 76 ++
.../phoenix/hive/util/PhoenixConnectionUtil.java | 116 ++
.../hive/util/PhoenixStorageHandlerUtil.java | 288 +++++
.../org/apache/phoenix/hive/util/PhoenixUtil.java | 210 ++++
.../apache/phoenix/hive/PrimaryKeyDataTest.java | 79 ++
.../hive/query/PhoenixQueryBuilderTest.java | 173 +++
phoenix-kafka/pom.xml | 420 +++++++
phoenix-kafka/pom.xml~ | 420 +++++++
.../apache/phoenix/kafka/PhoenixConsumerIT.java | 276 +++++
phoenix-kafka/src/it/resources/consumer.props | 32 +
phoenix-kafka/src/it/resources/producer.props | 24 +
.../org/apache/phoenix/kafka/KafkaConstants.java | 52 +
.../phoenix/kafka/consumer/PhoenixConsumer.java | 276 +++++
.../kafka/consumer/PhoenixConsumerTool.java | 107 ++
phoenix-pig/pom.xml | 460 +++++++
phoenix-pig/pom.xml~ | 460 +++++++
.../it/java/org/apache/phoenix/pig/BasePigIT.java | 87 ++
.../apache/phoenix/pig/PhoenixHBaseLoaderIT.java | 838 +++++++++++++
.../apache/phoenix/pig/PhoenixHBaseStorerIT.java | 292 +++++
.../phoenix/pig/udf/ReserveNSequenceTestIT.java | 306 +++++
.../org/apache/phoenix/pig/PhoenixHBaseLoader.java | 265 ++++
.../apache/phoenix/pig/PhoenixHBaseStorage.java | 236 ++++
.../apache/phoenix/pig/udf/ReserveNSequence.java | 129 ++
.../phoenix/pig/util/PhoenixPigSchemaUtil.java | 90 ++
.../pig/util/QuerySchemaParserFunction.java | 118 ++
.../pig/util/SqlQueryToColumnInfoFunction.java | 82 ++
.../pig/util/TableSchemaParserFunction.java | 52 +
.../java/org/apache/phoenix/pig/util/TypeUtil.java | 349 ++++++
.../phoenix/pig/util/PhoenixPigSchemaUtilTest.java | 92 ++
.../pig/util/QuerySchemaParserFunctionTest.java | 97 ++
.../pig/util/SqlQueryToColumnInfoFunctionTest.java | 63 +
.../pig/util/TableSchemaParserFunctionTest.java | 54 +
.../org/apache/phoenix/pig/util/TypeUtilTest.java | 83 ++
phoenix-spark/README.md | 164 +++
phoenix-spark/pom.xml | 598 +++++++++
phoenix-spark/pom.xml~ | 598 +++++++++
.../java/org/apache/phoenix/spark/AggregateIT.java | 91 ++
.../java/org/apache/phoenix/spark/OrderByIT.java | 449 +++++++
.../org/apache/phoenix/spark/SaltedTableIT.java | 53 +
.../java/org/apache/phoenix/spark/SparkUtil.java | 80 ++
phoenix-spark/src/it/resources/globalSetup.sql | 64 +
phoenix-spark/src/it/resources/log4j.xml | 70 ++
phoenix-spark/src/it/resources/tenantSetup.sql | 18 +
.../phoenix/spark/AbstractPhoenixSparkIT.scala | 117 ++
.../org/apache/phoenix/spark/PhoenixSparkIT.scala | 733 +++++++++++
.../spark/PhoenixSparkITTenantSpecific.scala | 135 +++
.../org/apache/phoenix/spark/SparkResultSet.java | 1056 ++++++++++++++++
.../spark/datasource/v2/PhoenixDataSource.java | 82 ++
.../v2/reader/PhoenixDataSourceReadOptions.java | 51 +
.../v2/reader/PhoenixDataSourceReader.java | 201 +++
.../v2/reader/PhoenixInputPartition.java | 44 +
.../v2/reader/PhoenixInputPartitionReader.java | 168 +++
.../v2/writer/PhoenixDataSourceWriteOptions.java | 109 ++
.../datasource/v2/writer/PhoenixDataWriter.java | 117 ++
.../v2/writer/PhoenixDataWriterFactory.java | 36 +
.../v2/writer/PhoenixDatasourceWriter.java | 51 +
...org.apache.spark.sql.sources.DataSourceRegister | 18 +
.../apache/phoenix/spark/ConfigurationUtil.scala | 100 ++
.../apache/phoenix/spark/DataFrameFunctions.scala | 79 ++
.../org/apache/phoenix/spark/DefaultSource.scala | 60 +
.../phoenix/spark/FilterExpressionCompiler.scala | 135 +++
.../org/apache/phoenix/spark/PhoenixRDD.scala | 150 +++
.../phoenix/spark/PhoenixRecordWritable.scala | 115 ++
.../org/apache/phoenix/spark/PhoenixRelation.scala | 69 ++
.../apache/phoenix/spark/ProductRDDFunctions.scala | 64 +
.../phoenix/spark/SparkContextFunctions.scala | 42 +
.../org/apache/phoenix/spark/SparkSchemaUtil.scala | 84 ++
.../phoenix/spark/SparkSqlContextFunctions.scala | 42 +
.../scala/org/apache/phoenix/spark/package.scala | 36 +
.../datasources/jdbc/PhoenixJdbcDialect.scala | 39 +
.../execution/datasources/jdbc/SparkJdbcUtil.scala | 309 +++++
pom.xml | 1081 +++++++++++++++++
presto-phoenix-shaded/LICENSE | 202 +++
presto-phoenix-shaded/README.md | 18 +
presto-phoenix-shaded/pom.xml | 69 ++
.../presto-hbase-testing-util-shaded/.gitignore | 6 +
.../presto-hbase-testing-util-shaded/pom.xml | 148 +++
.../presto-phoenix-client-shaded/.gitignore | 6 +
.../presto-phoenix-client-shaded/pom.xml | 356 ++++++
141 files changed, 26275 insertions(+)
diff --git a/phoenix-flume/pom.xml b/phoenix-flume/pom.xml
new file mode 100644
index 0000000..64d9bc7
--- /dev/null
+++ b/phoenix-flume/pom.xml
@@ -0,0 +1,199 @@
+<?xml version='1.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.
+
+-->
+
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+ xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+ <modelVersion>4.0.0</modelVersion>
+ <parent>
+ <groupId>org.apache.phoenix</groupId>
+ <artifactId>phoenix-connectors</artifactId>
+ <version>1.0.0-SNAPSHOT</version>
+ </parent>
+ <artifactId>phoenix-flume</artifactId>
+ <name>Phoenix - Flume</name>
+
+ <properties>
+ <top.dir>${project.basedir}/..</top.dir>
+ </properties>
+
+ <dependencies>
+ <dependency>
+ <groupId>org.apache.phoenix</groupId>
+ <artifactId>phoenix-core</artifactId>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.phoenix</groupId>
+ <artifactId>phoenix-core</artifactId>
+ <classifier>tests</classifier>
+ <scope>test</scope>
+ </dependency>
+
+ <!-- Test Dependencies -->
+ <dependency>
+ <groupId>junit</groupId>
+ <artifactId>junit</artifactId>
+ </dependency>
+ <dependency>
+ <groupId>org.mockito</groupId>
+ <artifactId>mockito-all</artifactId>
+ </dependency>
+ <dependency>
+ <groupId>com.github.stephenc.high-scale-lib</groupId>
+ <artifactId>high-scale-lib</artifactId>
+ <version>1.1.1</version>
+ <scope>test</scope>
+ </dependency>
+ <dependency>
+ <groupId>com.yammer.metrics</groupId>
+ <artifactId>metrics-core</artifactId>
+ <version>2.1.2</version>
+ <scope>test</scope>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.htrace</groupId>
+ <artifactId>htrace-core</artifactId>
+ </dependency>
+ <dependency>
+ <groupId>commons-codec</groupId>
+ <artifactId>commons-codec</artifactId>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.hbase</groupId>
+ <artifactId>hbase-testing-util</artifactId>
+ <scope>test</scope>
+ <optional>true</optional>
+ <exclusions>
+ <exclusion>
+ <groupId>org.jruby</groupId>
+ <artifactId>jruby-complete</artifactId>
+ </exclusion>
+ </exclusions>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.hbase</groupId>
+ <artifactId>hbase-it</artifactId>
+ <type>test-jar</type>
+ <scope>test</scope>
+ <exclusions>
+ <exclusion>
+ <groupId>org.jruby</groupId>
+ <artifactId>jruby-complete</artifactId>
+ </exclusion>
+ </exclusions>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.hbase</groupId>
+ <artifactId>hbase-common</artifactId>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.hbase</groupId>
+ <artifactId>hbase-protocol</artifactId>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.hbase</groupId>
+ <artifactId>hbase-client</artifactId>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.hbase</groupId>
+ <artifactId>hbase-hadoop-compat</artifactId>
+ <scope>test</scope>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.hbase</groupId>
+ <artifactId>hbase-hadoop-compat</artifactId>
+ <type>test-jar</type>
+ <scope>test</scope>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.hbase</groupId>
+ <artifactId>hbase-hadoop2-compat</artifactId>
+ <scope>test</scope>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.hbase</groupId>
+ <artifactId>hbase-hadoop2-compat</artifactId>
+ <type>test-jar</type>
+ <scope>test</scope>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.hadoop</groupId>
+ <artifactId>hadoop-common</artifactId>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.hadoop</groupId>
+ <artifactId>hadoop-annotations</artifactId>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.hadoop</groupId>
+ <artifactId>hadoop-mapreduce-client-core</artifactId>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.hadoop</groupId>
+ <artifactId>hadoop-minicluster</artifactId>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.tephra</groupId>
+ <artifactId>tephra-core</artifactId>
+ <type>test-jar</type>
+ <scope>test</scope>
+ </dependency>
+
+ <!-- to work with json data using flume -->
+ <dependency>
+ <groupId>com.tdunning</groupId>
+ <artifactId>json</artifactId>
+ <version>1.8</version>
+ </dependency>
+ <dependency>
+ <groupId>com.jayway.jsonpath</groupId>
+ <artifactId>json-path</artifactId>
+ <version>2.2.0</version>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.commons</groupId>
+ <artifactId>commons-csv</artifactId>
+ <version>${commons-csv.version}</version>
+ </dependency>
+ <!-- Main dependency on flume. The last to avoid using old commons-io in IT -->
+ <dependency>
+ <groupId>org.apache.flume</groupId>
+ <artifactId>flume-ng-core</artifactId>
+ </dependency>
+ </dependencies>
+
+ <build>
+ <plugins>
+ <plugin>
+ <groupId>org.codehaus.mojo</groupId>
+ <artifactId>build-helper-maven-plugin</artifactId>
+ </plugin>
+ <plugin>
+ <groupId>org.apache.maven.plugins</groupId>
+ <artifactId>maven-failsafe-plugin</artifactId>
+ </plugin>
+ <plugin>
+ <artifactId>maven-dependency-plugin</artifactId>
+ <version>${maven-dependency-plugin.version}</version>
+ </plugin>
+ </plugins>
+ </build>
+</project>
diff --git a/phoenix-flume/pom.xml~ b/phoenix-flume/pom.xml~
new file mode 100644
index 0000000..384f14e
--- /dev/null
+++ b/phoenix-flume/pom.xml~
@@ -0,0 +1,199 @@
+<?xml version='1.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.
+
+-->
+
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+ xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+ <modelVersion>4.0.0</modelVersion>
+ <parent>
+ <groupId>org.apache.phoenix</groupId>
+ <artifactId>phoenix</artifactId>
+ <version>1.0.0-SNAPSHOT</version>
+ </parent>
+ <artifactId>phoenix-flume</artifactId>
+ <name>Phoenix - Flume</name>
+
+ <properties>
+ <top.dir>${project.basedir}/..</top.dir>
+ </properties>
+
+ <dependencies>
+ <dependency>
+ <groupId>org.apache.phoenix</groupId>
+ <artifactId>phoenix-core</artifactId>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.phoenix</groupId>
+ <artifactId>phoenix-core</artifactId>
+ <classifier>tests</classifier>
+ <scope>test</scope>
+ </dependency>
+
+ <!-- Test Dependencies -->
+ <dependency>
+ <groupId>junit</groupId>
+ <artifactId>junit</artifactId>
+ </dependency>
+ <dependency>
+ <groupId>org.mockito</groupId>
+ <artifactId>mockito-all</artifactId>
+ </dependency>
+ <dependency>
+ <groupId>com.github.stephenc.high-scale-lib</groupId>
+ <artifactId>high-scale-lib</artifactId>
+ <version>1.1.1</version>
+ <scope>test</scope>
+ </dependency>
+ <dependency>
+ <groupId>com.yammer.metrics</groupId>
+ <artifactId>metrics-core</artifactId>
+ <version>2.1.2</version>
+ <scope>test</scope>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.htrace</groupId>
+ <artifactId>htrace-core</artifactId>
+ </dependency>
+ <dependency>
+ <groupId>commons-codec</groupId>
+ <artifactId>commons-codec</artifactId>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.hbase</groupId>
+ <artifactId>hbase-testing-util</artifactId>
+ <scope>test</scope>
+ <optional>true</optional>
+ <exclusions>
+ <exclusion>
+ <groupId>org.jruby</groupId>
+ <artifactId>jruby-complete</artifactId>
+ </exclusion>
+ </exclusions>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.hbase</groupId>
+ <artifactId>hbase-it</artifactId>
+ <type>test-jar</type>
+ <scope>test</scope>
+ <exclusions>
+ <exclusion>
+ <groupId>org.jruby</groupId>
+ <artifactId>jruby-complete</artifactId>
+ </exclusion>
+ </exclusions>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.hbase</groupId>
+ <artifactId>hbase-common</artifactId>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.hbase</groupId>
+ <artifactId>hbase-protocol</artifactId>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.hbase</groupId>
+ <artifactId>hbase-client</artifactId>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.hbase</groupId>
+ <artifactId>hbase-hadoop-compat</artifactId>
+ <scope>test</scope>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.hbase</groupId>
+ <artifactId>hbase-hadoop-compat</artifactId>
+ <type>test-jar</type>
+ <scope>test</scope>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.hbase</groupId>
+ <artifactId>hbase-hadoop2-compat</artifactId>
+ <scope>test</scope>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.hbase</groupId>
+ <artifactId>hbase-hadoop2-compat</artifactId>
+ <type>test-jar</type>
+ <scope>test</scope>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.hadoop</groupId>
+ <artifactId>hadoop-common</artifactId>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.hadoop</groupId>
+ <artifactId>hadoop-annotations</artifactId>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.hadoop</groupId>
+ <artifactId>hadoop-mapreduce-client-core</artifactId>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.hadoop</groupId>
+ <artifactId>hadoop-minicluster</artifactId>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.tephra</groupId>
+ <artifactId>tephra-core</artifactId>
+ <type>test-jar</type>
+ <scope>test</scope>
+ </dependency>
+
+ <!-- to work with json data using flume -->
+ <dependency>
+ <groupId>com.tdunning</groupId>
+ <artifactId>json</artifactId>
+ <version>1.8</version>
+ </dependency>
+ <dependency>
+ <groupId>com.jayway.jsonpath</groupId>
+ <artifactId>json-path</artifactId>
+ <version>2.2.0</version>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.commons</groupId>
+ <artifactId>commons-csv</artifactId>
+ <version>${commons-csv.version}</version>
+ </dependency>
+ <!-- Main dependency on flume. The last to avoid using old commons-io in IT -->
+ <dependency>
+ <groupId>org.apache.flume</groupId>
+ <artifactId>flume-ng-core</artifactId>
+ </dependency>
+ </dependencies>
+
+ <build>
+ <plugins>
+ <plugin>
+ <groupId>org.codehaus.mojo</groupId>
+ <artifactId>build-helper-maven-plugin</artifactId>
+ </plugin>
+ <plugin>
+ <groupId>org.apache.maven.plugins</groupId>
+ <artifactId>maven-failsafe-plugin</artifactId>
+ </plugin>
+ <plugin>
+ <artifactId>maven-dependency-plugin</artifactId>
+ <version>${maven-dependency-plugin.version}</version>
+ </plugin>
+ </plugins>
+ </build>
+</project>
diff --git a/phoenix-flume/src/it/java/org/apache/phoenix/flume/CsvEventSerializerIT.java b/phoenix-flume/src/it/java/org/apache/phoenix/flume/CsvEventSerializerIT.java
new file mode 100644
index 0000000..842db04
--- /dev/null
+++ b/phoenix-flume/src/it/java/org/apache/phoenix/flume/CsvEventSerializerIT.java
@@ -0,0 +1,416 @@
+/*
+ * 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.phoenix.flume;
+
+import static org.apache.phoenix.util.TestUtil.TEST_PROPERTIES;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+import java.sql.Connection;
+import java.sql.DriverManager;
+import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.util.List;
+import java.util.Map;
+import java.util.Properties;
+
+import org.apache.flume.Channel;
+import org.apache.flume.Context;
+import org.apache.flume.Event;
+import org.apache.flume.EventDeliveryException;
+import org.apache.flume.Transaction;
+import org.apache.flume.channel.MemoryChannel;
+import org.apache.flume.conf.Configurables;
+import org.apache.flume.event.EventBuilder;
+import org.apache.flume.lifecycle.LifecycleState;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.phoenix.end2end.BaseHBaseManagedTimeIT;
+import org.apache.phoenix.flume.serializer.EventSerializers;
+import org.apache.phoenix.flume.sink.PhoenixSink;
+import org.apache.phoenix.util.PropertiesUtil;
+import org.junit.Test;
+
+import com.google.common.base.Preconditions;
+import com.google.common.collect.Lists;
+import com.google.common.collect.Maps;
+
+public class CsvEventSerializerIT extends BaseHBaseManagedTimeIT {
+
+ private Context sinkContext;
+ private PhoenixSink sink;
+
+ @Test
+ public void testWithDefaultDelimiters() throws EventDeliveryException, SQLException {
+
+ final String fullTableName = "FLUME_CSV_TEST";
+
+ String ddl = "CREATE TABLE IF NOT EXISTS " + fullTableName
+ + " (flume_time timestamp not null, col1 varchar , col2 double, col3 varchar[], col4 integer[]"
+ + " CONSTRAINT pk PRIMARY KEY (flume_time))\n";
+ String columns = "col1,col2,col3,col4";
+ String rowkeyType = DefaultKeyGenerator.TIMESTAMP.name();
+ initSinkContext(fullTableName, ddl, columns, null, null, null, null, rowkeyType, null);
+
+ sink = new PhoenixSink();
+ Configurables.configure(sink, sinkContext);
+
+ assertEquals(LifecycleState.IDLE, sink.getLifecycleState());
+
+ final Channel channel = this.initChannel();
+ sink.setChannel(channel);
+
+ sink.start();
+
+ final String eventBody = "kalyan,10.5,\"abc,pqr,xyz\",\"1,2,3,4\"";
+ final Event event = EventBuilder.withBody(Bytes.toBytes(eventBody));
+ // put event in channel
+ Transaction transaction = channel.getTransaction();
+ transaction.begin();
+ channel.put(event);
+ transaction.commit();
+ transaction.close();
+
+ sink.process();
+
+ int rowsInDb = countRows(fullTableName);
+ assertEquals(1, rowsInDb);
+
+ sink.stop();
+ assertEquals(LifecycleState.STOP, sink.getLifecycleState());
+
+ dropTable(fullTableName);
+ }
+
+ @Test
+ public void testKeyGenerator() throws EventDeliveryException, SQLException {
+
+ final String fullTableName = "FLUME_CSV_TEST";
+ initSinkContextWithDefaults(fullTableName);
+
+ sink = new PhoenixSink();
+ Configurables.configure(sink, sinkContext);
+
+ assertEquals(LifecycleState.IDLE, sink.getLifecycleState());
+
+ final Channel channel = this.initChannel();
+ sink.setChannel(channel);
+
+ sink.start();
+ final String eventBody = "kalyan,10.5,\"abc,pqr,xyz\",\"1,2,3,4\"";
+ final Event event = EventBuilder.withBody(Bytes.toBytes(eventBody));
+ // put event in channel
+ Transaction transaction = channel.getTransaction();
+ transaction.begin();
+ channel.put(event);
+ transaction.commit();
+ transaction.close();
+
+ sink.process();
+
+ int rowsInDb = countRows(fullTableName);
+ assertEquals(1, rowsInDb);
+
+ sink.stop();
+ assertEquals(LifecycleState.STOP, sink.getLifecycleState());
+
+ dropTable(fullTableName);
+ }
+
+ @Test
+ public void testMismatchKeyGenerator() throws EventDeliveryException, SQLException {
+
+ final String fullTableName = "FLUME_CSV_TEST";
+ initSinkContextWithDefaults(fullTableName);
+ setConfig(FlumeConstants.CONFIG_SERIALIZER_PREFIX + FlumeConstants.CONFIG_ROWKEY_TYPE_GENERATOR,
+ DefaultKeyGenerator.UUID.name());
+
+ sink = new PhoenixSink();
+ Configurables.configure(sink, sinkContext);
+ assertEquals(LifecycleState.IDLE, sink.getLifecycleState());
+
+ final Channel channel = this.initChannel();
+ sink.setChannel(channel);
+
+ sink.start();
+ final String eventBody = "kalyan,10.5,\"abc,pqr,xyz\",\"1,2,3,4\"";
+ final Event event = EventBuilder.withBody(Bytes.toBytes(eventBody));
+ // put event in channel
+ Transaction transaction = channel.getTransaction();
+ transaction.begin();
+ channel.put(event);
+ transaction.commit();
+ transaction.close();
+
+ try {
+ sink.process();
+ fail();
+ } catch (Exception ex) {
+ assertTrue(ex.getCause().getMessage().contains("java.lang.IllegalArgumentException: Invalid format:"));
+ }
+
+ dropTable(fullTableName);
+ }
+
+ @Test
+ public void testMissingColumnsInEvent() throws EventDeliveryException, SQLException {
+
+ final String fullTableName = "FLUME_CSV_TEST";
+ initSinkContextWithDefaults(fullTableName);
+
+ sink = new PhoenixSink();
+ Configurables.configure(sink, sinkContext);
+ assertEquals(LifecycleState.IDLE, sink.getLifecycleState());
+
+ final Channel channel = this.initChannel();
+ sink.setChannel(channel);
+
+ sink.start();
+ final String eventBody = "kalyan,\"abc,pqr,xyz\",\"1,2,3,4\"";
+ final Event event = EventBuilder.withBody(Bytes.toBytes(eventBody));
+ // put event in channel
+ Transaction transaction = channel.getTransaction();
+ transaction.begin();
+ channel.put(event);
+ transaction.commit();
+ transaction.close();
+
+ sink.process();
+
+ int rowsInDb = countRows(fullTableName);
+ assertEquals(0, rowsInDb);
+
+ sink.stop();
+ assertEquals(LifecycleState.STOP, sink.getLifecycleState());
+
+ dropTable(fullTableName);
+ }
+
+ @Test
+ public void testBatchEvents() throws EventDeliveryException, SQLException {
+
+ final String fullTableName = "FLUME_CSV_TEST";
+ initSinkContextWithDefaults(fullTableName);
+
+ sink = new PhoenixSink();
+ Configurables.configure(sink, sinkContext);
+ assertEquals(LifecycleState.IDLE, sink.getLifecycleState());
+
+ final Channel channel = this.initChannel();
+ sink.setChannel(channel);
+
+ sink.start();
+ int numEvents = 150;
+ String col1 = "val1";
+ String a1 = "\"aaa,bbb,ccc\"";
+ String a2 = "\"1,2,3,4\"";
+ String eventBody = null;
+ List<Event> eventList = Lists.newArrayListWithCapacity(numEvents);
+ for (int i = 0; i < eventList.size(); i++) {
+ eventBody = (col1 + i) + "," + i * 10.5 + "," + a1 + "," + a2;
+ Event event = EventBuilder.withBody(Bytes.toBytes(eventBody));
+ eventList.add(event);
+ }
+
+ // put event in channel
+ Transaction transaction = channel.getTransaction();
+ transaction.begin();
+ for (Event event : eventList) {
+ channel.put(event);
+ }
+ transaction.commit();
+ transaction.close();
+
+ sink.process();
+
+ int rowsInDb = countRows(fullTableName);
+ assertEquals(eventList.size(), rowsInDb);
+
+ sink.stop();
+ assertEquals(LifecycleState.STOP, sink.getLifecycleState());
+
+ dropTable(fullTableName);
+ }
+
+ @Test
+ public void testEventsWithHeaders() throws Exception {
+
+ sinkContext = new Context();
+ final String fullTableName = "FLUME_CSV_TEST";
+ final String ddl = "CREATE TABLE IF NOT EXISTS "
+ + fullTableName
+ + " (rowkey VARCHAR not null, col1 varchar , col2 double, col3 varchar[], col4 integer[], host varchar , source varchar \n"
+ + " CONSTRAINT pk PRIMARY KEY (rowkey))\n";
+ String columns = "col1,col2,col3,col4";
+ String rowkeyType = DefaultKeyGenerator.UUID.name();
+ String headers = "host,source";
+ initSinkContext(fullTableName, ddl, columns, null, null, null, null, rowkeyType, headers);
+
+ sink = new PhoenixSink();
+ Configurables.configure(sink, sinkContext);
+ assertEquals(LifecycleState.IDLE, sink.getLifecycleState());
+
+ final Channel channel = this.initChannel();
+ sink.setChannel(channel);
+
+ sink.start();
+
+ int numEvents = 10;
+ String col1 = "val1";
+ String a1 = "\"aaa,bbb,ccc\"";
+ String a2 = "\"1,2,3,4\"";
+ String hostHeader = "host1";
+ String sourceHeader = "source1";
+ String eventBody = null;
+ List<Event> eventList = Lists.newArrayListWithCapacity(numEvents);
+ for (int i = 0; i < numEvents; i++) {
+ eventBody = (col1 + i) + "," + i * 10.5 + "," + a1 + "," + a2;
+ Map<String, String> headerMap = Maps.newHashMapWithExpectedSize(2);
+ headerMap.put("host", hostHeader);
+ headerMap.put("source", sourceHeader);
+ Event event = EventBuilder.withBody(Bytes.toBytes(eventBody), headerMap);
+ eventList.add(event);
+ }
+
+ // put event in channel
+ Transaction transaction = channel.getTransaction();
+ transaction.begin();
+ for (Event event : eventList) {
+ channel.put(event);
+ }
+ transaction.commit();
+ transaction.close();
+
+ sink.process();
+
+ final String query = " SELECT * FROM \n " + fullTableName;
+ Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
+ final ResultSet rs;
+ final Connection conn = DriverManager.getConnection(getUrl(), props);
+ try {
+ rs = conn.createStatement().executeQuery(query);
+ assertTrue(rs.next());
+ assertEquals("host1", rs.getString("host"));
+ assertEquals("source1", rs.getString("source"));
+
+ assertTrue(rs.next());
+ assertEquals("host1", rs.getString("host"));
+ assertEquals("source1", rs.getString("source"));
+ } finally {
+ if (conn != null) {
+ conn.close();
+ }
+ }
+ sink.stop();
+ assertEquals(LifecycleState.STOP, sink.getLifecycleState());
+
+ dropTable(fullTableName);
+ }
+
+ private Channel initChannel() {
+ // Channel configuration
+ Context channelContext = new Context();
+ channelContext.put("capacity", "10000");
+ channelContext.put("transactionCapacity", "200");
+
+ Channel channel = new MemoryChannel();
+ channel.setName("memorychannel");
+ Configurables.configure(channel, channelContext);
+ return channel;
+ }
+
+ private void initSinkContext(final String fullTableName, final String ddl, final String columns,
+ final String csvDelimiter, final String csvQuote, final String csvEscape, final String csvArrayDelimiter,
+ final String rowkeyType, final String headers) {
+ Preconditions.checkNotNull(fullTableName);
+ sinkContext = new Context();
+ sinkContext.put(FlumeConstants.CONFIG_TABLE, fullTableName);
+ sinkContext.put(FlumeConstants.CONFIG_JDBC_URL, getUrl());
+ sinkContext.put(FlumeConstants.CONFIG_SERIALIZER, EventSerializers.CSV.name());
+ sinkContext.put(FlumeConstants.CONFIG_TABLE_DDL, ddl);
+ sinkContext.put(FlumeConstants.CONFIG_SERIALIZER_PREFIX + FlumeConstants.CONFIG_COLUMN_NAMES, columns);
+ if (null != csvDelimiter)
+ sinkContext.put(FlumeConstants.CONFIG_SERIALIZER_PREFIX + FlumeConstants.CSV_DELIMITER, csvDelimiter);
+ if (null != csvQuote)
+ sinkContext.put(FlumeConstants.CONFIG_SERIALIZER_PREFIX + FlumeConstants.CSV_QUOTE, csvQuote);
+ if (null != csvEscape)
+ sinkContext.put(FlumeConstants.CONFIG_SERIALIZER_PREFIX + FlumeConstants.CSV_ESCAPE, csvEscape);
+ if (null != csvArrayDelimiter)
+ sinkContext.put(FlumeConstants.CONFIG_SERIALIZER_PREFIX + FlumeConstants.CSV_ARRAY_DELIMITER,
+ csvArrayDelimiter);
+ if (null != rowkeyType)
+ sinkContext.put(FlumeConstants.CONFIG_SERIALIZER_PREFIX + FlumeConstants.CONFIG_ROWKEY_TYPE_GENERATOR,
+ rowkeyType);
+ if (null != headers)
+ sinkContext.put(FlumeConstants.CONFIG_SERIALIZER_PREFIX + FlumeConstants.CONFIG_HEADER_NAMES, headers);
+ }
+
+ private void initSinkContextWithDefaults(final String fullTableName) {
+ String ddl = "CREATE TABLE IF NOT EXISTS " + fullTableName
+ + " (flume_time timestamp not null, col1 varchar , col2 double, col3 varchar[], col4 integer[]"
+ + " CONSTRAINT pk PRIMARY KEY (flume_time))\n";
+ String columns = "col1,col2,col3,col4";
+ String rowkeyType = DefaultKeyGenerator.TIMESTAMP.name();
+ initSinkContext(fullTableName, ddl, columns, null, null, null, null, rowkeyType, null);
+ }
+
+ private void setConfig(final String configName, final String configValue) {
+ Preconditions.checkNotNull(sinkContext);
+ Preconditions.checkNotNull(configName);
+ Preconditions.checkNotNull(configValue);
+ sinkContext.put(configName, configValue);
+ }
+
+ private int countRows(final String fullTableName) throws SQLException {
+ Preconditions.checkNotNull(fullTableName);
+ Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
+ final Connection conn = DriverManager.getConnection(getUrl(), props);
+ ResultSet rs = null;
+ try {
+ rs = conn.createStatement().executeQuery("select count(*) from " + fullTableName);
+ int rowsCount = 0;
+ while (rs.next()) {
+ rowsCount = rs.getInt(1);
+ }
+ return rowsCount;
+
+ } finally {
+ if (rs != null) {
+ rs.close();
+ }
+ if (conn != null) {
+ conn.close();
+ }
+ }
+
+ }
+
+ private void dropTable(final String fullTableName) throws SQLException {
+ Preconditions.checkNotNull(fullTableName);
+ Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
+ final Connection conn = DriverManager.getConnection(getUrl(), props);
+ try {
+ conn.createStatement().execute("drop table if exists " + fullTableName);
+ } finally {
+ if (conn != null) {
+ conn.close();
+ }
+ }
+ }
+
+}
diff --git a/phoenix-flume/src/it/java/org/apache/phoenix/flume/JsonEventSerializerIT.java b/phoenix-flume/src/it/java/org/apache/phoenix/flume/JsonEventSerializerIT.java
new file mode 100644
index 0000000..0210bad
--- /dev/null
+++ b/phoenix-flume/src/it/java/org/apache/phoenix/flume/JsonEventSerializerIT.java
@@ -0,0 +1,541 @@
+/*
+ * 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.phoenix.flume;
+
+import static org.apache.phoenix.util.TestUtil.TEST_PROPERTIES;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+import java.sql.Connection;
+import java.sql.DriverManager;
+import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.util.List;
+import java.util.Map;
+import java.util.Properties;
+
+import org.apache.flume.Channel;
+import org.apache.flume.Context;
+import org.apache.flume.Event;
+import org.apache.flume.EventDeliveryException;
+import org.apache.flume.Transaction;
+import org.apache.flume.channel.MemoryChannel;
+import org.apache.flume.conf.Configurables;
+import org.apache.flume.event.EventBuilder;
+import org.apache.flume.lifecycle.LifecycleState;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.phoenix.end2end.BaseHBaseManagedTimeIT;
+import org.apache.phoenix.flume.serializer.EventSerializers;
+import org.apache.phoenix.flume.sink.PhoenixSink;
+import org.apache.phoenix.util.PropertiesUtil;
+import org.junit.Test;
+
+import com.google.common.base.Preconditions;
+import com.google.common.collect.Lists;
+import com.google.common.collect.Maps;
+
+public class JsonEventSerializerIT extends BaseHBaseManagedTimeIT {
+
+ private Context sinkContext;
+ private PhoenixSink sink;
+
+ @Test
+ public void testWithOutColumnsMapping() throws EventDeliveryException, SQLException {
+
+ final String fullTableName = "FLUME_JSON_TEST";
+
+ String ddl = "CREATE TABLE IF NOT EXISTS " + fullTableName
+ + " (flume_time timestamp not null, col1 varchar , col2 double, col3 varchar[], col4 integer[]"
+ + " CONSTRAINT pk PRIMARY KEY (flume_time))\n";
+ String columns = "col1,col2,col3,col4";
+ String rowkeyType = DefaultKeyGenerator.TIMESTAMP.name();
+ initSinkContext(fullTableName, ddl, columns, null, rowkeyType, null);
+
+ sink = new PhoenixSink();
+ Configurables.configure(sink, sinkContext);
+
+ assertEquals(LifecycleState.IDLE, sink.getLifecycleState());
+
+ final Channel channel = this.initChannel();
+ sink.setChannel(channel);
+
+ sink.start();
+ final String eventBody = "{\"col1\" : \"kalyan\", \"col2\" : 10.5, \"col3\" : [\"abc\",\"pqr\",\"xyz\"], \"col4\" : [1,2,3,4]}";
+ final Event event = EventBuilder.withBody(Bytes.toBytes(eventBody));
+ // put event in channel
+ Transaction transaction = channel.getTransaction();
+ transaction.begin();
+ channel.put(event);
+ transaction.commit();
+ transaction.close();
+
+ sink.process();
+
+ int rowsInDb = countRows(fullTableName);
+ assertEquals(1, rowsInDb);
+
+ sink.stop();
+ assertEquals(LifecycleState.STOP, sink.getLifecycleState());
+
+ dropTable(fullTableName);
+ }
+
+ @Test
+ public void testDifferentColumnNames() throws EventDeliveryException, SQLException {
+
+ final String fullTableName = "FLUME_JSON_TEST";
+
+ String ddl = "CREATE TABLE IF NOT EXISTS " + fullTableName
+ + " (flume_time timestamp not null, col1 varchar , col2 double, col3 varchar[], col4 integer[]"
+ + " CONSTRAINT pk PRIMARY KEY (flume_time))\n";
+ String columns = "col1,col2,col3,col4";
+ String rowkeyType = DefaultKeyGenerator.TIMESTAMP.name();
+ String columnsMapping = "{\"col1\":\"col1\",\"col2\":\"f2\",\"col3\":\"f3\",\"col4\":\"col4\"}";
+
+ initSinkContext(fullTableName, ddl, columns, columnsMapping, rowkeyType, null);
+
+ sink = new PhoenixSink();
+ Configurables.configure(sink, sinkContext);
+
+ assertEquals(LifecycleState.IDLE, sink.getLifecycleState());
+
+ final Channel channel = this.initChannel();
+ sink.setChannel(channel);
+
+ sink.start();
+ final String eventBody = "{\"col1\" : \"kalyan\", \"f2\" : 10.5, \"f3\" : [\"abc\",\"pqr\",\"xyz\"], \"col4\" : [1,2,3,4]}";
+ final Event event = EventBuilder.withBody(Bytes.toBytes(eventBody));
+ // put event in channel
+ Transaction transaction = channel.getTransaction();
+ transaction.begin();
+ channel.put(event);
+ transaction.commit();
+ transaction.close();
+
+ sink.process();
+
+ int rowsInDb = countRows(fullTableName);
+ assertEquals(1, rowsInDb);
+
+ sink.stop();
+ assertEquals(LifecycleState.STOP, sink.getLifecycleState());
+
+ dropTable(fullTableName);
+ }
+
+ @Test
+ public void testInnerColumns() throws EventDeliveryException, SQLException {
+
+ final String fullTableName = "FLUME_JSON_TEST";
+
+ String ddl = "CREATE TABLE IF NOT EXISTS " + fullTableName
+ + " (flume_time timestamp not null, col1 varchar , col2 double, col3 varchar[], col4 integer[]"
+ + " CONSTRAINT pk PRIMARY KEY (flume_time))\n";
+ String columns = "col1,col2,col3,col4";
+ String rowkeyType = DefaultKeyGenerator.TIMESTAMP.name();
+ String columnsMapping = "{\"col1\":\"col1\",\"col2\":\"x.y\",\"col3\":\"a.b1.c\",\"col4\":\"col4\"}";
+
+ initSinkContext(fullTableName, ddl, columns, columnsMapping, rowkeyType, null);
+
+ sink = new PhoenixSink();
+ Configurables.configure(sink, sinkContext);
+
+ assertEquals(LifecycleState.IDLE, sink.getLifecycleState());
+
+ final Channel channel = this.initChannel();
+ sink.setChannel(channel);
+
+ sink.start();
+ final String eventBody = "{\"col1\" : \"kalyan\", \"x\" : {\"y\" : 10.5}, \"a\" : {\"b1\" : {\"c\" : [\"abc\",\"pqr\",\"xyz\"] }, \"b2\" : 111}, \"col4\" : [1,2,3,4]}";
+ final Event event = EventBuilder.withBody(Bytes.toBytes(eventBody));
+ // put event in channel
+ Transaction transaction = channel.getTransaction();
+ transaction.begin();
+ channel.put(event);
+ transaction.commit();
+ transaction.close();
+
+ sink.process();
+
+ int rowsInDb = countRows(fullTableName);
+ assertEquals(1, rowsInDb);
+
+ sink.stop();
+ assertEquals(LifecycleState.STOP, sink.getLifecycleState());
+
+ dropTable(fullTableName);
+ }
+
+ @Test
+ public void testInnerColumnsWithArrayMapping() throws EventDeliveryException, SQLException {
+
+ final String fullTableName = "FLUME_JSON_TEST";
+
+ String ddl = "CREATE TABLE IF NOT EXISTS " + fullTableName
+ + " (flume_time timestamp not null, col1 varchar , col2 double, col3 varchar[], col4 integer[]"
+ + " CONSTRAINT pk PRIMARY KEY (flume_time))\n";
+ String columns = "col1,col2,col3,col4";
+ String rowkeyType = DefaultKeyGenerator.TIMESTAMP.name();
+ String columnsMapping = "{\"col1\":\"col1\",\"col2\":\"x.y\",\"col3\":\"a.b[*].c\",\"col4\":\"col4\"}";
+
+ initSinkContext(fullTableName, ddl, columns, columnsMapping, rowkeyType, null);
+
+ sink = new PhoenixSink();
+ Configurables.configure(sink, sinkContext);
+
+ assertEquals(LifecycleState.IDLE, sink.getLifecycleState());
+
+ final Channel channel = this.initChannel();
+ sink.setChannel(channel);
+
+ sink.start();
+ final String eventBody = "{\"col1\" : \"kalyan\", \"x\" : {\"y\" : 10.5}, \"a\" : {\"b\" : [{\"c\" : \"abc\"}, {\"c\" : \"pqr\"}, {\"c\" : \"xyz\"}] , \"b2\" : 111}, \"col4\" : [1,2,3,4]}";
+ final Event event = EventBuilder.withBody(Bytes.toBytes(eventBody));
+ // put event in channel
+ Transaction transaction = channel.getTransaction();
+ transaction.begin();
+ channel.put(event);
+ transaction.commit();
+ transaction.close();
+
+ sink.process();
+
+ int rowsInDb = countRows(fullTableName);
+ assertEquals(1, rowsInDb);
+
+ sink.stop();
+ assertEquals(LifecycleState.STOP, sink.getLifecycleState());
+
+ dropTable(fullTableName);
+ }
+
+ @Test
+ public void testKeyGenerator() throws EventDeliveryException, SQLException {
+
+ final String fullTableName = "FLUME_JSON_TEST";
+ initSinkContextWithDefaults(fullTableName);
+
+ sink = new PhoenixSink();
+ Configurables.configure(sink, sinkContext);
+
+ assertEquals(LifecycleState.IDLE, sink.getLifecycleState());
+
+ final Channel channel = this.initChannel();
+ sink.setChannel(channel);
+
+ sink.start();
+ final String eventBody = "{\"col1\" : \"kalyan\", \"col2\" : 10.5, \"col3\" : [\"abc\",\"pqr\",\"xyz\"], \"col4\" : [1,2,3,4]}";
+ final Event event = EventBuilder.withBody(Bytes.toBytes(eventBody));
+ // put event in channel
+ Transaction transaction = channel.getTransaction();
+ transaction.begin();
+ channel.put(event);
+ transaction.commit();
+ transaction.close();
+
+ sink.process();
+
+ int rowsInDb = countRows(fullTableName);
+ assertEquals(1, rowsInDb);
+
+ sink.stop();
+ assertEquals(LifecycleState.STOP, sink.getLifecycleState());
+
+ dropTable(fullTableName);
+ }
+
+ @Test
+ public void testMismatchKeyGenerator() throws EventDeliveryException, SQLException {
+
+ final String fullTableName = "FLUME_JSON_TEST";
+ initSinkContextWithDefaults(fullTableName);
+ setConfig(FlumeConstants.CONFIG_SERIALIZER_PREFIX + FlumeConstants.CONFIG_ROWKEY_TYPE_GENERATOR,
+ DefaultKeyGenerator.UUID.name());
+
+ sink = new PhoenixSink();
+ Configurables.configure(sink, sinkContext);
+ assertEquals(LifecycleState.IDLE, sink.getLifecycleState());
+
+ final Channel channel = this.initChannel();
+ sink.setChannel(channel);
+
+ sink.start();
+ final String eventBody = "{\"col1\" : \"kalyan\", \"col2\" : 10.5, \"col3\" : [\"abc\",\"pqr\",\"xyz\"], \"col4\" : [1,2,3,4]}";
+ final Event event = EventBuilder.withBody(Bytes.toBytes(eventBody));
+ // put event in channel
+ Transaction transaction = channel.getTransaction();
+ transaction.begin();
+ channel.put(event);
+ transaction.commit();
+ transaction.close();
+
+ try {
+ sink.process();
+ fail();
+ } catch (Exception ex) {
+ assertTrue(ex.getCause().getMessage().contains("java.lang.IllegalArgumentException: Invalid format:"));
+ }
+
+ dropTable(fullTableName);
+ }
+
+ @Test
+ public void testMissingColumnsInEvent() throws EventDeliveryException, SQLException {
+
+ final String fullTableName = "FLUME_JSON_TEST";
+ initSinkContextWithDefaults(fullTableName);
+
+ sink = new PhoenixSink();
+ Configurables.configure(sink, sinkContext);
+ assertEquals(LifecycleState.IDLE, sink.getLifecycleState());
+
+ final Channel channel = this.initChannel();
+ sink.setChannel(channel);
+
+ sink.start();
+ final String eventBody = "{\"col1\" : \"kalyan\", \"col3\" : [\"abc\",\"pqr\",\"xyz\"], \"col4\" : [1,2,3,4]}";
+ final Event event = EventBuilder.withBody(Bytes.toBytes(eventBody));
+ // put event in channel
+ Transaction transaction = channel.getTransaction();
+ transaction.begin();
+ channel.put(event);
+ transaction.commit();
+ transaction.close();
+
+ sink.process();
+
+ int rowsInDb = countRows(fullTableName);
+ assertEquals(0, rowsInDb);
+
+ sink.stop();
+ assertEquals(LifecycleState.STOP, sink.getLifecycleState());
+
+ dropTable(fullTableName);
+ }
+
+ @Test
+ public void testBatchEvents() throws EventDeliveryException, SQLException {
+
+ final String fullTableName = "FLUME_JSON_TEST";
+ initSinkContextWithDefaults(fullTableName);
+
+ sink = new PhoenixSink();
+ Configurables.configure(sink, sinkContext);
+ assertEquals(LifecycleState.IDLE, sink.getLifecycleState());
+
+ final Channel channel = this.initChannel();
+ sink.setChannel(channel);
+
+ sink.start();
+ int numEvents = 150;
+ String col1 = "val1";
+ String a1 = "[aaa,bbb,ccc]";
+ String a2 = "[1,2,3,4]";
+ String eventBody = null;
+ List<Event> eventList = Lists.newArrayListWithCapacity(numEvents);
+ for (int i = 0; i < eventList.size(); i++) {
+ eventBody = "{\"col1\" : \"" + (col1 + i) + "\", \"col2\" : " + i * 10.5 + " , \"col3\" : " + a1
+ + " , \"col4\" : " + a2 + "}";
+ Event event = EventBuilder.withBody(Bytes.toBytes(eventBody));
+ eventList.add(event);
+ }
+
+ // put event in channel
+ Transaction transaction = channel.getTransaction();
+ transaction.begin();
+ for (Event event : eventList) {
+ channel.put(event);
+ }
+ transaction.commit();
+ transaction.close();
+
+ sink.process();
+
+ int rowsInDb = countRows(fullTableName);
+ assertEquals(eventList.size(), rowsInDb);
+
+ sink.stop();
+ assertEquals(LifecycleState.STOP, sink.getLifecycleState());
+
+ dropTable(fullTableName);
+ }
+
+ @Test
+ public void testEventsWithHeaders() throws Exception {
+
+ sinkContext = new Context();
+ final String fullTableName = "FLUME_JSON_TEST";
+ final String ddl = "CREATE TABLE IF NOT EXISTS "
+ + fullTableName
+ + " (rowkey VARCHAR not null, col1 varchar , col2 double, col3 varchar[], col4 integer[], host varchar , source varchar \n"
+ + " CONSTRAINT pk PRIMARY KEY (rowkey))\n";
+ String columns = "col1,col2,col3,col4";
+ String columnsMapping = "{\"col1\":\"col1\",\"col2\":\"col2\",\"col3\":\"col3\",\"col4\":\"col4\"}";
+ String rowkeyType = DefaultKeyGenerator.UUID.name();
+ String headers = "host,source";
+ initSinkContext(fullTableName, ddl, columns, columnsMapping, rowkeyType, headers);
+
+ sink = new PhoenixSink();
+ Configurables.configure(sink, sinkContext);
+ assertEquals(LifecycleState.IDLE, sink.getLifecycleState());
+
+ final Channel channel = this.initChannel();
+ sink.setChannel(channel);
+
+ sink.start();
+
+ int numEvents = 10;
+ String col1 = "val1";
+ String a1 = "[aaa,bbb,ccc]";
+ String a2 = "[1,2,3,4]";
+ String hostHeader = "host1";
+ String sourceHeader = "source1";
+ String eventBody = null;
+ List<Event> eventList = Lists.newArrayListWithCapacity(numEvents);
+ for (int i = 0; i < numEvents; i++) {
+ eventBody = "{\"col1\" : \"" + (col1 + i) + "\", \"col2\" : " + i * 10.5 + " , \"col3\" : " + a1
+ + " , \"col4\" : " + a2 + "}";
+ Map<String, String> headerMap = Maps.newHashMapWithExpectedSize(2);
+ headerMap.put("host", hostHeader);
+ headerMap.put("source", sourceHeader);
+ Event event = EventBuilder.withBody(Bytes.toBytes(eventBody), headerMap);
+ eventList.add(event);
+ }
+
+ // put event in channel
+ Transaction transaction = channel.getTransaction();
+ transaction.begin();
+ for (Event event : eventList) {
+ channel.put(event);
+ }
+ transaction.commit();
+ transaction.close();
+
+ sink.process();
+
+ final String query = " SELECT * FROM \n " + fullTableName;
+ Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
+ final ResultSet rs;
+ final Connection conn = DriverManager.getConnection(getUrl(), props);
+ try {
+ rs = conn.createStatement().executeQuery(query);
+ assertTrue(rs.next());
+ assertEquals("host1", rs.getString("host"));
+ assertEquals("source1", rs.getString("source"));
+
+ assertTrue(rs.next());
+ assertEquals("host1", rs.getString("host"));
+ assertEquals("source1", rs.getString("source"));
+ } finally {
+ if (conn != null) {
+ conn.close();
+ }
+ }
+ sink.stop();
+ assertEquals(LifecycleState.STOP, sink.getLifecycleState());
+
+ dropTable(fullTableName);
+ }
+
+ private Channel initChannel() {
+ // Channel configuration
+ Context channelContext = new Context();
+ channelContext.put("capacity", "10000");
+ channelContext.put("transactionCapacity", "200");
+
+ Channel channel = new MemoryChannel();
+ channel.setName("memorychannel");
+ Configurables.configure(channel, channelContext);
+ return channel;
+ }
+
+ private void initSinkContext(final String fullTableName, final String ddl, final String columns,
+ final String columnsMapping, final String rowkeyType, final String headers) {
+ Preconditions.checkNotNull(fullTableName);
+ sinkContext = new Context();
+ sinkContext.put(FlumeConstants.CONFIG_TABLE, fullTableName);
+ sinkContext.put(FlumeConstants.CONFIG_JDBC_URL, getUrl());
+ sinkContext.put(FlumeConstants.CONFIG_SERIALIZER, EventSerializers.JSON.name());
+ sinkContext.put(FlumeConstants.CONFIG_TABLE_DDL, ddl);
+ sinkContext.put(FlumeConstants.CONFIG_SERIALIZER_PREFIX + FlumeConstants.CONFIG_COLUMN_NAMES, columns);
+ if (null != columnsMapping)
+ sinkContext.put(FlumeConstants.CONFIG_SERIALIZER_PREFIX + FlumeConstants.CONFIG_COLUMNS_MAPPING,
+ columnsMapping);
+ if (null != rowkeyType)
+ sinkContext.put(FlumeConstants.CONFIG_SERIALIZER_PREFIX + FlumeConstants.CONFIG_ROWKEY_TYPE_GENERATOR,
+ rowkeyType);
+ if (null != headers)
+ sinkContext.put(FlumeConstants.CONFIG_SERIALIZER_PREFIX + FlumeConstants.CONFIG_HEADER_NAMES, headers);
+ }
+
+ private void initSinkContextWithDefaults(final String fullTableName) {
+ String ddl = "CREATE TABLE IF NOT EXISTS " + fullTableName
+ + " (flume_time timestamp not null, col1 varchar , col2 double, col3 varchar[], col4 integer[]"
+ + " CONSTRAINT pk PRIMARY KEY (flume_time))\n";
+ String columns = "col1,col2,col3,col4";
+ String columnsMapping = "{\"col1\":\"col1\",\"col2\":\"col2\",\"col3\":\"col3\",\"col4\":\"col4\"}";
+ String rowkeyType = DefaultKeyGenerator.TIMESTAMP.name();
+ initSinkContext(fullTableName, ddl, columns, columnsMapping, rowkeyType, null);
+ }
+
+ private void setConfig(final String configName, final String configValue) {
+ Preconditions.checkNotNull(sinkContext);
+ Preconditions.checkNotNull(configName);
+ Preconditions.checkNotNull(configValue);
+ sinkContext.put(configName, configValue);
+ }
+
+ private int countRows(final String fullTableName) throws SQLException {
+ Preconditions.checkNotNull(fullTableName);
+ Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
+ final Connection conn = DriverManager.getConnection(getUrl(), props);
+ ResultSet rs = null;
+ try {
+ rs = conn.createStatement().executeQuery("select count(*) from " + fullTableName);
+ int rowsCount = 0;
+ while (rs.next()) {
+ rowsCount = rs.getInt(1);
+ }
+ return rowsCount;
+
+ } finally {
+ if (rs != null) {
+ rs.close();
+ }
+ if (conn != null) {
+ conn.close();
+ }
+ }
+
+ }
+
+ private void dropTable(final String fullTableName) throws SQLException {
+ Preconditions.checkNotNull(fullTableName);
+ Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
+ final Connection conn = DriverManager.getConnection(getUrl(), props);
+ try {
+ conn.createStatement().execute("drop table if exists " + fullTableName);
+ } finally {
+ if (conn != null) {
+ conn.close();
+ }
+ }
+ }
+
+}
diff --git a/phoenix-flume/src/it/java/org/apache/phoenix/flume/PhoenixSinkIT.java b/phoenix-flume/src/it/java/org/apache/phoenix/flume/PhoenixSinkIT.java
new file mode 100644
index 0000000..867d1ad
--- /dev/null
+++ b/phoenix-flume/src/it/java/org/apache/phoenix/flume/PhoenixSinkIT.java
@@ -0,0 +1,271 @@
+/*
+ * 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.phoenix.flume;
+
+import static org.apache.phoenix.util.TestUtil.TEST_PROPERTIES;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.verify;
+
+import java.sql.Connection;
+import java.sql.DriverManager;
+import java.sql.ResultSet;
+import java.util.Properties;
+
+import org.apache.flume.Channel;
+import org.apache.flume.Context;
+import org.apache.flume.Sink;
+import org.apache.flume.SinkFactory;
+import org.apache.flume.Transaction;
+import org.apache.flume.channel.MemoryChannel;
+import org.apache.flume.conf.Configurables;
+import org.apache.flume.event.EventBuilder;
+import org.apache.flume.lifecycle.LifecycleState;
+import org.apache.flume.sink.DefaultSinkFactory;
+import org.apache.hadoop.hbase.client.HBaseAdmin;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.phoenix.end2end.BaseHBaseManagedTimeIT;
+import org.apache.phoenix.flume.serializer.CustomSerializer;
+import org.apache.phoenix.flume.serializer.EventSerializers;
+import org.apache.phoenix.flume.sink.NullPhoenixSink;
+import org.apache.phoenix.flume.sink.PhoenixSink;
+import org.apache.phoenix.util.PropertiesUtil;
+import org.apache.phoenix.util.TestUtil;
+import org.junit.Assert;
+import org.junit.Test;
+
+
+public class PhoenixSinkIT extends BaseHBaseManagedTimeIT {
+
+ private Context sinkContext;
+ private PhoenixSink sink;
+
+
+ @Test
+ public void testSinkCreation() {
+ SinkFactory factory = new DefaultSinkFactory ();
+ Sink sink = factory.create("PhoenixSink__", "org.apache.phoenix.flume.sink.PhoenixSink");
+ Assert.assertNotNull(sink);
+ Assert.assertTrue(PhoenixSink.class.isInstance(sink));
+ }
+ @Test
+ public void testConfiguration () {
+
+ sinkContext = new Context ();
+ sinkContext.put(FlumeConstants.CONFIG_TABLE, "test");
+ sinkContext.put(FlumeConstants.CONFIG_JDBC_URL, getUrl());
+ sinkContext.put(FlumeConstants.CONFIG_SERIALIZER,EventSerializers.REGEX.name());
+ sinkContext.put(FlumeConstants.CONFIG_SERIALIZER_PREFIX + FlumeConstants.CONFIG_COLUMN_NAMES,"col1,col2");
+ sinkContext.put(FlumeConstants.CONFIG_SERIALIZER_PREFIX + FlumeConstants.CONFIG_ROWKEY_TYPE_GENERATOR,DefaultKeyGenerator.TIMESTAMP.name());
+
+ sink = new PhoenixSink();
+ Configurables.configure(sink, sinkContext);
+ }
+
+
+
+ @Test(expected= NullPointerException.class)
+ public void testInvalidConfiguration () {
+
+ sinkContext = new Context ();
+ sinkContext.put(FlumeConstants.CONFIG_JDBC_URL, getUrl());
+ sinkContext.put(FlumeConstants.CONFIG_SERIALIZER,EventSerializers.REGEX.name());
+ sinkContext.put(FlumeConstants.CONFIG_SERIALIZER_PREFIX + FlumeConstants.CONFIG_COLUMN_NAMES,"col1,col2");
+ sinkContext.put(FlumeConstants.CONFIG_SERIALIZER_PREFIX + FlumeConstants.CONFIG_ROWKEY_TYPE_GENERATOR,DefaultKeyGenerator.TIMESTAMP.name());
+
+ sink = new PhoenixSink();
+ Configurables.configure(sink, sinkContext);
+ }
+
+ @Test(expected=RuntimeException.class)
+ public void testInvalidConfigurationOfSerializer () {
+
+ sinkContext = new Context ();
+ sinkContext.put(FlumeConstants.CONFIG_TABLE, "test");
+ sinkContext.put(FlumeConstants.CONFIG_JDBC_URL, getUrl());
+ sinkContext.put(FlumeConstants.CONFIG_SERIALIZER,"unknown");
+ sinkContext.put(FlumeConstants.CONFIG_SERIALIZER_PREFIX + FlumeConstants.CONFIG_COLUMN_NAMES,"col1,col2");
+ sinkContext.put(FlumeConstants.CONFIG_SERIALIZER_PREFIX + FlumeConstants.CONFIG_ROWKEY_TYPE_GENERATOR,DefaultKeyGenerator.TIMESTAMP.name());
+
+ sink = new PhoenixSink();
+ Configurables.configure(sink, sinkContext);
+ }
+
+ @Test
+ public void testInvalidTable() {
+ sinkContext = new Context ();
+ sinkContext.put(FlumeConstants.CONFIG_TABLE, "flume_test");
+ sinkContext.put(FlumeConstants.CONFIG_JDBC_URL, getUrl());
+ sinkContext.put(FlumeConstants.CONFIG_SERIALIZER, EventSerializers.REGEX.name());
+ sinkContext.put(FlumeConstants.CONFIG_SERIALIZER_PREFIX + FlumeConstants.CONFIG_COLUMN_NAMES, "col1,col2");
+ sinkContext.put(FlumeConstants.CONFIG_SERIALIZER_PREFIX + FlumeConstants.CONFIG_ROWKEY_TYPE_GENERATOR,DefaultKeyGenerator.TIMESTAMP.name());
+
+ sink = new PhoenixSink();
+ Configurables.configure(sink, sinkContext);
+
+ final Channel channel = this.initChannel();
+ sink.setChannel(channel);
+ try {
+ sink.start();
+ fail();
+ }catch(Exception e) {
+ assertTrue(e.getMessage(), e.getMessage().contains("ERROR 1012 (42M03): Table undefined."));
+ }
+ }
+
+ @Test
+ public void testSinkLifecycle () {
+ String tableName = generateUniqueName();
+
+ String ddl = "CREATE TABLE " + tableName +
+ " (flume_time timestamp not null, col1 varchar , col2 varchar" +
+ " CONSTRAINT pk PRIMARY KEY (flume_time))\n";
+
+ sinkContext = new Context ();
+ sinkContext.put(FlumeConstants.CONFIG_TABLE, tableName);
+ sinkContext.put(FlumeConstants.CONFIG_JDBC_URL, getUrl());
+ sinkContext.put(FlumeConstants.CONFIG_SERIALIZER,EventSerializers.REGEX.name());
+ sinkContext.put(FlumeConstants.CONFIG_TABLE_DDL, ddl);
+ sinkContext.put(FlumeConstants.CONFIG_SERIALIZER_PREFIX + FlumeConstants.CONFIG_REGULAR_EXPRESSION,"^([^\t]+)\t([^\t]+)$");
+ sinkContext.put(FlumeConstants.CONFIG_SERIALIZER_PREFIX + FlumeConstants.CONFIG_COLUMN_NAMES,"col1,col2");
+ sinkContext.put(FlumeConstants.CONFIG_SERIALIZER_PREFIX + FlumeConstants.CONFIG_ROWKEY_TYPE_GENERATOR,DefaultKeyGenerator.TIMESTAMP.name());
+
+
+ sink = new PhoenixSink();
+ Configurables.configure(sink, sinkContext);
+ Assert.assertEquals(LifecycleState.IDLE, sink.getLifecycleState());
+
+ final Channel channel = this.initChannel();
+ sink.setChannel(channel);
+
+ sink.start();
+ Assert.assertEquals(LifecycleState.START, sink.getLifecycleState());
+ sink.stop();
+ Assert.assertEquals(LifecycleState.STOP, sink.getLifecycleState());
+ }
+
+ @Test
+ public void testCreateTable () throws Exception {
+ String tableName = generateUniqueName();
+ String ddl = "CREATE TABLE " + tableName + " " +
+ " (flume_time timestamp not null, col1 varchar , col2 varchar" +
+ " CONSTRAINT pk PRIMARY KEY (flume_time))\n";
+
+ final String fullTableName = tableName;
+ sinkContext = new Context ();
+ sinkContext.put(FlumeConstants.CONFIG_TABLE, fullTableName);
+ sinkContext.put(FlumeConstants.CONFIG_JDBC_URL, getUrl());
+ sinkContext.put(FlumeConstants.CONFIG_SERIALIZER,EventSerializers.REGEX.name());
+ sinkContext.put(FlumeConstants.CONFIG_TABLE_DDL, ddl);
+ sinkContext.put(FlumeConstants.CONFIG_SERIALIZER_PREFIX + FlumeConstants.CONFIG_REGULAR_EXPRESSION,"^([^\t]+)\t([^\t]+)$");
+ sinkContext.put(FlumeConstants.CONFIG_SERIALIZER_PREFIX + FlumeConstants.CONFIG_COLUMN_NAMES,"col1,col2");
+ sinkContext.put(FlumeConstants.CONFIG_SERIALIZER_PREFIX + FlumeConstants.CONFIG_ROWKEY_TYPE_GENERATOR, DefaultKeyGenerator.TIMESTAMP.name());
+
+
+ sink = new PhoenixSink();
+ Configurables.configure(sink, sinkContext);
+ Assert.assertEquals(LifecycleState.IDLE, sink.getLifecycleState());
+
+ final Channel channel = this.initChannel();
+ sink.setChannel(channel);
+
+ sink.start();
+ HBaseAdmin admin = driver.getConnectionQueryServices(getUrl(), TestUtil.TEST_PROPERTIES).getAdmin();
+ try {
+ boolean exists = admin.tableExists(fullTableName);
+ Assert.assertTrue(exists);
+ }finally {
+ admin.close();
+ }
+ }
+
+ @Test
+ public void testExtendedSink() throws Exception {
+ // Create a mock NullPhoenixSink which extends PhoenixSink, and verify configure is invoked()
+
+ PhoenixSink sink = mock(NullPhoenixSink.class);
+ sinkContext = new Context();
+ sinkContext.put(FlumeConstants.CONFIG_TABLE, "FLUME_TEST_EXTENDED");
+ sinkContext.put(FlumeConstants.CONFIG_JDBC_URL, getUrl());
+ sinkContext.put(FlumeConstants.CONFIG_SERIALIZER, CustomSerializer.class.getName());
+ sinkContext.put(FlumeConstants.CONFIG_SERIALIZER_PREFIX + FlumeConstants.CONFIG_COLUMN_NAMES, "ID, COUNTS");
+ sinkContext.put(FlumeConstants.CONFIG_SERIALIZER_PREFIX + FlumeConstants.CONFIG_ROWKEY_TYPE_GENERATOR, DefaultKeyGenerator.TIMESTAMP.name());
+
+ Configurables.configure(sink, sinkContext);
+ verify(sink).configure(sinkContext);
+ }
+
+ @Test
+ public void testExtendedSerializer() throws Exception {
+ /*
+ Sadly, we can't mock a serializer, as the PhoenixSink does a Class.forName() to instantiate
+ it. Instead. we'll setup a Flume channel and verify the data our custom serializer wrote.
+ */
+
+ final String fullTableName = "FLUME_TEST_EXTENDED";
+ final String ddl = "CREATE TABLE " + fullTableName + " (ID BIGINT NOT NULL PRIMARY KEY, COUNTS UNSIGNED_LONG)";
+ Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
+ final Connection conn = DriverManager.getConnection(getUrl(), props);
+ conn.createStatement().execute(ddl);
+ conn.commit();
+
+ sinkContext = new Context();
+ sinkContext.put(FlumeConstants.CONFIG_TABLE, "FLUME_TEST_EXTENDED");
+ sinkContext.put(FlumeConstants.CONFIG_JDBC_URL, getUrl());
+ sinkContext.put(FlumeConstants.CONFIG_SERIALIZER, CustomSerializer.class.getName());
+ sinkContext.put(FlumeConstants.CONFIG_SERIALIZER_PREFIX + FlumeConstants.CONFIG_COLUMN_NAMES, "ID, COUNTS");
+ sinkContext.put(FlumeConstants.CONFIG_SERIALIZER_PREFIX + FlumeConstants.CONFIG_ROWKEY_TYPE_GENERATOR, DefaultKeyGenerator.TIMESTAMP.name());
+
+ PhoenixSink sink = new PhoenixSink();
+ Configurables.configure(sink, sinkContext);
+
+ // Send a test event through Flume, using our custom serializer
+ final Channel channel = this.initChannel();
+ sink.setChannel(channel);
+ sink.start();
+
+ final Transaction transaction = channel.getTransaction();
+ transaction.begin();
+ channel.put(EventBuilder.withBody(Bytes.toBytes("test event")));
+ transaction.commit();
+ transaction.close();
+
+ sink.process();
+ sink.stop();
+
+ // Verify our serializer wrote out data
+ ResultSet rs = conn.createStatement().executeQuery("SELECT * FROM FLUME_TEST_EXTENDED");
+ assertTrue(rs.next());
+ assertTrue(rs.getLong(1) == 1L);
+ }
+
+ private Channel initChannel() {
+ //Channel configuration
+ Context channelContext = new Context();
+ channelContext.put("capacity", "10000");
+ channelContext.put("transactionCapacity", "200");
+
+ Channel channel = new MemoryChannel();
+ channel.setName("memorychannel");
+ Configurables.configure(channel, channelContext);
+ return channel;
+ }
+
+
+}
diff --git a/phoenix-flume/src/it/java/org/apache/phoenix/flume/RegexEventSerializerIT.java b/phoenix-flume/src/it/java/org/apache/phoenix/flume/RegexEventSerializerIT.java
new file mode 100644
index 0000000..9548e65
--- /dev/null
+++ b/phoenix-flume/src/it/java/org/apache/phoenix/flume/RegexEventSerializerIT.java
@@ -0,0 +1,417 @@
+/*
+ * 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.phoenix.flume;
+
+import static org.apache.phoenix.util.TestUtil.TEST_PROPERTIES;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+import java.sql.Connection;
+import java.sql.DriverManager;
+import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.util.List;
+import java.util.Map;
+import java.util.Properties;
+
+import org.apache.flume.Channel;
+import org.apache.flume.Context;
+import org.apache.flume.Event;
+import org.apache.flume.EventDeliveryException;
+import org.apache.flume.Transaction;
+import org.apache.flume.channel.MemoryChannel;
+import org.apache.flume.conf.Configurables;
+import org.apache.flume.event.EventBuilder;
+import org.apache.flume.lifecycle.LifecycleState;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.phoenix.end2end.BaseHBaseManagedTimeIT;
+import org.apache.phoenix.flume.serializer.EventSerializers;
+import org.apache.phoenix.flume.sink.PhoenixSink;
+import org.apache.phoenix.util.PropertiesUtil;
+import org.junit.Test;
+
+import com.google.common.base.Preconditions;
+import com.google.common.collect.Lists;
+import com.google.common.collect.Maps;
+
+
+public class RegexEventSerializerIT extends BaseHBaseManagedTimeIT {
+
+ private Context sinkContext;
+ private PhoenixSink sink;
+
+ @Test
+ public void testKeyGenerator() throws EventDeliveryException, SQLException {
+
+ final String fullTableName = generateUniqueName();
+ initSinkContextWithDefaults(fullTableName);
+
+ sink = new PhoenixSink();
+ Configurables.configure(sink, sinkContext);
+
+ assertEquals(LifecycleState.IDLE, sink.getLifecycleState());
+
+ final Channel channel = this.initChannel();
+ sink.setChannel(channel);
+
+ sink.start();
+ final String eventBody = "val1\tval2";
+ final Event event = EventBuilder.withBody(Bytes.toBytes(eventBody));
+ // put event in channel
+ Transaction transaction = channel.getTransaction();
+ transaction.begin();
+ channel.put(event);
+ transaction.commit();
+ transaction.close();
+
+ sink.process();
+
+ int rowsInDb = countRows(fullTableName);
+ assertEquals(1 , rowsInDb);
+
+ sink.stop();
+ assertEquals(LifecycleState.STOP, sink.getLifecycleState());
+
+ }
+
+
+ @Test
+ public void testMismatchKeyGenerator() throws EventDeliveryException, SQLException {
+
+ final String fullTableName = generateUniqueName();
+ initSinkContextWithDefaults(fullTableName);
+ setConfig(FlumeConstants.CONFIG_SERIALIZER_PREFIX + FlumeConstants.CONFIG_ROWKEY_TYPE_GENERATOR,DefaultKeyGenerator.UUID.name());
+
+ sink = new PhoenixSink();
+ Configurables.configure(sink, sinkContext);
+ assertEquals(LifecycleState.IDLE, sink.getLifecycleState());
+
+
+ final Channel channel = this.initChannel();
+ sink.setChannel(channel);
+
+ sink.start();
+ final String eventBody = "val1\tval2";
+ final Event event = EventBuilder.withBody(Bytes.toBytes(eventBody));
+ // put event in channel
+ Transaction transaction = channel.getTransaction();
+ transaction.begin();
+ channel.put(event);
+ transaction.commit();
+ transaction.close();
+
+ try {
+ sink.process();
+ fail();
+ }catch(Exception ex){
+ assertTrue(ex.getCause().getMessage().contains("java.lang.IllegalArgumentException: Invalid format:"));
+ }
+ }
+
+ @Test
+ public void testMissingColumnsInEvent() throws EventDeliveryException, SQLException {
+
+ final String fullTableName = generateUniqueName();
+ initSinkContextWithDefaults(fullTableName);
+
+ sink = new PhoenixSink();
+ Configurables.configure(sink, sinkContext);
+ assertEquals(LifecycleState.IDLE, sink.getLifecycleState());
+
+ final Channel channel = this.initChannel();
+ sink.setChannel(channel);
+
+ sink.start();
+ final String eventBody = "val1";
+ final Event event = EventBuilder.withBody(Bytes.toBytes(eventBody));
+ // put event in channel
+ Transaction transaction = channel.getTransaction();
+ transaction.begin();
+ channel.put(event);
+ transaction.commit();
+ transaction.close();
+
+ sink.process();
+
+ int rowsInDb = countRows(fullTableName);
+ assertEquals(0 , rowsInDb);
+
+ sink.stop();
+ assertEquals(LifecycleState.STOP, sink.getLifecycleState());
+
+ }
+
+ @Test
+ public void testBatchEvents() throws EventDeliveryException, SQLException {
+
+ final String fullTableName = generateUniqueName();
+ initSinkContextWithDefaults(fullTableName);
+
+ sink = new PhoenixSink();
+ Configurables.configure(sink, sinkContext);
+ assertEquals(LifecycleState.IDLE, sink.getLifecycleState());
+
+ final Channel channel = this.initChannel();
+ sink.setChannel(channel);
+
+ sink.start();
+ int numEvents = 150;
+ String col1 = "val1";
+ String col2 = "val2";
+ String eventBody = null;
+ List<Event> eventList = Lists.newArrayListWithCapacity(numEvents);
+ for(int i = 0 ; i < eventList.size() ; i++) {
+ eventBody = (col1 + i) + "\t" + (col2 + i);
+ Event event = EventBuilder.withBody(Bytes.toBytes(eventBody));
+ eventList.add(event);
+ }
+
+ // put event in channel
+ Transaction transaction = channel.getTransaction();
+ transaction.begin();
+ for(Event event : eventList) {
+ channel.put(event);
+ }
+ transaction.commit();
+ transaction.close();
+
+ sink.process();
+
+ int rowsInDb = countRows(fullTableName);
+ assertEquals(eventList.size(), rowsInDb);
+
+ sink.stop();
+ assertEquals(LifecycleState.STOP, sink.getLifecycleState());
+
+ }
+
+ @Test
+ public void testApacheLogRegex() throws Exception {
+
+ sinkContext = new Context ();
+ final String fullTableName = generateUniqueName();
+ final String logRegex = "([^ ]*) ([^ ]*) ([^ ]*) (-|\\[[^\\]]*\\]) \"([^ ]+) ([^ ]+)" +
+ " ([^\"]+)\" (-|[0-9]*) (-|[0-9]*)(?: ([^ \"]*|\"[^\"]*\")" +
+ " ([^ \"]*|\"[^\"]*\"))?";
+
+ final String columns = "host,identity,user,time,method,request,protocol,status,size,referer,agent";
+
+ String ddl = "CREATE TABLE " + fullTableName +
+ " (uid VARCHAR NOT NULL, user VARCHAR, time varchar, host varchar , identity varchar, method varchar, request varchar , protocol varchar," +
+ " status integer , size integer , referer varchar , agent varchar CONSTRAINT pk PRIMARY KEY (uid))\n";
+
+ sinkContext.put(FlumeConstants.CONFIG_TABLE, fullTableName);
+ sinkContext.put(FlumeConstants.CONFIG_JDBC_URL, getUrl());
+ sinkContext.put(FlumeConstants.CONFIG_SERIALIZER,EventSerializers.REGEX.name());
+ sinkContext.put(FlumeConstants.CONFIG_TABLE_DDL, ddl);
+ sinkContext.put(FlumeConstants.CONFIG_SERIALIZER_PREFIX + FlumeConstants.CONFIG_REGULAR_EXPRESSION,logRegex);
+ sinkContext.put(FlumeConstants.CONFIG_SERIALIZER_PREFIX + FlumeConstants.CONFIG_COLUMN_NAMES,columns);
+ sinkContext.put(FlumeConstants.CONFIG_SERIALIZER_PREFIX + FlumeConstants.CONFIG_ROWKEY_TYPE_GENERATOR,DefaultKeyGenerator.UUID.name());
+
+ String message1 = "33.22.11.00 - user1 [12/Dec/2013:07:01:19 +0000] " +
+ "\"GET /wp-admin/css/install.css HTTP/1.0\" 200 813 " +
+ "\"http://www.google.com\" \"Mozilla/5.0 (comp" +
+ "atible; Yahoo! Slurp; http://help.yahoo.com/help/us/ysearch/slurp)\"";
+
+ String message2 = "192.168.20.1 - user2 [13/Dec/2013:06:05:19 +0000] " +
+ "\"GET /wp-admin/css/install.css HTTP/1.0\" 400 363 " +
+ "\"http://www.salesforce.com/in/?ir=1\" \"Mozilla/5.0 (comp" +
+ "atible;)\"";
+
+
+ sink = new PhoenixSink();
+ Configurables.configure(sink, sinkContext);
+ assertEquals(LifecycleState.IDLE, sink.getLifecycleState());
+
+ final Channel channel = this.initChannel();
+ sink.setChannel(channel);
+
+ sink.start();
+
+ final Event event1 = EventBuilder.withBody(Bytes.toBytes(message1));
+ final Event event2 = EventBuilder.withBody(Bytes.toBytes(message2));
+
+ final Transaction transaction = channel.getTransaction();
+ transaction.begin();
+ channel.put(event1);
+ channel.put(event2);
+ transaction.commit();
+ transaction.close();
+
+ sink.process();
+
+ final String query = " SELECT * FROM \n " + fullTableName;
+ Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
+ final ResultSet rs ;
+ final Connection conn = DriverManager.getConnection(getUrl(), props);
+ try{
+ rs = conn.createStatement().executeQuery(query);
+ assertTrue(rs.next());
+ assertTrue(rs.next());
+
+ }finally {
+ if(conn != null) {
+ conn.close();
+ }
+ }
+ sink.stop();
+ assertEquals(LifecycleState.STOP, sink.getLifecycleState());
+
+ }
+
+
+ @Test
+ public void testEventsWithHeaders() throws Exception {
+
+ sinkContext = new Context ();
+ final String fullTableName = generateUniqueName();
+ final String ddl = "CREATE TABLE " + fullTableName +
+ " (rowkey VARCHAR not null, col1 varchar , cf1.col2 varchar , host varchar , source varchar \n" +
+ " CONSTRAINT pk PRIMARY KEY (rowkey))\n";
+
+ sinkContext.put(FlumeConstants.CONFIG_TABLE, fullTableName);
+ sinkContext.put(FlumeConstants.CONFIG_JDBC_URL, getUrl());
+ sinkContext.put(FlumeConstants.CONFIG_SERIALIZER,EventSerializers.REGEX.name());
+ sinkContext.put(FlumeConstants.CONFIG_TABLE_DDL, ddl);
+ sinkContext.put(FlumeConstants.CONFIG_SERIALIZER_PREFIX + FlumeConstants.CONFIG_REGULAR_EXPRESSION,"^([^\t]+)\t([^\t]+)$");
+ sinkContext.put(FlumeConstants.CONFIG_SERIALIZER_PREFIX + FlumeConstants.CONFIG_COLUMN_NAMES,"col1,cf1.col2");
+ sinkContext.put(FlumeConstants.CONFIG_SERIALIZER_PREFIX + FlumeConstants.CONFIG_HEADER_NAMES,"host,source");
+ sinkContext.put(FlumeConstants.CONFIG_SERIALIZER_PREFIX + FlumeConstants.CONFIG_ROWKEY_TYPE_GENERATOR,DefaultKeyGenerator.UUID.name());
+
+ sink = new PhoenixSink();
+ Configurables.configure(sink, sinkContext);
+ assertEquals(LifecycleState.IDLE, sink.getLifecycleState());
+
+ final Channel channel = this.initChannel();
+ sink.setChannel(channel);
+
+ sink.start();
+
+ int numEvents = 10;
+ String col1 = "val1";
+ String col2 = "val2";
+ String hostHeader = "host1";
+ String sourceHeader = "source1";
+ String eventBody = null;
+ List<Event> eventList = Lists.newArrayListWithCapacity(numEvents);
+ for(int i = 0 ; i < numEvents ; i++) {
+ eventBody = (col1 + i) + "\t" + (col2 + i);
+ Map<String, String> headerMap = Maps.newHashMapWithExpectedSize(2);
+ headerMap.put("host",hostHeader);
+ headerMap.put("source",sourceHeader);
+ Event event = EventBuilder.withBody(Bytes.toBytes(eventBody),headerMap);
+ eventList.add(event);
+ }
+
+ // put event in channel
+ Transaction transaction = channel.getTransaction();
+ transaction.begin();
+ for(Event event : eventList) {
+ channel.put(event);
+ }
+ transaction.commit();
+ transaction.close();
+
+ sink.process();
+
+ final String query = " SELECT * FROM \n " + fullTableName;
+ Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
+ final ResultSet rs ;
+ final Connection conn = DriverManager.getConnection(getUrl(), props);
+ try{
+ rs = conn.createStatement().executeQuery(query);
+ assertTrue(rs.next());
+ assertEquals("host1",rs.getString("host"));
+ assertEquals("source1",rs.getString("source"));
+
+ assertTrue(rs.next());
+ assertEquals("host1",rs.getString("host"));
+ assertEquals("source1",rs.getString("source"));
+ }finally {
+ if(conn != null) {
+ conn.close();
+ }
+ }
+ sink.stop();
+ assertEquals(LifecycleState.STOP, sink.getLifecycleState());
+
+ }
+
+ private Channel initChannel() {
+ //Channel configuration
+ Context channelContext = new Context();
+ channelContext.put("capacity", "10000");
+ channelContext.put("transactionCapacity", "200");
+
+ Channel channel = new MemoryChannel();
+ channel.setName("memorychannel");
+ Configurables.configure(channel, channelContext);
+ return channel;
+ }
+
+ private void initSinkContextWithDefaults(final String fullTableName) {
+ Preconditions.checkNotNull(fullTableName);
+ sinkContext = new Context ();
+ String ddl = "CREATE TABLE " + fullTableName +
+ " (flume_time timestamp not null, col1 varchar , col2 varchar" +
+ " CONSTRAINT pk PRIMARY KEY (flume_time))\n";
+
+ sinkContext.put(FlumeConstants.CONFIG_TABLE, fullTableName);
+ sinkContext.put(FlumeConstants.CONFIG_JDBC_URL, getUrl());
+ sinkContext.put(FlumeConstants.CONFIG_SERIALIZER,EventSerializers.REGEX.name());
+ sinkContext.put(FlumeConstants.CONFIG_TABLE_DDL, ddl);
+ sinkContext.put(FlumeConstants.CONFIG_SERIALIZER_PREFIX + FlumeConstants.CONFIG_REGULAR_EXPRESSION,"^([^\t]+)\t([^\t]+)$");
+ sinkContext.put(FlumeConstants.CONFIG_SERIALIZER_PREFIX + FlumeConstants.CONFIG_COLUMN_NAMES,"col1,col2");
+ sinkContext.put(FlumeConstants.CONFIG_SERIALIZER_PREFIX + FlumeConstants.CONFIG_ROWKEY_TYPE_GENERATOR,DefaultKeyGenerator.TIMESTAMP.name());
+
+ }
+
+ private void setConfig(final String configName , final String configValue) {
+ Preconditions.checkNotNull(sinkContext);
+ Preconditions.checkNotNull(configName);
+ Preconditions.checkNotNull(configValue);
+ sinkContext.put(configName, configValue);
+ }
+
+ private int countRows(final String fullTableName) throws SQLException {
+ Preconditions.checkNotNull(fullTableName);
+ Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
+ final Connection conn = DriverManager.getConnection(getUrl(), props);
+ ResultSet rs = null ;
+ try{
+ rs = conn.createStatement().executeQuery("select count(*) from "+fullTableName);
+ int rowsCount = 0;
+ while(rs.next()) {
+ rowsCount = rs.getInt(1);
+ }
+ return rowsCount;
+
+ } finally {
+ if(rs != null) {
+ rs.close();
+ }
+ if(conn != null) {
+ conn.close();
+ }
+ }
+
+
+ }
+
+}
diff --git a/phoenix-flume/src/it/java/org/apache/phoenix/flume/serializer/CustomSerializer.java b/phoenix-flume/src/it/java/org/apache/phoenix/flume/serializer/CustomSerializer.java
new file mode 100644
index 0000000..5db5fa6
--- /dev/null
+++ b/phoenix-flume/src/it/java/org/apache/phoenix/flume/serializer/CustomSerializer.java
@@ -0,0 +1,43 @@
+/*
+ * 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.phoenix.flume.serializer;
+
+import java.sql.SQLException;
+import java.util.List;
+
+import org.apache.flume.Context;
+import org.apache.flume.Event;
+
+public class CustomSerializer extends BaseEventSerializer {
+ @Override
+ public void doConfigure(Context context) {
+
+ }
+
+ @Override
+ public void doInitialize() throws SQLException {
+
+ }
+
+ @Override
+ public void upsertEvents(List<Event> events) throws SQLException {
+ // Just execute a sample UPSERT
+ connection.createStatement().execute("UPSERT INTO FLUME_TEST_EXTENDED(ID, COUNTS) VALUES(1, 1)");
+ connection.commit();
+ }
+}
diff --git a/phoenix-flume/src/it/java/org/apache/phoenix/flume/sink/NullPhoenixSink.java b/phoenix-flume/src/it/java/org/apache/phoenix/flume/sink/NullPhoenixSink.java
new file mode 100644
index 0000000..1df52e1
--- /dev/null
+++ b/phoenix-flume/src/it/java/org/apache/phoenix/flume/sink/NullPhoenixSink.java
@@ -0,0 +1,21 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.phoenix.flume.sink;
+
+public class NullPhoenixSink extends PhoenixSink {
+}
diff --git a/phoenix-flume/src/main/java/org/apache/phoenix/flume/DefaultKeyGenerator.java b/phoenix-flume/src/main/java/org/apache/phoenix/flume/DefaultKeyGenerator.java
new file mode 100644
index 0000000..3820c2a
--- /dev/null
+++ b/phoenix-flume/src/main/java/org/apache/phoenix/flume/DefaultKeyGenerator.java
@@ -0,0 +1,69 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.phoenix.flume;
+
+import java.sql.Date;
+import java.sql.Timestamp;
+import java.util.Random;
+
+import org.apache.phoenix.util.DateUtil;
+
+public enum DefaultKeyGenerator implements KeyGenerator {
+
+ UUID {
+
+ @Override
+ public String generate() {
+ return String.valueOf(java.util.UUID.randomUUID());
+ }
+
+ },
+ TIMESTAMP {
+
+ @Override
+ public String generate() {
+ java.sql.Timestamp ts = new Timestamp(System.currentTimeMillis());
+ return DateUtil.DEFAULT_DATE_FORMATTER.format(ts);
+ }
+
+ },
+ DATE {
+
+ @Override
+ public String generate() {
+ Date dt = new Date(System.currentTimeMillis());
+ return DateUtil.DEFAULT_DATE_FORMATTER.format(dt);
+ }
+ },
+ RANDOM {
+
+ @Override
+ public String generate() {
+ return String.valueOf(new Random().nextLong());
+ }
+
+ },
+ NANOTIMESTAMP {
+
+ @Override
+ public String generate() {
+ return String.valueOf(System.nanoTime());
+ }
+
+ };
+}
diff --git a/phoenix-flume/src/main/java/org/apache/phoenix/flume/FlumeConstants.java b/phoenix-flume/src/main/java/org/apache/phoenix/flume/FlumeConstants.java
new file mode 100644
index 0000000..a146bbe
--- /dev/null
+++ b/phoenix-flume/src/main/java/org/apache/phoenix/flume/FlumeConstants.java
@@ -0,0 +1,94 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.phoenix.flume;
+
+public final class FlumeConstants {
+
+ /**
+ * The Hbase table which the sink should write to.
+ */
+ public static final String CONFIG_TABLE = "table";
+ /**
+ * The ddl query for the Hbase table where events are ingested to.
+ */
+ public static final String CONFIG_TABLE_DDL = "ddl";
+ /**
+ * Maximum number of events the sink should take from the channel per transaction, if available.
+ */
+ public static final String CONFIG_BATCHSIZE = "batchSize";
+ /**
+ * The fully qualified class name of the serializer the sink should use.
+ */
+ public static final String CONFIG_SERIALIZER = "serializer";
+ /**
+ * Configuration to pass to the serializer.
+ */
+ public static final String CONFIG_SERIALIZER_PREFIX = CONFIG_SERIALIZER + ".";
+
+ /**
+ * Configuration for the zookeeper quorum.
+ */
+ public static final String CONFIG_ZK_QUORUM = "zookeeperQuorum";
+
+ /**
+ * Configuration for the jdbc url.
+ */
+ public static final String CONFIG_JDBC_URL = "jdbcUrl";
+
+ /**
+ * Default batch size .
+ */
+ public static final Integer DEFAULT_BATCH_SIZE = 100;
+
+ /** Regular expression used to parse groups from event data. */
+ public static final String CONFIG_REGULAR_EXPRESSION = "regex";
+ public static final String REGEX_DEFAULT = "(.*)";
+
+ /** Whether to ignore case when performing regex matches. */
+ public static final String IGNORE_CASE_CONFIG = "regexIgnoreCase";
+ public static final boolean IGNORE_CASE_DEFAULT = false;
+
+ /** JSON expression used to parse groups from event data. */
+ public static final String CONFIG_COLUMNS_MAPPING = "columnsMapping";
+ public static final String CONFIG_PARTIAL_SCHEMA = "partialSchema";
+ public static final String JSON_DEFAULT = "{}";
+
+ /** CSV expression used to parse groups from event data. */
+ public static final String CSV_DELIMITER = "csvDelimiter";
+ public static final String CSV_DELIMITER_DEFAULT = ",";
+ public static final String CSV_QUOTE = "csvQuote";
+ public static final String CSV_QUOTE_DEFAULT = "\"";
+ public static final String CSV_ESCAPE = "csvEscape";
+ public static final String CSV_ESCAPE_DEFAULT = "\\";
+ public static final String CSV_ARRAY_DELIMITER = "csvArrayDelimiter";
+ public static final String CSV_ARRAY_DELIMITER_DEFAULT = ",";
+
+ /** Comma separated list of column names . */
+ public static final String CONFIG_COLUMN_NAMES = "columns";
+
+ /** The header columns to persist as columns into the default column family. */
+ public static final String CONFIG_HEADER_NAMES = "headers";
+
+ /** The rowkey type generator . */
+ public static final String CONFIG_ROWKEY_TYPE_GENERATOR = "rowkeyType";
+
+ /**
+ * The default delimiter for columns and headers
+ */
+ public static final String DEFAULT_COLUMNS_DELIMITER = ",";
+}
diff --git a/phoenix-flume/src/main/java/org/apache/phoenix/flume/KeyGenerator.java b/phoenix-flume/src/main/java/org/apache/phoenix/flume/KeyGenerator.java
new file mode 100644
index 0000000..d823a56
--- /dev/null
+++ b/phoenix-flume/src/main/java/org/apache/phoenix/flume/KeyGenerator.java
@@ -0,0 +1,24 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.phoenix.flume;
+
+public interface KeyGenerator {
+
+ public String generate();
+}
+
diff --git a/phoenix-flume/src/main/java/org/apache/phoenix/flume/SchemaHandler.java b/phoenix-flume/src/main/java/org/apache/phoenix/flume/SchemaHandler.java
new file mode 100644
index 0000000..8b14b64
--- /dev/null
+++ b/phoenix-flume/src/main/java/org/apache/phoenix/flume/SchemaHandler.java
@@ -0,0 +1,47 @@
+/*
+ * 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.phoenix.flume;
+
+import java.sql.Connection;
+import java.sql.SQLException;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.google.common.base.Preconditions;
+import com.google.common.base.Throwables;
+
+public class SchemaHandler {
+
+ private static final Logger logger = LoggerFactory.getLogger(SchemaHandler.class);
+
+ public static boolean createTable(Connection connection, String createTableDdl) {
+ Preconditions.checkNotNull(connection);
+ Preconditions.checkNotNull(createTableDdl);
+ boolean status = true;
+ try {
+ status = connection.createStatement().execute(createTableDdl);
+ } catch (SQLException e) {
+ logger.error("An error occurred during executing the create table ddl {} ",createTableDdl);
+ Throwables.propagate(e);
+ }
+ return status;
+
+ }
+
+}
diff --git a/phoenix-flume/src/main/java/org/apache/phoenix/flume/serializer/BaseEventSerializer.java b/phoenix-flume/src/main/java/org/apache/phoenix/flume/serializer/BaseEventSerializer.java
new file mode 100644
index 0000000..24527e3
--- /dev/null
+++ b/phoenix-flume/src/main/java/org/apache/phoenix/flume/serializer/BaseEventSerializer.java
@@ -0,0 +1,245 @@
+/*
+ * 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.phoenix.flume.serializer;
+
+import static org.apache.phoenix.flume.FlumeConstants.CONFIG_COLUMN_NAMES;
+import static org.apache.phoenix.flume.FlumeConstants.CONFIG_HEADER_NAMES;
+import static org.apache.phoenix.flume.FlumeConstants.CONFIG_ROWKEY_TYPE_GENERATOR;
+import static org.apache.phoenix.flume.FlumeConstants.DEFAULT_COLUMNS_DELIMITER;
+import static org.apache.phoenix.util.PhoenixRuntime.UPSERT_BATCH_SIZE_ATTRIB;
+
+import java.sql.Connection;
+import java.sql.DriverManager;
+import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.util.Arrays;
+import java.util.List;
+import java.util.Map;
+import java.util.Properties;
+
+import org.apache.flume.Context;
+import org.apache.flume.conf.ComponentConfiguration;
+import org.apache.phoenix.exception.SQLExceptionCode;
+import org.apache.phoenix.exception.SQLExceptionInfo;
+import org.apache.phoenix.flume.DefaultKeyGenerator;
+import org.apache.phoenix.flume.FlumeConstants;
+import org.apache.phoenix.flume.KeyGenerator;
+import org.apache.phoenix.flume.SchemaHandler;
+import org.apache.phoenix.util.ColumnInfo;
+import org.apache.phoenix.util.QueryUtil;
+import org.apache.phoenix.util.SchemaUtil;
+import org.apache.phoenix.util.StringUtil;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.google.common.base.Preconditions;
+import com.google.common.base.Splitter;
+import com.google.common.base.Strings;
+import com.google.common.base.Throwables;
+import com.google.common.collect.Lists;
+import com.google.common.collect.Maps;
+
+public abstract class BaseEventSerializer implements EventSerializer {
+
+ private static final Logger logger = LoggerFactory.getLogger(BaseEventSerializer.class);
+
+ protected Connection connection;
+ protected String fullTableName;
+ protected ColumnInfo[] columnMetadata;
+ protected boolean autoGenerateKey = false;
+ protected KeyGenerator keyGenerator;
+ protected List<String> colNames = Lists.newArrayListWithExpectedSize(10);
+ protected List<String> headers = Lists.newArrayListWithExpectedSize(5);
+ protected String upsertStatement;
+ private String jdbcUrl;
+ private Integer batchSize;
+ private String createTableDdl;
+
+
+
+
+
+ @Override
+ public void configure(Context context) {
+
+ this.createTableDdl = context.getString(FlumeConstants.CONFIG_TABLE_DDL);
+ this.fullTableName = context.getString(FlumeConstants.CONFIG_TABLE);
+ final String zookeeperQuorum = context.getString(FlumeConstants.CONFIG_ZK_QUORUM);
+ final String ipJdbcURL = context.getString(FlumeConstants.CONFIG_JDBC_URL);
+ this.batchSize = context.getInteger(FlumeConstants.CONFIG_BATCHSIZE, FlumeConstants.DEFAULT_BATCH_SIZE);
+ final String columnNames = context.getString(CONFIG_COLUMN_NAMES);
+ final String headersStr = context.getString(CONFIG_HEADER_NAMES);
+ final String keyGeneratorType = context.getString(CONFIG_ROWKEY_TYPE_GENERATOR);
+
+ Preconditions.checkNotNull(this.fullTableName,"Table name cannot be empty, please specify in the configuration file");
+ if(!Strings.isNullOrEmpty(zookeeperQuorum)) {
+ this.jdbcUrl = QueryUtil.getUrl(zookeeperQuorum);
+ }
+ if(!Strings.isNullOrEmpty(ipJdbcURL)) {
+ this.jdbcUrl = ipJdbcURL;
+ }
+ Preconditions.checkNotNull(this.jdbcUrl,"Please specify either the zookeeper quorum or the jdbc url in the configuration file");
+ Preconditions.checkNotNull(columnNames,"Column names cannot be empty, please specify in configuration file");
+ for(String s : Splitter.on(DEFAULT_COLUMNS_DELIMITER).split(columnNames)) {
+ colNames.add(s);
+ }
+
+ if(!Strings.isNullOrEmpty(headersStr)) {
+ for(String s : Splitter.on(DEFAULT_COLUMNS_DELIMITER).split(headersStr)) {
+ headers.add(s);
+ }
+ }
+
+ if(!Strings.isNullOrEmpty(keyGeneratorType)) {
+ try {
+ keyGenerator = DefaultKeyGenerator.valueOf(keyGeneratorType.toUpperCase());
+ this.autoGenerateKey = true;
+ } catch(IllegalArgumentException iae) {
+ logger.error("An invalid key generator {} was specified in configuration file. Specify one of {}",keyGeneratorType,DefaultKeyGenerator.values());
+ Throwables.propagate(iae);
+ }
+ }
+
+ logger.debug(" the jdbcUrl configured is {}",jdbcUrl);
+ logger.debug(" columns configured are {}",colNames.toString());
+ logger.debug(" headers configured are {}",headersStr);
+ logger.debug(" the keyGenerator configured is {} ",keyGeneratorType);
+
+ doConfigure(context);
+
+ }
+
+ @Override
+ public void configure(ComponentConfiguration conf) {
+ // NO-OP
+
+ }
+
+
+ @Override
+ public void initialize() throws SQLException {
+ final Properties props = new Properties();
+ props.setProperty(UPSERT_BATCH_SIZE_ATTRIB, String.valueOf(this.batchSize));
+ ResultSet rs = null;
+ try {
+ this.connection = DriverManager.getConnection(this.jdbcUrl, props);
+ this.connection.setAutoCommit(false);
+ if(this.createTableDdl != null) {
+ SchemaHandler.createTable(connection,createTableDdl);
+ }
+
+
+ final Map<String,Integer> qualifiedColumnMap = Maps.newLinkedHashMap();
+ final Map<String,Integer> unqualifiedColumnMap = Maps.newLinkedHashMap();
+ final String schemaName = SchemaUtil.getSchemaNameFromFullName(fullTableName);
+ final String tableName = SchemaUtil.getTableNameFromFullName(fullTableName);
+
+ String rowkey = null;
+ String cq = null;
+ String cf = null;
+ Integer dt = null;
+ rs = connection.getMetaData().getColumns("", StringUtil.escapeLike(SchemaUtil.normalizeIdentifier(schemaName)), StringUtil.escapeLike(SchemaUtil.normalizeIdentifier(tableName)), null);
+ while (rs.next()) {
+ cf = rs.getString(QueryUtil.COLUMN_FAMILY_POSITION);
+ cq = rs.getString(QueryUtil.COLUMN_NAME_POSITION);
+ // TODO: Fix this .. change `DATA_TYPE_POSITION` value 5 to 26
+ // dt = rs.getInt(QueryUtil.DATA_TYPE_POSITION);
+ dt = rs.getInt(26);
+ if(Strings.isNullOrEmpty(cf)) {
+ rowkey = cq; // this is required only when row key is auto generated
+ } else {
+ qualifiedColumnMap.put(SchemaUtil.getColumnDisplayName(cf, cq), dt);
+ }
+ unqualifiedColumnMap.put(SchemaUtil.getColumnDisplayName(null, cq), dt);
+ }
+
+ //can happen when table not found in Hbase.
+ if(unqualifiedColumnMap.isEmpty()) {
+ throw new SQLExceptionInfo.Builder(SQLExceptionCode.TABLE_UNDEFINED)
+ .setTableName(tableName).build().buildException();
+ }
+
+ int colSize = colNames.size();
+ int headersSize = headers.size();
+ int totalSize = colSize + headersSize + ( autoGenerateKey ? 1 : 0);
+ columnMetadata = new ColumnInfo[totalSize] ;
+
+ int position = 0;
+ position = this.addToColumnMetadataInfo(colNames, qualifiedColumnMap, unqualifiedColumnMap, position);
+ position = this.addToColumnMetadataInfo(headers, qualifiedColumnMap, unqualifiedColumnMap, position);
+
+ if(autoGenerateKey) {
+ Integer sqlType = unqualifiedColumnMap.get(rowkey);
+ if (sqlType == null) {
+ throw new SQLExceptionInfo.Builder(SQLExceptionCode.PRIMARY_KEY_MISSING)
+ .setColumnName(rowkey).setTableName(fullTableName).build().buildException();
+ }
+ columnMetadata[position] = new ColumnInfo(rowkey, sqlType);
+ position++;
+ }
+
+ this.upsertStatement = QueryUtil.constructUpsertStatement(fullTableName, Arrays.asList(columnMetadata));
+ logger.info(" the upsert statement is {} " ,this.upsertStatement);
+
+ } catch (SQLException e) {
+ logger.error("error {} occurred during initializing connection ",e.getMessage());
+ throw e;
+ } finally {
+ if(rs != null) {
+ rs.close();
+ }
+ }
+ doInitialize();
+ }
+
+ private int addToColumnMetadataInfo(final List<String> columns , final Map<String,Integer> qualifiedColumnsInfoMap, Map<String, Integer> unqualifiedColumnsInfoMap, int position) throws SQLException {
+ Preconditions.checkNotNull(columns);
+ Preconditions.checkNotNull(qualifiedColumnsInfoMap);
+ Preconditions.checkNotNull(unqualifiedColumnsInfoMap);
+ for (int i = 0 ; i < columns.size() ; i++) {
+ String columnName = SchemaUtil.normalizeIdentifier(columns.get(i).trim());
+ Integer sqlType = unqualifiedColumnsInfoMap.get(columnName);
+ if (sqlType == null) {
+ sqlType = qualifiedColumnsInfoMap.get(columnName);
+ if (sqlType == null) {
+ throw new SQLExceptionInfo.Builder(SQLExceptionCode.COLUMN_NOT_FOUND)
+ .setColumnName(columnName).setTableName(this.fullTableName).build().buildException();
+ }
+ }
+ columnMetadata[position] = new ColumnInfo(columnName, sqlType);
+ position++;
+ }
+ return position;
+ }
+
+ public abstract void doConfigure(Context context);
+
+ public abstract void doInitialize() throws SQLException;
+
+
+ @Override
+ public void close() {
+ if(connection != null) {
+ try {
+ connection.close();
+ } catch (SQLException e) {
+ logger.error(" Error while closing connection {} ");
+ }
+ }
+ }
+}
diff --git a/phoenix-flume/src/main/java/org/apache/phoenix/flume/serializer/CsvEventSerializer.java b/phoenix-flume/src/main/java/org/apache/phoenix/flume/serializer/CsvEventSerializer.java
new file mode 100644
index 0000000..a856c3e
--- /dev/null
+++ b/phoenix-flume/src/main/java/org/apache/phoenix/flume/serializer/CsvEventSerializer.java
@@ -0,0 +1,196 @@
+/*
+ * 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.phoenix.flume.serializer;
+
+import static org.apache.phoenix.flume.FlumeConstants.CSV_DELIMITER;
+import static org.apache.phoenix.flume.FlumeConstants.CSV_DELIMITER_DEFAULT;
+import static org.apache.phoenix.flume.FlumeConstants.CSV_QUOTE;
+import static org.apache.phoenix.flume.FlumeConstants.CSV_QUOTE_DEFAULT;
+import static org.apache.phoenix.flume.FlumeConstants.CSV_ESCAPE;
+import static org.apache.phoenix.flume.FlumeConstants.CSV_ESCAPE_DEFAULT;
+import static org.apache.phoenix.flume.FlumeConstants.CSV_ARRAY_DELIMITER;
+import static org.apache.phoenix.flume.FlumeConstants.CSV_ARRAY_DELIMITER_DEFAULT;
+
+import java.io.IOException;
+import java.io.StringReader;
+import java.sql.Array;
+import java.sql.PreparedStatement;
+import java.sql.SQLException;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.commons.csv.CSVFormat;
+import org.apache.commons.csv.CSVParser;
+import org.apache.commons.csv.CSVRecord;
+import org.apache.flume.Context;
+import org.apache.flume.Event;
+import org.apache.phoenix.schema.types.PDataType;
+import org.json.JSONArray;
+import org.json.JSONTokener;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.google.common.base.Preconditions;
+import com.google.common.collect.Iterables;
+
+public class CsvEventSerializer extends BaseEventSerializer {
+
+ private static final Logger logger = LoggerFactory.getLogger(CsvEventSerializer.class);
+
+ private String csvDelimiter;
+ private String csvQuote;
+ private String csvEscape;
+ private String csvArrayDelimiter;
+ private CsvLineParser csvLineParser;
+
+ /**
+ *
+ */
+ @Override
+ public void doConfigure(Context context) {
+ csvDelimiter = context.getString(CSV_DELIMITER, CSV_DELIMITER_DEFAULT);
+ csvQuote = context.getString(CSV_QUOTE, CSV_QUOTE_DEFAULT);
+ csvEscape = context.getString(CSV_ESCAPE, CSV_ESCAPE_DEFAULT);
+ csvArrayDelimiter = context.getString(CSV_ARRAY_DELIMITER, CSV_ARRAY_DELIMITER_DEFAULT);
+ csvLineParser = new CsvLineParser(csvDelimiter.toCharArray()[0], csvQuote.toCharArray()[0],
+ csvEscape.toCharArray()[0]);
+ }
+
+ /**
+ *
+ */
+ @Override
+ public void doInitialize() throws SQLException {
+ // NO-OP
+ }
+
+ @Override
+ public void upsertEvents(List<Event> events) throws SQLException {
+ Preconditions.checkNotNull(events);
+ Preconditions.checkNotNull(connection);
+ Preconditions.checkNotNull(this.upsertStatement);
+
+ boolean wasAutoCommit = connection.getAutoCommit();
+ connection.setAutoCommit(false);
+ try (PreparedStatement colUpsert = connection.prepareStatement(upsertStatement)) {
+ String value = null;
+ Integer sqlType = null;
+ for (Event event : events) {
+ byte[] payloadBytes = event.getBody();
+ if (payloadBytes == null || payloadBytes.length == 0) {
+ continue;
+ }
+ String payload = new String(payloadBytes);
+ CSVRecord csvRecord = csvLineParser.parse(payload);
+ if (colNames.size() != csvRecord.size()) {
+ logger.debug("payload data {} doesn't match the fields mapping {} ", payload, colNames);
+ continue;
+ }
+ Map<String, String> data = new HashMap<String, String>();
+ for (int i = 0; i < csvRecord.size(); i++) {
+ data.put(colNames.get(i), csvRecord.get(i));
+ }
+ Collection<String> values = data.values();
+ if (values.contains(null)) {
+ logger.debug("payload data {} doesn't match the fields mapping {} ", payload, colNames);
+ continue;
+ }
+
+ int index = 1;
+ int offset = 0;
+ for (int i = 0; i < colNames.size(); i++, offset++) {
+ if (columnMetadata[offset] == null) {
+ continue;
+ }
+ String colName = colNames.get(i);
+ value = data.get(colName);
+ sqlType = columnMetadata[offset].getSqlType();
+ PDataType pDataType = PDataType.fromTypeId(sqlType);
+ Object upsertValue;
+ if (pDataType.isArrayType()) {
+ String arrayJson = Arrays.toString(value.split(csvArrayDelimiter));
+ JSONArray jsonArray = new JSONArray(new JSONTokener(arrayJson));
+ Object[] vals = new Object[jsonArray.length()];
+ for (int x = 0; x < jsonArray.length(); x++) {
+ vals[x] = jsonArray.get(x);
+ }
+ String baseTypeSqlName = PDataType.arrayBaseType(pDataType).getSqlTypeName();
+ Array array = connection.createArrayOf(baseTypeSqlName, vals);
+ upsertValue = pDataType.toObject(array, pDataType);
+ } else {
+ upsertValue = pDataType.toObject(value);
+ }
+ if (upsertValue != null) {
+ colUpsert.setObject(index++, upsertValue, sqlType);
+ } else {
+ colUpsert.setNull(index++, sqlType);
+ }
+ }
+
+ // add headers if necessary
+ Map<String, String> headerValues = event.getHeaders();
+ for (int i = 0; i < headers.size(); i++, offset++) {
+ String headerName = headers.get(i);
+ String headerValue = headerValues.get(headerName);
+ sqlType = columnMetadata[offset].getSqlType();
+ Object upsertValue = PDataType.fromTypeId(sqlType).toObject(headerValue);
+ if (upsertValue != null) {
+ colUpsert.setObject(index++, upsertValue, sqlType);
+ } else {
+ colUpsert.setNull(index++, sqlType);
+ }
+ }
+
+ if (autoGenerateKey) {
+ sqlType = columnMetadata[offset].getSqlType();
+ String generatedRowValue = this.keyGenerator.generate();
+ Object rowkeyValue = PDataType.fromTypeId(sqlType).toObject(generatedRowValue);
+ colUpsert.setObject(index++, rowkeyValue, sqlType);
+ }
+ colUpsert.execute();
+ }
+ connection.commit();
+ } catch (Exception ex) {
+ logger.error("An error {} occurred during persisting the event ", ex.getMessage());
+ throw new SQLException(ex.getMessage());
+ } finally {
+ if (wasAutoCommit) {
+ connection.setAutoCommit(true);
+ }
+ }
+
+ }
+
+ static class CsvLineParser {
+ private final CSVFormat csvFormat;
+
+ CsvLineParser(char fieldDelimiter, char quote, char escape) {
+ this.csvFormat = CSVFormat.DEFAULT.withIgnoreEmptyLines(true).withDelimiter(fieldDelimiter)
+ .withEscape(escape).withQuote(quote);
+ }
+
+ public CSVRecord parse(String input) throws IOException {
+ CSVParser csvParser = new CSVParser(new StringReader(input), this.csvFormat);
+ return Iterables.getFirst(csvParser, null);
+ }
+ }
+
+}
\ No newline at end of file
diff --git a/phoenix-flume/src/main/java/org/apache/phoenix/flume/serializer/EventSerializer.java b/phoenix-flume/src/main/java/org/apache/phoenix/flume/serializer/EventSerializer.java
new file mode 100644
index 0000000..80959f5
--- /dev/null
+++ b/phoenix-flume/src/main/java/org/apache/phoenix/flume/serializer/EventSerializer.java
@@ -0,0 +1,42 @@
+/*
+ * 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.phoenix.flume.serializer;
+
+import java.sql.SQLException;
+import java.util.List;
+
+import org.apache.flume.Event;
+import org.apache.flume.conf.Configurable;
+import org.apache.flume.conf.ConfigurableComponent;
+
+import org.apache.phoenix.util.SQLCloseable;
+
+public interface EventSerializer extends Configurable,ConfigurableComponent,SQLCloseable {
+
+ /**
+ * called during the start of the process to initialize the table columns.
+ */
+ public void initialize() throws SQLException;
+
+ /**
+ * @param events to be written to HBase.
+ * @throws SQLException
+ */
+ public void upsertEvents(List<Event> events) throws SQLException;
+
+}
diff --git a/phoenix-flume/src/main/java/org/apache/phoenix/flume/serializer/EventSerializers.java b/phoenix-flume/src/main/java/org/apache/phoenix/flume/serializer/EventSerializers.java
new file mode 100644
index 0000000..8c99d7d
--- /dev/null
+++ b/phoenix-flume/src/main/java/org/apache/phoenix/flume/serializer/EventSerializers.java
@@ -0,0 +1,36 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.phoenix.flume.serializer;
+
+public enum EventSerializers {
+
+ REGEX(RegexEventSerializer.class.getName()), JSON(JsonEventSerializer.class.getName()), CSV(CsvEventSerializer.class.getName());
+
+ private final String className;
+
+ private EventSerializers(String serializerClassName) {
+ this.className = serializerClassName;
+ }
+
+ /**
+ * @return Returns the serializer className.
+ */
+ public String getClassName() {
+ return className;
+ }
+}
\ No newline at end of file
diff --git a/phoenix-flume/src/main/java/org/apache/phoenix/flume/serializer/JsonEventSerializer.java b/phoenix-flume/src/main/java/org/apache/phoenix/flume/serializer/JsonEventSerializer.java
new file mode 100644
index 0000000..9226017
--- /dev/null
+++ b/phoenix-flume/src/main/java/org/apache/phoenix/flume/serializer/JsonEventSerializer.java
@@ -0,0 +1,226 @@
+/*
+ * 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.phoenix.flume.serializer;
+
+import static org.apache.phoenix.flume.FlumeConstants.JSON_DEFAULT;
+import static org.apache.phoenix.flume.FlumeConstants.CONFIG_COLUMNS_MAPPING;
+import static org.apache.phoenix.flume.FlumeConstants.CONFIG_PARTIAL_SCHEMA;
+
+import java.sql.Array;
+import java.sql.PreparedStatement;
+import java.sql.SQLException;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.commons.collections.CollectionUtils;
+import org.apache.flume.Context;
+import org.apache.flume.Event;
+import org.apache.phoenix.schema.types.PDataType;
+import org.json.JSONArray;
+import org.json.JSONException;
+import org.json.JSONObject;
+import org.json.JSONTokener;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.google.common.base.Preconditions;
+import com.jayway.jsonpath.Configuration;
+import com.jayway.jsonpath.JsonPath;
+import com.jayway.jsonpath.spi.json.JsonOrgJsonProvider;
+import com.jayway.jsonpath.spi.mapper.JsonOrgMappingProvider;
+
+public class JsonEventSerializer extends BaseEventSerializer {
+
+ private static final Logger logger = LoggerFactory.getLogger(JsonEventSerializer.class);
+
+ private JSONObject jsonSchema;
+ private boolean isProperMapping;
+ private boolean partialSchema;
+
+ /**
+ *
+ */
+ @Override
+ public void doConfigure(Context context) {
+ final String jsonData = context.getString(CONFIG_COLUMNS_MAPPING, JSON_DEFAULT);
+ try {
+ jsonSchema = new JSONObject(jsonData);
+ if (jsonSchema.length() == 0) {
+ for (String colName : colNames) {
+ jsonSchema.put(colName, colName);
+ }
+ isProperMapping = true;
+ } else {
+ Iterator<String> keys = jsonSchema.keys();
+ List<String> keylist = new ArrayList<String>();
+ while (keys.hasNext()) {
+ keylist.add(keys.next());
+ }
+ isProperMapping = CollectionUtils.isEqualCollection(keylist, colNames);
+ }
+ } catch (JSONException e) {
+ e.printStackTrace();
+ logger.debug("json mapping not proper, verify the data {} ", jsonData);
+ }
+ partialSchema = context.getBoolean(CONFIG_PARTIAL_SCHEMA, false);
+ }
+
+ /**
+ *
+ */
+ @Override
+ public void doInitialize() throws SQLException {
+ // NO-OP
+ }
+
+ @Override
+ public void upsertEvents(List<Event> events) throws SQLException {
+ Preconditions.checkNotNull(events);
+ Preconditions.checkNotNull(connection);
+ Preconditions.checkNotNull(this.upsertStatement);
+ Preconditions.checkArgument(isProperMapping, "Please verify fields mapping is not properly done..");
+
+ boolean wasAutoCommit = connection.getAutoCommit();
+ connection.setAutoCommit(false);
+ try (PreparedStatement colUpsert = connection.prepareStatement(upsertStatement)) {
+ String value = null;
+ Integer sqlType = null;
+ JSONObject inputJson = new JSONObject();
+ for (Event event : events) {
+ byte[] payloadBytes = event.getBody();
+ if (payloadBytes == null || payloadBytes.length == 0) {
+ continue;
+ }
+ String payload = new String(payloadBytes);
+
+ try {
+ inputJson = new JSONObject(payload);
+ } catch (Exception e) {
+ logger.debug("payload is not proper json");
+ continue;
+ }
+
+ Map<String, String> data = new HashMap<String, String>();
+ for (String colName : colNames) {
+ String pattern = colName;
+ if (jsonSchema.has(colName)) {
+ Object obj = jsonSchema.opt(colName);
+ if (null != obj) {
+ pattern = obj.toString();
+ }
+ }
+ pattern = "$." + pattern;
+ value = getPatternData(inputJson, pattern);
+
+ // if field mapping data is null then look for column data
+ if (null == value && partialSchema) {
+ pattern = "$." + colName;
+ value = getPatternData(inputJson, pattern);
+ }
+
+ data.put(colName, value);
+ }
+
+ Collection<String> values = data.values();
+ if (values.contains(null)) {
+ logger.debug("payload data {} doesn't match the fields mapping {} ", inputJson, jsonSchema);
+ continue;
+ }
+
+ int index = 1;
+ int offset = 0;
+ for (int i = 0; i < colNames.size(); i++, offset++) {
+ if (columnMetadata[offset] == null) {
+ continue;
+ }
+ String colName = colNames.get(i);
+ value = data.get(colName);
+ sqlType = columnMetadata[offset].getSqlType();
+ PDataType pDataType = PDataType.fromTypeId(sqlType);
+ Object upsertValue;
+ if (pDataType.isArrayType()) {
+ JSONArray jsonArray = new JSONArray(new JSONTokener(value));
+ Object[] vals = new Object[jsonArray.length()];
+ for (int x = 0; x < jsonArray.length(); x++) {
+ vals[x] = jsonArray.get(x);
+ }
+ String baseTypeSqlName = PDataType.arrayBaseType(pDataType).getSqlTypeName();
+ Array array = connection.createArrayOf(baseTypeSqlName, vals);
+ upsertValue = pDataType.toObject(array, pDataType);
+ } else {
+ upsertValue = pDataType.toObject(value);
+ }
+ if (upsertValue != null) {
+ colUpsert.setObject(index++, upsertValue, sqlType);
+ } else {
+ colUpsert.setNull(index++, sqlType);
+ }
+ }
+
+ // add headers if necessary
+ Map<String, String> headerValues = event.getHeaders();
+ for (int i = 0; i < headers.size(); i++, offset++) {
+ String headerName = headers.get(i);
+ String headerValue = headerValues.get(headerName);
+ sqlType = columnMetadata[offset].getSqlType();
+ Object upsertValue = PDataType.fromTypeId(sqlType).toObject(headerValue);
+ if (upsertValue != null) {
+ colUpsert.setObject(index++, upsertValue, sqlType);
+ } else {
+ colUpsert.setNull(index++, sqlType);
+ }
+ }
+
+ if (autoGenerateKey) {
+ sqlType = columnMetadata[offset].getSqlType();
+ String generatedRowValue = this.keyGenerator.generate();
+ Object rowkeyValue = PDataType.fromTypeId(sqlType).toObject(generatedRowValue);
+ colUpsert.setObject(index++, rowkeyValue, sqlType);
+ }
+ colUpsert.execute();
+ }
+ connection.commit();
+ } catch (Exception ex) {
+ logger.error("An error {} occurred during persisting the event ", ex.getMessage());
+ throw new SQLException(ex.getMessage());
+ } finally {
+ if (wasAutoCommit) {
+ connection.setAutoCommit(true);
+ }
+ }
+
+ }
+
+ private String getPatternData(JSONObject json, String pattern) {
+ Configuration JSON_ORG_CONFIGURATION = Configuration.builder().mappingProvider(new JsonOrgMappingProvider())
+ .jsonProvider(new JsonOrgJsonProvider()).build();
+ String value;
+ try {
+ Object object = JsonPath.using(JSON_ORG_CONFIGURATION).parse(json).read(pattern);
+ value = object.toString();
+ } catch (Exception e) {
+ value = null;
+ }
+ return value;
+ }
+
+}
\ No newline at end of file
diff --git a/phoenix-flume/src/main/java/org/apache/phoenix/flume/serializer/RegexEventSerializer.java b/phoenix-flume/src/main/java/org/apache/phoenix/flume/serializer/RegexEventSerializer.java
new file mode 100644
index 0000000..b636481
--- /dev/null
+++ b/phoenix-flume/src/main/java/org/apache/phoenix/flume/serializer/RegexEventSerializer.java
@@ -0,0 +1,145 @@
+/*
+ * 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.phoenix.flume.serializer;
+
+import static org.apache.phoenix.flume.FlumeConstants.CONFIG_REGULAR_EXPRESSION;
+import static org.apache.phoenix.flume.FlumeConstants.IGNORE_CASE_CONFIG;
+import static org.apache.phoenix.flume.FlumeConstants.IGNORE_CASE_DEFAULT;
+import static org.apache.phoenix.flume.FlumeConstants.REGEX_DEFAULT;
+
+import java.sql.PreparedStatement;
+import java.sql.SQLException;
+import java.util.List;
+import java.util.Map;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+
+import org.apache.flume.Context;
+import org.apache.flume.Event;
+import org.apache.phoenix.schema.types.PDataType;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.google.common.base.Preconditions;
+
+public class RegexEventSerializer extends BaseEventSerializer {
+
+ private static final Logger logger = LoggerFactory.getLogger(RegexEventSerializer.class);
+
+ private Pattern inputPattern;
+
+ /**
+ *
+ */
+ @Override
+ public void doConfigure(Context context) {
+ final String regex = context.getString(CONFIG_REGULAR_EXPRESSION, REGEX_DEFAULT);
+ final boolean regexIgnoreCase = context.getBoolean(IGNORE_CASE_CONFIG,IGNORE_CASE_DEFAULT);
+ inputPattern = Pattern.compile(regex, Pattern.DOTALL + (regexIgnoreCase ? Pattern.CASE_INSENSITIVE : 0));
+ }
+
+
+ /**
+ *
+ */
+ @Override
+ public void doInitialize() throws SQLException {
+ // NO-OP
+ }
+
+
+ @Override
+ public void upsertEvents(List<Event> events) throws SQLException {
+ Preconditions.checkNotNull(events);
+ Preconditions.checkNotNull(connection);
+ Preconditions.checkNotNull(this.upsertStatement);
+
+ boolean wasAutoCommit = connection.getAutoCommit();
+ connection.setAutoCommit(false);
+ try (PreparedStatement colUpsert = connection.prepareStatement(upsertStatement)) {
+ String value = null;
+ Integer sqlType = null;
+ for(Event event : events) {
+ byte [] payloadBytes = event.getBody();
+ if(payloadBytes == null || payloadBytes.length == 0) {
+ continue;
+ }
+ String payload = new String(payloadBytes);
+ Matcher m = inputPattern.matcher(payload.trim());
+
+ if (!m.matches()) {
+ logger.debug("payload {} doesn't match the pattern {} ", payload, inputPattern.toString());
+ continue;
+ }
+ if (m.groupCount() != colNames.size()) {
+ logger.debug("payload {} size doesn't match the pattern {} ", m.groupCount(), colNames.size());
+ continue;
+ }
+ int index = 1 ;
+ int offset = 0;
+ for (int i = 0 ; i < colNames.size() ; i++,offset++) {
+ if (columnMetadata[offset] == null ) {
+ continue;
+ }
+
+ value = m.group(i + 1);
+ sqlType = columnMetadata[offset].getSqlType();
+ Object upsertValue = PDataType.fromTypeId(sqlType).toObject(value);
+ if (upsertValue != null) {
+ colUpsert.setObject(index++, upsertValue, sqlType);
+ } else {
+ colUpsert.setNull(index++, sqlType);
+ }
+ }
+
+ //add headers if necessary
+ Map<String,String> headerValues = event.getHeaders();
+ for(int i = 0 ; i < headers.size() ; i++ , offset++) {
+
+ String headerName = headers.get(i);
+ String headerValue = headerValues.get(headerName);
+ sqlType = columnMetadata[offset].getSqlType();
+ Object upsertValue = PDataType.fromTypeId(sqlType).toObject(headerValue);
+ if (upsertValue != null) {
+ colUpsert.setObject(index++, upsertValue, sqlType);
+ } else {
+ colUpsert.setNull(index++, sqlType);
+ }
+ }
+
+ if(autoGenerateKey) {
+ sqlType = columnMetadata[offset].getSqlType();
+ String generatedRowValue = this.keyGenerator.generate();
+ Object rowkeyValue = PDataType.fromTypeId(sqlType).toObject(generatedRowValue);
+ colUpsert.setObject(index++, rowkeyValue ,sqlType);
+ }
+ colUpsert.execute();
+ }
+ connection.commit();
+ } catch(Exception ex){
+ logger.error("An error {} occurred during persisting the event ",ex.getMessage());
+ throw new SQLException(ex.getMessage());
+ } finally {
+ if(wasAutoCommit) {
+ connection.setAutoCommit(true);
+ }
+ }
+
+ }
+
+}
diff --git a/phoenix-flume/src/main/java/org/apache/phoenix/flume/sink/PhoenixSink.java b/phoenix-flume/src/main/java/org/apache/phoenix/flume/sink/PhoenixSink.java
new file mode 100644
index 0000000..2b102a2
--- /dev/null
+++ b/phoenix-flume/src/main/java/org/apache/phoenix/flume/sink/PhoenixSink.java
@@ -0,0 +1,212 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.phoenix.flume.sink;
+
+import java.sql.SQLException;
+import java.util.List;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicInteger;
+
+import org.apache.flume.Channel;
+import org.apache.flume.ChannelException;
+import org.apache.flume.Context;
+import org.apache.flume.Event;
+import org.apache.flume.EventDeliveryException;
+import org.apache.flume.Transaction;
+import org.apache.flume.conf.Configurable;
+import org.apache.flume.instrumentation.SinkCounter;
+import org.apache.flume.sink.AbstractSink;
+import org.apache.phoenix.flume.FlumeConstants;
+import org.apache.phoenix.flume.serializer.EventSerializer;
+import org.apache.phoenix.flume.serializer.EventSerializers;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.google.common.base.Joiner;
+import com.google.common.base.Preconditions;
+import com.google.common.base.Throwables;
+import com.google.common.collect.Lists;
+
+public class PhoenixSink extends AbstractSink implements Configurable {
+ private static final Logger logger = LoggerFactory.getLogger(PhoenixSink.class);
+ private static AtomicInteger counter = new AtomicInteger();
+ private static final String NAME = "Phoenix Sink__";
+
+ private SinkCounter sinkCounter;
+ private Integer batchSize;
+ private EventSerializer serializer;
+
+ public PhoenixSink(){
+ }
+
+ @Override
+ public void configure(Context context){
+ this.setName(NAME + counter.incrementAndGet());
+ this.batchSize = context.getInteger(FlumeConstants.CONFIG_BATCHSIZE, FlumeConstants.DEFAULT_BATCH_SIZE);
+ final String eventSerializerType = context.getString(FlumeConstants.CONFIG_SERIALIZER);
+
+ Preconditions.checkNotNull(eventSerializerType,"Event serializer cannot be empty, please specify in the configuration file");
+ initializeSerializer(context,eventSerializerType);
+ this.sinkCounter = new SinkCounter(this.getName());
+ }
+
+ /**
+ * Initializes the serializer for flume events.
+ * @param eventSerializerType
+ */
+ private void initializeSerializer(final Context context,final String eventSerializerType) {
+ String serializerClazz = null;
+ EventSerializers eventSerializer = null;
+
+ try {
+ eventSerializer = EventSerializers.valueOf(eventSerializerType.toUpperCase());
+ } catch(IllegalArgumentException iae) {
+ serializerClazz = eventSerializerType;
+ }
+
+ final Context serializerContext = new Context();
+ serializerContext.putAll(context.getSubProperties(FlumeConstants.CONFIG_SERIALIZER_PREFIX));
+ copyPropertiesToSerializerContext(context,serializerContext);
+
+ try {
+ @SuppressWarnings("unchecked")
+ Class<? extends EventSerializer> clazz = null;
+ if(serializerClazz == null) {
+ clazz = (Class<? extends EventSerializer>) Class.forName(eventSerializer.getClassName());
+ }
+ else {
+ clazz = (Class<? extends EventSerializer>) Class.forName(serializerClazz);
+ }
+
+ serializer = clazz.newInstance();
+ serializer.configure(serializerContext);
+
+ } catch (Exception e) {
+ logger.error("Could not instantiate event serializer." , e);
+ Throwables.propagate(e);
+ }
+ }
+
+ private void copyPropertiesToSerializerContext(Context context, Context serializerContext) {
+
+ serializerContext.put(FlumeConstants.CONFIG_TABLE_DDL,context.getString(FlumeConstants.CONFIG_TABLE_DDL));
+ serializerContext.put(FlumeConstants.CONFIG_TABLE,context.getString(FlumeConstants.CONFIG_TABLE));
+ serializerContext.put(FlumeConstants.CONFIG_ZK_QUORUM,context.getString(FlumeConstants.CONFIG_ZK_QUORUM));
+ serializerContext.put(FlumeConstants.CONFIG_JDBC_URL,context.getString(FlumeConstants.CONFIG_JDBC_URL));
+ serializerContext.put(FlumeConstants.CONFIG_BATCHSIZE,context.getString(FlumeConstants.CONFIG_BATCHSIZE));
+ }
+
+ @Override
+ public void start() {
+ logger.info("Starting sink {} ",this.getName());
+ sinkCounter.start();
+ try {
+ serializer.initialize();
+ sinkCounter.incrementConnectionCreatedCount();
+ } catch(Exception ex) {
+ sinkCounter.incrementConnectionFailedCount();
+ logger.error("Error {} in initializing the serializer.",ex.getMessage());
+ Throwables.propagate(ex);
+ }
+ super.start();
+ }
+
+ @Override
+ public void stop(){
+ super.stop();
+ try {
+ serializer.close();
+ } catch (SQLException e) {
+ logger.error(" Error while closing connection {} for sink {} ",e.getMessage(),this.getName());
+ }
+ sinkCounter.incrementConnectionClosedCount();
+ sinkCounter.stop();
+ }
+
+ @Override
+ public Status process() throws EventDeliveryException {
+
+ Status status = Status.READY;
+ Channel channel = getChannel();
+ Transaction transaction = null;
+ List<Event> events = Lists.newArrayListWithExpectedSize(this.batchSize);
+ long startTime = System.nanoTime();
+ try {
+ transaction = channel.getTransaction();
+ transaction.begin();
+
+ for(long i = 0; i < this.batchSize; i++) {
+ Event event = channel.take();
+ if(event == null){
+ status = Status.BACKOFF;
+ if (i == 0) {
+ sinkCounter.incrementBatchEmptyCount();
+ } else {
+ sinkCounter.incrementBatchUnderflowCount();
+ }
+ break;
+ } else {
+ events.add(event);
+ }
+ }
+ if (!events.isEmpty()) {
+ if (events.size() == this.batchSize) {
+ sinkCounter.incrementBatchCompleteCount();
+ }
+ else {
+ sinkCounter.incrementBatchUnderflowCount();
+ status = Status.BACKOFF;
+ }
+ // save to Hbase
+ serializer.upsertEvents(events);
+ sinkCounter.addToEventDrainSuccessCount(events.size());
+ }
+ else {
+ logger.debug("no events to process ");
+ sinkCounter.incrementBatchEmptyCount();
+ status = Status.BACKOFF;
+ }
+ transaction.commit();
+ } catch (ChannelException e) {
+ transaction.rollback();
+ status = Status.BACKOFF;
+ sinkCounter.incrementConnectionFailedCount();
+ }
+ catch (SQLException e) {
+ sinkCounter.incrementConnectionFailedCount();
+ transaction.rollback();
+ logger.error("exception while persisting to Hbase ", e);
+ throw new EventDeliveryException("Failed to persist message to Hbase", e);
+ }
+ catch (Throwable e) {
+ transaction.rollback();
+ logger.error("exception while processing in Phoenix Sink", e);
+ throw new EventDeliveryException("Failed to persist message", e);
+ }
+ finally {
+ logger.info(String.format("Time taken to process [%s] events was [%s] seconds",
+ events.size(),
+ TimeUnit.SECONDS.convert(System.nanoTime() - startTime, TimeUnit.NANOSECONDS)));
+ if( transaction != null ) {
+ transaction.close();
+ }
+ }
+ return status;
+ }
+
+}
diff --git a/phoenix-hive/pom.xml b/phoenix-hive/pom.xml
new file mode 100644
index 0000000..705ad34
--- /dev/null
+++ b/phoenix-hive/pom.xml
@@ -0,0 +1,192 @@
+<?xml version='1.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.
+
+-->
+
+<project xmlns="http://maven.apache.org/POM/4.0.0"
+ xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+ xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+ <modelVersion>4.0.0</modelVersion>
+ <parent>
+ <groupId>org.apache.phoenix</groupId>
+ <artifactId>phoenix-connectors</artifactId>
+ <version>1.0.0-SNAPSHOT</version>
+ </parent>
+ <artifactId>phoenix-hive</artifactId>
+ <name>Phoenix - Hive</name>
+
+ <dependencies>
+ <dependency>
+ <groupId>org.apache.phoenix</groupId>
+ <artifactId>phoenix-core</artifactId>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.hive</groupId>
+ <artifactId>hive-cli</artifactId>
+ <version>${hive.version}</version>
+ <scope>provided</scope>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.hive</groupId>
+ <artifactId>hive-exec</artifactId>
+ <version>${hive.version}</version>
+ <scope>provided</scope>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.hadoop</groupId>
+ <artifactId>hadoop-common</artifactId>
+ <scope>provided</scope>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.hadoop</groupId>
+ <artifactId>hadoop-mapreduce-client-core</artifactId>
+ </dependency>
+
+ <!-- Test dependencies -->
+ <dependency>
+ <groupId>org.apache.phoenix</groupId>
+ <artifactId>phoenix-core</artifactId>
+ <classifier>tests</classifier>
+ <scope>test</scope>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.hbase</groupId>
+ <artifactId>hbase-testing-util</artifactId>
+ <scope>test</scope>
+ <optional>true</optional>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.hbase</groupId>
+ <artifactId>hbase-it</artifactId>
+ <type>test-jar</type>
+ <scope>test</scope>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.hadoop</groupId>
+ <artifactId>hadoop-hdfs</artifactId>
+ <scope>test</scope>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.hadoop</groupId>
+ <artifactId>hadoop-hdfs</artifactId>
+ <type>test-jar</type>
+ <scope>test</scope>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.hadoop</groupId>
+ <artifactId>hadoop-auth</artifactId>
+ <scope>test</scope>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.hadoop</groupId>
+ <artifactId>hadoop-mapreduce-client-common</artifactId>
+ <scope>test</scope>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.hadoop</groupId>
+ <artifactId>hadoop-mapreduce-client-jobclient</artifactId>
+ <scope>test</scope>
+ </dependency>
+
+ <dependency>
+ <groupId>org.apache.hadoop</groupId>
+ <artifactId>hadoop-minicluster</artifactId>
+ <scope>test</scope>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.tez</groupId>
+ <artifactId>tez-tests</artifactId>
+ <scope>test</scope>
+ <version>0.8.4</version>
+ <type>test-jar</type>
+ <exclusions>
+ <exclusion>
+ <groupId>org.apache.hadoop</groupId>
+ <artifactId>hadoop-yarn-api</artifactId>
+ </exclusion>
+ </exclusions>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.tez</groupId>
+ <artifactId>tez-dag</artifactId>
+ <scope>test</scope>
+ <version>0.8.4</version>
+ <exclusions>
+ <exclusion>
+ <groupId>org.apache.hadoop</groupId>
+ <artifactId>hadoop-yarn-api</artifactId>
+ </exclusion>
+ </exclusions>
+ </dependency>
+ <dependency>
+ <groupId>org.mockito</groupId>
+ <artifactId>mockito-all</artifactId>
+ <version>${mockito-all.version}</version>
+ <scope>test</scope>
+ </dependency>
+ </dependencies>
+
+ <build>
+ <plugins>
+ <plugin>
+ <groupId>org.codehaus.mojo</groupId>
+ <artifactId>build-helper-maven-plugin</artifactId>
+ </plugin>
+ <plugin>
+ <groupId>org.apache.maven.plugins</groupId>
+ <artifactId>maven-failsafe-plugin</artifactId>
+ </plugin>
+ <plugin>
+ <artifactId>maven-dependency-plugin</artifactId>
+ <version>${maven-dependency-plugin.version}</version>
+ <executions>
+ <execution>
+ <id>copy-dependencies</id>
+ <phase>package</phase>
+ <goals>
+ <goal>copy-dependencies</goal>
+ </goals>
+ </execution>
+ </executions>
+ </plugin>
+ <plugin>
+ <artifactId>maven-assembly-plugin</artifactId>
+ <configuration>
+ <descriptorRefs>
+ <descriptorRef>jar-with-dependencies</descriptorRef>
+ </descriptorRefs>
+ </configuration>
+ <executions>
+ <execution>
+ <id>make-jar-with-dependencies</id>
+ <phase>package</phase>
+ <goals>
+ <goal>single</goal>
+ </goals>
+ <configuration>
+ <appendAssemblyId>false</appendAssemblyId>
+ <finalName>phoenix-${project.version}-hive</finalName>
+ </configuration>
+ </execution>
+ </executions>
+ </plugin>
+ </plugins>
+ </build>
+</project>
diff --git a/phoenix-hive/pom.xml~ b/phoenix-hive/pom.xml~
new file mode 100644
index 0000000..0918e67
--- /dev/null
+++ b/phoenix-hive/pom.xml~
@@ -0,0 +1,192 @@
+<?xml version='1.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.
+
+-->
+
+<project xmlns="http://maven.apache.org/POM/4.0.0"
+ xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+ xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+ <modelVersion>4.0.0</modelVersion>
+ <parent>
+ <groupId>org.apache.phoenix</groupId>
+ <artifactId>phoenix</artifactId>
+ <version>1.0.0-SNAPSHOT</version>
+ </parent>
+ <artifactId>phoenix-hive</artifactId>
+ <name>Phoenix - Hive</name>
+
+ <dependencies>
+ <dependency>
+ <groupId>org.apache.phoenix</groupId>
+ <artifactId>phoenix-core</artifactId>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.hive</groupId>
+ <artifactId>hive-cli</artifactId>
+ <version>${hive.version}</version>
+ <scope>provided</scope>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.hive</groupId>
+ <artifactId>hive-exec</artifactId>
+ <version>${hive.version}</version>
+ <scope>provided</scope>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.hadoop</groupId>
+ <artifactId>hadoop-common</artifactId>
+ <scope>provided</scope>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.hadoop</groupId>
+ <artifactId>hadoop-mapreduce-client-core</artifactId>
+ </dependency>
+
+ <!-- Test dependencies -->
+ <dependency>
+ <groupId>org.apache.phoenix</groupId>
+ <artifactId>phoenix-core</artifactId>
+ <classifier>tests</classifier>
+ <scope>test</scope>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.hbase</groupId>
+ <artifactId>hbase-testing-util</artifactId>
+ <scope>test</scope>
+ <optional>true</optional>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.hbase</groupId>
+ <artifactId>hbase-it</artifactId>
+ <type>test-jar</type>
+ <scope>test</scope>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.hadoop</groupId>
+ <artifactId>hadoop-hdfs</artifactId>
+ <scope>test</scope>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.hadoop</groupId>
+ <artifactId>hadoop-hdfs</artifactId>
+ <type>test-jar</type>
+ <scope>test</scope>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.hadoop</groupId>
+ <artifactId>hadoop-auth</artifactId>
+ <scope>test</scope>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.hadoop</groupId>
+ <artifactId>hadoop-mapreduce-client-common</artifactId>
+ <scope>test</scope>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.hadoop</groupId>
+ <artifactId>hadoop-mapreduce-client-jobclient</artifactId>
+ <scope>test</scope>
+ </dependency>
+
+ <dependency>
+ <groupId>org.apache.hadoop</groupId>
+ <artifactId>hadoop-minicluster</artifactId>
+ <scope>test</scope>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.tez</groupId>
+ <artifactId>tez-tests</artifactId>
+ <scope>test</scope>
+ <version>0.8.4</version>
+ <type>test-jar</type>
+ <exclusions>
+ <exclusion>
+ <groupId>org.apache.hadoop</groupId>
+ <artifactId>hadoop-yarn-api</artifactId>
+ </exclusion>
+ </exclusions>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.tez</groupId>
+ <artifactId>tez-dag</artifactId>
+ <scope>test</scope>
+ <version>0.8.4</version>
+ <exclusions>
+ <exclusion>
+ <groupId>org.apache.hadoop</groupId>
+ <artifactId>hadoop-yarn-api</artifactId>
+ </exclusion>
+ </exclusions>
+ </dependency>
+ <dependency>
+ <groupId>org.mockito</groupId>
+ <artifactId>mockito-all</artifactId>
+ <version>${mockito-all.version}</version>
+ <scope>test</scope>
+ </dependency>
+ </dependencies>
+
+ <build>
+ <plugins>
+ <plugin>
+ <groupId>org.codehaus.mojo</groupId>
+ <artifactId>build-helper-maven-plugin</artifactId>
+ </plugin>
+ <plugin>
+ <groupId>org.apache.maven.plugins</groupId>
+ <artifactId>maven-failsafe-plugin</artifactId>
+ </plugin>
+ <plugin>
+ <artifactId>maven-dependency-plugin</artifactId>
+ <version>${maven-dependency-plugin.version}</version>
+ <executions>
+ <execution>
+ <id>copy-dependencies</id>
+ <phase>package</phase>
+ <goals>
+ <goal>copy-dependencies</goal>
+ </goals>
+ </execution>
+ </executions>
+ </plugin>
+ <plugin>
+ <artifactId>maven-assembly-plugin</artifactId>
+ <configuration>
+ <descriptorRefs>
+ <descriptorRef>jar-with-dependencies</descriptorRef>
+ </descriptorRefs>
+ </configuration>
+ <executions>
+ <execution>
+ <id>make-jar-with-dependencies</id>
+ <phase>package</phase>
+ <goals>
+ <goal>single</goal>
+ </goals>
+ <configuration>
+ <appendAssemblyId>false</appendAssemblyId>
+ <finalName>phoenix-${project.version}-hive</finalName>
+ </configuration>
+ </execution>
+ </executions>
+ </plugin>
+ </plugins>
+ </build>
+</project>
diff --git a/phoenix-hive/src/it/java/org/apache/phoenix/hive/BaseHivePhoenixStoreIT.java b/phoenix-hive/src/it/java/org/apache/phoenix/hive/BaseHivePhoenixStoreIT.java
new file mode 100644
index 0000000..c705e2d
--- /dev/null
+++ b/phoenix-hive/src/it/java/org/apache/phoenix/hive/BaseHivePhoenixStoreIT.java
@@ -0,0 +1,168 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.phoenix.hive;
+
+import org.apache.commons.io.FileUtils;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.HBaseTestingUtility;
+import org.apache.hadoop.hbase.MiniHBaseCluster;
+import org.apache.hadoop.hdfs.MiniDFSCluster;
+import org.apache.phoenix.end2end.NeedsOwnMiniClusterTest;
+import org.apache.phoenix.jdbc.PhoenixDriver;
+import org.apache.phoenix.query.QueryServices;
+import org.apache.phoenix.util.PhoenixRuntime;
+import org.apache.phoenix.util.PropertiesUtil;
+import org.apache.phoenix.util.TestUtil;
+import org.junit.AfterClass;
+import org.junit.experimental.categories.Category;
+
+import java.io.File;
+import java.io.IOException;
+import java.sql.*;
+import java.util.Properties;
+
+import static org.apache.phoenix.query.BaseTest.setUpConfigForMiniCluster;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+/**
+ * Base class for all Hive Phoenix integration tests that may be run with Tez or MR mini cluster
+ */
+@Category(NeedsOwnMiniClusterTest.class)
+public class BaseHivePhoenixStoreIT {
+
+ private static final Log LOG = LogFactory.getLog(BaseHivePhoenixStoreIT.class);
+ protected static HBaseTestingUtility hbaseTestUtil;
+ protected static MiniHBaseCluster hbaseCluster;
+ private static String zkQuorum;
+ protected static Connection conn;
+ private static Configuration conf;
+ protected static HiveTestUtil qt;
+ protected static String hiveOutputDir;
+ protected static String hiveLogDir;
+
+
+ public static void setup(HiveTestUtil.MiniClusterType clusterType)throws Exception {
+ String hadoopConfDir = System.getenv("HADOOP_CONF_DIR");
+ if (null != hadoopConfDir && !hadoopConfDir.isEmpty()) {
+ LOG.warn("WARNING: HADOOP_CONF_DIR is set in the environment which may cause "
+ + "issues with test execution via MiniDFSCluster");
+ }
+ hbaseTestUtil = new HBaseTestingUtility();
+ conf = hbaseTestUtil.getConfiguration();
+ setUpConfigForMiniCluster(conf);
+ conf.set(QueryServices.DROP_METADATA_ATTRIB, Boolean.toString(true));
+ hiveOutputDir = new Path(hbaseTestUtil.getDataTestDir(), "hive_output").toString();
+ File outputDir = new File(hiveOutputDir);
+ outputDir.mkdirs();
+ hiveLogDir = new Path(hbaseTestUtil.getDataTestDir(), "hive_log").toString();
+ File logDir = new File(hiveLogDir);
+ logDir.mkdirs();
+ // Setup Hive mini Server
+ Path testRoot = hbaseTestUtil.getDataTestDir();
+ System.setProperty("test.tmp.dir", testRoot.toString());
+ System.setProperty("test.warehouse.dir", (new Path(testRoot, "warehouse")).toString());
+
+ try {
+ qt = new HiveTestUtil(hiveOutputDir, hiveLogDir, clusterType, null);
+ } catch (Exception e) {
+ LOG.error("Unexpected exception in setup", e);
+ fail("Unexpected exception in setup");
+ }
+
+ //Start HBase cluster
+ hbaseCluster = hbaseTestUtil.startMiniCluster(1);
+ MiniDFSCluster x = hbaseTestUtil.getDFSCluster();
+ Class.forName(PhoenixDriver.class.getName());
+ zkQuorum = "localhost:" + hbaseTestUtil.getZkCluster().getClientPort();
+ Properties props = PropertiesUtil.deepCopy(TestUtil.TEST_PROPERTIES);
+ props.put(QueryServices.DROP_METADATA_ATTRIB, Boolean.toString(true));
+ conn = DriverManager.getConnection(PhoenixRuntime.JDBC_PROTOCOL +
+ PhoenixRuntime.JDBC_PROTOCOL_SEPARATOR + zkQuorum, props);
+ // Setup Hive Output Folder
+
+ Statement stmt = conn.createStatement();
+ stmt.execute("create table t(a integer primary key,b varchar)");
+ }
+
+ protected void runTest(String fname, String fpath) throws Exception {
+ long startTime = System.currentTimeMillis();
+ try {
+ LOG.info("Begin query: " + fname);
+ qt.addFile(fpath);
+
+ if (qt.shouldBeSkipped(fname)) {
+ LOG.info("Test " + fname + " skipped");
+ return;
+ }
+
+ qt.cliInit(fname);
+ qt.clearTestSideEffects();
+ int ecode = qt.executeClient(fname);
+ if (ecode != 0) {
+ qt.failed(ecode, fname, null);
+ return;
+ }
+
+ ecode = qt.checkCliDriverResults(fname);
+ if (ecode != 0) {
+ qt.failedDiff(ecode, fname, null);
+ }
+ qt.clearPostTestEffects();
+
+ } catch (Throwable e) {
+ qt.failed(e, fname, null);
+ }
+
+ long elapsedTime = System.currentTimeMillis() - startTime;
+ LOG.info("Done query: " + fname + " elapsedTime=" + elapsedTime / 1000 + "s");
+ assertTrue("Test passed", true);
+ }
+
+ protected void createFile(String content, String fullName) throws IOException {
+ FileUtils.write(new File(fullName), content);
+ }
+
+ @AfterClass
+ public static void tearDownAfterClass() throws Exception {
+ if (qt != null) {
+ try {
+ qt.shutdown();
+ } catch (Exception e) {
+ LOG.error("Unexpected exception in setup", e);
+ fail("Unexpected exception in tearDown");
+ }
+ }
+ try {
+ conn.close();
+ } finally {
+ try {
+ PhoenixDriver.INSTANCE.close();
+ } finally {
+ try {
+ DriverManager.deregisterDriver(PhoenixDriver.INSTANCE);
+ } finally {
+ hbaseTestUtil.shutdownMiniCluster();
+ }
+ }
+ }
+ }
+}
diff --git a/phoenix-hive/src/it/java/org/apache/phoenix/hive/HiveMapReduceIT.java b/phoenix-hive/src/it/java/org/apache/phoenix/hive/HiveMapReduceIT.java
new file mode 100644
index 0000000..4bc5a7d
--- /dev/null
+++ b/phoenix-hive/src/it/java/org/apache/phoenix/hive/HiveMapReduceIT.java
@@ -0,0 +1,41 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.phoenix.hive;
+
+import static org.junit.Assert.fail;
+
+import java.util.Map;
+
+import org.apache.phoenix.end2end.NeedsOwnMiniClusterTest;
+import org.junit.BeforeClass;
+import org.junit.experimental.categories.Category;
+import org.junit.Ignore;
+
+@Category(NeedsOwnMiniClusterTest.class)
+public class HiveMapReduceIT extends HivePhoenixStoreIT {
+
+ @BeforeClass
+ public static void setUpBeforeClass() throws Exception {
+ final String hadoopConfDir = System.getenv("HADOOP_CONF_DIR");
+ if (hadoopConfDir != null && hadoopConfDir.length() != 0) {
+ fail("HADOOP_CONF_DIR is non-empty in the current shell environment which will very likely cause this test to fail.");
+ }
+ setup(HiveTestUtil.MiniClusterType.mr);
+ }
+}
diff --git a/phoenix-hive/src/it/java/org/apache/phoenix/hive/HivePhoenixStoreIT.java b/phoenix-hive/src/it/java/org/apache/phoenix/hive/HivePhoenixStoreIT.java
new file mode 100644
index 0000000..66f99ad
--- /dev/null
+++ b/phoenix-hive/src/it/java/org/apache/phoenix/hive/HivePhoenixStoreIT.java
@@ -0,0 +1,341 @@
+/*
+ * 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.phoenix.hive;
+
+import org.apache.hadoop.fs.Path;
+import org.apache.phoenix.end2end.NeedsOwnMiniClusterTest;
+import org.apache.phoenix.util.StringUtil;
+import org.junit.Ignore;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+import java.sql.PreparedStatement;
+import java.sql.ResultSet;
+
+import static org.junit.Assert.assertTrue;
+
+/**
+ * Test methods only. All supporting methods should be placed to BaseHivePhoenixStoreIT
+ */
+
+@Category(NeedsOwnMiniClusterTest.class)
+@Ignore("This class contains only test methods and should not be executed directly")
+public class HivePhoenixStoreIT extends BaseHivePhoenixStoreIT {
+
+ /**
+ * Create a table with two column, insert 1 row, check that phoenix table is created and
+ * the row is there
+ *
+ * @throws Exception
+ */
+ @Test
+ public void simpleTest() throws Exception {
+ String testName = "simpleTest";
+ hbaseTestUtil.getTestFileSystem().createNewFile(new Path(hiveLogDir, testName + ".out"));
+ createFile(StringUtil.EMPTY_STRING, new Path(hiveLogDir, testName + ".out").toString());
+ createFile(StringUtil.EMPTY_STRING, new Path(hiveOutputDir, testName + ".out").toString());
+ StringBuilder sb = new StringBuilder();
+ sb.append("CREATE TABLE phoenix_table(ID STRING, SALARY STRING)" + HiveTestUtil.CRLF +
+ " STORED BY \"org.apache.phoenix.hive.PhoenixStorageHandler\"" + HiveTestUtil
+ .CRLF + " TBLPROPERTIES(" + HiveTestUtil.CRLF +
+ " 'phoenix.table.name'='phoenix_table'," + HiveTestUtil.CRLF +
+ " 'phoenix.zookeeper.znode.parent'='/hbase'," + HiveTestUtil.CRLF +
+ " 'phoenix.zookeeper.quorum'='localhost'," + HiveTestUtil.CRLF +
+ " 'phoenix.zookeeper.client.port'='" +
+ hbaseTestUtil.getZkCluster().getClientPort() + "'," + HiveTestUtil.CRLF +
+ " 'phoenix.rowkeys'='id');");
+ sb.append("INSERT INTO TABLE phoenix_table" + HiveTestUtil.CRLF +
+ "VALUES ('10', '1000');" + HiveTestUtil.CRLF);
+ String fullPath = new Path(hbaseTestUtil.getDataTestDir(), testName).toString();
+ createFile(sb.toString(), fullPath);
+ runTest(testName, fullPath);
+
+ String phoenixQuery = "SELECT * FROM phoenix_table";
+ PreparedStatement statement = conn.prepareStatement(phoenixQuery);
+ ResultSet rs = statement.executeQuery();
+ assert (rs.getMetaData().getColumnCount() == 2);
+ assertTrue(rs.next());
+ assert (rs.getString(1).equals("10"));
+ assert (rs.getString(2).equals("1000"));
+ }
+
+ /**
+ * Create hive table with custom column mapping
+ * @throws Exception
+ */
+
+ @Test
+ public void simpleColumnMapTest() throws Exception {
+ String testName = "cmTest";
+ hbaseTestUtil.getTestFileSystem().createNewFile(new Path(hiveLogDir, testName + ".out"));
+ createFile(StringUtil.EMPTY_STRING, new Path(hiveLogDir, testName + ".out").toString());
+ createFile(StringUtil.EMPTY_STRING, new Path(hiveOutputDir, testName + ".out").toString());
+ StringBuilder sb = new StringBuilder();
+ sb.append("CREATE TABLE column_table(ID STRING, P1 STRING, p2 STRING)" + HiveTestUtil.CRLF +
+ " STORED BY \"org.apache.phoenix.hive.PhoenixStorageHandler\"" + HiveTestUtil
+ .CRLF + " TBLPROPERTIES(" + HiveTestUtil.CRLF +
+ " 'phoenix.table.name'='column_table'," + HiveTestUtil.CRLF +
+ " 'phoenix.zookeeper.znode.parent'='/hbase'," + HiveTestUtil.CRLF +
+ " 'phoenix.column.mapping' = 'id:C1, p1:c2, p2:C3'," + HiveTestUtil.CRLF +
+ " 'phoenix.zookeeper.quorum'='localhost'," + HiveTestUtil.CRLF +
+ " 'phoenix.zookeeper.client.port'='" +
+ hbaseTestUtil.getZkCluster().getClientPort() + "'," + HiveTestUtil.CRLF +
+ " 'phoenix.rowkeys'='id');");
+ sb.append("INSERT INTO TABLE column_table" + HiveTestUtil.CRLF +
+ "VALUES ('1', '2', '3');" + HiveTestUtil.CRLF);
+ String fullPath = new Path(hbaseTestUtil.getDataTestDir(), testName).toString();
+ createFile(sb.toString(), fullPath);
+ runTest(testName, fullPath);
+
+ String phoenixQuery = "SELECT C1, \"c2\", C3 FROM column_table";
+ PreparedStatement statement = conn.prepareStatement(phoenixQuery);
+ ResultSet rs = statement.executeQuery();
+ assert (rs.getMetaData().getColumnCount() == 3);
+ assertTrue(rs.next());
+ assert (rs.getString(1).equals("1"));
+ assert (rs.getString(2).equals("2"));
+ assert (rs.getString(3).equals("3"));
+
+ }
+
+
+ /**
+ * Datatype Test
+ *
+ * @throws Exception
+ */
+ @Test
+ public void dataTypeTest() throws Exception {
+ String testName = "dataTypeTest";
+ hbaseTestUtil.getTestFileSystem().createNewFile(new Path(hiveLogDir, testName + ".out"));
+ createFile(StringUtil.EMPTY_STRING, new Path(hiveLogDir, testName + ".out").toString());
+ createFile(StringUtil.EMPTY_STRING, new Path(hiveOutputDir, testName + ".out").toString());
+ StringBuilder sb = new StringBuilder();
+ sb.append("CREATE TABLE phoenix_datatype(ID int, description STRING, ts TIMESTAMP, db " +
+ "DOUBLE,fl FLOAT, us INT)" + HiveTestUtil.CRLF +
+ " STORED BY \"org.apache.phoenix.hive.PhoenixStorageHandler\"" + HiveTestUtil
+ .CRLF + " TBLPROPERTIES(" + HiveTestUtil.CRLF +
+ " 'phoenix.hbase.table.name'='phoenix_datatype'," + HiveTestUtil.CRLF +
+ " 'phoenix.zookeeper.znode.parent'='/hbase'," + HiveTestUtil.CRLF +
+ " 'phoenix.zookeeper.quorum'='localhost'," + HiveTestUtil.CRLF +
+ " 'phoenix.zookeeper.client.port'='" +
+ hbaseTestUtil.getZkCluster().getClientPort() + "'," + HiveTestUtil.CRLF +
+ " 'phoenix.rowkeys'='id');");
+ sb.append("INSERT INTO TABLE phoenix_datatype" + HiveTestUtil.CRLF +
+ "VALUES (10, \"foodesc\", \"2013-01-05 01:01:01\", 200,2.0,-1);" + HiveTestUtil.CRLF);
+ String fullPath = new Path(hbaseTestUtil.getDataTestDir(), testName).toString();
+ createFile(sb.toString(), fullPath);
+ runTest(testName, fullPath);
+
+ String phoenixQuery = "SELECT * FROM phoenix_datatype";
+ PreparedStatement statement = conn.prepareStatement(phoenixQuery);
+ ResultSet rs = statement.executeQuery();
+ assert (rs.getMetaData().getColumnCount() == 6);
+ while (rs.next()) {
+ assert (rs.getInt(1) == 10);
+ assert (rs.getString(2).equalsIgnoreCase("foodesc"));
+ assert (rs.getDouble(4) == 200);
+ assert (rs.getFloat(5) == 2.0);
+ assert (rs.getInt(6) == -1);
+ }
+ }
+
+ /**
+ * Datatype Test
+ *
+ * @throws Exception
+ */
+ @Test
+ public void MultiKey() throws Exception {
+ String testName = "MultiKey";
+ hbaseTestUtil.getTestFileSystem().createNewFile(new Path(hiveLogDir, testName + ".out"));
+ createFile(StringUtil.EMPTY_STRING, new Path(hiveLogDir, testName + ".out").toString());
+ createFile(StringUtil.EMPTY_STRING, new Path(hiveOutputDir, testName + ".out").toString());
+ StringBuilder sb = new StringBuilder();
+ sb.append("CREATE TABLE phoenix_MultiKey(ID int, ID2 String,description STRING," +
+ "db DOUBLE,fl FLOAT, us INT)" + HiveTestUtil.CRLF +
+ " STORED BY \"org.apache.phoenix.hive.PhoenixStorageHandler\"" + HiveTestUtil
+ .CRLF +
+ " TBLPROPERTIES(" + HiveTestUtil.CRLF +
+ " 'phoenix.hbase.table.name'='phoenix_MultiKey'," + HiveTestUtil.CRLF +
+ " 'phoenix.zookeeper.znode.parent'='/hbase'," + HiveTestUtil.CRLF +
+ " 'phoenix.zookeeper.quorum'='localhost'," + HiveTestUtil.CRLF +
+ " 'phoenix.zookeeper.client.port'='" +
+ hbaseTestUtil.getZkCluster().getClientPort() + "'," + HiveTestUtil.CRLF +
+ " 'phoenix.rowkeys'='id,id2');" + HiveTestUtil.CRLF);
+ sb.append("INSERT INTO TABLE phoenix_MultiKey VALUES (10, \"part2\",\"foodesc\",200,2.0,-1);" +
+ HiveTestUtil.CRLF);
+ String fullPath = new Path(hbaseTestUtil.getDataTestDir(), testName).toString();
+ createFile(sb.toString(), fullPath);
+ runTest(testName, fullPath);
+
+ String phoenixQuery = "SELECT * FROM phoenix_MultiKey";
+ PreparedStatement statement = conn.prepareStatement(phoenixQuery);
+ ResultSet rs = statement.executeQuery();
+ assert (rs.getMetaData().getColumnCount() == 6);
+ while (rs.next()) {
+ assert (rs.getInt(1) == 10);
+ assert (rs.getString(2).equalsIgnoreCase("part2"));
+ assert (rs.getString(3).equalsIgnoreCase("foodesc"));
+ assert (rs.getDouble(4) == 200);
+ assert (rs.getFloat(5) == 2.0);
+ assert (rs.getInt(6) == -1);
+ }
+ }
+
+ /**
+ * Test that hive is able to access Phoenix data during MR job (creating two tables and perform join on it)
+ *
+ * @throws Exception
+ */
+ @Test
+ public void testJoinNoColumnMaps() throws Exception {
+ String testName = "testJoin";
+ hbaseTestUtil.getTestFileSystem().createNewFile(new Path(hiveLogDir, testName + ".out"));
+ createFile(StringUtil.EMPTY_STRING, new Path(hiveLogDir, testName + ".out").toString());
+ createFile("10\tpart2\tfoodesc\t200.0\t2.0\t-1\t10\tpart2\tfoodesc\t200.0\t2.0\t-1\n",
+ new Path(hiveOutputDir, testName + ".out").toString());
+ StringBuilder sb = new StringBuilder();
+ sb.append("CREATE TABLE joinTable1(ID int, ID2 String,description STRING," +
+ "db DOUBLE,fl FLOAT, us INT)" + HiveTestUtil.CRLF +
+ " STORED BY \"org.apache.phoenix.hive.PhoenixStorageHandler\"" + HiveTestUtil
+ .CRLF +
+ " TBLPROPERTIES(" + HiveTestUtil.CRLF +
+ " 'phoenix.hbase.table.name'='joinTable1'," + HiveTestUtil.CRLF +
+ " 'phoenix.zookeeper.znode.parent'='/hbase'," + HiveTestUtil.CRLF +
+ " 'phoenix.zookeeper.quorum'='localhost'," + HiveTestUtil.CRLF +
+ " 'phoenix.zookeeper.client.port'='" +
+ hbaseTestUtil.getZkCluster().getClientPort() + "'," + HiveTestUtil.CRLF +
+ " 'phoenix.rowkeys'='id,id2');" + HiveTestUtil.CRLF);
+ sb.append("CREATE TABLE joinTable2(ID int, ID2 String,description STRING," +
+ "db DOUBLE,fl FLOAT, us INT)" + HiveTestUtil.CRLF +
+ " STORED BY \"org.apache.phoenix.hive.PhoenixStorageHandler\"" + HiveTestUtil
+ .CRLF +
+ " TBLPROPERTIES(" + HiveTestUtil.CRLF +
+ " 'phoenix.hbase.table.name'='joinTable2'," + HiveTestUtil.CRLF +
+ " 'phoenix.zookeeper.znode.parent'='/hbase'," + HiveTestUtil.CRLF +
+ " 'phoenix.zookeeper.quorum'='localhost'," + HiveTestUtil.CRLF +
+ " 'phoenix.zookeeper.client.port'='" +
+ hbaseTestUtil.getZkCluster().getClientPort() + "'," + HiveTestUtil.CRLF +
+ " 'phoenix.rowkeys'='id,id2');" + HiveTestUtil.CRLF);
+
+ sb.append("INSERT INTO TABLE joinTable1 VALUES (5, \"part2\",\"foodesc\",200,2.0,-1);" + HiveTestUtil.CRLF);
+ sb.append("INSERT INTO TABLE joinTable1 VALUES (10, \"part2\",\"foodesc\",200,2.0,-1);" + HiveTestUtil.CRLF);
+
+ sb.append("INSERT INTO TABLE joinTable2 VALUES (5, \"part2\",\"foodesc\",200,2.0,-1);" + HiveTestUtil.CRLF);
+ sb.append("INSERT INTO TABLE joinTable2 VALUES (10, \"part2\",\"foodesc\",200,2.0,-1);" + HiveTestUtil.CRLF);
+
+ sb.append("SELECT * from joinTable1 A join joinTable2 B on A.ID = B.ID WHERE A.ID=10;" +
+ HiveTestUtil.CRLF);
+
+ String fullPath = new Path(hbaseTestUtil.getDataTestDir(), testName).toString();
+ createFile(sb.toString(), fullPath);
+ runTest(testName, fullPath);
+ }
+
+ /**
+ * Test that hive is able to access Phoenix data during MR job (creating two tables and perform join on it)
+ *
+ * @throws Exception
+ */
+ @Test
+ public void testJoinColumnMaps() throws Exception {
+ String testName = "testJoin";
+ hbaseTestUtil.getTestFileSystem().createNewFile(new Path(hiveLogDir, testName + ".out"));
+ createFile("10\t200.0\tpart2\n", new Path(hiveOutputDir, testName + ".out").toString());
+ createFile(StringUtil.EMPTY_STRING, new Path(hiveLogDir, testName + ".out").toString());
+
+ StringBuilder sb = new StringBuilder();
+ sb.append("CREATE TABLE joinTable3(ID int, ID2 String,description STRING," +
+ "db DOUBLE,fl FLOAT, us INT)" + HiveTestUtil.CRLF +
+ " STORED BY \"org.apache.phoenix.hive.PhoenixStorageHandler\"" + HiveTestUtil
+ .CRLF +
+ " TBLPROPERTIES(" + HiveTestUtil.CRLF +
+ " 'phoenix.hbase.table.name'='joinTable3'," + HiveTestUtil.CRLF +
+ " 'phoenix.zookeeper.znode.parent'='/hbase'," + HiveTestUtil.CRLF +
+ " 'phoenix.zookeeper.quorum'='localhost'," + HiveTestUtil.CRLF +
+ " 'phoenix.zookeeper.client.port'='" +
+ hbaseTestUtil.getZkCluster().getClientPort() + "'," + HiveTestUtil.CRLF +
+ " 'phoenix.column.mapping' = 'id:i1, id2:I2'," + HiveTestUtil.CRLF +
+ " 'phoenix.rowkeys'='id,id2');" + HiveTestUtil.CRLF);
+ sb.append("CREATE TABLE joinTable4(ID int, ID2 String,description STRING," +
+ "db DOUBLE,fl FLOAT, us INT)" + HiveTestUtil.CRLF +
+ " STORED BY \"org.apache.phoenix.hive.PhoenixStorageHandler\"" + HiveTestUtil
+ .CRLF +
+ " TBLPROPERTIES(" + HiveTestUtil.CRLF +
+ " 'phoenix.hbase.table.name'='joinTable4'," + HiveTestUtil.CRLF +
+ " 'phoenix.zookeeper.znode.parent'='/hbase'," + HiveTestUtil.CRLF +
+ " 'phoenix.zookeeper.quorum'='localhost'," + HiveTestUtil.CRLF +
+ " 'phoenix.zookeeper.client.port'='" +
+ hbaseTestUtil.getZkCluster().getClientPort() + "'," + HiveTestUtil.CRLF +
+ " 'phoenix.column.mapping' = 'id:i1, id2:I2'," + HiveTestUtil.CRLF +
+ " 'phoenix.rowkeys'='id,id2');" + HiveTestUtil.CRLF);
+
+ sb.append("INSERT INTO TABLE joinTable3 VALUES (5, \"part1\",\"foodesc\",200,2.0,-1);" + HiveTestUtil.CRLF);
+ sb.append("INSERT INTO TABLE joinTable3 VALUES (10, \"part1\",\"foodesc\",200,2.0,-1);" + HiveTestUtil.CRLF);
+
+ sb.append("INSERT INTO TABLE joinTable4 VALUES (5, \"part2\",\"foodesc\",200,2.0,-1);" + HiveTestUtil.CRLF);
+ sb.append("INSERT INTO TABLE joinTable4 VALUES (10, \"part2\",\"foodesc\",200,2.0,-1);" + HiveTestUtil.CRLF);
+
+ sb.append("SELECT A.ID, a.db, B.ID2 from joinTable3 A join joinTable4 B on A.ID = B.ID WHERE A.ID=10;" +
+ HiveTestUtil.CRLF);
+
+ String fullPath = new Path(hbaseTestUtil.getDataTestDir(), testName).toString();
+ createFile(sb.toString(), fullPath);
+ runTest(testName, fullPath);
+ //Test that Phoenix has correctly mapped columns. We are checking both, primary key and
+ // regular columns mapped and not mapped
+ String phoenixQuery = "SELECT \"i1\", \"I2\", \"db\" FROM joinTable3 where \"i1\" = 10 AND \"I2\" = 'part1' AND \"db\" = 200";
+ PreparedStatement statement = conn.prepareStatement(phoenixQuery);
+ ResultSet rs = statement.executeQuery();
+ assert (rs.getMetaData().getColumnCount() == 3);
+ while (rs.next()) {
+ assert (rs.getInt(1) == 10);
+ assert (rs.getString(2).equalsIgnoreCase("part1"));
+ assert (rs.getDouble(3) == 200);
+ }
+ }
+
+ @Test
+ public void testTimestampPredicate() throws Exception {
+ String testName = "testTimeStampPredicate";
+ hbaseTestUtil.getTestFileSystem().createNewFile(new Path(hiveLogDir, testName + ".out"));
+ createFile("10\t2013-01-02 01:01:01.123456\n", new Path(hiveOutputDir, testName + ".out").toString());
+ createFile(StringUtil.EMPTY_STRING, new Path(hiveLogDir, testName + ".out").toString());
+
+ StringBuilder sb = new StringBuilder();
+ sb.append("CREATE TABLE timeStampTable(ID int,ts TIMESTAMP)" + HiveTestUtil.CRLF +
+ " STORED BY \"org.apache.phoenix.hive.PhoenixStorageHandler\"" + HiveTestUtil
+ .CRLF +
+ " TBLPROPERTIES(" + HiveTestUtil.CRLF +
+ " 'phoenix.hbase.table.name'='TIMESTAMPTABLE'," + HiveTestUtil.CRLF +
+ " 'phoenix.zookeeper.znode.parent'='/hbase'," + HiveTestUtil.CRLF +
+ " 'phoenix.zookeeper.quorum'='localhost'," + HiveTestUtil.CRLF +
+ " 'phoenix.zookeeper.client.port'='" +
+ hbaseTestUtil.getZkCluster().getClientPort() + "'," + HiveTestUtil.CRLF +
+ " 'phoenix.column.mapping' = 'id:ID, ts:TS'," + HiveTestUtil.CRLF +
+ " 'phoenix.rowkeys'='id');" + HiveTestUtil.CRLF);
+ sb.append("INSERT INTO TABLE timeStampTable VALUES (10, \"2013-01-02 01:01:01.123456\");" + HiveTestUtil.CRLF);
+ sb.append("SELECT * from timeStampTable WHERE ts between '2013-01-02 01:01:01.123455' and " +
+ " '2013-01-02 12:01:02.123457789' AND id = 10;" + HiveTestUtil.CRLF);
+
+ String fullPath = new Path(hbaseTestUtil.getDataTestDir(), testName).toString();
+ createFile(sb.toString(), fullPath);
+ runTest(testName, fullPath);
+ }
+}
diff --git a/phoenix-hive/src/it/java/org/apache/phoenix/hive/HiveTestUtil.java b/phoenix-hive/src/it/java/org/apache/phoenix/hive/HiveTestUtil.java
new file mode 100644
index 0000000..b4c4e46
--- /dev/null
+++ b/phoenix-hive/src/it/java/org/apache/phoenix/hive/HiveTestUtil.java
@@ -0,0 +1,1280 @@
+/*
+ * 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.phoenix.hive;
+
+import com.google.common.collect.ImmutableList;
+import junit.framework.Assert;
+import org.apache.commons.io.FileUtils;
+import org.apache.commons.io.IOUtils;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.zookeeper.MiniZooKeeperCluster;
+import org.apache.hadoop.hive.cli.CliDriver;
+import org.apache.hadoop.hive.cli.CliSessionState;
+import org.apache.hadoop.hive.common.io.CachingPrintStream;
+import org.apache.hadoop.hive.common.io.DigestPrintStream;
+import org.apache.hadoop.hive.common.io.SortAndDigestPrintStream;
+import org.apache.hadoop.hive.common.io.SortPrintStream;
+import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.conf.HiveConf.ConfVars;
+import org.apache.hadoop.hive.metastore.api.Index;
+import org.apache.hadoop.hive.ql.exec.FunctionRegistry;
+import org.apache.hadoop.hive.ql.exec.Utilities;
+import org.apache.hadoop.hive.ql.lockmgr.zookeeper.ZooKeeperHiveLockManager;
+import org.apache.hadoop.hive.ql.metadata.Hive;
+import org.apache.hadoop.hive.ql.metadata.Table;
+import org.apache.hadoop.hive.ql.parse.ASTNode;
+import org.apache.hadoop.hive.ql.parse.BaseSemanticAnalyzer;
+import org.apache.hadoop.hive.ql.parse.ParseDriver;
+import org.apache.hadoop.hive.ql.parse.SemanticAnalyzer;
+import org.apache.hadoop.hive.ql.parse.SemanticException;
+import org.apache.hadoop.hive.ql.session.SessionState;
+import org.apache.hadoop.hive.shims.HadoopShims;
+import org.apache.hadoop.hive.shims.ShimLoader;
+import org.apache.hadoop.util.Shell;
+import org.apache.hive.common.util.StreamPrinter;
+import org.apache.tools.ant.BuildException;
+import org.apache.zookeeper.WatchedEvent;
+import org.apache.zookeeper.Watcher;
+import org.apache.zookeeper.ZooKeeper;
+
+import java.io.BufferedInputStream;
+import java.io.BufferedOutputStream;
+import java.io.BufferedReader;
+import java.io.BufferedWriter;
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.FileNotFoundException;
+import java.io.FileOutputStream;
+import java.io.FilenameFilter;
+import java.io.IOException;
+import java.io.InputStreamReader;
+import java.io.OutputStream;
+import java.io.OutputStreamWriter;
+import java.io.PrintStream;
+import java.io.StringWriter;
+import java.net.URL;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Deque;
+import java.util.HashSet;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Set;
+import java.util.TreeMap;
+import java.util.UUID;
+import java.util.concurrent.TimeUnit;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+
+/**
+ * HiveTestUtil cloned from Hive QTestUtil. Can be outdated and may require update once a problem
+ * found.
+ */
+public class HiveTestUtil {
+
+ public static final String UTF_8 = "UTF-8";
+ private static final Log LOG = LogFactory.getLog("HiveTestUtil");
+ private static final String QTEST_LEAVE_FILES = "QTEST_LEAVE_FILES";
+ public static final String DEFAULT_DATABASE_NAME = "default";
+
+ private String testWarehouse;
+ private final String testFiles;
+ protected final String outDir;
+ protected final String logDir;
+ private final TreeMap<String, String> qMap;
+ private final Set<String> qSkipSet;
+ private final Set<String> qSortSet;
+ private final Set<String> qSortQuerySet;
+ private final Set<String> qHashQuerySet;
+ private final Set<String> qSortNHashQuerySet;
+ private final Set<String> qJavaVersionSpecificOutput;
+ private static final String SORT_SUFFIX = ".sorted";
+ private static MiniClusterType clusterType = MiniClusterType.none;
+ private ParseDriver pd;
+ protected Hive db;
+ protected HiveConf conf;
+ private BaseSemanticAnalyzer sem;
+ protected final boolean overWrite;
+ private CliDriver cliDriver;
+ private HadoopShims.MiniMrShim mr = null;
+ private HadoopShims.MiniDFSShim dfs = null;
+ private String hadoopVer = null;
+ private HiveTestSetup setup = null;
+ private boolean isSessionStateStarted = false;
+ private static final String javaVersion = getJavaVersion();
+
+ private String initScript = "";
+ private String cleanupScript = "";
+
+ public HiveConf getConf() {
+ return conf;
+ }
+
+ public boolean deleteDirectory(File path) {
+ if (path.exists()) {
+ File[] files = path.listFiles();
+ for (File file : files) {
+ if (file.isDirectory()) {
+ deleteDirectory(file);
+ } else {
+ file.delete();
+ }
+ }
+ }
+ return (path.delete());
+ }
+
+ public void copyDirectoryToLocal(Path src, Path dest) throws Exception {
+
+ FileSystem srcFs = src.getFileSystem(conf);
+ FileSystem destFs = dest.getFileSystem(conf);
+ if (srcFs.exists(src)) {
+ FileStatus[] files = srcFs.listStatus(src);
+ for (FileStatus file : files) {
+ String name = file.getPath().getName();
+ Path dfs_path = file.getPath();
+ Path local_path = new Path(dest, name);
+
+ if (file.isDir()) {
+ if (!destFs.exists(local_path)) {
+ destFs.mkdirs(local_path);
+ }
+ copyDirectoryToLocal(dfs_path, local_path);
+ } else {
+ srcFs.copyToLocalFile(dfs_path, local_path);
+ }
+ }
+ }
+ }
+
+ static Pattern mapTok = Pattern.compile("(\\.?)(.*)_map_(.*)");
+ static Pattern reduceTok = Pattern.compile("(.*)(reduce_[^\\.]*)((\\..*)?)");
+
+ public void normalizeNames(File path) throws Exception {
+ if (path.isDirectory()) {
+ File[] files = path.listFiles();
+ for (File file : files) {
+ normalizeNames(file);
+ }
+ } else {
+ Matcher m = reduceTok.matcher(path.getName());
+ if (m.matches()) {
+ String name = m.group(1) + "reduce" + m.group(3);
+ path.renameTo(new File(path.getParent(), name));
+ } else {
+ m = mapTok.matcher(path.getName());
+ if (m.matches()) {
+ String name = m.group(1) + "map_" + m.group(3);
+ path.renameTo(new File(path.getParent(), name));
+ }
+ }
+ }
+ }
+
+ public String getOutputDirectory() {
+ return outDir;
+ }
+
+ public String getLogDirectory() {
+ return logDir;
+ }
+
+ private String getHadoopMainVersion(String input) {
+ if (input == null) {
+ return null;
+ }
+ Pattern p = Pattern.compile("^(\\d+\\.\\d+).*");
+ Matcher m = p.matcher(input);
+ if (m.matches()) {
+ return m.group(1);
+ }
+ return null;
+ }
+
+ public void initConf() throws Exception {
+ // Plug verifying metastore in for testing.
+ conf.setVar(HiveConf.ConfVars.METASTORE_RAW_STORE_IMPL,
+ "org.apache.hadoop.hive.metastore.VerifyingObjectStore");
+
+ if (mr != null) {
+ assert dfs != null;
+
+ mr.setupConfiguration(conf);
+
+ // set fs.default.name to the uri of mini-dfs
+ String dfsUriString = WindowsPathUtil.getHdfsUriString(dfs.getFileSystem().getUri()
+ .toString());
+ conf.setVar(HiveConf.ConfVars.HADOOPFS, dfsUriString);
+ // hive.metastore.warehouse.dir needs to be set relative to the mini-dfs
+ conf.setVar(HiveConf.ConfVars.METASTOREWAREHOUSE,
+ (new Path(dfsUriString,
+ "/build/ql/test/data/warehouse/")).toString());
+ }
+
+ // Windows paths should be converted after MiniMrShim.setupConfiguration()
+ // since setupConfiguration may overwrite configuration values.
+ if (Shell.WINDOWS) {
+ WindowsPathUtil.convertPathsFromWindowsToHdfs(conf);
+ }
+ }
+
+ public enum MiniClusterType {
+ mr,
+ tez,
+ none;
+
+ public static MiniClusterType valueForString(String type) {
+ if (type.equals("miniMR")) {
+ return mr;
+ } else if (type.equals("tez")) {
+ return tez;
+ } else {
+ return none;
+ }
+ }
+ }
+
+ public HiveTestUtil(String outDir, String logDir, MiniClusterType clusterType, String hadoopVer)
+ throws Exception {
+ this(outDir, logDir, clusterType, null, hadoopVer);
+ }
+
+ public HiveTestUtil(String outDir, String logDir, MiniClusterType clusterType, String confDir,
+ String hadoopVer)
+ throws Exception {
+ this.outDir = outDir;
+ this.logDir = logDir;
+ if (confDir != null && !confDir.isEmpty()) {
+ HiveConf.setHiveSiteLocation(new URL("file://" + new File(confDir).toURI().getPath()
+ + "/hive-site.xml"));
+ LOG.info("Setting hive-site: " + HiveConf.getHiveSiteLocation());
+ }
+ conf = new HiveConf();
+ String tmpBaseDir = System.getProperty("test.tmp.dir");
+ if (tmpBaseDir == null || tmpBaseDir == "") {
+ tmpBaseDir = System.getProperty("java.io.tmpdir");
+ }
+ String metaStoreURL = "jdbc:derby:" + tmpBaseDir + File.separator + "metastore_dbtest;" +
+ "create=true";
+ conf.set(ConfVars.METASTORECONNECTURLKEY.varname, metaStoreURL);
+ System.setProperty(HiveConf.ConfVars.METASTORECONNECTURLKEY.varname, metaStoreURL);
+
+ //set where derby logs
+ File derbyLogFile = new File(tmpBaseDir + "/derby.log");
+ derbyLogFile.createNewFile();
+ System.setProperty("derby.stream.error.file", derbyLogFile.getPath());
+
+ this.hadoopVer = getHadoopMainVersion(hadoopVer);
+ qMap = new TreeMap<String, String>();
+ qSkipSet = new HashSet<String>();
+ qSortSet = new HashSet<String>();
+ qSortQuerySet = new HashSet<String>();
+ qHashQuerySet = new HashSet<String>();
+ qSortNHashQuerySet = new HashSet<String>();
+ qJavaVersionSpecificOutput = new HashSet<String>();
+ this.clusterType = clusterType;
+
+ // Using randomUUID for dfs cluster
+ System.setProperty("test.build.data", "target/test-data/hive-" + UUID.randomUUID().toString
+ ());
+
+ HadoopShims shims = ShimLoader.getHadoopShims();
+ int numberOfDataNodes = 1;
+
+ if (clusterType != MiniClusterType.none) {
+ dfs = shims.getMiniDfs(conf, numberOfDataNodes, true, null);
+ FileSystem fs = dfs.getFileSystem();
+ String uriString = WindowsPathUtil.getHdfsUriString(fs.getUri().toString());
+ if (clusterType == MiniClusterType.tez) {
+ conf.set("hive.execution.engine", "tez");
+ mr = shims.getMiniTezCluster(conf, 1, uriString, 1);
+ } else {
+ conf.set("hive.execution.engine", "mr");
+ mr = shims.getMiniMrCluster(conf, 1, uriString, 1);
+
+ }
+ }
+
+ initConf();
+
+ // Use the current directory if it is not specified
+ String dataDir = conf.get("test.data.files");
+ if (dataDir == null) {
+ dataDir = new File(".").getAbsolutePath() + "/data/files";
+ }
+
+ testFiles = dataDir;
+
+ // Use the current directory if it is not specified
+ String scriptsDir = conf.get("test.data.scripts");
+ if (scriptsDir == null) {
+ scriptsDir = new File(".").getAbsolutePath() + "/data/scripts";
+ }
+ if (!initScript.isEmpty()) {
+ this.initScript = scriptsDir + "/" + initScript;
+ }
+ if (!cleanupScript.isEmpty()) {
+ this.cleanupScript = scriptsDir + "/" + cleanupScript;
+ }
+
+ overWrite = "true".equalsIgnoreCase(System.getProperty("test.output.overwrite"));
+
+ setup = new HiveTestSetup();
+ setup.preTest(conf);
+ init();
+ }
+
+ public void shutdown() throws Exception {
+ cleanUp();
+ setup.tearDown();
+ if (mr != null) {
+ mr.shutdown();
+ mr = null;
+ }
+ FileSystem.closeAll();
+ if (dfs != null) {
+ dfs.shutdown();
+ dfs = null;
+ }
+ }
+
+ public String readEntireFileIntoString(File queryFile) throws IOException {
+ InputStreamReader isr = new InputStreamReader(
+ new BufferedInputStream(new FileInputStream(queryFile)), HiveTestUtil.UTF_8);
+ StringWriter sw = new StringWriter();
+ try {
+ IOUtils.copy(isr, sw);
+ } finally {
+ if (isr != null) {
+ isr.close();
+ }
+ }
+ return sw.toString();
+ }
+
+ public void addFile(String queryFile) throws IOException {
+ addFile(queryFile, false);
+ }
+
+ public void addFile(String queryFile, boolean partial) throws IOException {
+ addFile(new File(queryFile));
+ }
+
+ public void addFile(File qf) throws IOException {
+ addFile(qf, false);
+ }
+
+ public void addFile(File qf, boolean partial) throws IOException {
+ String query = readEntireFileIntoString(qf);
+ qMap.put(qf.getName(), query);
+ if (partial) return;
+
+ if (matches(SORT_BEFORE_DIFF, query)) {
+ qSortSet.add(qf.getName());
+ } else if (matches(SORT_QUERY_RESULTS, query)) {
+ qSortQuerySet.add(qf.getName());
+ } else if (matches(HASH_QUERY_RESULTS, query)) {
+ qHashQuerySet.add(qf.getName());
+ } else if (matches(SORT_AND_HASH_QUERY_RESULTS, query)) {
+ qSortNHashQuerySet.add(qf.getName());
+ }
+ }
+
+ private static final Pattern SORT_BEFORE_DIFF = Pattern.compile("-- SORT_BEFORE_DIFF");
+ private static final Pattern SORT_QUERY_RESULTS = Pattern.compile("-- SORT_QUERY_RESULTS");
+ private static final Pattern HASH_QUERY_RESULTS = Pattern.compile("-- HASH_QUERY_RESULTS");
+ private static final Pattern SORT_AND_HASH_QUERY_RESULTS = Pattern.compile("-- " +
+ "SORT_AND_HASH_QUERY_RESULTS");
+
+ private boolean matches(Pattern pattern, String query) {
+ Matcher matcher = pattern.matcher(query);
+ if (matcher.find()) {
+ return true;
+ }
+ return false;
+ }
+
+ /**
+ * Get formatted Java version to include minor version, but
+ * exclude patch level.
+ *
+ * @return Java version formatted as major_version.minor_version
+ */
+ private static String getJavaVersion() {
+ String version = System.getProperty("java.version");
+ if (version == null) {
+ throw new NullPointerException("No java version could be determined " +
+ "from system properties");
+ }
+
+ // "java version" system property is formatted
+ // major_version.minor_version.patch_level.
+ // Find second dot, instead of last dot, to be safe
+ int pos = version.indexOf('.');
+ pos = version.indexOf('.', pos + 1);
+ return version.substring(0, pos);
+ }
+
+ /**
+ * Clear out any side effects of running tests
+ */
+ public void clearPostTestEffects() throws Exception {
+ setup.postTest(conf);
+ }
+
+ /**
+ * Clear out any side effects of running tests
+ */
+ public void clearTablesCreatedDuringTests() throws Exception {
+ if (System.getenv(QTEST_LEAVE_FILES) != null) {
+ return;
+ }
+
+ // Delete any tables other than the source tables
+ // and any databases other than the default database.
+ for (String dbName : db.getAllDatabases()) {
+ SessionState.get().setCurrentDatabase(dbName);
+ for (String tblName : db.getAllTables()) {
+ if (!DEFAULT_DATABASE_NAME.equals(dbName)) {
+ Table tblObj = db.getTable(tblName);
+ // dropping index table can not be dropped directly. Dropping the base
+ // table will automatically drop all its index table
+ if (tblObj.isIndexTable()) {
+ continue;
+ }
+ db.dropTable(dbName, tblName);
+ } else {
+ // this table is defined in srcTables, drop all indexes on it
+ List<Index> indexes = db.getIndexes(dbName, tblName, (short) -1);
+ if (indexes != null && indexes.size() > 0) {
+ for (Index index : indexes) {
+ db.dropIndex(dbName, tblName, index.getIndexName(), true, true);
+ }
+ }
+ }
+ }
+ if (!DEFAULT_DATABASE_NAME.equals(dbName)) {
+ // Drop cascade, may need to drop functions
+ db.dropDatabase(dbName, true, true, true);
+ }
+ }
+
+ // delete remaining directories for external tables (can affect stats for following tests)
+ try {
+ Path p = new Path(testWarehouse);
+ FileSystem fileSystem = p.getFileSystem(conf);
+ if (fileSystem.exists(p)) {
+ for (FileStatus status : fileSystem.listStatus(p)) {
+ if (status.isDir()) {
+ fileSystem.delete(status.getPath(), true);
+ }
+ }
+ }
+ } catch (IllegalArgumentException e) {
+ // ignore.. provides invalid url sometimes intentionally
+ }
+ SessionState.get().setCurrentDatabase(DEFAULT_DATABASE_NAME);
+
+ List<String> roleNames = db.getAllRoleNames();
+ for (String roleName : roleNames) {
+ if (!"PUBLIC".equalsIgnoreCase(roleName) && !"ADMIN".equalsIgnoreCase(roleName)) {
+ db.dropRole(roleName);
+ }
+ }
+ }
+
+ /**
+ * Clear out any side effects of running tests
+ */
+ public void clearTestSideEffects() throws Exception {
+ if (System.getenv(QTEST_LEAVE_FILES) != null) {
+ return;
+ }
+
+ clearTablesCreatedDuringTests();
+ }
+
+ public void cleanUp() throws Exception {
+ if (!isSessionStateStarted) {
+ startSessionState();
+ }
+ if (System.getenv(QTEST_LEAVE_FILES) != null) {
+ return;
+ }
+
+ clearTablesCreatedDuringTests();
+
+ SessionState.get().getConf().setBoolean("hive.test.shutdown.phase", true);
+
+ if (cleanupScript != "") {
+ String cleanupCommands = readEntireFileIntoString(new File(cleanupScript));
+ LOG.info("Cleanup (" + cleanupScript + "):\n" + cleanupCommands);
+ if (cliDriver == null) {
+ cliDriver = new CliDriver();
+ }
+ cliDriver.processLine(cleanupCommands);
+ }
+
+ SessionState.get().getConf().setBoolean("hive.test.shutdown.phase", false);
+
+ // delete any contents in the warehouse dir
+ Path p = new Path(testWarehouse);
+ FileSystem fs = p.getFileSystem(conf);
+
+ try {
+ FileStatus[] ls = fs.listStatus(p);
+ for (int i = 0; (ls != null) && (i < ls.length); i++) {
+ fs.delete(ls[i].getPath(), true);
+ }
+ } catch (FileNotFoundException e) {
+ // Best effort
+ }
+
+ FunctionRegistry.unregisterTemporaryUDF("test_udaf");
+ FunctionRegistry.unregisterTemporaryUDF("test_error");
+ }
+
+ public void createSources() throws Exception {
+ if (!isSessionStateStarted) {
+ startSessionState();
+ }
+ conf.setBoolean("hive.test.init.phase", true);
+
+ if (cliDriver == null) {
+ cliDriver = new CliDriver();
+ }
+ cliDriver.processLine("set test.data.dir=" + testFiles + ";");
+
+ conf.setBoolean("hive.test.init.phase", false);
+ }
+
+ public void init() throws Exception {
+ testWarehouse = conf.getVar(HiveConf.ConfVars.METASTOREWAREHOUSE);
+ conf.setBoolVar(HiveConf.ConfVars.SUBMITLOCALTASKVIACHILD, false);
+ String execEngine = conf.get("hive.execution.engine");
+ conf.set("hive.execution.engine", "mr");
+ SessionState.start(conf);
+ conf.set("hive.execution.engine", execEngine);
+ db = Hive.get(conf);
+ pd = new ParseDriver();
+ sem = new SemanticAnalyzer(conf);
+ }
+
+ public void init(String tname) throws Exception {
+ cleanUp();
+ createSources();
+ cliDriver.processCmd("set hive.cli.print.header=true;");
+ }
+
+ public void cliInit(String tname) throws Exception {
+ cliInit(tname, true);
+ }
+
+ public String cliInit(String tname, boolean recreate) throws Exception {
+ if (recreate) {
+ cleanUp();
+ createSources();
+ }
+
+ HiveConf.setVar(conf, HiveConf.ConfVars.HIVE_AUTHENTICATOR_MANAGER,
+ "org.apache.hadoop.hive.ql.security.HadoopDefaultAuthenticator");
+ Utilities.clearWorkMap();
+ CliSessionState ss = new CliSessionState(conf);
+ assert ss != null;
+ ss.in = System.in;
+
+ String outFileExtension = getOutFileExtension(tname);
+ String stdoutName = null;
+ if (outDir != null) {
+ File qf = new File(outDir, tname);
+ stdoutName = qf.getName().concat(outFileExtension);
+ } else {
+ stdoutName = tname + outFileExtension;
+ }
+
+ File outf = new File(logDir, stdoutName);
+ OutputStream fo = new BufferedOutputStream(new FileOutputStream(outf));
+ if (qSortQuerySet.contains(tname)) {
+ ss.out = new SortPrintStream(fo, "UTF-8");
+ } else if (qHashQuerySet.contains(tname)) {
+ ss.out = new DigestPrintStream(fo, "UTF-8");
+ } else if (qSortNHashQuerySet.contains(tname)) {
+ ss.out = new SortAndDigestPrintStream(fo, "UTF-8");
+ } else {
+ ss.out = new PrintStream(fo, true, "UTF-8");
+ }
+ ss.err = new CachingPrintStream(fo, true, "UTF-8");
+ ss.setIsSilent(true);
+ SessionState oldSs = SessionState.get();
+
+ if (oldSs != null && clusterType == MiniClusterType.tez) {
+ oldSs.close();
+ }
+
+ if (oldSs != null && oldSs.out != null && oldSs.out != System.out) {
+ oldSs.out.close();
+ }
+ SessionState.start(ss);
+
+ cliDriver = new CliDriver();
+ cliDriver.processInitFiles(ss);
+
+ return outf.getAbsolutePath();
+ }
+
+ private CliSessionState startSessionState()
+ throws IOException {
+
+ HiveConf.setVar(conf, HiveConf.ConfVars.HIVE_AUTHENTICATOR_MANAGER,
+ "org.apache.hadoop.hive.ql.security.HadoopDefaultAuthenticator");
+
+ String execEngine = conf.get("hive.execution.engine");
+ conf.set("hive.execution.engine", "mr");
+ CliSessionState ss = new CliSessionState(conf);
+ assert ss != null;
+ ss.in = System.in;
+ ss.out = System.out;
+ ss.err = System.out;
+
+ SessionState oldSs = SessionState.get();
+ if (oldSs != null && clusterType == MiniClusterType.tez) {
+ oldSs.close();
+ }
+ if (oldSs != null && oldSs.out != null && oldSs.out != System.out) {
+ oldSs.out.close();
+ }
+ SessionState.start(ss);
+
+ isSessionStateStarted = true;
+
+ conf.set("hive.execution.engine", execEngine);
+ return ss;
+ }
+
+ public int executeOne(String tname) {
+ String q = qMap.get(tname);
+
+ if (q.indexOf(";") == -1) {
+ return -1;
+ }
+
+ String q1 = q.substring(0, q.indexOf(";") + 1);
+ String qrest = q.substring(q.indexOf(";") + 1);
+ qMap.put(tname, qrest);
+
+ LOG.info("Executing " + q1);
+ return cliDriver.processLine(q1);
+ }
+
+ public static final String CRLF = System.getProperty("line.separator");
+
+ public int executeClient(String tname1, String tname2) {
+ String commands = getCommands(tname1) + CRLF + getCommands(tname2);
+ return cliDriver.processLine(commands);
+ }
+
+ public int executeClient(String tname) {
+ conf.set("mapreduce.job.name", "test");
+ return cliDriver.processLine(getCommands(tname), false);
+ }
+
+ private String getCommands(String tname) {
+ String commands = qMap.get(tname);
+ StringBuilder newCommands = new StringBuilder(commands.length());
+ int lastMatchEnd = 0;
+ Matcher commentMatcher = Pattern.compile("^--.*$", Pattern.MULTILINE).matcher(commands);
+ while (commentMatcher.find()) {
+ newCommands.append(commands.substring(lastMatchEnd, commentMatcher.start()));
+ newCommands.append(commentMatcher.group().replaceAll("(?<!\\\\);", "\\\\;"));
+ lastMatchEnd = commentMatcher.end();
+ }
+ newCommands.append(commands.substring(lastMatchEnd, commands.length()));
+ commands = newCommands.toString();
+ return commands;
+ }
+
+ public boolean shouldBeSkipped(String tname) {
+ return qSkipSet.contains(tname);
+ }
+
+ private String getOutFileExtension(String fname) {
+ String outFileExtension = ".out";
+ if (qJavaVersionSpecificOutput.contains(fname)) {
+ outFileExtension = ".java" + javaVersion + ".out";
+ }
+
+ return outFileExtension;
+ }
+
+ /**
+ * Given the current configurations (e.g., hadoop version and execution mode), return
+ * the correct file name to compare with the current test run output.
+ *
+ * @param outDir The directory where the reference log files are stored.
+ * @param testName The test file name (terminated by ".out").
+ * @return The file name appended with the configuration values if it exists.
+ */
+ public String outPath(String outDir, String testName) {
+ String ret = (new File(outDir, testName)).getPath();
+ // List of configurations. Currently the list consists of hadoop version and execution
+ // mode only
+ List<String> configs = new ArrayList<String>();
+ configs.add(this.hadoopVer);
+
+ Deque<String> stack = new LinkedList<String>();
+ StringBuilder sb = new StringBuilder();
+ sb.append(testName);
+ stack.push(sb.toString());
+
+ // example file names are input1.q.out_0.20.0_minimr or input2.q.out_0.17
+ for (String s : configs) {
+ sb.append('_');
+ sb.append(s);
+ stack.push(sb.toString());
+ }
+ while (stack.size() > 0) {
+ String fileName = stack.pop();
+ File f = new File(outDir, fileName);
+ if (f.exists()) {
+ ret = f.getPath();
+ break;
+ }
+ }
+ return ret;
+ }
+
+ private Pattern[] toPattern(String[] patternStrs) {
+ Pattern[] patterns = new Pattern[patternStrs.length];
+ for (int i = 0; i < patternStrs.length; i++) {
+ patterns[i] = Pattern.compile(patternStrs[i]);
+ }
+ return patterns;
+ }
+
+ private void maskPatterns(Pattern[] patterns, String fname) throws Exception {
+ String maskPattern = "#### A masked pattern was here ####";
+
+ String line;
+ BufferedReader in;
+ BufferedWriter out;
+
+ File file = new File(fname);
+ File fileOrig = new File(fname + ".orig");
+ FileUtils.copyFile(file, fileOrig);
+
+ in = new BufferedReader(new InputStreamReader(new FileInputStream(fileOrig), "UTF-8"));
+ out = new BufferedWriter(new OutputStreamWriter(new FileOutputStream(file), "UTF-8"));
+
+ boolean lastWasMasked = false;
+ while (null != (line = in.readLine())) {
+ for (Pattern pattern : patterns) {
+ line = pattern.matcher(line).replaceAll(maskPattern);
+ }
+
+ if (line.equals(maskPattern)) {
+ // We're folding multiple masked lines into one.
+ if (!lastWasMasked) {
+ out.write(line);
+ out.write("\n");
+ lastWasMasked = true;
+ }
+ } else {
+ out.write(line);
+ out.write("\n");
+ lastWasMasked = false;
+ }
+ }
+
+ in.close();
+ out.close();
+ }
+
+ private final Pattern[] planMask = toPattern(new String[]{
+ ".*file:.*",
+ ".*pfile:.*",
+ ".*hdfs:.*",
+ ".*/tmp/.*",
+ ".*invalidscheme:.*",
+ ".*lastUpdateTime.*",
+ ".*lastAccessTime.*",
+ ".*lastModifiedTime.*",
+ ".*[Oo]wner.*",
+ ".*CreateTime.*",
+ ".*LastAccessTime.*",
+ ".*Location.*",
+ ".*LOCATION '.*",
+ ".*transient_lastDdlTime.*",
+ ".*last_modified_.*",
+ ".*at org.*",
+ ".*at sun.*",
+ ".*at java.*",
+ ".*at junit.*",
+ ".*Caused by:.*",
+ ".*LOCK_QUERYID:.*",
+ ".*LOCK_TIME:.*",
+ ".*grantTime.*",
+ ".*[.][.][.] [0-9]* more.*",
+ ".*job_[0-9_]*.*",
+ ".*job_local[0-9_]*.*",
+ ".*USING 'java -cp.*",
+ "^Deleted.*",
+ ".*DagName:.*",
+ ".*Input:.*/data/files/.*",
+ ".*Output:.*/data/files/.*",
+ ".*total number of created files now is.*"
+ });
+
+ public int checkCliDriverResults(String tname) throws Exception {
+ assert (qMap.containsKey(tname));
+
+ String outFileExtension = getOutFileExtension(tname);
+ String outFileName = outPath(outDir, tname + outFileExtension);
+
+ File f = new File(logDir, tname + outFileExtension);
+
+ maskPatterns(planMask, f.getPath());
+ int exitVal = executeDiffCommand(f.getPath(),
+ outFileName, false,
+ qSortSet.contains(tname));
+
+ if (exitVal != 0 && overWrite) {
+ exitVal = overwriteResults(f.getPath(), outFileName);
+ }
+
+ return exitVal;
+ }
+
+
+ public int checkCompareCliDriverResults(String tname, List<String> outputs) throws Exception {
+ assert outputs.size() > 1;
+ maskPatterns(planMask, outputs.get(0));
+ for (int i = 1; i < outputs.size(); ++i) {
+ maskPatterns(planMask, outputs.get(i));
+ int ecode = executeDiffCommand(
+ outputs.get(i - 1), outputs.get(i), false, qSortSet.contains(tname));
+ if (ecode != 0) {
+ LOG.info("Files don't match: " + outputs.get(i - 1) + " and " + outputs.get(i));
+ return ecode;
+ }
+ }
+ return 0;
+ }
+
+ private static int overwriteResults(String inFileName, String outFileName) throws Exception {
+ // This method can be replaced with Files.copy(source, target, REPLACE_EXISTING)
+ // once Hive uses JAVA 7.
+ LOG.info("Overwriting results " + inFileName + " to " + outFileName);
+ return executeCmd(new String[]{
+ "cp",
+ getQuotedString(inFileName),
+ getQuotedString(outFileName)
+ });
+ }
+
+ private static int executeDiffCommand(String inFileName,
+ String outFileName,
+ boolean ignoreWhiteSpace,
+ boolean sortResults
+ ) throws Exception {
+
+ int result = 0;
+
+ if (sortResults) {
+ // sort will try to open the output file in write mode on windows. We need to
+ // close it first.
+ SessionState ss = SessionState.get();
+ if (ss != null && ss.out != null && ss.out != System.out) {
+ ss.out.close();
+ }
+
+ String inSorted = inFileName + SORT_SUFFIX;
+ String outSorted = outFileName + SORT_SUFFIX;
+
+ result = sortFiles(inFileName, inSorted);
+ result |= sortFiles(outFileName, outSorted);
+ if (result != 0) {
+ LOG.error("ERROR: Could not sort files before comparing");
+ return result;
+ }
+ inFileName = inSorted;
+ outFileName = outSorted;
+ }
+
+ ArrayList<String> diffCommandArgs = new ArrayList<String>();
+ diffCommandArgs.add("diff");
+
+ // Text file comparison
+ diffCommandArgs.add("-a");
+
+ // Ignore changes in the amount of white space
+ if (ignoreWhiteSpace || Shell.WINDOWS) {
+ diffCommandArgs.add("-b");
+ }
+
+ // Files created on Windows machines have different line endings
+ // than files created on Unix/Linux. Windows uses carriage return and line feed
+ // ("\r\n") as a line ending, whereas Unix uses just line feed ("\n").
+ // Also StringBuilder.toString(), Stream to String conversions adds extra
+ // spaces at the end of the line.
+ if (Shell.WINDOWS) {
+ diffCommandArgs.add("--strip-trailing-cr"); // Strip trailing carriage return on input
+ diffCommandArgs.add("-B"); // Ignore changes whose lines are all blank
+ }
+ // Add files to compare to the arguments list
+ diffCommandArgs.add(getQuotedString(inFileName));
+ diffCommandArgs.add(getQuotedString(outFileName));
+
+ result = executeCmd(diffCommandArgs);
+
+ if (sortResults) {
+ new File(inFileName).delete();
+ new File(outFileName).delete();
+ }
+
+ return result;
+ }
+
+ private static int sortFiles(String in, String out) throws Exception {
+ return executeCmd(new String[]{
+ "sort",
+ getQuotedString(in),
+ }, out, null);
+ }
+
+ private static int executeCmd(Collection<String> args) throws Exception {
+ return executeCmd(args, null, null);
+ }
+
+ private static int executeCmd(String[] args) throws Exception {
+ return executeCmd(args, null, null);
+ }
+
+ private static int executeCmd(Collection<String> args, String outFile, String errFile) throws
+ Exception {
+ String[] cmdArray = args.toArray(new String[args.size()]);
+ return executeCmd(cmdArray, outFile, errFile);
+ }
+
+ private static int executeCmd(String[] args, String outFile, String errFile) throws Exception {
+ LOG.info("Running: " + org.apache.commons.lang.StringUtils.join(args, ' '));
+
+ PrintStream out = outFile == null ?
+ SessionState.getConsole().getChildOutStream() :
+ new PrintStream(new FileOutputStream(outFile), true);
+ PrintStream err = errFile == null ?
+ SessionState.getConsole().getChildErrStream() :
+ new PrintStream(new FileOutputStream(errFile), true);
+
+ Process executor = Runtime.getRuntime().exec(args);
+
+ StreamPrinter errPrinter = new StreamPrinter(executor.getErrorStream(), null, err);
+ StreamPrinter outPrinter = new StreamPrinter(executor.getInputStream(), null, out);
+
+ outPrinter.start();
+ errPrinter.start();
+
+ int result = executor.waitFor();
+
+ outPrinter.join();
+ errPrinter.join();
+
+ if (outFile != null) {
+ out.close();
+ }
+
+ if (errFile != null) {
+ err.close();
+ }
+
+ return result;
+ }
+
+ private static String getQuotedString(String str) {
+ return Shell.WINDOWS ? String.format("\"%s\"", str) : str;
+ }
+
+ public ASTNode parseQuery(String tname) throws Exception {
+ return pd.parse(qMap.get(tname));
+ }
+
+ public void resetParser() throws SemanticException {
+ pd = new ParseDriver();
+ sem = new SemanticAnalyzer(conf);
+ }
+
+ public TreeMap<String, String> getQMap() {
+ return qMap;
+ }
+
+ /**
+ * HiveTestSetup defines test fixtures which are reused across testcases,
+ * and are needed before any test can be run
+ */
+ public static class HiveTestSetup {
+ private MiniZooKeeperCluster zooKeeperCluster = null;
+ private int zkPort;
+ private ZooKeeper zooKeeper;
+
+ public HiveTestSetup() {
+ }
+
+ public void preTest(HiveConf conf) throws Exception {
+
+ if (zooKeeperCluster == null) {
+ //create temp dir
+ String tmpBaseDir = System.getProperty("test.tmp.dir");
+ File tmpDir = Utilities.createTempDir(tmpBaseDir);
+
+ zooKeeperCluster = new MiniZooKeeperCluster();
+ zkPort = zooKeeperCluster.startup(tmpDir);
+ }
+
+ if (zooKeeper != null) {
+ zooKeeper.close();
+ }
+
+ int sessionTimeout = (int) conf.getTimeVar(HiveConf.ConfVars
+ .HIVE_ZOOKEEPER_SESSION_TIMEOUT, TimeUnit.MILLISECONDS);
+ zooKeeper = new ZooKeeper("localhost:" + zkPort, sessionTimeout, new Watcher() {
+ @Override
+ public void process(WatchedEvent arg0) {
+ }
+ });
+
+ String zkServer = "localhost";
+ conf.set("hive.zookeeper.quorum", zkServer);
+ conf.set("hive.zookeeper.client.port", "" + zkPort);
+ }
+
+ public void postTest(HiveConf conf) throws Exception {
+ if (zooKeeperCluster == null) {
+ return;
+ }
+
+ if (zooKeeper != null) {
+ zooKeeper.close();
+ }
+
+ ZooKeeperHiveLockManager.releaseAllLocks(conf);
+ }
+
+ public void tearDown() throws Exception {
+ if (zooKeeperCluster != null) {
+ zooKeeperCluster.shutdown();
+ zooKeeperCluster = null;
+ }
+ }
+ }
+
+ /**
+ * QTRunner: Runnable class for running a a single query file.
+ **/
+ public static class HiveTestRunner implements Runnable {
+ private final HiveTestUtil qt;
+ private final String fname;
+
+ public HiveTestRunner(HiveTestUtil qt, String fname) {
+ this.qt = qt;
+ this.fname = fname;
+ }
+
+ @Override
+ public void run() {
+ try {
+ // assumption is that environment has already been cleaned once globally
+ // hence each thread does not call cleanUp() and createSources() again
+ qt.cliInit(fname, false);
+ qt.executeClient(fname);
+ } catch (Throwable e) {
+ LOG.error("Query file " + fname + " failed with exception ", e);
+ e.printStackTrace();
+ outputTestFailureHelpMessage();
+ }
+ }
+ }
+
+ /**
+ * Executes a set of query files in sequence.
+ *
+ * @param qfiles array of input query files containing arbitrary number of hive
+ * queries
+ * @param qt array of HiveTestUtils, one per qfile
+ * @return true if all queries passed, false otw
+ */
+ public static boolean queryListRunnerSingleThreaded(File[] qfiles, HiveTestUtil[] qt)
+ throws Exception {
+ boolean failed = false;
+ qt[0].cleanUp();
+ qt[0].createSources();
+ for (int i = 0; i < qfiles.length && !failed; i++) {
+ qt[i].clearTestSideEffects();
+ qt[i].cliInit(qfiles[i].getName(), false);
+ qt[i].executeClient(qfiles[i].getName());
+ int ecode = qt[i].checkCliDriverResults(qfiles[i].getName());
+ if (ecode != 0) {
+ failed = true;
+ LOG.error("Test " + qfiles[i].getName()
+ + " results check failed with error code " + ecode);
+ outputTestFailureHelpMessage();
+ }
+ qt[i].clearPostTestEffects();
+ }
+ return (!failed);
+ }
+
+ public static void outputTestFailureHelpMessage() {
+ LOG.error("See ./ql/target/tmp/log/hive.log or ./itests/qtest/target/tmp/log/hive.log, "
+ + "or check ./ql/target/surefire-reports or " +
+ "./itests/qtest/target/surefire-reports/ for specific test cases logs.");
+ }
+
+ public static String ensurePathEndsInSlash(String path) {
+ if (path == null) {
+ throw new NullPointerException("Path cannot be null");
+ }
+ if (path.endsWith(File.separator)) {
+ return path;
+ } else {
+ return path + File.separator;
+ }
+ }
+
+ private static String[] cachedQvFileList = null;
+ private static ImmutableList<String> cachedDefaultQvFileList = null;
+ private static Pattern qvSuffix = Pattern.compile("_[0-9]+.qv$", Pattern.CASE_INSENSITIVE);
+
+ public static List<String> getVersionFiles(String queryDir, String tname) {
+ ensureQvFileList(queryDir);
+ List<String> result = getVersionFilesInternal(tname);
+ if (result == null) {
+ result = cachedDefaultQvFileList;
+ }
+ return result;
+ }
+
+ private static void ensureQvFileList(String queryDir) {
+ if (cachedQvFileList != null) return;
+ // Not thread-safe.
+ LOG.info("Getting versions from " + queryDir);
+ cachedQvFileList = (new File(queryDir)).list(new FilenameFilter() {
+ @Override
+ public boolean accept(File dir, String name) {
+ return name.toLowerCase().endsWith(".qv");
+ }
+ });
+ if (cachedQvFileList == null) return; // no files at all
+ Arrays.sort(cachedQvFileList, String.CASE_INSENSITIVE_ORDER);
+ List<String> defaults = getVersionFilesInternal("default");
+ cachedDefaultQvFileList = (defaults != null)
+ ? ImmutableList.copyOf(defaults) : ImmutableList.<String>of();
+ }
+
+ private static List<String> getVersionFilesInternal(String tname) {
+ if (cachedQvFileList == null) {
+ return new ArrayList<String>();
+ }
+ int pos = Arrays.binarySearch(cachedQvFileList, tname, String.CASE_INSENSITIVE_ORDER);
+ if (pos >= 0) {
+ throw new BuildException("Unexpected file list element: " + cachedQvFileList[pos]);
+ }
+ List<String> result = null;
+ for (pos = (-pos - 1); pos < cachedQvFileList.length; ++pos) {
+ String candidate = cachedQvFileList[pos];
+ if (candidate.length() <= tname.length()
+ || !tname.equalsIgnoreCase(candidate.substring(0, tname.length()))
+ || !qvSuffix.matcher(candidate.substring(tname.length())).matches()) {
+ break;
+ }
+ if (result == null) {
+ result = new ArrayList<String>();
+ }
+ result.add(candidate);
+ }
+ return result;
+ }
+
+ public void failed(int ecode, String fname, String debugHint) {
+ String command = SessionState.get() != null ? SessionState.get().getLastCommand() : null;
+ Assert.fail("Client Execution failed with error code = " + ecode +
+ (command != null ? " running " + command : "") + (debugHint != null ? debugHint :
+ ""));
+ }
+
+ // for negative tests, which is succeeded.. no need to print the query string
+ public void failed(String fname, String debugHint) {
+ Assert.fail("Client Execution was expected to fail, but succeeded with error code 0 " +
+ (debugHint != null ? debugHint : ""));
+ }
+
+ public void failedDiff(int ecode, String fname, String debugHint) {
+ Assert.fail("Client Execution results failed with error code = " + ecode +
+ (debugHint != null ? debugHint : ""));
+ }
+
+ public void failed(Throwable e, String fname, String debugHint) {
+ String command = SessionState.get() != null ? SessionState.get().getLastCommand() : null;
+ LOG.error("Exception: ", e);
+ e.printStackTrace();
+ LOG.error("Failed query: " + fname);
+ Assert.fail("Unexpected exception " +
+ org.apache.hadoop.util.StringUtils.stringifyException(e) + "\n" +
+ (command != null ? " running " + command : "") +
+ (debugHint != null ? debugHint : ""));
+ }
+
+ public static class WindowsPathUtil {
+
+ public static void convertPathsFromWindowsToHdfs(HiveConf conf) {
+ // Following local paths are used as HDFS paths in unit tests.
+ // It works well in Unix as the path notation in Unix and HDFS is more or less same.
+ // But when it comes to Windows, drive letter separator ':' & backslash '\" are invalid
+ // characters in HDFS so we need to converts these local paths to HDFS paths before
+ // using them
+ // in unit tests.
+
+ String orgWarehouseDir = conf.getVar(HiveConf.ConfVars.METASTOREWAREHOUSE);
+ conf.setVar(HiveConf.ConfVars.METASTOREWAREHOUSE, getHdfsUriString(orgWarehouseDir));
+
+ String orgTestTempDir = System.getProperty("test.tmp.dir");
+ System.setProperty("test.tmp.dir", getHdfsUriString(orgTestTempDir));
+
+ String orgTestWarehouseDir = System.getProperty("test.warehouse.dir");
+ System.setProperty("test.warehouse.dir", getHdfsUriString(orgTestWarehouseDir));
+
+ String orgScratchDir = conf.getVar(HiveConf.ConfVars.SCRATCHDIR);
+ conf.setVar(HiveConf.ConfVars.SCRATCHDIR, getHdfsUriString(orgScratchDir));
+ }
+
+ public static String getHdfsUriString(String uriStr) {
+ assert uriStr != null;
+ if (Shell.WINDOWS) {
+ // If the URI conversion is from Windows to HDFS then replace the '\' with '/'
+ // and remove the windows single drive letter & colon from absolute path.
+ return uriStr.replace('\\', '/')
+ .replaceFirst("/[c-zC-Z]:", "/")
+ .replaceFirst("^[c-zC-Z]:", "");
+ }
+ return uriStr;
+ }
+ }
+}
diff --git a/phoenix-hive/src/it/java/org/apache/phoenix/hive/HiveTezIT.java b/phoenix-hive/src/it/java/org/apache/phoenix/hive/HiveTezIT.java
new file mode 100644
index 0000000..8dc3309
--- /dev/null
+++ b/phoenix-hive/src/it/java/org/apache/phoenix/hive/HiveTezIT.java
@@ -0,0 +1,33 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.phoenix.hive;
+
+import org.apache.phoenix.end2end.NeedsOwnMiniClusterTest;
+import org.junit.BeforeClass;
+import org.junit.experimental.categories.Category;
+import org.junit.Ignore;
+
+@Category(NeedsOwnMiniClusterTest.class)
+public class HiveTezIT extends HivePhoenixStoreIT {
+
+ @BeforeClass
+ public static void setUpBeforeClass() throws Exception {
+ setup(HiveTestUtil.MiniClusterType.tez);
+ }
+}
diff --git a/phoenix-hive/src/main/java/org/apache/phoenix/hive/PhoenixMetaHook.java b/phoenix-hive/src/main/java/org/apache/phoenix/hive/PhoenixMetaHook.java
new file mode 100644
index 0000000..c35634a
--- /dev/null
+++ b/phoenix-hive/src/main/java/org/apache/phoenix/hive/PhoenixMetaHook.java
@@ -0,0 +1,229 @@
+/*
+ * 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.phoenix.hive;
+
+import com.google.common.base.CharMatcher;
+import com.google.common.base.Splitter;
+import com.google.common.collect.Lists;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.hive.metastore.HiveMetaHook;
+import org.apache.hadoop.hive.metastore.TableType;
+import org.apache.hadoop.hive.metastore.api.FieldSchema;
+import org.apache.hadoop.hive.metastore.api.MetaException;
+import org.apache.hadoop.hive.metastore.api.Table;
+import org.apache.phoenix.hive.constants.PhoenixStorageHandlerConstants;
+import org.apache.phoenix.hive.util.PhoenixConnectionUtil;
+import org.apache.phoenix.hive.util.PhoenixStorageHandlerUtil;
+import org.apache.phoenix.hive.util.PhoenixUtil;
+
+import java.sql.Connection;
+import java.sql.SQLException;
+import java.util.Collections;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+
+import static org.apache.phoenix.hive.util.ColumnMappingUtils.getColumnMappingMap;
+
+/**
+ * Implementation for notification methods which are invoked as part of transactions against the
+ * hive metastore,allowing Phoenix metadata to be kept in sync with Hive'smetastore.
+ */
+public class PhoenixMetaHook implements HiveMetaHook {
+
+ private static final Log LOG = LogFactory.getLog(PhoenixMetaHook.class);
+
+ @Override
+ public void preCreateTable(Table table) throws MetaException {
+ if (LOG.isDebugEnabled()) {
+ LOG.debug("Precreate table : " + table.getTableName());
+ }
+
+ try (Connection conn = PhoenixConnectionUtil.getConnection(table)) {
+ String tableType = table.getTableType();
+ String tableName = PhoenixStorageHandlerUtil.getTargetTableName(table);
+
+ if (TableType.EXTERNAL_TABLE.name().equals(tableType)) {
+ // Check whether phoenix table exists.
+ if (!PhoenixUtil.existTable(conn, tableName)) {
+ // Error if phoenix table not exist.
+ throw new MetaException("Phoenix table " + tableName + " doesn't exist");
+ }
+ } else if (TableType.MANAGED_TABLE.name().equals(tableType)) {
+ // Check whether phoenix table exists.
+ if (PhoenixUtil.existTable(conn, tableName)) {
+ // Error if phoenix table already exist.
+ throw new MetaException("Phoenix table " + tableName + " already exist.");
+ }
+
+ PhoenixUtil.createTable(conn, createTableStatement(table));
+ } else {
+ throw new MetaException("Unsupported table Type: " + table.getTableType());
+ }
+
+ if (LOG.isDebugEnabled()) {
+ LOG.debug("Phoenix table " + tableName + " was created");
+ }
+ } catch (SQLException e) {
+ throw new MetaException(e.getMessage());
+ }
+ }
+
+ private String createTableStatement(Table table) throws MetaException {
+ Map<String, String> tableParameterMap = table.getParameters();
+
+ String tableName = PhoenixStorageHandlerUtil.getTargetTableName(table);
+ StringBuilder ddl = new StringBuilder("create table ").append(tableName).append(" (\n");
+
+ String phoenixRowKeys = tableParameterMap.get(PhoenixStorageHandlerConstants
+ .PHOENIX_ROWKEYS);
+ StringBuilder realRowKeys = new StringBuilder();
+ List<String> phoenixRowKeyList = Lists.newArrayList(Splitter.on
+ (PhoenixStorageHandlerConstants.COMMA).trimResults().split(phoenixRowKeys));
+ Map<String, String> columnMappingMap = getColumnMappingMap(tableParameterMap.get
+ (PhoenixStorageHandlerConstants.PHOENIX_COLUMN_MAPPING));
+
+ List<FieldSchema> fieldSchemaList = table.getSd().getCols();
+ for (int i = 0, limit = fieldSchemaList.size(); i < limit; i++) {
+ FieldSchema fieldSchema = fieldSchemaList.get(i);
+ String fieldName = fieldSchema.getName();
+ String fieldType = fieldSchema.getType();
+ String columnType = PhoenixUtil.getPhoenixType(fieldType);
+
+ String rowKeyName = getRowKeyMapping(fieldName, phoenixRowKeyList);
+ if (rowKeyName != null) {
+ String columnName = columnMappingMap.get(fieldName);
+ if(columnName != null) {
+ rowKeyName = columnName;
+ }
+ // In case of RowKey
+ if ("binary".equals(columnType)) {
+ // Phoenix must define max length of binary when type definition. Obtaining
+ // information from the column mapping. ex) phoenix.rowkeys = "r1, r2(100), ..."
+ List<String> tokenList = Lists.newArrayList(Splitter.on(CharMatcher.is('(')
+ .or(CharMatcher.is(')'))).trimResults().split(rowKeyName));
+ columnType = columnType + "(" + tokenList.get(1) + ")";
+ rowKeyName = tokenList.get(0);
+ }
+
+ ddl.append(" ").append("\"").append(rowKeyName).append("\"").append(" ").append(columnType).append(" not " +
+ "null,\n");
+ realRowKeys.append("\"").append(rowKeyName).append("\",");
+ } else {
+ // In case of Column
+ String columnName = columnMappingMap.get(fieldName);
+
+ if (columnName == null) {
+ // Use field definition.
+ columnName = fieldName;
+ }
+
+ if ("binary".equals(columnType)) {
+ // Phoenix must define max length of binary when type definition. Obtaining
+ // information from the column mapping. ex) phoenix.column.mapping=c1:c1(100)
+ List<String> tokenList = Lists.newArrayList(Splitter.on(CharMatcher.is('(')
+ .or(CharMatcher.is(')'))).trimResults().split(columnName));
+ columnType = columnType + "(" + tokenList.get(1) + ")";
+ columnName = tokenList.get(0);
+ }
+
+ ddl.append(" ").append("\"").append(columnName).append("\"").append(" ").append(columnType).append(",\n");
+ }
+ }
+ ddl.append(" ").append("constraint pk_").append(PhoenixUtil.getTableSchema(tableName.toUpperCase())[1]).append(" primary key(")
+ .append(realRowKeys.deleteCharAt(realRowKeys.length() - 1)).append(")\n)\n");
+
+ String tableOptions = tableParameterMap.get(PhoenixStorageHandlerConstants
+ .PHOENIX_TABLE_OPTIONS);
+ if (tableOptions != null) {
+ ddl.append(tableOptions);
+ }
+
+ String statement = ddl.toString();
+
+ if (LOG.isDebugEnabled()) {
+ LOG.debug("DDL : " + statement);
+ }
+
+ return statement;
+ }
+
+ private String getRowKeyMapping(String rowKeyName, List<String> phoenixRowKeyList) {
+ String rowKeyMapping = null;
+
+ for (String phoenixRowKey : phoenixRowKeyList) {
+ if (phoenixRowKey.equals(rowKeyName)) {
+ rowKeyMapping = phoenixRowKey;
+ break;
+ } else if (phoenixRowKey.startsWith(rowKeyName + "(") && phoenixRowKey.endsWith(")")) {
+ rowKeyMapping = phoenixRowKey;
+ break;
+ }
+ }
+
+ return rowKeyMapping;
+ }
+
+ @Override
+ public void rollbackCreateTable(Table table) throws MetaException {
+ if (LOG.isDebugEnabled()) {
+ LOG.debug("Rollback for table : " + table.getTableName());
+ }
+
+ dropTableIfExist(table);
+ }
+
+ @Override
+ public void commitCreateTable(Table table) throws MetaException {
+
+ }
+
+ @Override
+ public void preDropTable(Table table) throws MetaException {
+ }
+
+ @Override
+ public void rollbackDropTable(Table table) throws MetaException {
+ }
+
+ @Override
+ public void commitDropTable(Table table, boolean deleteData) throws MetaException {
+ if (LOG.isDebugEnabled()) {
+ LOG.debug("Commit drop table : " + table.getTableName());
+ }
+
+ dropTableIfExist(table);
+ }
+
+ private void dropTableIfExist(Table table) throws MetaException {
+ try (Connection conn = PhoenixConnectionUtil.getConnection(table)) {
+ String tableType = table.getTableType();
+ String tableName = PhoenixStorageHandlerUtil.getTargetTableName(table);
+
+ if (TableType.MANAGED_TABLE.name().equals(tableType)) {
+ // Drop if phoenix table exist.
+ if (PhoenixUtil.existTable(conn, tableName)) {
+ PhoenixUtil.dropTable(conn, tableName);
+ }
+ }
+ } catch (SQLException e) {
+ throw new MetaException(e.getMessage());
+ }
+ }
+}
diff --git a/phoenix-hive/src/main/java/org/apache/phoenix/hive/PhoenixRecordUpdater.java b/phoenix-hive/src/main/java/org/apache/phoenix/hive/PhoenixRecordUpdater.java
new file mode 100644
index 0000000..089a299
--- /dev/null
+++ b/phoenix-hive/src/main/java/org/apache/phoenix/hive/PhoenixRecordUpdater.java
@@ -0,0 +1,336 @@
+/*
+ * 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.phoenix.hive;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hive.ql.io.AcidOutputFormat;
+import org.apache.hadoop.hive.ql.io.RecordUpdater;
+import org.apache.hadoop.hive.serde2.SerDeException;
+import org.apache.hadoop.hive.serde2.SerDeStats;
+import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector;
+import org.apache.phoenix.hive.PhoenixSerializer.DmlType;
+import org.apache.phoenix.hive.constants.PhoenixStorageHandlerConstants;
+import org.apache.phoenix.hive.mapreduce.PhoenixResultWritable;
+import org.apache.phoenix.hive.util.PhoenixConnectionUtil;
+import org.apache.phoenix.hive.util.PhoenixStorageHandlerUtil;
+import org.apache.phoenix.hive.util.PhoenixUtil;
+import org.apache.phoenix.jdbc.PhoenixConnection;
+import org.apache.phoenix.mapreduce.util.PhoenixConfigurationUtil;
+import org.apache.phoenix.schema.ConcurrentTableMutationException;
+import org.apache.phoenix.schema.MetaDataClient;
+import org.apache.phoenix.util.QueryUtil;
+
+import java.io.IOException;
+import java.sql.Connection;
+import java.sql.PreparedStatement;
+import java.sql.SQLException;
+import java.util.Properties;
+
+public class PhoenixRecordUpdater implements RecordUpdater {
+
+ private static final Log LOG = LogFactory.getLog(PhoenixRecordUpdater.class);
+
+ private final Connection conn;
+ private final PreparedStatement pstmt;
+ private final long batchSize;
+ private long numRecords = 0;
+
+ private Configuration config;
+ private String tableName;
+ private MetaDataClient metaDataClient;
+ private boolean restoreWalMode;
+
+ private long rowCountDelta = 0;
+
+ private PhoenixSerializer phoenixSerializer;
+ private ObjectInspector objInspector;
+ private PreparedStatement pstmtForDelete;
+
+ public PhoenixRecordUpdater(Path path, AcidOutputFormat.Options options) throws IOException {
+ this.config = options.getConfiguration();
+ tableName = config.get(PhoenixStorageHandlerConstants.PHOENIX_TABLE_NAME);
+
+ Properties props = new Properties();
+
+ try {
+ // Disable WAL
+ String walConfigName = tableName.toLowerCase() + PhoenixStorageHandlerConstants
+ .DISABLE_WAL;
+ boolean disableWal = config.getBoolean(walConfigName, false);
+ if (disableWal) {
+ if (LOG.isDebugEnabled()) {
+ LOG.debug(walConfigName + " is true. batch.mode will be set true.");
+ }
+
+ props.setProperty(PhoenixStorageHandlerConstants.BATCH_MODE, "true");
+ }
+
+ this.conn = PhoenixConnectionUtil.getInputConnection(config, props);
+
+ if (disableWal) {
+ metaDataClient = new MetaDataClient((PhoenixConnection) conn);
+
+ if (!PhoenixUtil.isDisabledWal(metaDataClient, tableName)) {
+ // execute alter tablel statement if disable_wal is not true.
+ try {
+ PhoenixUtil.alterTableForWalDisable(conn, tableName, true);
+ } catch (ConcurrentTableMutationException e) {
+ if (LOG.isWarnEnabled()) {
+ LOG.warn("Concurrent modification of disableWAL");
+ }
+ }
+
+ if (LOG.isDebugEnabled()) {
+ LOG.debug(tableName + "s wal disabled.");
+ }
+
+ // restore original value of disable_wal at the end.
+ restoreWalMode = true;
+ }
+ }
+
+ this.batchSize = PhoenixConfigurationUtil.getBatchSize(config);
+
+ if (LOG.isDebugEnabled()) {
+ LOG.debug("Batch-size : " + batchSize);
+ }
+
+ String upsertQuery = QueryUtil.constructUpsertStatement(tableName, PhoenixUtil
+ .getColumnInfoList(conn, tableName));
+
+ if (LOG.isDebugEnabled()) {
+ LOG.debug("Upsert-query : " + upsertQuery);
+ }
+ this.pstmt = this.conn.prepareStatement(upsertQuery);
+ } catch (SQLException e) {
+ throw new IOException(e);
+ }
+
+ this.objInspector = options.getInspector();
+ try {
+ phoenixSerializer = new PhoenixSerializer(config, options.getTableProperties());
+ } catch (SerDeException e) {
+ throw new IOException(e);
+ }
+ }
+
+ /* (non-Javadoc)
+ * @see org.apache.hadoop.hive.ql.io.RecordUpdater#insert(long, java.lang.Object)
+ */
+ @Override
+ public void insert(long currentTransaction, Object row) throws IOException {
+ if (LOG.isTraceEnabled()) {
+ LOG.trace("Insert - currentTranscation : " + currentTransaction + ", row : " +
+ PhoenixStorageHandlerUtil.toString(row));
+ }
+
+ PhoenixResultWritable pResultWritable = (PhoenixResultWritable) phoenixSerializer
+ .serialize(row, objInspector, DmlType.INSERT);
+
+ if (LOG.isTraceEnabled()) {
+ LOG.trace("Data : " + pResultWritable.getValueList());
+ }
+
+ write(pResultWritable);
+
+ rowCountDelta++;
+ }
+
+ /* (non-Javadoc)
+ * @see org.apache.hadoop.hive.ql.io.RecordUpdater#update(long, java.lang.Object)
+ */
+ @Override
+ public void update(long currentTransaction, Object row) throws IOException {
+ if (LOG.isTraceEnabled()) {
+ LOG.trace("Update - currentTranscation : " + currentTransaction + ", row : " +
+ PhoenixStorageHandlerUtil.toString(row));
+ }
+
+ PhoenixResultWritable pResultWritable = (PhoenixResultWritable) phoenixSerializer
+ .serialize(row, objInspector, DmlType.UPDATE);
+
+ if (LOG.isTraceEnabled()) {
+ LOG.trace("Data : " + pResultWritable.getValueList());
+ }
+
+ write(pResultWritable);
+ }
+
+ /* (non-Javadoc)
+ * @see org.apache.hadoop.hive.ql.io.RecordUpdater#delete(long, java.lang.Object)
+ */
+ @Override
+ public void delete(long currentTransaction, Object row) throws IOException {
+ if (LOG.isTraceEnabled()) {
+ LOG.trace("Delete - currentTranscation : " + currentTransaction + ", row : " +
+ PhoenixStorageHandlerUtil.toString(row));
+ }
+
+ PhoenixResultWritable pResultWritable = (PhoenixResultWritable) phoenixSerializer
+ .serialize(row, objInspector, DmlType.DELETE);
+
+ if (LOG.isTraceEnabled()) {
+ LOG.trace("Data : " + pResultWritable.getValueList());
+ }
+
+ if (pstmtForDelete == null) {
+ try {
+ String deleteQuery = PhoenixUtil.constructDeleteStatement(conn, tableName);
+
+ if (LOG.isDebugEnabled()) {
+ LOG.debug("Delete query : " + deleteQuery);
+ }
+
+ pstmtForDelete = conn.prepareStatement(deleteQuery);
+ } catch (SQLException e) {
+ throw new IOException(e);
+ }
+ }
+
+ delete(pResultWritable);
+
+ rowCountDelta--;
+ }
+
+ private void delete(PhoenixResultWritable pResultWritable) throws IOException {
+ try {
+ pResultWritable.delete(pstmtForDelete);
+ numRecords++;
+ pstmtForDelete.executeUpdate();
+
+ if (numRecords % batchSize == 0) {
+ LOG.debug("Commit called on a batch of size : " + batchSize);
+ conn.commit();
+ }
+ } catch (SQLException e) {
+ throw new IOException("Exception while deleting to table.", e);
+ }
+ }
+
+ private void write(PhoenixResultWritable pResultWritable) throws IOException {
+ try {
+ pResultWritable.write(pstmt);
+ numRecords++;
+ pstmt.executeUpdate();
+
+ if (numRecords % batchSize == 0) {
+ LOG.debug("Commit called on a batch of size : " + batchSize);
+ conn.commit();
+ }
+ } catch (SQLException e) {
+ throw new IOException("Exception while writing to table.", e);
+ }
+ }
+
+ /* (non-Javadoc)
+ * @see org.apache.hadoop.hive.ql.io.RecordUpdater#flush()
+ */
+ @Override
+ public void flush() throws IOException {
+ if (LOG.isDebugEnabled()) {
+ LOG.debug("Flush called");
+ }
+
+ try {
+ conn.commit();
+
+ if (LOG.isInfoEnabled()) {
+ LOG.info("Written row : " + numRecords);
+ }
+ } catch (SQLException e) {
+ LOG.error("SQLException while performing the commit for the task.");
+ throw new IOException(e);
+ }
+ }
+
+ /* (non-Javadoc)
+ * @see org.apache.hadoop.hive.ql.io.RecordUpdater#close(boolean)
+ */
+ @Override
+ public void close(boolean abort) throws IOException {
+ if (LOG.isDebugEnabled()) {
+ LOG.debug("abort : " + abort);
+ }
+
+ try {
+ conn.commit();
+
+ if (LOG.isInfoEnabled()) {
+ LOG.info("Written row : " + numRecords);
+ }
+ } catch (SQLException e) {
+ LOG.error("SQLException while performing the commit for the task.");
+ throw new IOException(e);
+ } finally {
+ try {
+ if (restoreWalMode && PhoenixUtil.isDisabledWal(metaDataClient, tableName)) {
+ try {
+ PhoenixUtil.alterTableForWalDisable(conn, tableName, false);
+ } catch (ConcurrentTableMutationException e) {
+ if (LOG.isWarnEnabled()) {
+ LOG.warn("Concurrent modification of disableWAL");
+ }
+ }
+
+ if (LOG.isDebugEnabled()) {
+ LOG.debug(tableName + "s wal enabled.");
+ }
+ }
+
+ // flush when [table-name].auto.flush is true.
+ String autoFlushConfigName = tableName.toLowerCase() +
+ PhoenixStorageHandlerConstants.AUTO_FLUSH;
+ boolean autoFlush = config.getBoolean(autoFlushConfigName, false);
+ if (autoFlush) {
+ if (LOG.isDebugEnabled()) {
+ LOG.debug("autoFlush is " + autoFlush);
+ }
+
+ PhoenixUtil.flush(conn, tableName);
+ }
+
+ PhoenixUtil.closeResource(pstmt);
+ PhoenixUtil.closeResource(pstmtForDelete);
+ PhoenixUtil.closeResource(conn);
+ } catch (SQLException ex) {
+ LOG.error("SQLException while closing the connection for the task.");
+ throw new IOException(ex);
+ }
+ }
+ }
+
+ /* (non-Javadoc)
+ * @see org.apache.hadoop.hive.ql.io.RecordUpdater#getStats()
+ */
+ @Override
+ public SerDeStats getStats() {
+ if (LOG.isDebugEnabled()) {
+ LOG.debug("getStats called");
+ }
+
+ SerDeStats stats = new SerDeStats();
+ stats.setRowCount(rowCountDelta);
+ // Don't worry about setting raw data size diff. There is no reasonable way to calculate
+ // that without finding the row we are updating or deleting, which would be a mess.
+ return stats;
+ }
+
+}
diff --git a/phoenix-hive/src/main/java/org/apache/phoenix/hive/PhoenixRow.java b/phoenix-hive/src/main/java/org/apache/phoenix/hive/PhoenixRow.java
new file mode 100644
index 0000000..fa307ce
--- /dev/null
+++ b/phoenix-hive/src/main/java/org/apache/phoenix/hive/PhoenixRow.java
@@ -0,0 +1,64 @@
+/*
+ * 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.phoenix.hive;
+
+import com.google.common.collect.Lists;
+import org.apache.hadoop.hive.serde2.StructObject;
+
+import java.util.List;
+import java.util.Map;
+
+/**
+ * Implementation for Hive SerDe StructObject
+ */
+public class PhoenixRow implements StructObject {
+
+ private List<String> columnList;
+ private Map<String, Object> resultRowMap;
+
+ public PhoenixRow(List<String> columnList) {
+ this.columnList = columnList;
+ }
+
+ public PhoenixRow setResultRowMap(Map<String, Object> resultRowMap) {
+ this.resultRowMap = resultRowMap;
+ return this;
+ }
+
+ /* (non-Javadoc)
+ * @see org.apache.hadoop.hive.serde2.StructObject#getField(int)
+ */
+ @Override
+ public Object getField(int fieldID) {
+ return resultRowMap.get(columnList.get(fieldID));
+ }
+
+ /* (non-Javadoc)
+ * @see org.apache.hadoop.hive.serde2.StructObject#getFieldsAsList()
+ */
+ @Override
+ public List<Object> getFieldsAsList() {
+ return Lists.newArrayList(resultRowMap.values());
+ }
+
+
+ @Override
+ public String toString() {
+ return resultRowMap.toString();
+ }
+}
diff --git a/phoenix-hive/src/main/java/org/apache/phoenix/hive/PhoenixRowKey.java b/phoenix-hive/src/main/java/org/apache/phoenix/hive/PhoenixRowKey.java
new file mode 100644
index 0000000..a963fba
--- /dev/null
+++ b/phoenix-hive/src/main/java/org/apache/phoenix/hive/PhoenixRowKey.java
@@ -0,0 +1,62 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.phoenix.hive;
+
+import org.apache.hadoop.hive.ql.io.RecordIdentifier;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.util.Map;
+
+/**
+ * Hive's RecordIdentifier implementation.
+ */
+
+public class PhoenixRowKey extends RecordIdentifier {
+
+ private PrimaryKeyData rowKeyMap = PrimaryKeyData.EMPTY;
+
+ public PhoenixRowKey() {
+
+ }
+
+ public void setRowKeyMap(Map<String, Object> rowKeyMap) {
+ this.rowKeyMap = new PrimaryKeyData(rowKeyMap);
+ }
+
+ @Override
+ public void write(DataOutput dataOutput) throws IOException {
+ super.write(dataOutput);
+
+ rowKeyMap.serialize((OutputStream) dataOutput);
+ }
+
+ @Override
+ public void readFields(DataInput dataInput) throws IOException {
+ super.readFields(dataInput);
+
+ try {
+ rowKeyMap = PrimaryKeyData.deserialize((InputStream) dataInput);
+ } catch (ClassNotFoundException e) {
+ throw new RuntimeException(e);
+ }
+ }
+}
diff --git a/phoenix-hive/src/main/java/org/apache/phoenix/hive/PhoenixSerDe.java b/phoenix-hive/src/main/java/org/apache/phoenix/hive/PhoenixSerDe.java
new file mode 100644
index 0000000..9ef0158
--- /dev/null
+++ b/phoenix-hive/src/main/java/org/apache/phoenix/hive/PhoenixSerDe.java
@@ -0,0 +1,152 @@
+/*
+ * 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.phoenix.hive;
+
+import com.google.common.base.Function;
+import com.google.common.collect.Lists;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+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.SerDeStats;
+import org.apache.hadoop.hive.serde2.lazy.LazySerDeParameters;
+import org.apache.hadoop.hive.serde2.lazy.objectinspector.LazyObjectInspectorFactory;
+import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector;
+import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorFactory.ObjectInspectorOptions;
+import org.apache.hadoop.hive.serde2.typeinfo.TypeInfo;
+import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoUtils;
+import org.apache.hadoop.io.Writable;
+import org.apache.phoenix.hive.PhoenixSerializer.DmlType;
+import org.apache.phoenix.hive.constants.PhoenixStorageHandlerConstants;
+import org.apache.phoenix.hive.mapreduce.PhoenixResultWritable;
+import org.apache.phoenix.hive.objectinspector.PhoenixObjectInspectorFactory;
+
+import java.util.Arrays;
+import java.util.List;
+import java.util.Properties;
+
+/**
+ * SerDe implementation for Phoenix Hive Storage
+ *
+ */
+public class PhoenixSerDe extends AbstractSerDe {
+
+ public static final Log LOG = LogFactory.getLog(PhoenixSerDe.class);
+
+ private PhoenixSerializer serializer;
+ private ObjectInspector objectInspector;
+
+ private LazySerDeParameters serdeParams;
+ private PhoenixRow row;
+
+ private Properties tableProperties;
+
+ /**
+ * @throws SerDeException
+ */
+ public PhoenixSerDe() throws SerDeException {
+ if (LOG.isDebugEnabled()) {
+ LOG.debug("PhoenixSerDe created");
+ }
+ }
+
+ @Override
+ public void initialize(Configuration conf, Properties tbl) throws SerDeException {
+ tableProperties = tbl;
+
+ if (LOG.isDebugEnabled()) {
+ LOG.debug("SerDe initialize : " + tbl.getProperty("name"));
+ }
+
+ serdeParams = new LazySerDeParameters(conf, tbl, getClass().getName());
+ objectInspector = createLazyPhoenixInspector(conf, tbl);
+
+ String inOutWork = tbl.getProperty(PhoenixStorageHandlerConstants.IN_OUT_WORK);
+ if (inOutWork == null) {
+ return;
+ }
+
+ serializer = new PhoenixSerializer(conf, tbl);
+ row = new PhoenixRow(serdeParams.getColumnNames());
+ }
+
+ @Override
+ public Object deserialize(Writable result) throws SerDeException {
+ if (!(result instanceof PhoenixResultWritable)) {
+ throw new SerDeException(result.getClass().getName() + ": expects " +
+ "PhoenixResultWritable!");
+ }
+
+ return row.setResultRowMap(((PhoenixResultWritable) result).getResultMap());
+ }
+
+ @Override
+ public Class<? extends Writable> getSerializedClass() {
+ return PhoenixResultWritable.class;
+ }
+
+ @Override
+ public Writable serialize(Object obj, ObjectInspector objInspector) throws SerDeException {
+ try {
+ return serializer.serialize(obj, objInspector, DmlType.NONE);
+ } catch (Exception e) {
+ throw new SerDeException(e);
+ }
+ }
+
+ @Override
+ public SerDeStats getSerDeStats() {
+ // no support for statistics
+ return null;
+ }
+
+ public Properties getTableProperties() {
+ return tableProperties;
+ }
+
+ public LazySerDeParameters getSerdeParams() {
+ return serdeParams;
+ }
+
+ @Override
+ public ObjectInspector getObjectInspector() throws SerDeException {
+ return objectInspector;
+ }
+
+ private ObjectInspector createLazyPhoenixInspector(Configuration conf, Properties tbl) throws
+ SerDeException {
+ List<String> columnNameList = Arrays.asList(tbl.getProperty(serdeConstants.LIST_COLUMNS)
+ .split(PhoenixStorageHandlerConstants.COMMA));
+ List<TypeInfo> columnTypeList = TypeInfoUtils.getTypeInfosFromTypeString(tbl.getProperty
+ (serdeConstants.LIST_COLUMN_TYPES));
+
+ List<ObjectInspector> columnObjectInspectors = Lists.newArrayListWithExpectedSize
+ (columnTypeList.size());
+
+ for (TypeInfo typeInfo : columnTypeList) {
+ columnObjectInspectors.add(PhoenixObjectInspectorFactory.createObjectInspector
+ (typeInfo, serdeParams));
+ }
+
+ return LazyObjectInspectorFactory.getLazySimpleStructObjectInspector(columnNameList,
+ columnObjectInspectors, null, serdeParams.getSeparators()[0], serdeParams,
+ ObjectInspectorOptions.JAVA);
+ }
+}
diff --git a/phoenix-hive/src/main/java/org/apache/phoenix/hive/PhoenixSerializer.java b/phoenix-hive/src/main/java/org/apache/phoenix/hive/PhoenixSerializer.java
new file mode 100644
index 0000000..852407a
--- /dev/null
+++ b/phoenix-hive/src/main/java/org/apache/phoenix/hive/PhoenixSerializer.java
@@ -0,0 +1,173 @@
+/*
+ * 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.phoenix.hive;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+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.serde2.SerDeException;
+import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector;
+import org.apache.hadoop.hive.serde2.objectinspector.PrimitiveObjectInspector;
+import org.apache.hadoop.hive.serde2.objectinspector.StandardStructObjectInspector;
+import org.apache.hadoop.hive.serde2.objectinspector.StructField;
+import org.apache.hadoop.hive.serde2.objectinspector.StructObjectInspector;
+import org.apache.hadoop.io.BytesWritable;
+import org.apache.hadoop.io.Writable;
+import org.apache.phoenix.hive.constants.PhoenixStorageHandlerConstants;
+import org.apache.phoenix.hive.mapreduce.PhoenixResultWritable;
+import org.apache.phoenix.hive.util.PhoenixConnectionUtil;
+import org.apache.phoenix.hive.util.PhoenixStorageHandlerUtil;
+import org.apache.phoenix.hive.util.PhoenixUtil;
+import org.apache.phoenix.util.ColumnInfo;
+
+import java.io.IOException;
+import java.sql.Connection;
+import java.sql.SQLException;
+import java.util.List;
+import java.util.Map;
+import java.util.Properties;
+
+/**
+ * Serializer used in PhoenixSerDe and PhoenixRecordUpdater to produce Writable.
+ */
+public class PhoenixSerializer {
+
+ private static final Log LOG = LogFactory.getLog(PhoenixSerializer.class);
+
+ public static enum DmlType {
+ NONE,
+ SELECT,
+ INSERT,
+ UPDATE,
+ DELETE
+ }
+
+ private int columnCount = 0;
+ private PhoenixResultWritable pResultWritable;
+
+ public PhoenixSerializer(Configuration config, Properties tbl) throws SerDeException {
+ String mapping = tbl.getProperty(PhoenixStorageHandlerConstants.PHOENIX_COLUMN_MAPPING, null);
+ if(mapping!=null ) {
+ config.set(PhoenixStorageHandlerConstants.PHOENIX_COLUMN_MAPPING, mapping);
+ }
+ try (Connection conn = PhoenixConnectionUtil.getInputConnection(config, tbl)) {
+ List<ColumnInfo> columnMetadata = PhoenixUtil.getColumnInfoList(conn, tbl.getProperty
+ (PhoenixStorageHandlerConstants.PHOENIX_TABLE_NAME));
+
+ columnCount = columnMetadata.size();
+
+ if (LOG.isDebugEnabled()) {
+ LOG.debug("Column-meta : " + columnMetadata);
+ }
+
+ pResultWritable = new PhoenixResultWritable(config, columnMetadata);
+ } catch (SQLException | IOException e) {
+ throw new SerDeException(e);
+ }
+ }
+
+ public Writable serialize(Object values, ObjectInspector objInspector, DmlType dmlType) {
+ pResultWritable.clear();
+
+ final StructObjectInspector structInspector = (StructObjectInspector) objInspector;
+ final List<? extends StructField> fieldList = structInspector.getAllStructFieldRefs();
+
+ if (LOG.isTraceEnabled()) {
+ LOG.trace("FieldList : " + fieldList + " values(" + values.getClass() + ") : " +
+ values);
+ }
+
+ int fieldCount = columnCount;
+ if (dmlType == DmlType.UPDATE || dmlType == DmlType.DELETE) {
+ fieldCount++;
+ }
+
+ for (int i = 0; i < fieldCount; i++) {
+ if (fieldList.size() <= i) {
+ break;
+ }
+
+ StructField structField = fieldList.get(i);
+
+ if (LOG.isTraceEnabled()) {
+ LOG.trace("structField[" + i + "] : " + structField);
+ }
+
+ if (structField != null) {
+ Object fieldValue = structInspector.getStructFieldData(values, structField);
+ ObjectInspector fieldOI = structField.getFieldObjectInspector();
+
+ String fieldName = structField.getFieldName();
+
+ if (LOG.isTraceEnabled()) {
+ LOG.trace("Field " + fieldName + "[" + i + "] : " + fieldValue + ", " +
+ fieldOI);
+ }
+
+ Object value = null;
+ switch (fieldOI.getCategory()) {
+ case PRIMITIVE:
+ value = ((PrimitiveObjectInspector) fieldOI).getPrimitiveJavaObject
+ (fieldValue);
+
+ if (LOG.isTraceEnabled()) {
+ LOG.trace("Field " + fieldName + "[" + i + "] : " + value + "(" + value
+ .getClass() + ")");
+ }
+
+ if (value instanceof HiveDecimal) {
+ value = ((HiveDecimal) value).bigDecimalValue();
+ } else if (value instanceof HiveChar) {
+ value = ((HiveChar) value).getValue().trim();
+ }
+
+ pResultWritable.add(value);
+ break;
+ case LIST:
+ // Not support for arrays in insert statement yet
+ break;
+ case STRUCT:
+ if (dmlType == DmlType.DELETE) {
+ // When update/delete, First value is struct<transactionid:bigint,
+ // bucketid:int,rowid:bigint,primaryKey:binary>>
+ List<Object> fieldValueList = ((StandardStructObjectInspector)
+ fieldOI).getStructFieldsDataAsList(fieldValue);
+
+ // convert to map from binary of primary key.
+ @SuppressWarnings("unchecked")
+ Map<String, Object> primaryKeyMap = (Map<String, Object>)
+ PhoenixStorageHandlerUtil.toMap(((BytesWritable)
+ fieldValueList.get(3)).getBytes());
+ for (Object pkValue : primaryKeyMap.values()) {
+ pResultWritable.add(pkValue);
+ }
+ }
+
+ break;
+ default:
+ new SerDeException("Phoenix Unsupported column type: " + fieldOI
+ .getCategory());
+ }
+ }
+ }
+
+ return pResultWritable;
+ }
+}
diff --git a/phoenix-hive/src/main/java/org/apache/phoenix/hive/PhoenixStorageHandler.java b/phoenix-hive/src/main/java/org/apache/phoenix/hive/PhoenixStorageHandler.java
new file mode 100644
index 0000000..4e9f465
--- /dev/null
+++ b/phoenix-hive/src/main/java/org/apache/phoenix/hive/PhoenixStorageHandler.java
@@ -0,0 +1,276 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.phoenix.hive;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.HBaseConfiguration;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.mapred.TableMapReduceUtil;
+import org.apache.hadoop.hive.common.JavaUtils;
+import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.metastore.HiveMetaHook;
+import org.apache.hadoop.hive.metastore.api.hive_metastoreConstants;
+import org.apache.hadoop.hive.ql.exec.TableScanOperator;
+import org.apache.hadoop.hive.ql.metadata.DefaultStorageHandler;
+import org.apache.hadoop.hive.ql.metadata.HiveException;
+import org.apache.hadoop.hive.ql.metadata.HiveStoragePredicateHandler;
+import org.apache.hadoop.hive.ql.metadata.InputEstimator;
+import org.apache.hadoop.hive.ql.plan.ExprNodeDesc;
+import org.apache.hadoop.hive.ql.plan.TableDesc;
+import org.apache.hadoop.hive.ql.session.SessionState;
+import org.apache.hadoop.hive.serde2.Deserializer;
+import org.apache.hadoop.hive.serde2.SerDe;
+import org.apache.hadoop.hive.shims.ShimLoader;
+import org.apache.hadoop.mapred.InputFormat;
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.hadoop.mapred.OutputFormat;
+import org.apache.phoenix.hive.constants.PhoenixStorageHandlerConstants;
+import org.apache.phoenix.hive.mapreduce.PhoenixInputFormat;
+import org.apache.phoenix.hive.mapreduce.PhoenixOutputFormat;
+import org.apache.phoenix.hive.ppd.PhoenixPredicateDecomposer;
+import org.apache.phoenix.mapreduce.util.PhoenixConfigurationUtil;
+
+import java.io.IOException;
+import java.util.List;
+import java.util.Map;
+import java.util.Properties;
+import java.util.UUID;
+
+/**
+ * This class manages all the Phoenix/Hive table initial configurations and SerDe Election
+ */
+@SuppressWarnings("deprecation")
+public class PhoenixStorageHandler extends DefaultStorageHandler implements
+ HiveStoragePredicateHandler, InputEstimator {
+
+
+ private Configuration jobConf;
+ private Configuration hbaseConf;
+
+
+ @Override
+ public void setConf(Configuration conf) {
+ jobConf = conf;
+ hbaseConf = HBaseConfiguration.create(conf);
+ }
+
+ @Override
+ public Configuration getConf() {
+ return hbaseConf;
+ }
+
+ private static final Log LOG = LogFactory.getLog(PhoenixStorageHandler.class);
+
+ public PhoenixStorageHandler() {
+ if (LOG.isDebugEnabled()) {
+ LOG.debug("PhoenixStorageHandler created");
+ }
+ }
+
+ @Override
+ public HiveMetaHook getMetaHook() {
+ return new PhoenixMetaHook();
+ }
+
+ @Override
+ public void configureJobConf(TableDesc tableDesc, JobConf jobConf) {
+ try {
+ TableMapReduceUtil.addDependencyJars(jobConf);
+ org.apache.hadoop.hbase.mapreduce.TableMapReduceUtil.addDependencyJars(jobConf,
+ PhoenixStorageHandler.class);
+ JobConf hbaseJobConf = new JobConf(getConf());
+ org.apache.hadoop.hbase.mapred.TableMapReduceUtil.initCredentials(hbaseJobConf);
+ ShimLoader.getHadoopShims().mergeCredentials(jobConf, hbaseJobConf);
+ } catch (IOException e) {
+ throw new RuntimeException(e);
+ }
+
+
+ }
+
+ @SuppressWarnings("rawtypes")
+ @Override
+ public Class<? extends OutputFormat> getOutputFormatClass() {
+ return PhoenixOutputFormat.class;
+ }
+
+ @SuppressWarnings("rawtypes")
+ @Override
+ public Class<? extends InputFormat> getInputFormatClass() {
+ return PhoenixInputFormat.class;
+ }
+
+ @Override
+ public void configureInputJobProperties(TableDesc tableDesc, Map<String, String>
+ jobProperties) {
+ configureJobProperties(tableDesc, jobProperties);
+
+ if (LOG.isDebugEnabled()) {
+ LOG.debug("Configuring input job for table : " + tableDesc.getTableName());
+ }
+
+ // initialization efficiency. Inform to SerDe about in/out work.
+ tableDesc.getProperties().setProperty(PhoenixStorageHandlerConstants.IN_OUT_WORK,
+ PhoenixStorageHandlerConstants.IN_WORK);
+ }
+
+ @Override
+ public void configureOutputJobProperties(TableDesc tableDesc, Map<String, String>
+ jobProperties) {
+ configureJobProperties(tableDesc, jobProperties);
+
+ if (LOG.isDebugEnabled()) {
+ LOG.debug("Configuring output job for table : " + tableDesc.getTableName());
+ }
+
+ // initialization efficiency. Inform to SerDe about in/out work.
+ tableDesc.getProperties().setProperty(PhoenixStorageHandlerConstants.IN_OUT_WORK,
+ PhoenixStorageHandlerConstants.OUT_WORK);
+ }
+
+ @Override
+ public void configureTableJobProperties(TableDesc tableDesc, Map<String, String>
+ jobProperties) {
+ configureJobProperties(tableDesc, jobProperties);
+ }
+
+ @SuppressWarnings({"unchecked", "rawtypes"})
+ protected void configureJobProperties(TableDesc tableDesc, Map<String, String> jobProperties) {
+ Properties tableProperties = tableDesc.getProperties();
+
+ String inputFormatClassName =
+ tableProperties.getProperty(PhoenixStorageHandlerConstants
+ .HBASE_INPUT_FORMAT_CLASS);
+
+ if (LOG.isDebugEnabled()) {
+ LOG.debug(PhoenixStorageHandlerConstants.HBASE_INPUT_FORMAT_CLASS + " is " +
+ inputFormatClassName);
+ }
+
+ Class<?> inputFormatClass;
+ try {
+ if (inputFormatClassName != null) {
+ inputFormatClass = JavaUtils.loadClass(inputFormatClassName);
+ } else {
+ inputFormatClass = PhoenixInputFormat.class;
+ }
+ } catch (Exception e) {
+ LOG.error(e.getMessage(), e);
+ throw new RuntimeException(e);
+ }
+
+ if (inputFormatClass != null) {
+ tableDesc.setInputFileFormatClass((Class<? extends InputFormat>) inputFormatClass);
+ }
+
+ String tableName = tableProperties.getProperty(PhoenixStorageHandlerConstants
+ .PHOENIX_TABLE_NAME);
+ if (tableName == null) {
+ tableName = tableDesc.getTableName();
+ tableProperties.setProperty(PhoenixStorageHandlerConstants.PHOENIX_TABLE_NAME,
+ tableName);
+ }
+ SessionState sessionState = SessionState.get();
+
+ String sessionId;
+ if(sessionState!= null) {
+ sessionId = sessionState.getSessionId();
+ } else {
+ sessionId = UUID.randomUUID().toString();
+ }
+ jobProperties.put(PhoenixConfigurationUtil.SESSION_ID, sessionId);
+ jobProperties.put(PhoenixConfigurationUtil.INPUT_TABLE_NAME, tableName);
+ jobProperties.put(PhoenixStorageHandlerConstants.ZOOKEEPER_QUORUM, tableProperties
+ .getProperty(PhoenixStorageHandlerConstants.ZOOKEEPER_QUORUM,
+ PhoenixStorageHandlerConstants.DEFAULT_ZOOKEEPER_QUORUM));
+ jobProperties.put(PhoenixStorageHandlerConstants.ZOOKEEPER_PORT, tableProperties
+ .getProperty(PhoenixStorageHandlerConstants.ZOOKEEPER_PORT, String.valueOf
+ (PhoenixStorageHandlerConstants.DEFAULT_ZOOKEEPER_PORT)));
+ jobProperties.put(PhoenixStorageHandlerConstants.ZOOKEEPER_PARENT, tableProperties
+ .getProperty(PhoenixStorageHandlerConstants.ZOOKEEPER_PARENT,
+ PhoenixStorageHandlerConstants.DEFAULT_ZOOKEEPER_PARENT));
+ String columnMapping = tableProperties
+ .getProperty(PhoenixStorageHandlerConstants.PHOENIX_COLUMN_MAPPING);
+ if(columnMapping != null) {
+ jobProperties.put(PhoenixStorageHandlerConstants.PHOENIX_COLUMN_MAPPING, columnMapping);
+ }
+
+ jobProperties.put(hive_metastoreConstants.META_TABLE_STORAGE, this.getClass().getName());
+
+ // set configuration when direct work with HBase.
+ jobProperties.put(HConstants.ZOOKEEPER_QUORUM, jobProperties.get
+ (PhoenixStorageHandlerConstants.ZOOKEEPER_QUORUM));
+ jobProperties.put(HConstants.ZOOKEEPER_CLIENT_PORT, jobProperties.get
+ (PhoenixStorageHandlerConstants.ZOOKEEPER_PORT));
+ jobProperties.put(HConstants.ZOOKEEPER_ZNODE_PARENT, jobProperties.get
+ (PhoenixStorageHandlerConstants.ZOOKEEPER_PARENT));
+ addHBaseResources(jobConf, jobProperties);
+ }
+
+ /**
+ * Utility method to add hbase-default.xml and hbase-site.xml properties to a new map
+ * if they are not already present in the jobConf.
+ * @param jobConf Job configuration
+ * @param newJobProperties Map to which new properties should be added
+ */
+ private void addHBaseResources(Configuration jobConf,
+ Map<String, String> newJobProperties) {
+ Configuration conf = new Configuration(false);
+ HBaseConfiguration.addHbaseResources(conf);
+ for (Map.Entry<String, String> entry : conf) {
+ if (jobConf.get(entry.getKey()) == null) {
+ newJobProperties.put(entry.getKey(), entry.getValue());
+ }
+ }
+ }
+
+ @Override
+ public Class<? extends SerDe> getSerDeClass() {
+ return PhoenixSerDe.class;
+ }
+
+ @Override
+ public DecomposedPredicate decomposePredicate(JobConf jobConf, Deserializer deserializer,
+ ExprNodeDesc predicate) {
+ PhoenixSerDe phoenixSerDe = (PhoenixSerDe) deserializer;
+ List<String> columnNameList = phoenixSerDe.getSerdeParams().getColumnNames();
+
+ return PhoenixPredicateDecomposer.create(columnNameList).decomposePredicate(predicate);
+ }
+
+ @Override
+ public Estimation estimate(JobConf job, TableScanOperator ts, long remaining) throws
+ HiveException {
+ String hiveTableName = ts.getConf().getTableMetadata().getTableName();
+ int reducerCount = job.getInt(hiveTableName + PhoenixStorageHandlerConstants
+ .PHOENIX_REDUCER_NUMBER, 1);
+
+ if (LOG.isDebugEnabled()) {
+ LOG.debug("Estimating input size for table: " + hiveTableName + " with reducer count " +
+ reducerCount + ". Remaining : " + remaining);
+ }
+
+ long bytesPerReducer = job.getLong(HiveConf.ConfVars.BYTESPERREDUCER.varname,
+ Long.parseLong(HiveConf.ConfVars.BYTESPERREDUCER.getDefaultValue()));
+ long totalLength = reducerCount * bytesPerReducer;
+
+ return new Estimation(0, totalLength);
+ }
+}
diff --git a/phoenix-hive/src/main/java/org/apache/phoenix/hive/PrimaryKeyData.java b/phoenix-hive/src/main/java/org/apache/phoenix/hive/PrimaryKeyData.java
new file mode 100644
index 0000000..7773997
--- /dev/null
+++ b/phoenix-hive/src/main/java/org/apache/phoenix/hive/PrimaryKeyData.java
@@ -0,0 +1,88 @@
+/*
+ * 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.phoenix.hive;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.InvalidClassException;
+import java.io.ObjectInputStream;
+import java.io.ObjectOutputStream;
+import java.io.ObjectStreamClass;
+import java.io.OutputStream;
+import java.io.Serializable;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Objects;
+
+/**
+ * Wrapper around the primary key data for Hive.
+ */
+public class PrimaryKeyData implements Serializable{
+ public static final PrimaryKeyData EMPTY = new PrimaryKeyData(Collections.<String,Object> emptyMap());
+ private static final long serialVersionUID = 1L;
+
+ // Based on https://www.ibm.com/developerworks/library/se-lookahead/. Prevents unexpected
+ // deserialization of other objects of an unexpected class.
+ private static class LookAheadObjectInputStream extends ObjectInputStream {
+ public LookAheadObjectInputStream(InputStream in) throws IOException {
+ super(in);
+ }
+
+ @Override
+ protected Class<?> resolveClass(ObjectStreamClass desc) throws IOException, ClassNotFoundException {
+ if (!desc.getName().equals(PrimaryKeyData.class.getName()) &&
+ !desc.getName().startsWith("java.lang.") &&
+ !desc.getName().startsWith("java.util.") &&
+ !desc.getName().startsWith("java.sql.")) {
+ throw new InvalidClassException(desc.getName(), "Expected an instance of PrimaryKeyData");
+ }
+ return super.resolveClass(desc);
+ }
+ }
+
+ private final HashMap<String,Object> data;
+
+ public PrimaryKeyData(Map<String,Object> data) {
+ if (data instanceof HashMap) {
+ this.data = (HashMap<String,Object>) data;
+ } else {
+ this.data = new HashMap<>(Objects.requireNonNull(data));
+ }
+ }
+
+ public HashMap<String,Object> getData() {
+ return data;
+ }
+
+ public void serialize(OutputStream output) throws IOException {
+ try (ObjectOutputStream oos = new ObjectOutputStream(output)) {
+ oos.writeObject(this);
+ oos.flush();
+ }
+ }
+
+ public static PrimaryKeyData deserialize(InputStream input) throws IOException, ClassNotFoundException {
+ try (LookAheadObjectInputStream ois = new LookAheadObjectInputStream(input)) {
+ Object obj = ois.readObject();
+ if (obj instanceof PrimaryKeyData) {
+ return (PrimaryKeyData) obj;
+ }
+ throw new InvalidClassException(obj == null ? "null" : obj.getClass().getName(), "Disallowed serialized class");
+ }
+ }
+}
diff --git a/phoenix-hive/src/main/java/org/apache/phoenix/hive/constants/PhoenixStorageHandlerConstants.java b/phoenix-hive/src/main/java/org/apache/phoenix/hive/constants/PhoenixStorageHandlerConstants.java
new file mode 100644
index 0000000..e3c7d84
--- /dev/null
+++ b/phoenix-hive/src/main/java/org/apache/phoenix/hive/constants/PhoenixStorageHandlerConstants.java
@@ -0,0 +1,108 @@
+/*
+ * 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.phoenix.hive.constants;
+
+import com.google.common.collect.Lists;
+import org.apache.hadoop.io.IntWritable;
+
+import java.util.List;
+
+/**
+ * Constants using for Hive Storage Handler implementation
+ */
+public class PhoenixStorageHandlerConstants {
+
+ public static final String HBASE_INPUT_FORMAT_CLASS = "phoenix.input.format.class";
+
+ public static final String PHOENIX_TABLE_NAME = "phoenix.table.name";
+
+ public static final String DEFAULT_PHOENIX_INPUT_CLASS = "org.apache.phoenix.hive.mapreduce" +
+ ".PhoenixResultWritable";
+
+ public static final String ZOOKEEPER_QUORUM = "phoenix.zookeeper.quorum";
+ public static final String ZOOKEEPER_PORT = "phoenix.zookeeper.client.port";
+ public static final String ZOOKEEPER_PARENT = "phoenix.zookeeper.znode.parent";
+ public static final String DEFAULT_ZOOKEEPER_QUORUM = "localhost";
+ public static final int DEFAULT_ZOOKEEPER_PORT = 2181;
+ public static final String DEFAULT_ZOOKEEPER_PARENT = "/hbase";
+
+ public static final String PHOENIX_ROWKEYS = "phoenix.rowkeys";
+ public static final String PHOENIX_COLUMN_MAPPING = "phoenix.column.mapping";
+ public static final String PHOENIX_TABLE_OPTIONS = "phoenix.table.options";
+
+ public static final String PHOENIX_TABLE_QUERY_HINT = ".query.hint";
+ public static final String PHOENIX_REDUCER_NUMBER = ".reducer.count";
+ public static final String DISABLE_WAL = ".disable.wal";
+ public static final String BATCH_MODE = "batch.mode";
+ public static final String AUTO_FLUSH = ".auto.flush";
+
+ public static final String COLON = ":";
+ public static final String COMMA = ",";
+ public static final String EMPTY_STRING = "";
+ public static final String SPACE = " ";
+ public static final String LEFT_ROUND_BRACKET = "(";
+ public static final String RIGHT_ROUND_BRACKET = ")";
+ public static final String QUOTATION_MARK = "'";
+ public static final String EQUAL = "=";
+ public static final String IS = "is";
+ public static final String QUESTION = "?";
+
+ public static final String SPLIT_BY_STATS = "split.by.stats";
+ public static final String HBASE_SCAN_CACHE = "hbase.scan.cache";
+ public static final String HBASE_SCAN_CACHEBLOCKS = "hbase.scan.cacheblock";
+ public static final String HBASE_DATE_FORMAT = "hbase.date.format";
+ public static final String HBASE_TIMESTAMP_FORMAT = "hbase.timestamp.format";
+ public static final String DEFAULT_DATE_FORMAT = "yyyy-MM-dd";
+ public static final String DEFAULT_TIMESTAMP_FORMAT = "yyyy-MM-dd HH:mm:ss.SSS";
+
+ public static final String IN_OUT_WORK = "in.out.work";
+ public static final String IN_WORK = "input";
+ public static final String OUT_WORK = "output";
+
+ public static final String MR = "mr";
+ public static final String TEZ = "tez";
+ public static final String SPARK = "spark";
+
+ public static final String DATE_TYPE = "date";
+ public static final String TIMESTAMP_TYPE = "timestamp";
+ public static final String BETWEEN_COMPARATOR = "between";
+ public static final String IN_COMPARATOR = "in";
+ public static final List<String> COMMON_COMPARATOR = Lists.newArrayList("=", "<", ">", "<=",
+ ">=");
+
+ // date/timestamp
+ public static final String COLUMNE_MARKER = "$columnName$";
+ public static final String PATERN_MARKER = "$targetPattern$";
+ public static final String DATE_PATTERN = "'?\\d{4}-\\d{2}-\\d{2}'?";
+ public static final String TIMESTAMP_PATTERN = "'?\\d{4}-\\d{2}-\\d{2} \\d{2}:\\d{2}:\\d{2}\\" +
+ ".?\\d{0,9}'?";
+ public static final String COMMON_OPERATOR_PATTERN = "(\\(?\"?" + COLUMNE_MARKER + "\"?\\)?\\s*" +
+ "(=|>|<|<=|>=)\\s*(" + PATERN_MARKER + "))";
+ public static final String BETWEEN_OPERATOR_PATTERN = "(\\(?\"?" + COLUMNE_MARKER + "\"?\\)?\\s*(" +
+ "(?i)not)?\\s*(?i)between\\s*(" + PATERN_MARKER + ")\\s*(?i)and\\s*(" + PATERN_MARKER
+ + "))";
+ public static final String IN_OPERATOR_PATTERN = "(\\(?\"?" + COLUMNE_MARKER + "\"?\\)?\\s*((?i)" +
+ "not)?\\s*(?i)in\\s*\\((" + PATERN_MARKER + ",?\\s*)+\\))";
+
+ public static final String FUNCTION_VALUE_MARKER = "$value$";
+ public static final String DATE_FUNCTION_TEMPLETE = "to_date(" + FUNCTION_VALUE_MARKER + ")";
+ public static final String TIMESTAMP_FUNCTION_TEMPLATE = "to_timestamp(" +
+ FUNCTION_VALUE_MARKER + ")";
+
+ public static final IntWritable INT_ZERO = new IntWritable(0);
+}
diff --git a/phoenix-hive/src/main/java/org/apache/phoenix/hive/mapreduce/PhoenixInputFormat.java b/phoenix-hive/src/main/java/org/apache/phoenix/hive/mapreduce/PhoenixInputFormat.java
new file mode 100644
index 0000000..b550e32
--- /dev/null
+++ b/phoenix-hive/src/main/java/org/apache/phoenix/hive/mapreduce/PhoenixInputFormat.java
@@ -0,0 +1,270 @@
+/*
+ * 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.phoenix.hive.mapreduce;
+
+import com.google.common.base.Preconditions;
+import com.google.common.collect.Lists;
+import java.io.IOException;
+import java.sql.Connection;
+import java.sql.Statement;
+import java.util.Arrays;
+import java.util.List;
+import java.util.Map;
+import java.util.Properties;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.HRegionLocation;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.HConnection;
+import org.apache.hadoop.hbase.client.HConnectionManager;
+import org.apache.hadoop.hbase.client.RegionLocator;
+import org.apache.hadoop.hbase.client.Scan;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.RegionSizeCalculator;
+import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.ql.exec.Utilities;
+import org.apache.hadoop.hive.ql.plan.ExprNodeGenericFuncDesc;
+import org.apache.hadoop.hive.ql.plan.TableScanDesc;
+import org.apache.hadoop.hive.serde.serdeConstants;
+import org.apache.hadoop.hive.serde2.typeinfo.TypeInfo;
+import org.apache.hadoop.hive.shims.ShimLoader;
+import org.apache.hadoop.io.WritableComparable;
+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 org.apache.hadoop.mapreduce.Job;
+import org.apache.hadoop.mapreduce.lib.db.DBWritable;
+import org.apache.hadoop.mapreduce.lib.input.FileInputFormat;
+import org.apache.phoenix.compile.QueryPlan;
+import org.apache.phoenix.coprocessor.BaseScannerRegionObserver;
+import org.apache.phoenix.hive.constants.PhoenixStorageHandlerConstants;
+import org.apache.phoenix.hive.ppd.PhoenixPredicateDecomposer;
+import org.apache.phoenix.hive.ql.index.IndexSearchCondition;
+import org.apache.phoenix.hive.query.PhoenixQueryBuilder;
+import org.apache.phoenix.hive.util.PhoenixConnectionUtil;
+import org.apache.phoenix.hive.util.PhoenixStorageHandlerUtil;
+import org.apache.phoenix.iterate.MapReduceParallelScanGrouper;
+import org.apache.phoenix.jdbc.PhoenixStatement;
+import org.apache.phoenix.mapreduce.util.PhoenixConfigurationUtil;
+import org.apache.phoenix.query.KeyRange;
+import org.apache.phoenix.util.PhoenixRuntime;
+
+/**
+ * Custom InputFormat to feed into Hive
+ */
+@SuppressWarnings({"deprecation", "rawtypes"})
+public class PhoenixInputFormat<T extends DBWritable> implements InputFormat<WritableComparable,
+ T> {
+
+ private static final Log LOG = LogFactory.getLog(PhoenixInputFormat.class);
+
+ public PhoenixInputFormat() {
+ if (LOG.isDebugEnabled()) {
+ LOG.debug("PhoenixInputFormat created");
+ }
+ }
+
+ @Override
+ public InputSplit[] getSplits(JobConf jobConf, int numSplits) throws IOException {
+ String tableName = jobConf.get(PhoenixStorageHandlerConstants.PHOENIX_TABLE_NAME);
+
+ String query;
+ String executionEngine = jobConf.get(HiveConf.ConfVars.HIVE_EXECUTION_ENGINE.varname,
+ HiveConf.ConfVars.HIVE_EXECUTION_ENGINE.getDefaultValue());
+ if (LOG.isDebugEnabled()) {
+ LOG.debug("Target table name at split phase : " + tableName + "with whereCondition :" +
+ jobConf.get(TableScanDesc.FILTER_TEXT_CONF_STR) +
+ " and " + HiveConf.ConfVars.HIVE_EXECUTION_ENGINE.varname + " : " +
+ executionEngine);
+ }
+
+ if (PhoenixStorageHandlerConstants.MR.equals(executionEngine)) {
+ List<IndexSearchCondition> conditionList = null;
+ String filterExprSerialized = jobConf.get(TableScanDesc.FILTER_EXPR_CONF_STR);
+ if (filterExprSerialized != null) {
+ ExprNodeGenericFuncDesc filterExpr =
+ Utilities.deserializeExpression(filterExprSerialized);
+ PhoenixPredicateDecomposer predicateDecomposer =
+ PhoenixPredicateDecomposer.create(Arrays.asList(jobConf.get(serdeConstants.LIST_COLUMNS).split(",")));
+ predicateDecomposer.decomposePredicate(filterExpr);
+ if (predicateDecomposer.isCalledPPD()) {
+ conditionList = predicateDecomposer.getSearchConditionList();
+ }
+ }
+
+ query = PhoenixQueryBuilder.getInstance().buildQuery(jobConf, tableName,
+ PhoenixStorageHandlerUtil.getReadColumnNames(jobConf), conditionList);
+ } else if (PhoenixStorageHandlerConstants.TEZ.equals(executionEngine)) {
+ Map<String, TypeInfo> columnTypeMap =
+ PhoenixStorageHandlerUtil.createColumnTypeMap(jobConf);
+ if (LOG.isDebugEnabled()) {
+ LOG.debug("Column type map for TEZ : " + columnTypeMap);
+ }
+
+ String whereClause = jobConf.get(TableScanDesc.FILTER_TEXT_CONF_STR);
+ query = PhoenixQueryBuilder.getInstance().buildQuery(jobConf, tableName,
+ PhoenixStorageHandlerUtil.getReadColumnNames(jobConf), whereClause, columnTypeMap);
+ } else {
+ throw new IOException(executionEngine + " execution engine unsupported yet.");
+ }
+
+ final QueryPlan queryPlan = getQueryPlan(jobConf, query);
+ final List<KeyRange> allSplits = queryPlan.getSplits();
+ final List<InputSplit> splits = generateSplits(jobConf, queryPlan, allSplits, query);
+
+ return splits.toArray(new InputSplit[splits.size()]);
+ }
+
+ private List<InputSplit> generateSplits(final JobConf jobConf, final QueryPlan qplan,
+ final List<KeyRange> splits, String query) throws
+ IOException {
+ Preconditions.checkNotNull(qplan);
+ Preconditions.checkNotNull(splits);
+ final List<InputSplit> psplits = Lists.newArrayListWithExpectedSize(splits.size());
+
+ Path[] tablePaths = FileInputFormat.getInputPaths(ShimLoader.getHadoopShims()
+ .newJobContext(new Job(jobConf)));
+ boolean splitByStats = jobConf.getBoolean(PhoenixStorageHandlerConstants.SPLIT_BY_STATS,
+ false);
+
+ setScanCacheSize(jobConf);
+
+ // Adding Localization
+ try (HConnection connection = HConnectionManager.createConnection(PhoenixConnectionUtil.getConfiguration(jobConf))) {
+ RegionLocator regionLocator = connection.getRegionLocator(TableName.valueOf(qplan
+ .getTableRef().getTable().getPhysicalName().toString()));
+ RegionSizeCalculator sizeCalculator = new RegionSizeCalculator(regionLocator, connection
+ .getAdmin());
+
+ for (List<Scan> scans : qplan.getScans()) {
+ PhoenixInputSplit inputSplit;
+
+ HRegionLocation location = regionLocator.getRegionLocation(scans.get(0).getStartRow()
+ , false);
+ long regionSize = sizeCalculator.getRegionSize(location.getRegionInfo().getRegionName
+ ());
+ String regionLocation = PhoenixStorageHandlerUtil.getRegionLocation(location, LOG);
+
+ if (splitByStats) {
+ for (Scan aScan : scans) {
+ if (LOG.isDebugEnabled()) {
+ LOG.debug("Split for scan : " + aScan + "with scanAttribute : " + aScan
+ .getAttributesMap() + " [scanCache, cacheBlock, scanBatch] : [" +
+ aScan.getCaching() + ", " + aScan.getCacheBlocks() + ", " + aScan
+ .getBatch() + "] and regionLocation : " + regionLocation);
+ }
+
+ inputSplit = new PhoenixInputSplit(Lists.newArrayList(aScan), tablePaths[0],
+ regionLocation, regionSize);
+ inputSplit.setQuery(query);
+ psplits.add(inputSplit);
+ }
+ } else {
+ if (LOG.isDebugEnabled()) {
+ LOG.debug("Scan count[" + scans.size() + "] : " + Bytes.toStringBinary(scans
+ .get(0).getStartRow()) + " ~ " + Bytes.toStringBinary(scans.get(scans
+ .size() - 1).getStopRow()));
+ LOG.debug("First scan : " + scans.get(0) + "with scanAttribute : " + scans
+ .get(0).getAttributesMap() + " [scanCache, cacheBlock, scanBatch] : " +
+ "[" + scans.get(0).getCaching() + ", " + scans.get(0).getCacheBlocks()
+ + ", " + scans.get(0).getBatch() + "] and regionLocation : " +
+ regionLocation);
+
+ for (int i = 0, limit = scans.size(); i < limit; i++) {
+ LOG.debug("EXPECTED_UPPER_REGION_KEY[" + i + "] : " + Bytes
+ .toStringBinary(scans.get(i).getAttribute
+ (BaseScannerRegionObserver.EXPECTED_UPPER_REGION_KEY)));
+ }
+ }
+
+ inputSplit = new PhoenixInputSplit(scans, tablePaths[0], regionLocation,
+ regionSize);
+ inputSplit.setQuery(query);
+ psplits.add(inputSplit);
+ }
+ }
+ }
+
+ return psplits;
+ }
+
+ private void setScanCacheSize(JobConf jobConf) {
+ int scanCacheSize = jobConf.getInt(PhoenixStorageHandlerConstants.HBASE_SCAN_CACHE, -1);
+ if (scanCacheSize > 0) {
+ jobConf.setInt(HConstants.HBASE_CLIENT_SCANNER_CACHING, scanCacheSize);
+ }
+
+ if (LOG.isDebugEnabled()) {
+ LOG.debug("Generating splits with scanCacheSize : " + scanCacheSize);
+ }
+ }
+
+ @Override
+ public RecordReader<WritableComparable, T> getRecordReader(InputSplit split, JobConf job,
+ Reporter reporter) throws
+ IOException {
+ final QueryPlan queryPlan = getQueryPlan(job, ((PhoenixInputSplit) split).getQuery());
+ @SuppressWarnings("unchecked")
+ final Class<T> inputClass = (Class<T>) job.getClass(PhoenixConfigurationUtil.INPUT_CLASS,
+ PhoenixResultWritable.class);
+
+ PhoenixRecordReader<T> recordReader = new PhoenixRecordReader<T>(inputClass, job,
+ queryPlan);
+ recordReader.initialize(split);
+
+ return recordReader;
+ }
+
+ /**
+ * Returns the query plan associated with the select query.
+ */
+ private QueryPlan getQueryPlan(final Configuration configuration, String selectStatement)
+ throws IOException {
+ try {
+ final String currentScnValue = configuration.get(PhoenixConfigurationUtil
+ .CURRENT_SCN_VALUE);
+ final Properties overridingProps = new Properties();
+ if (currentScnValue != null) {
+ overridingProps.put(PhoenixRuntime.CURRENT_SCN_ATTRIB, currentScnValue);
+ }
+ final Connection connection = PhoenixConnectionUtil.getInputConnection(configuration,
+ overridingProps);
+ Preconditions.checkNotNull(selectStatement);
+ final Statement statement = connection.createStatement();
+ final PhoenixStatement pstmt = statement.unwrap(PhoenixStatement.class);
+
+ if (LOG.isDebugEnabled()) {
+ LOG.debug("Compiled query : " + selectStatement);
+ }
+
+ // Optimize the query plan so that we potentially use secondary indexes
+ final QueryPlan queryPlan = pstmt.optimizeQuery(selectStatement);
+ // Initialize the query plan so it sets up the parallel scans
+ queryPlan.iterator(MapReduceParallelScanGrouper.getInstance());
+ return queryPlan;
+ } catch (Exception exception) {
+ LOG.error(String.format("Failed to get the query plan with error [%s]", exception.getMessage()));
+ throw new RuntimeException(exception);
+ }
+ }
+}
diff --git a/phoenix-hive/src/main/java/org/apache/phoenix/hive/mapreduce/PhoenixInputSplit.java b/phoenix-hive/src/main/java/org/apache/phoenix/hive/mapreduce/PhoenixInputSplit.java
new file mode 100644
index 0000000..d76e863
--- /dev/null
+++ b/phoenix-hive/src/main/java/org/apache/phoenix/hive/mapreduce/PhoenixInputSplit.java
@@ -0,0 +1,160 @@
+/*
+ * 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.phoenix.hive.mapreduce;
+
+import com.google.common.base.Preconditions;
+import com.google.common.collect.Lists;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.client.Scan;
+import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
+import org.apache.hadoop.hbase.protobuf.generated.ClientProtos;
+import org.apache.hadoop.io.WritableUtils;
+import org.apache.hadoop.mapred.FileSplit;
+import org.apache.hadoop.mapred.InputSplit;
+import org.apache.phoenix.query.KeyRange;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.util.List;
+
+/**
+ * InputSplit implementation. Represents the data to be processed by an individual Mapper
+ */
+public class PhoenixInputSplit extends FileSplit implements InputSplit {
+
+ private List<Scan> scans;
+ private KeyRange keyRange;
+
+ private long regionSize;
+
+ // query is in the split because it is not delivered in jobConf.
+ private String query;
+
+ public PhoenixInputSplit() {
+ }
+
+ public PhoenixInputSplit(final List<Scan> scans, Path dummyPath, String regionLocation, long
+ length) {
+ super(dummyPath, 0, 0, new String[]{regionLocation});
+
+ regionSize = length;
+
+ Preconditions.checkNotNull(scans);
+ Preconditions.checkState(!scans.isEmpty());
+ this.scans = scans;
+ init();
+ }
+
+ public List<Scan> getScans() {
+ return scans;
+ }
+
+ public KeyRange getKeyRange() {
+ return keyRange;
+ }
+
+ public String getQuery() {
+ return query;
+ }
+
+ public void setQuery(String query) {
+ this.query = query;
+ }
+
+ private void init() {
+ this.keyRange = KeyRange.getKeyRange(scans.get(0).getStartRow(), scans.get(scans.size() -
+ 1).getStopRow());
+ }
+
+ @Override
+ public void write(DataOutput out) throws IOException {
+ super.write(out);
+
+ Preconditions.checkNotNull(scans);
+ WritableUtils.writeVInt(out, scans.size());
+ for (Scan scan : scans) {
+ ClientProtos.Scan protoScan = ProtobufUtil.toScan(scan);
+ byte[] protoScanBytes = protoScan.toByteArray();
+ WritableUtils.writeVInt(out, protoScanBytes.length);
+ out.write(protoScanBytes);
+ }
+
+ WritableUtils.writeString(out, query);
+ WritableUtils.writeVLong(out, regionSize);
+ }
+
+ @Override
+ public void readFields(DataInput in) throws IOException {
+ super.readFields(in);
+
+ int count = WritableUtils.readVInt(in);
+ scans = Lists.newArrayListWithExpectedSize(count);
+ for (int i = 0; i < count; i++) {
+ byte[] protoScanBytes = new byte[WritableUtils.readVInt(in)];
+ in.readFully(protoScanBytes);
+ ClientProtos.Scan protoScan = ClientProtos.Scan.parseFrom(protoScanBytes);
+ Scan scan = ProtobufUtil.toScan(protoScan);
+ scans.add(scan);
+ }
+ init();
+
+ query = WritableUtils.readString(in);
+ regionSize = WritableUtils.readVLong(in);
+ }
+
+ @Override
+ public long getLength() {
+ return regionSize;
+ }
+
+ @Override
+ public String[] getLocations() throws IOException {
+ return new String[]{};
+ }
+
+ @Override
+ public int hashCode() {
+ final int prime = 31;
+ int result = 1;
+ result = prime * result + keyRange.hashCode();
+ return result;
+ }
+
+ @Override
+ public boolean equals(Object obj) {
+ if (this == obj) {
+ return true;
+ }
+ if (obj == null) {
+ return false;
+ }
+ if (!(obj instanceof PhoenixInputSplit)) {
+ return false;
+ }
+ PhoenixInputSplit other = (PhoenixInputSplit) obj;
+ if (keyRange == null) {
+ if (other.keyRange != null) {
+ return false;
+ }
+ } else if (!keyRange.equals(other.keyRange)) {
+ return false;
+ }
+ return true;
+ }
+}
diff --git a/phoenix-hive/src/main/java/org/apache/phoenix/hive/mapreduce/PhoenixOutputFormat.java b/phoenix-hive/src/main/java/org/apache/phoenix/hive/mapreduce/PhoenixOutputFormat.java
new file mode 100644
index 0000000..ed47176
--- /dev/null
+++ b/phoenix-hive/src/main/java/org/apache/phoenix/hive/mapreduce/PhoenixOutputFormat.java
@@ -0,0 +1,112 @@
+/*
+ * 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.phoenix.hive.mapreduce;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hive.ql.io.AcidOutputFormat;
+import org.apache.hadoop.hive.ql.io.RecordUpdater;
+import org.apache.hadoop.io.NullWritable;
+import org.apache.hadoop.io.Writable;
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.hadoop.mapred.OutputFormat;
+import org.apache.hadoop.mapred.RecordWriter;
+import org.apache.hadoop.mapreduce.lib.db.DBWritable;
+import org.apache.hadoop.util.Progressable;
+import org.apache.phoenix.hive.util.PhoenixStorageHandlerUtil;
+
+import java.io.IOException;
+import java.sql.SQLException;
+import java.util.Properties;
+
+/**
+ * Custom OutputFormat to feed into Hive. Describes the output-specification for a Map-Reduce job.
+ */
+public class PhoenixOutputFormat<T extends DBWritable> implements OutputFormat<NullWritable, T>,
+ AcidOutputFormat<NullWritable, T> {
+
+ private static final Log LOG = LogFactory.getLog(PhoenixOutputFormat.class);
+
+ public PhoenixOutputFormat() {
+ if (LOG.isDebugEnabled()) {
+ LOG.debug("PhoenixOutputFormat created");
+ }
+ }
+
+ @Override
+ public RecordWriter<NullWritable, T> getRecordWriter(FileSystem ignored, JobConf jobConf,
+ String name, Progressable progress)
+ throws IOException {
+ return createRecordWriter(jobConf, new Properties());
+ }
+
+ @Override
+ public void checkOutputSpecs(FileSystem ignored, JobConf job) throws IOException {
+
+ }
+
+ @Override
+ public org.apache.hadoop.hive.ql.exec.FileSinkOperator.RecordWriter getHiveRecordWriter
+ (JobConf jobConf, Path finalOutPath, Class<? extends Writable> valueClass, boolean
+ isCompressed, Properties tableProperties, Progressable progress) throws
+ IOException {
+ if (LOG.isDebugEnabled()) {
+ LOG.debug("Get RecordWriter for finalOutPath : " + finalOutPath + ", valueClass" +
+ " : " +
+ valueClass
+ .getName() + ", isCompressed : " + isCompressed + ", tableProperties " +
+ ": " + tableProperties + ", progress : " + progress);
+ }
+
+ return createRecordWriter(jobConf, new Properties());
+ }
+
+ @Override
+ public RecordUpdater getRecordUpdater(Path path, org.apache.hadoop.hive.ql.io
+ .AcidOutputFormat.Options options) throws IOException {
+ if (LOG.isDebugEnabled()) {
+ LOG.debug("Get RecordWriter for path : " + path + ", options : " +
+ PhoenixStorageHandlerUtil
+ .getOptionsValue(options));
+ }
+ return new PhoenixRecordWriter<T>(path, options);
+ }
+
+ @Override
+ public org.apache.hadoop.hive.ql.exec.FileSinkOperator.RecordWriter getRawRecordWriter(Path path,
+ org.apache.hadoop.hive.ql.io.AcidOutputFormat.Options options) throws IOException {
+ if (LOG.isDebugEnabled()) {
+ LOG.debug("Get RawRecordWriter for path : " + path + ", options : " +
+ PhoenixStorageHandlerUtil.getOptionsValue(options));
+ }
+
+ return new PhoenixRecordWriter<T>(path, options);
+ }
+
+ private PhoenixRecordWriter<T> createRecordWriter(Configuration config, Properties properties) {
+ try {
+ return new PhoenixRecordWriter<T>(config, properties);
+ } catch (SQLException e) {
+ LOG.error("Error during PhoenixRecordWriter instantiation :" + e.getMessage());
+ throw new RuntimeException(e);
+ }
+ }
+}
diff --git a/phoenix-hive/src/main/java/org/apache/phoenix/hive/mapreduce/PhoenixRecordReader.java b/phoenix-hive/src/main/java/org/apache/phoenix/hive/mapreduce/PhoenixRecordReader.java
new file mode 100644
index 0000000..2ff8aca
--- /dev/null
+++ b/phoenix-hive/src/main/java/org/apache/phoenix/hive/mapreduce/PhoenixRecordReader.java
@@ -0,0 +1,217 @@
+/*
+ * 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.phoenix.hive.mapreduce;
+
+import java.io.IOException;
+import java.sql.SQLException;
+import java.util.List;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.client.Scan;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.io.NullWritable;
+import org.apache.hadoop.io.WritableComparable;
+import org.apache.hadoop.mapred.InputSplit;
+import org.apache.hadoop.mapred.RecordReader;
+import org.apache.hadoop.mapreduce.lib.db.DBWritable;
+import org.apache.hadoop.util.ReflectionUtils;
+import org.apache.phoenix.compile.QueryPlan;
+import org.apache.phoenix.compile.StatementContext;
+import org.apache.phoenix.coprocessor.BaseScannerRegionObserver;
+import org.apache.phoenix.hive.PhoenixRowKey;
+import org.apache.phoenix.hive.util.PhoenixStorageHandlerUtil;
+import org.apache.phoenix.iterate.ConcatResultIterator;
+import org.apache.phoenix.iterate.LookAheadResultIterator;
+import org.apache.phoenix.iterate.MapReduceParallelScanGrouper;
+import org.apache.phoenix.iterate.PeekingResultIterator;
+import org.apache.phoenix.iterate.ResultIterator;
+import org.apache.phoenix.iterate.RoundRobinResultIterator;
+import org.apache.phoenix.iterate.SequenceResultIterator;
+import org.apache.phoenix.iterate.TableResultIterator;
+import org.apache.phoenix.jdbc.PhoenixResultSet;
+import org.apache.phoenix.monitoring.ReadMetricQueue;
+import org.apache.phoenix.monitoring.ScanMetricsHolder;
+
+import com.google.common.base.Throwables;
+import com.google.common.collect.Lists;
+
+/**
+ * @RecordReader implementation that iterates over the the records.
+ */
+@SuppressWarnings("rawtypes")
+public class PhoenixRecordReader<T extends DBWritable> implements
+ RecordReader<WritableComparable, T> {
+
+ private static final Log LOG = LogFactory.getLog(PhoenixRecordReader.class);
+
+ private final Configuration configuration;
+ private final QueryPlan queryPlan;
+ private WritableComparable key;
+ private T value = null;
+ private Class<T> inputClass;
+ private ResultIterator resultIterator = null;
+ private PhoenixResultSet resultSet;
+ private long readCount;
+
+
+ private boolean isTransactional;
+
+ public PhoenixRecordReader(Class<T> inputClass, final Configuration configuration, final
+ QueryPlan queryPlan) throws IOException {
+ this.inputClass = inputClass;
+ this.configuration = configuration;
+ this.queryPlan = queryPlan;
+
+ isTransactional = PhoenixStorageHandlerUtil.isTransactionalTable(configuration);
+ }
+
+ public void initialize(InputSplit split) throws IOException {
+ final PhoenixInputSplit pSplit = (PhoenixInputSplit) split;
+ final List<Scan> scans = pSplit.getScans();
+
+ if (LOG.isInfoEnabled()) {
+ LOG.info("Target table : " + queryPlan.getTableRef().getTable().getPhysicalName());
+ }
+
+ if (LOG.isDebugEnabled()) {
+ LOG.debug("Scan count[" + scans.size() + "] : " + Bytes.toStringBinary(scans.get(0)
+ .getStartRow()) + " ~ " + Bytes.toStringBinary(scans.get(scans.size() - 1)
+ .getStopRow()));
+ LOG.debug("First scan : " + scans.get(0) + " scanAttribute : " + scans.get(0)
+ .getAttributesMap());
+
+ for (int i = 0, limit = scans.size(); i < limit; i++) {
+ LOG.debug("EXPECTED_UPPER_REGION_KEY[" + i + "] : " +
+ Bytes.toStringBinary(scans.get(i).getAttribute(BaseScannerRegionObserver
+ .EXPECTED_UPPER_REGION_KEY)));
+ }
+ }
+
+ try {
+ List<PeekingResultIterator> iterators = Lists.newArrayListWithExpectedSize(scans.size
+ ());
+ StatementContext ctx = queryPlan.getContext();
+ ReadMetricQueue readMetrics = ctx.getReadMetricsQueue();
+ String tableName = queryPlan.getTableRef().getTable().getPhysicalName().getString();
+ long renewScannerLeaseThreshold = queryPlan.getContext().getConnection()
+ .getQueryServices().getRenewLeaseThresholdMilliSeconds();
+ for (Scan scan : scans) {
+ scan.setAttribute(BaseScannerRegionObserver.SKIP_REGION_BOUNDARY_CHECK, Bytes
+ .toBytes(true));
+ ScanMetricsHolder scanMetricsHolder = ScanMetricsHolder.getInstance(readMetrics, tableName, scan, ctx.getConnection().getLogLevel());
+ final TableResultIterator tableResultIterator = new TableResultIterator(
+ queryPlan.getContext().getConnection().getMutationState(), scan, scanMetricsHolder,
+ renewScannerLeaseThreshold, queryPlan, MapReduceParallelScanGrouper.getInstance());
+
+ PeekingResultIterator peekingResultIterator = LookAheadResultIterator.wrap
+ (tableResultIterator);
+ iterators.add(peekingResultIterator);
+ }
+ ResultIterator iterator = queryPlan.useRoundRobinIterator()
+ ? RoundRobinResultIterator.newIterator(iterators, queryPlan)
+ : ConcatResultIterator.newIterator(iterators);
+ if (queryPlan.getContext().getSequenceManager().getSequenceCount() > 0) {
+ iterator = new SequenceResultIterator(iterator, queryPlan.getContext()
+ .getSequenceManager());
+ }
+ this.resultIterator = iterator;
+ // Clone the row projector as it's not thread safe and would be used
+ // simultaneously by multiple threads otherwise.
+ this.resultSet = new PhoenixResultSet(this.resultIterator, queryPlan.getProjector()
+ .cloneIfNecessary(),
+ queryPlan.getContext());
+ } catch (SQLException e) {
+ LOG.error(String.format(" Error [%s] initializing PhoenixRecordReader. ", e
+ .getMessage()));
+ Throwables.propagate(e);
+ }
+ }
+
+ @Override
+ public boolean next(WritableComparable key, T value) throws IOException {
+ try {
+ if (!resultSet.next()) {
+ return false;
+ }
+ value.readFields(resultSet);
+
+ if (isTransactional) {
+ ((PhoenixResultWritable) value).readPrimaryKey((PhoenixRowKey) key);
+ }
+
+ ++readCount;
+
+ if (LOG.isTraceEnabled()) {
+ LOG.trace("Result[" + readCount + "] : " + ((PhoenixResultWritable) value)
+ .getResultMap());
+ }
+
+ return true;
+ } catch (SQLException e) {
+ LOG.error(String.format(" Error [%s] occurred while iterating over the resultset. ",
+ e.getMessage()));
+ throw new RuntimeException(e);
+ }
+ }
+
+ @Override
+ public WritableComparable createKey() {
+ if (isTransactional) {
+ key = new PhoenixRowKey();
+ } else {
+ key = NullWritable.get();
+ }
+
+ return key;
+ }
+
+ @Override
+ public T createValue() {
+ value = ReflectionUtils.newInstance(inputClass, this.configuration);
+ return value;
+ }
+
+ @Override
+ public long getPos() throws IOException {
+ return 0;
+ }
+
+ @Override
+ public void close() throws IOException {
+ if (LOG.isInfoEnabled()) {
+ LOG.info("Read Count : " + readCount);
+ }
+
+ if (resultIterator != null) {
+ try {
+ resultIterator.close();
+ } catch (SQLException e) {
+ LOG.error(" Error closing resultset.");
+ throw new RuntimeException(e);
+ }
+ }
+
+ }
+
+ @Override
+ public float getProgress() throws IOException {
+ return 0;
+ }
+}
diff --git a/phoenix-hive/src/main/java/org/apache/phoenix/hive/mapreduce/PhoenixRecordWriter.java b/phoenix-hive/src/main/java/org/apache/phoenix/hive/mapreduce/PhoenixRecordWriter.java
new file mode 100644
index 0000000..c6884df
--- /dev/null
+++ b/phoenix-hive/src/main/java/org/apache/phoenix/hive/mapreduce/PhoenixRecordWriter.java
@@ -0,0 +1,355 @@
+/*
+ * 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.phoenix.hive.mapreduce;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hive.ql.io.AcidOutputFormat;
+import org.apache.hadoop.hive.ql.io.RecordUpdater;
+import org.apache.hadoop.hive.serde2.SerDeException;
+import org.apache.hadoop.hive.serde2.SerDeStats;
+import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector;
+import org.apache.hadoop.io.NullWritable;
+import org.apache.hadoop.io.Writable;
+import org.apache.hadoop.mapred.RecordWriter;
+import org.apache.hadoop.mapred.Reporter;
+import org.apache.hadoop.mapreduce.lib.db.DBWritable;
+import org.apache.phoenix.hive.PhoenixSerializer;
+import org.apache.phoenix.hive.PhoenixSerializer.DmlType;
+import org.apache.phoenix.hive.constants.PhoenixStorageHandlerConstants;
+import org.apache.phoenix.hive.util.PhoenixConnectionUtil;
+import org.apache.phoenix.hive.util.PhoenixStorageHandlerUtil;
+import org.apache.phoenix.hive.util.PhoenixUtil;
+import org.apache.phoenix.jdbc.PhoenixConnection;
+import org.apache.phoenix.mapreduce.util.PhoenixConfigurationUtil;
+import org.apache.phoenix.schema.ConcurrentTableMutationException;
+import org.apache.phoenix.schema.MetaDataClient;
+import org.apache.phoenix.util.QueryUtil;
+
+import java.io.IOException;
+import java.sql.Connection;
+import java.sql.PreparedStatement;
+import java.sql.SQLException;
+import java.util.Properties;
+
+/**
+ *
+ * RecordWriter implementation. Writes records to the output
+ * WARNING : There is possibility that WAL disable setting not working properly due concurrent
+ * enabling/disabling WAL.
+ *
+ */
+public class PhoenixRecordWriter<T extends DBWritable> implements RecordWriter<NullWritable, T>,
+ org.apache.hadoop.hive.ql.exec.FileSinkOperator.RecordWriter, RecordUpdater {
+
+ private static final Log LOG = LogFactory.getLog(PhoenixRecordWriter.class);
+
+ private Connection conn;
+ private PreparedStatement pstmt;
+ private long batchSize;
+ private long numRecords = 0;
+
+ private Configuration config;
+ private String tableName;
+ private MetaDataClient metaDataClient;
+ private boolean restoreWalMode;
+
+ // For RecordUpdater
+ private long rowCountDelta = 0;
+ private PhoenixSerializer phoenixSerializer;
+ private ObjectInspector objInspector;
+ private PreparedStatement pstmtForDelete;
+
+ // For RecordUpdater
+ public PhoenixRecordWriter(Path path, AcidOutputFormat.Options options) throws IOException {
+ Configuration config = options.getConfiguration();
+ Properties props = new Properties();
+
+ try {
+ initialize(config, props);
+ } catch (SQLException e) {
+ throw new IOException(e);
+ }
+
+ this.objInspector = options.getInspector();
+ try {
+ phoenixSerializer = new PhoenixSerializer(config, options.getTableProperties());
+ } catch (SerDeException e) {
+ throw new IOException(e);
+ }
+ }
+
+ public PhoenixRecordWriter(final Configuration configuration, final Properties props) throws
+ SQLException {
+ initialize(configuration, props);
+ }
+
+ private void initialize(Configuration config, Properties properties) throws SQLException {
+ this.config = config;
+ tableName = config.get(PhoenixStorageHandlerConstants.PHOENIX_TABLE_NAME);
+
+ // Disable WAL
+ String walConfigName = tableName.toLowerCase() + PhoenixStorageHandlerConstants.DISABLE_WAL;
+ boolean disableWal = config.getBoolean(walConfigName, false);
+ if (disableWal) {
+ if (LOG.isDebugEnabled()) {
+ LOG.debug("Property " + walConfigName + " is true. batch.mode will be set true. ");
+ }
+
+ properties.setProperty(PhoenixStorageHandlerConstants.BATCH_MODE, "true");
+ }
+
+ this.conn = PhoenixConnectionUtil.getInputConnection(config, properties);
+
+ if (disableWal) {
+ metaDataClient = new MetaDataClient((PhoenixConnection) conn);
+
+ if (!PhoenixUtil.isDisabledWal(metaDataClient, tableName)) {
+ // execute alter tablel statement if disable_wal is not true.
+ try {
+ PhoenixUtil.alterTableForWalDisable(conn, tableName, true);
+ } catch (ConcurrentTableMutationException e) {
+ if (LOG.isWarnEnabled()) {
+ LOG.warn("Another mapper or task processing wal disable");
+ }
+ }
+
+ if (LOG.isDebugEnabled()) {
+ LOG.debug(tableName + "s wal disabled.");
+ }
+
+ // restore original value of disable_wal at the end.
+ restoreWalMode = true;
+ }
+ }
+
+ this.batchSize = PhoenixConfigurationUtil.getBatchSize(config);
+
+ if (LOG.isDebugEnabled()) {
+ LOG.debug("Batch-size : " + batchSize);
+ }
+
+ String upsertQuery = QueryUtil.constructUpsertStatement(tableName, PhoenixUtil
+ .getColumnInfoList(conn, tableName));
+
+ if (LOG.isDebugEnabled()) {
+ LOG.debug("Upsert-query : " + upsertQuery);
+ }
+ this.pstmt = this.conn.prepareStatement(upsertQuery);
+ }
+
+ @Override
+ public void write(NullWritable key, T record) throws IOException {
+ try {
+ record.write(pstmt);
+ numRecords++;
+ pstmt.executeUpdate();
+
+ if (numRecords % batchSize == 0) {
+ LOG.debug("Commit called on a batch of size : " + batchSize);
+ conn.commit();
+ }
+ } catch (SQLException e) {
+ throw new IOException("Exception while writing to table.", e);
+ }
+ }
+
+ @Override
+ public void close(Reporter reporter) throws IOException {
+ try {
+ conn.commit();
+
+ if (LOG.isInfoEnabled()) {
+ LOG.info("Wrote row : " + numRecords);
+ }
+ } catch (SQLException e) {
+ LOG.error("SQLException while performing the commit for the task.");
+ throw new IOException(e);
+ } finally {
+ try {
+ if (restoreWalMode && PhoenixUtil.isDisabledWal(metaDataClient, tableName)) {
+ try {
+ PhoenixUtil.alterTableForWalDisable(conn, tableName, false);
+ } catch (ConcurrentTableMutationException e) {
+ if (LOG.isWarnEnabled()) {
+ LOG.warn("Another mapper or task processing wal enable");
+ }
+ }
+
+ if (LOG.isDebugEnabled()) {
+ LOG.debug(tableName + "s wal enabled.");
+ }
+ }
+
+ // flush if [table-name].auto.flush is true.
+ String autoFlushConfigName = tableName.toLowerCase() +
+ PhoenixStorageHandlerConstants.AUTO_FLUSH;
+ boolean autoFlush = config.getBoolean(autoFlushConfigName, false);
+ if (autoFlush) {
+ if (LOG.isDebugEnabled()) {
+ LOG.debug("autoFlush is true.");
+ }
+
+ PhoenixUtil.flush(conn, tableName);
+ }
+
+ PhoenixUtil.closeResource(pstmt);
+ PhoenixUtil.closeResource(pstmtForDelete);
+ PhoenixUtil.closeResource(conn);
+ } catch (SQLException ex) {
+ LOG.error("SQLException while closing the connection for the task.");
+ throw new IOException(ex);
+ }
+ }
+ }
+
+ // For Testing
+ public boolean isRestoreWalMode() {
+ return restoreWalMode;
+ }
+
+ @SuppressWarnings("unchecked")
+ @Override
+ public void write(Writable w) throws IOException {
+ PhoenixResultWritable row = (PhoenixResultWritable) w;
+
+ write(NullWritable.get(), (T) row);
+ }
+
+ @Override
+ public void close(boolean abort) throws IOException {
+ close(Reporter.NULL);
+ }
+
+ @Override
+ public void insert(long currentTransaction, Object row) throws IOException {
+ if (LOG.isTraceEnabled()) {
+ LOG.trace("insert transaction : " + currentTransaction + ", row : " +
+ PhoenixStorageHandlerUtil.toString(row));
+ }
+
+ PhoenixResultWritable pResultWritable = (PhoenixResultWritable) phoenixSerializer
+ .serialize(row, objInspector, DmlType.INSERT);
+
+ if (LOG.isTraceEnabled()) {
+ LOG.trace("Data : " + pResultWritable.getValueList());
+ }
+
+ write(pResultWritable);
+ rowCountDelta++;
+ }
+
+ @Override
+ public void update(long currentTransaction, Object row) throws IOException {
+ if (LOG.isTraceEnabled()) {
+ LOG.trace("update transaction : " + currentTransaction + ", row : " +
+ PhoenixStorageHandlerUtil
+ .toString(row));
+ }
+
+ PhoenixResultWritable pResultWritable = (PhoenixResultWritable) phoenixSerializer
+ .serialize(row, objInspector, DmlType.UPDATE);
+
+ if (LOG.isTraceEnabled()) {
+ LOG.trace("Data : " + pResultWritable.getValueList());
+ }
+
+ write(pResultWritable);
+ }
+
+ @Override
+ public void delete(long currentTransaction, Object row) throws IOException {
+ if (LOG.isTraceEnabled()) {
+ LOG.trace("delete transaction : " + currentTransaction + ", row : " +
+ PhoenixStorageHandlerUtil.toString(row));
+ }
+
+ PhoenixResultWritable pResultWritable = (PhoenixResultWritable) phoenixSerializer
+ .serialize(row, objInspector, DmlType.DELETE);
+
+ if (LOG.isTraceEnabled()) {
+ LOG.trace("Data : " + pResultWritable.getValueList());
+ }
+
+ if (pstmtForDelete == null) {
+ try {
+ String deleteQuery = PhoenixUtil.constructDeleteStatement(conn, tableName);
+
+ if (LOG.isDebugEnabled()) {
+ LOG.debug("Delete query : " + deleteQuery);
+ }
+
+ pstmtForDelete = conn.prepareStatement(deleteQuery);
+ } catch (SQLException e) {
+ throw new IOException(e);
+ }
+ }
+
+ delete(pResultWritable);
+
+ rowCountDelta--;
+ }
+
+ private void delete(PhoenixResultWritable pResultWritable) throws IOException {
+ try {
+ pResultWritable.delete(pstmtForDelete);
+ numRecords++;
+ pstmtForDelete.executeUpdate();
+
+ if (numRecords % batchSize == 0) {
+ LOG.debug("Commit called on a batch of size : " + batchSize);
+ conn.commit();
+ }
+ } catch (SQLException e) {
+ throw new IOException("Exception while deleting to table.", e);
+ }
+ }
+
+ @Override
+ public void flush() throws IOException {
+ if (LOG.isDebugEnabled()) {
+ LOG.debug("Flush called");
+ }
+
+ try {
+ conn.commit();
+
+ if (LOG.isInfoEnabled()) {
+ LOG.info("Written row : " + numRecords);
+ }
+ } catch (SQLException e) {
+ LOG.error("SQLException while performing the commit for the task.");
+ throw new IOException(e);
+ }
+ }
+
+ @Override
+ public SerDeStats getStats() {
+ if (LOG.isDebugEnabled()) {
+ LOG.debug("getStats called");
+ }
+
+ SerDeStats stats = new SerDeStats();
+ stats.setRowCount(rowCountDelta);
+ // Don't worry about setting raw data size diff. There is no reasonable way to calculate
+ // that without finding the row we are updating or deleting, which would be a mess.
+ return stats;
+ }
+}
diff --git a/phoenix-hive/src/main/java/org/apache/phoenix/hive/mapreduce/PhoenixResultWritable.java b/phoenix-hive/src/main/java/org/apache/phoenix/hive/mapreduce/PhoenixResultWritable.java
new file mode 100644
index 0000000..2bdc7b2
--- /dev/null
+++ b/phoenix-hive/src/main/java/org/apache/phoenix/hive/mapreduce/PhoenixResultWritable.java
@@ -0,0 +1,217 @@
+/*
+ * 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.phoenix.hive.mapreduce;
+
+import com.google.common.collect.Lists;
+import com.google.common.collect.Maps;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configurable;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.io.Writable;
+import org.apache.hadoop.mapreduce.lib.db.DBWritable;
+import org.apache.phoenix.hive.PhoenixRowKey;
+import org.apache.phoenix.hive.constants.PhoenixStorageHandlerConstants;
+import org.apache.phoenix.hive.util.ColumnMappingUtils;
+import org.apache.phoenix.hive.util.PhoenixStorageHandlerUtil;
+import org.apache.phoenix.hive.util.PhoenixUtil;
+import org.apache.phoenix.util.ColumnInfo;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.sql.PreparedStatement;
+import java.sql.ResultSet;
+import java.sql.ResultSetMetaData;
+import java.sql.SQLException;
+import java.util.List;
+import java.util.Map;
+
+/**
+ * Serialized class for SerDe
+ *
+ */
+public class PhoenixResultWritable implements Writable, DBWritable, Configurable {
+
+ private static final Log LOG = LogFactory.getLog(PhoenixResultWritable.class);
+
+ private List<ColumnInfo> columnMetadataList;
+ private List<Object> valueList; // for output
+ private Map<String, Object> rowMap = Maps.newHashMap(); // for input
+ private Map<String, String> columnMap;
+
+ private int columnCount = -1;
+
+ private Configuration config;
+ private boolean isTransactional;
+ private Map<String, Object> rowKeyMap = Maps.newLinkedHashMap();
+ private List<String> primaryKeyColumnList;
+
+ public PhoenixResultWritable() {
+ }
+
+ public PhoenixResultWritable(Configuration config) throws IOException {
+ setConf(config);
+ }
+
+ public PhoenixResultWritable(Configuration config, List<ColumnInfo> columnMetadataList)
+ throws IOException {
+ this(config);
+ this.columnMetadataList = columnMetadataList;
+ valueList = Lists.newArrayListWithExpectedSize(columnMetadataList.size());
+ }
+
+ @Override
+ public void write(DataOutput out) throws IOException {
+ throw new UnsupportedOperationException();
+ }
+
+ @Override
+ public void readFields(DataInput in) throws IOException {
+ throw new UnsupportedOperationException();
+ }
+
+ // for write
+ public void clear() {
+ valueList.clear();
+ }
+
+ // for write
+ public void add(Object value) {
+ valueList.add(value);
+ }
+
+ @Override
+ public void write(PreparedStatement statement) throws SQLException {
+ ColumnInfo columnInfo = null;
+ Object value = null;
+
+ try {
+ for (int i = 0, limit = columnMetadataList.size(); i < limit; i++) {
+ columnInfo = columnMetadataList.get(i);
+
+ if (valueList.size() > i) {
+ value = valueList.get(i);
+ } else {
+ value = null;
+ }
+
+ if (value == null) {
+ statement.setNull(i + 1, columnInfo.getSqlType());
+ } else {
+ statement.setObject(i + 1, value, columnInfo.getSqlType());
+ }
+ }
+ } catch (SQLException | RuntimeException e) {
+ LOG.error("[column-info, value] : " + columnInfo + ", " + value);
+ throw e;
+ }
+ }
+
+ public void delete(PreparedStatement statement) throws SQLException {
+ ColumnInfo columnInfo = null;
+ Object value = null;
+
+ try {
+ for (int i = 0, limit = primaryKeyColumnList.size(); i < limit; i++) {
+ columnInfo = columnMetadataList.get(i);
+
+ if (valueList.size() > i) {
+ value = valueList.get(i);
+ } else {
+ value = null;
+ }
+
+ if (value == null) {
+ statement.setNull(i + 1, columnInfo.getSqlType());
+ } else {
+ statement.setObject(i + 1, value, columnInfo.getSqlType());
+ }
+ }
+ } catch (SQLException | RuntimeException e) {
+ LOG.error("[column-info, value] : " + columnInfo + ", " + value);
+ throw e;
+ }
+ }
+
+ @Override
+ public void readFields(ResultSet resultSet) throws SQLException {
+ ResultSetMetaData rsmd = resultSet.getMetaData();
+ if (columnCount == -1) {
+ this.columnCount = rsmd.getColumnCount();
+ }
+ rowMap.clear();
+
+ for (int i = 0; i < columnCount; i++) {
+ Object value = resultSet.getObject(i + 1);
+ String columnName = rsmd.getColumnName(i + 1);
+ String mapName = columnMap.get(columnName);
+ if(mapName != null) {
+ columnName = mapName;
+ }
+ rowMap.put(columnName, value);
+ }
+
+ // Adding row__id column.
+ if (isTransactional) {
+ rowKeyMap.clear();
+
+ for (String pkColumn : primaryKeyColumnList) {
+ rowKeyMap.put(pkColumn, rowMap.get(pkColumn));
+ }
+ }
+ }
+
+ public void readPrimaryKey(PhoenixRowKey rowKey) {
+ rowKey.setRowKeyMap(rowKeyMap);
+ }
+
+ public List<ColumnInfo> getColumnMetadataList() {
+ return columnMetadataList;
+ }
+
+ public void setColumnMetadataList(List<ColumnInfo> columnMetadataList) {
+ this.columnMetadataList = columnMetadataList;
+ }
+
+ public Map<String, Object> getResultMap() {
+ return rowMap;
+ }
+
+ public List<Object> getValueList() {
+ return valueList;
+ }
+
+ @Override
+ public void setConf(Configuration conf) {
+ config = conf;
+ this.columnMap = ColumnMappingUtils.getReverseColumnMapping(config.get(PhoenixStorageHandlerConstants.PHOENIX_COLUMN_MAPPING,""));
+
+ isTransactional = PhoenixStorageHandlerUtil.isTransactionalTable(config);
+
+ if (isTransactional) {
+ primaryKeyColumnList = PhoenixUtil.getPrimaryKeyColumnList(config, config.get
+ (PhoenixStorageHandlerConstants.PHOENIX_TABLE_NAME));
+ }
+ }
+
+ @Override
+ public Configuration getConf() {
+ return config;
+ }
+}
diff --git a/phoenix-hive/src/main/java/org/apache/phoenix/hive/objectinspector/AbstractPhoenixObjectInspector.java b/phoenix-hive/src/main/java/org/apache/phoenix/hive/objectinspector/AbstractPhoenixObjectInspector.java
new file mode 100644
index 0000000..1de1cc7
--- /dev/null
+++ b/phoenix-hive/src/main/java/org/apache/phoenix/hive/objectinspector/AbstractPhoenixObjectInspector.java
@@ -0,0 +1,59 @@
+/*
+ * 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.phoenix.hive.objectinspector;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.hive.serde2.lazy.objectinspector.primitive
+ .AbstractPrimitiveLazyObjectInspector;
+import org.apache.hadoop.hive.serde2.typeinfo.PrimitiveTypeInfo;
+import org.apache.hadoop.io.Writable;
+
+/**
+ * AbstractPhoenixObjectInspector for a LazyPrimitive object
+ */
+public abstract class AbstractPhoenixObjectInspector<T extends Writable>
+ extends AbstractPrimitiveLazyObjectInspector<T> {
+
+ private final Log log;
+
+ public AbstractPhoenixObjectInspector() {
+ super();
+
+ log = LogFactory.getLog(getClass());
+ }
+
+ protected AbstractPhoenixObjectInspector(PrimitiveTypeInfo typeInfo) {
+ super(typeInfo);
+
+ log = LogFactory.getLog(getClass());
+ }
+
+ @Override
+ public Object getPrimitiveJavaObject(Object o) {
+ return o == null ? null : o;
+ }
+
+ public void logExceptionMessage(Object value, String dataType) {
+ if (log.isDebugEnabled()) {
+ log.debug("Data not in the " + dataType + " data type range so converted to null. " +
+ "Given data is :"
+ + value.toString(), new Exception("For debugging purposes"));
+ }
+ }
+}
\ No newline at end of file
diff --git a/phoenix-hive/src/main/java/org/apache/phoenix/hive/objectinspector/PhoenixBinaryObjectInspector.java b/phoenix-hive/src/main/java/org/apache/phoenix/hive/objectinspector/PhoenixBinaryObjectInspector.java
new file mode 100644
index 0000000..2c642d2
--- /dev/null
+++ b/phoenix-hive/src/main/java/org/apache/phoenix/hive/objectinspector/PhoenixBinaryObjectInspector.java
@@ -0,0 +1,58 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.phoenix.hive.objectinspector;
+
+import org.apache.hadoop.hive.serde2.objectinspector.primitive.BinaryObjectInspector;
+import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoFactory;
+import org.apache.hadoop.io.BytesWritable;
+
+/**
+ * ObjectInspector for Binary type
+ */
+
+public class PhoenixBinaryObjectInspector extends AbstractPhoenixObjectInspector<BytesWritable>
+ implements BinaryObjectInspector {
+
+ public PhoenixBinaryObjectInspector() {
+ super(TypeInfoFactory.binaryTypeInfo);
+ }
+
+ @Override
+ public Object copyObject(Object o) {
+ byte[] clone = null;
+
+ if (o != null) {
+ byte[] source = (byte[]) o;
+ clone = new byte[source.length];
+ System.arraycopy(source, 0, clone, 0, source.length);
+ }
+
+ return clone;
+ }
+
+ @Override
+ public byte[] getPrimitiveJavaObject(Object o) {
+ return (byte[]) o;
+ }
+
+ @Override
+ public BytesWritable getPrimitiveWritableObject(Object o) {
+ return new BytesWritable((byte[]) o);
+ }
+
+}
diff --git a/phoenix-hive/src/main/java/org/apache/phoenix/hive/objectinspector/PhoenixBooleanObjectInspector.java b/phoenix-hive/src/main/java/org/apache/phoenix/hive/objectinspector/PhoenixBooleanObjectInspector.java
new file mode 100644
index 0000000..a767ca0
--- /dev/null
+++ b/phoenix-hive/src/main/java/org/apache/phoenix/hive/objectinspector/PhoenixBooleanObjectInspector.java
@@ -0,0 +1,55 @@
+/*
+ * 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.phoenix.hive.objectinspector;
+
+import org.apache.hadoop.hive.serde2.objectinspector.primitive.BooleanObjectInspector;
+import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoFactory;
+import org.apache.hadoop.io.BooleanWritable;
+
+public class PhoenixBooleanObjectInspector extends AbstractPhoenixObjectInspector<BooleanWritable>
+ implements BooleanObjectInspector {
+
+ public PhoenixBooleanObjectInspector() {
+ super(TypeInfoFactory.booleanTypeInfo);
+ }
+
+ @Override
+ public Object copyObject(Object o) {
+ return o == null ? null : new Boolean((Boolean) o);
+ }
+
+ @Override
+ public BooleanWritable getPrimitiveWritableObject(Object o) {
+ return new BooleanWritable(get(o));
+ }
+
+ @Override
+ public boolean get(Object o) {
+ Boolean value = null;
+
+ if (o != null) {
+ try {
+ value = (Boolean) o;
+ } catch (Exception e) {
+ logExceptionMessage(o, "BOOLEAN");
+ }
+ }
+
+ return value;
+ }
+}
diff --git a/phoenix-hive/src/main/java/org/apache/phoenix/hive/objectinspector/PhoenixByteObjectInspector.java b/phoenix-hive/src/main/java/org/apache/phoenix/hive/objectinspector/PhoenixByteObjectInspector.java
new file mode 100644
index 0000000..a19342a
--- /dev/null
+++ b/phoenix-hive/src/main/java/org/apache/phoenix/hive/objectinspector/PhoenixByteObjectInspector.java
@@ -0,0 +1,59 @@
+/*
+ * 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.phoenix.hive.objectinspector;
+
+import org.apache.hadoop.hive.serde2.objectinspector.primitive.ByteObjectInspector;
+import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoFactory;
+import org.apache.hadoop.io.ByteWritable;
+
+/**
+ * ObjectInspector for byte type
+ */
+public class PhoenixByteObjectInspector extends AbstractPhoenixObjectInspector<ByteWritable>
+ implements ByteObjectInspector {
+
+ public PhoenixByteObjectInspector() {
+ super(TypeInfoFactory.byteTypeInfo);
+ }
+
+ @Override
+ public Object copyObject(Object o) {
+ return o == null ? null : new Byte((Byte) o);
+ }
+
+ @Override
+ public ByteWritable getPrimitiveWritableObject(Object o) {
+ return new ByteWritable(get(o));
+ }
+
+ @Override
+ public byte get(Object o) {
+ Byte value = null;
+
+ if (o != null) {
+ try {
+ value = (Byte) o;
+ } catch (Exception e) {
+ logExceptionMessage(o, "BYTE");
+ }
+ }
+
+ return value;
+ }
+
+}
diff --git a/phoenix-hive/src/main/java/org/apache/phoenix/hive/objectinspector/PhoenixCharObjectInspector.java b/phoenix-hive/src/main/java/org/apache/phoenix/hive/objectinspector/PhoenixCharObjectInspector.java
new file mode 100644
index 0000000..17222a2
--- /dev/null
+++ b/phoenix-hive/src/main/java/org/apache/phoenix/hive/objectinspector/PhoenixCharObjectInspector.java
@@ -0,0 +1,56 @@
+/*
+ * 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.phoenix.hive.objectinspector;
+
+import org.apache.hadoop.hive.common.type.HiveChar;
+import org.apache.hadoop.hive.serde2.io.HiveCharWritable;
+import org.apache.hadoop.hive.serde2.objectinspector.primitive.HiveCharObjectInspector;
+import org.apache.hadoop.hive.serde2.typeinfo.PrimitiveTypeInfo;
+import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoFactory;
+
+/**
+ * ObjectInspector for char type
+ */
+public class PhoenixCharObjectInspector extends AbstractPhoenixObjectInspector<HiveCharWritable>
+ implements HiveCharObjectInspector {
+
+ public PhoenixCharObjectInspector() {
+ this(TypeInfoFactory.charTypeInfo);
+ }
+
+ public PhoenixCharObjectInspector(PrimitiveTypeInfo type) {
+ super(type);
+ }
+
+ @Override
+ public Object copyObject(Object o) {
+ return o == null ? null : new String((String) o);
+ }
+
+ @Override
+ public HiveCharWritable getPrimitiveWritableObject(Object o) {
+ return new HiveCharWritable(getPrimitiveJavaObject(o));
+ }
+
+ @Override
+ public HiveChar getPrimitiveJavaObject(Object o) {
+ String value = (String) o;
+ return new HiveChar(value, value.length());
+ }
+
+}
diff --git a/phoenix-hive/src/main/java/org/apache/phoenix/hive/objectinspector/PhoenixDateObjectInspector.java b/phoenix-hive/src/main/java/org/apache/phoenix/hive/objectinspector/PhoenixDateObjectInspector.java
new file mode 100644
index 0000000..d97993e
--- /dev/null
+++ b/phoenix-hive/src/main/java/org/apache/phoenix/hive/objectinspector/PhoenixDateObjectInspector.java
@@ -0,0 +1,63 @@
+/*
+ * 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.phoenix.hive.objectinspector;
+
+import org.apache.hadoop.hive.serde2.io.DateWritable;
+import org.apache.hadoop.hive.serde2.objectinspector.primitive.DateObjectInspector;
+import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoFactory;
+
+import java.sql.Date;
+
+/**
+ * ObjectInspector for date type
+ */
+
+public class PhoenixDateObjectInspector extends AbstractPhoenixObjectInspector<DateWritable>
+ implements DateObjectInspector {
+
+ public PhoenixDateObjectInspector() {
+ super(TypeInfoFactory.dateTypeInfo);
+ }
+
+ @Override
+ public Object copyObject(Object o) {
+ return o == null ? null : new Date(((Date) o).getTime());
+ }
+
+ @Override
+ public DateWritable getPrimitiveWritableObject(Object o) {
+ DateWritable value = null;
+
+ if (o != null) {
+ try {
+ value = new DateWritable((Date) o);
+ } catch (Exception e) {
+ logExceptionMessage(o, "DATE");
+ value = new DateWritable();
+ }
+ }
+
+ return value;
+ }
+
+ @Override
+ public Date getPrimitiveJavaObject(Object o) {
+ return (Date) o;
+ }
+
+}
diff --git a/phoenix-hive/src/main/java/org/apache/phoenix/hive/objectinspector/PhoenixDecimalObjectInspector.java b/phoenix-hive/src/main/java/org/apache/phoenix/hive/objectinspector/PhoenixDecimalObjectInspector.java
new file mode 100644
index 0000000..3853c18
--- /dev/null
+++ b/phoenix-hive/src/main/java/org/apache/phoenix/hive/objectinspector/PhoenixDecimalObjectInspector.java
@@ -0,0 +1,72 @@
+/*
+ * 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.phoenix.hive.objectinspector;
+
+import org.apache.hadoop.hive.common.type.HiveDecimal;
+import org.apache.hadoop.hive.metastore.api.Decimal;
+import org.apache.hadoop.hive.serde2.io.HiveDecimalWritable;
+import org.apache.hadoop.hive.serde2.objectinspector.primitive.HiveDecimalObjectInspector;
+import org.apache.hadoop.hive.serde2.typeinfo.DecimalTypeInfo;
+import org.apache.hadoop.hive.serde2.typeinfo.HiveDecimalUtils;
+import org.apache.hadoop.hive.serde2.typeinfo.PrimitiveTypeInfo;
+import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoFactory;
+
+import java.math.BigDecimal;
+
+public class PhoenixDecimalObjectInspector extends
+ AbstractPhoenixObjectInspector<HiveDecimalWritable>
+ implements HiveDecimalObjectInspector {
+
+ public PhoenixDecimalObjectInspector() {
+ this(TypeInfoFactory.decimalTypeInfo);
+ }
+
+ public PhoenixDecimalObjectInspector(PrimitiveTypeInfo typeInfo) {
+ super(typeInfo);
+ }
+
+ @Override
+ public Object copyObject(Object o) {
+ return o == null ? null : new BigDecimal(o.toString());
+ }
+
+ @Override
+ public HiveDecimal getPrimitiveJavaObject(Object o) {
+ if (o == null) {
+ return null;
+ }
+
+ return HiveDecimalUtils.enforcePrecisionScale(HiveDecimal.create((BigDecimal) o),(DecimalTypeInfo)typeInfo);
+ }
+
+ @Override
+ public HiveDecimalWritable getPrimitiveWritableObject(Object o) {
+ HiveDecimalWritable value = null;
+
+ if (o != null) {
+ try {
+ value = new HiveDecimalWritable(getPrimitiveJavaObject(o));
+ } catch (Exception e) {
+ logExceptionMessage(o, "DECIMAL");
+ }
+ }
+
+ return value;
+ }
+
+}
diff --git a/phoenix-hive/src/main/java/org/apache/phoenix/hive/objectinspector/PhoenixDoubleObjectInspector.java b/phoenix-hive/src/main/java/org/apache/phoenix/hive/objectinspector/PhoenixDoubleObjectInspector.java
new file mode 100644
index 0000000..9f440ed
--- /dev/null
+++ b/phoenix-hive/src/main/java/org/apache/phoenix/hive/objectinspector/PhoenixDoubleObjectInspector.java
@@ -0,0 +1,59 @@
+/*
+ * 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.phoenix.hive.objectinspector;
+
+import org.apache.hadoop.hive.serde2.objectinspector.primitive.DoubleObjectInspector;
+import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoFactory;
+import org.apache.hadoop.io.DoubleWritable;
+
+/**
+ * ObjectInspector for double type
+ */
+public class PhoenixDoubleObjectInspector extends AbstractPhoenixObjectInspector<DoubleWritable>
+ implements DoubleObjectInspector {
+
+ public PhoenixDoubleObjectInspector() {
+ super(TypeInfoFactory.doubleTypeInfo);
+ }
+
+ @Override
+ public Object copyObject(Object o) {
+ return o == null ? null : new Double((Double) o);
+ }
+
+ @Override
+ public DoubleWritable getPrimitiveWritableObject(Object o) {
+ return new DoubleWritable(get(o));
+ }
+
+ @Override
+ public double get(Object o) {
+ Double value = null;
+
+ if (o != null) {
+ try {
+ value = ((Double) o).doubleValue();
+ } catch (Exception e) {
+ logExceptionMessage(o, "LONG");
+ }
+ }
+
+ return value;
+ }
+
+}
diff --git a/phoenix-hive/src/main/java/org/apache/phoenix/hive/objectinspector/PhoenixFloatObjectInspector.java b/phoenix-hive/src/main/java/org/apache/phoenix/hive/objectinspector/PhoenixFloatObjectInspector.java
new file mode 100644
index 0000000..bf1badc
--- /dev/null
+++ b/phoenix-hive/src/main/java/org/apache/phoenix/hive/objectinspector/PhoenixFloatObjectInspector.java
@@ -0,0 +1,60 @@
+/*
+ * 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.phoenix.hive.objectinspector;
+
+import org.apache.hadoop.hive.serde2.objectinspector.primitive.FloatObjectInspector;
+import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoFactory;
+import org.apache.hadoop.io.FloatWritable;
+
+/**
+ * ObjectInspector for float type
+ */
+
+public class PhoenixFloatObjectInspector extends AbstractPhoenixObjectInspector<FloatWritable>
+ implements FloatObjectInspector {
+
+ public PhoenixFloatObjectInspector() {
+ super(TypeInfoFactory.floatTypeInfo);
+ }
+
+ @Override
+ public Object copyObject(Object o) {
+ return o == null ? null : new Float((Float) o);
+ }
+
+ @Override
+ public FloatWritable getPrimitiveWritableObject(Object o) {
+ return new FloatWritable(get(o));
+ }
+
+ @Override
+ public float get(Object o) {
+ Float value = null;
+
+ if (o != null) {
+ try {
+ value = ((Float) o).floatValue();
+ } catch (Exception e) {
+ logExceptionMessage(o, "LONG");
+ }
+ }
+
+ return value;
+ }
+
+}
diff --git a/phoenix-hive/src/main/java/org/apache/phoenix/hive/objectinspector/PhoenixIntObjectInspector.java b/phoenix-hive/src/main/java/org/apache/phoenix/hive/objectinspector/PhoenixIntObjectInspector.java
new file mode 100644
index 0000000..3511ee3
--- /dev/null
+++ b/phoenix-hive/src/main/java/org/apache/phoenix/hive/objectinspector/PhoenixIntObjectInspector.java
@@ -0,0 +1,62 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.phoenix.hive.objectinspector;
+
+import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector;
+import org.apache.hadoop.hive.serde2.objectinspector.primitive.IntObjectInspector;
+import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoFactory;
+import org.apache.hadoop.io.IntWritable;
+
+public class PhoenixIntObjectInspector extends AbstractPhoenixObjectInspector<IntWritable>
+ implements IntObjectInspector {
+
+ public PhoenixIntObjectInspector() {
+ super(TypeInfoFactory.intTypeInfo);
+ }
+
+ @Override
+ public Object copyObject(Object o) {
+ return o == null ? null : new Integer((Integer) o);
+ }
+
+ @Override
+ public Category getCategory() {
+ return Category.PRIMITIVE;
+ }
+
+ @Override
+ public IntWritable getPrimitiveWritableObject(Object o) {
+ return new IntWritable(get(o));
+ }
+
+ @Override
+ public int get(Object o) {
+ Integer value = null;
+
+ if (o != null) {
+ try {
+ value = ((Integer) o).intValue();
+ } catch (Exception e) {
+ logExceptionMessage(o, "INT");
+ }
+ }
+
+ return value;
+ }
+
+}
diff --git a/phoenix-hive/src/main/java/org/apache/phoenix/hive/objectinspector/PhoenixListObjectInspector.java b/phoenix-hive/src/main/java/org/apache/phoenix/hive/objectinspector/PhoenixListObjectInspector.java
new file mode 100644
index 0000000..c4f2d51
--- /dev/null
+++ b/phoenix-hive/src/main/java/org/apache/phoenix/hive/objectinspector/PhoenixListObjectInspector.java
@@ -0,0 +1,105 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.phoenix.hive.objectinspector;
+
+import com.google.common.collect.Lists;
+import org.apache.hadoop.hive.serde2.lazy.objectinspector.primitive.LazyObjectInspectorParameters;
+import org.apache.hadoop.hive.serde2.objectinspector.ListObjectInspector;
+import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector;
+import org.apache.phoenix.schema.types.PhoenixArray;
+
+import java.util.List;
+
+/**
+ * ObjectInspector for list objects.
+ */
+public class PhoenixListObjectInspector implements ListObjectInspector {
+
+ private ObjectInspector listElementObjectInspector;
+ private byte separator;
+ private LazyObjectInspectorParameters lazyParams;
+
+ public PhoenixListObjectInspector(ObjectInspector listElementObjectInspector,
+ byte separator, LazyObjectInspectorParameters lazyParams) {
+ this.listElementObjectInspector = listElementObjectInspector;
+ this.separator = separator;
+ this.lazyParams = lazyParams;
+ }
+
+ @Override
+ public String getTypeName() {
+ return org.apache.hadoop.hive.serde.serdeConstants.LIST_TYPE_NAME + "<" +
+ listElementObjectInspector.getTypeName() + ">";
+ }
+
+ @Override
+ public Category getCategory() {
+ return Category.LIST;
+ }
+
+ @Override
+ public ObjectInspector getListElementObjectInspector() {
+ return listElementObjectInspector;
+ }
+
+ @Override
+ public Object getListElement(Object data, int index) {
+ if (data == null) {
+ return null;
+ }
+
+ PhoenixArray array = (PhoenixArray) data;
+
+ return array.getElement(index);
+ }
+
+ @Override
+ public int getListLength(Object data) {
+ if (data == null) {
+ return -1;
+ }
+
+ PhoenixArray array = (PhoenixArray) data;
+ return array.getDimensions();
+ }
+
+ @Override
+ public List<?> getList(Object data) {
+ if (data == null) {
+ return null;
+ }
+
+ PhoenixArray array = (PhoenixArray) data;
+ int valueLength = array.getDimensions();
+ List<Object> valueList = Lists.newArrayListWithExpectedSize(valueLength);
+
+ for (int i = 0; i < valueLength; i++) {
+ valueList.add(array.getElement(i));
+ }
+
+ return valueList;
+ }
+
+ public byte getSeparator() {
+ return separator;
+ }
+
+ public LazyObjectInspectorParameters getLazyParams() {
+ return lazyParams;
+ }
+}
diff --git a/phoenix-hive/src/main/java/org/apache/phoenix/hive/objectinspector/PhoenixLongObjectInspector.java b/phoenix-hive/src/main/java/org/apache/phoenix/hive/objectinspector/PhoenixLongObjectInspector.java
new file mode 100644
index 0000000..554f2a4
--- /dev/null
+++ b/phoenix-hive/src/main/java/org/apache/phoenix/hive/objectinspector/PhoenixLongObjectInspector.java
@@ -0,0 +1,56 @@
+/*
+ * 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.phoenix.hive.objectinspector;
+
+import org.apache.hadoop.hive.serde2.objectinspector.primitive.LongObjectInspector;
+import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoFactory;
+import org.apache.hadoop.io.LongWritable;
+
+public class PhoenixLongObjectInspector extends AbstractPhoenixObjectInspector<LongWritable>
+ implements LongObjectInspector {
+
+ public PhoenixLongObjectInspector() {
+ super(TypeInfoFactory.longTypeInfo);
+ }
+
+ @Override
+ public Object copyObject(Object o) {
+ return o == null ? null : new Long((Long) o);
+ }
+
+ @Override
+ public LongWritable getPrimitiveWritableObject(Object o) {
+ return new LongWritable(get(o));
+ }
+
+ @Override
+ public long get(Object o) {
+ Long value = null;
+
+ if (o != null) {
+ try {
+ value = ((Long) o).longValue();
+ } catch (Exception e) {
+ logExceptionMessage(o, "LONG");
+ }
+ }
+
+ return value;
+ }
+
+}
diff --git a/phoenix-hive/src/main/java/org/apache/phoenix/hive/objectinspector/PhoenixObjectInspectorFactory.java b/phoenix-hive/src/main/java/org/apache/phoenix/hive/objectinspector/PhoenixObjectInspectorFactory.java
new file mode 100644
index 0000000..3a19ea7
--- /dev/null
+++ b/phoenix-hive/src/main/java/org/apache/phoenix/hive/objectinspector/PhoenixObjectInspectorFactory.java
@@ -0,0 +1,148 @@
+/*
+ * 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.phoenix.hive.objectinspector;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.hive.serde2.lazy.LazySerDeParameters;
+import org.apache.hadoop.hive.serde2.lazy.objectinspector.LazyObjectInspectorFactory;
+import org.apache.hadoop.hive.serde2.lazy.objectinspector.LazySimpleStructObjectInspector;
+import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector;
+import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorFactory.ObjectInspectorOptions;
+import org.apache.hadoop.hive.serde2.typeinfo.ListTypeInfo;
+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 java.util.ArrayList;
+import java.util.List;
+
+/**
+ * Factory for object inspectors. Matches hive type to the corresponding Phoenix object inspector.
+ */
+
+public class PhoenixObjectInspectorFactory {
+
+ private static final Log LOG = LogFactory.getLog(PhoenixObjectInspectorFactory.class);
+
+ private PhoenixObjectInspectorFactory() {
+
+ }
+
+ public static LazySimpleStructObjectInspector createStructObjectInspector(TypeInfo type,
+ LazySerDeParameters
+ serdeParams) {
+ StructTypeInfo structTypeInfo = (StructTypeInfo) type;
+ List<String> fieldNames = structTypeInfo.getAllStructFieldNames();
+ List<TypeInfo> fieldTypeInfos = structTypeInfo.getAllStructFieldTypeInfos();
+ List<ObjectInspector> fieldObjectInspectors = new ArrayList<ObjectInspector>
+ (fieldTypeInfos.size());
+
+ for (int i = 0; i < fieldTypeInfos.size(); i++) {
+ fieldObjectInspectors.add(createObjectInspector(fieldTypeInfos.get(i), serdeParams));
+ }
+
+ return LazyObjectInspectorFactory.getLazySimpleStructObjectInspector(
+ fieldNames, fieldObjectInspectors, null,
+ serdeParams.getSeparators()[1],
+ serdeParams, ObjectInspectorOptions.JAVA);
+ }
+
+ public static ObjectInspector createObjectInspector(TypeInfo type, LazySerDeParameters
+ serdeParams) {
+ ObjectInspector oi = null;
+
+ if (LOG.isDebugEnabled()) {
+ LOG.debug("Type : " + type);
+ }
+
+ switch (type.getCategory()) {
+ case PRIMITIVE:
+ switch (((PrimitiveTypeInfo) type).getPrimitiveCategory()) {
+ case BOOLEAN:
+ oi = new PhoenixBooleanObjectInspector();
+ break;
+ case BYTE:
+ oi = new PhoenixByteObjectInspector();
+ break;
+ case SHORT:
+ oi = new PhoenixShortObjectInspector();
+ break;
+ case INT:
+ oi = new PhoenixIntObjectInspector();
+ break;
+ case LONG:
+ oi = new PhoenixLongObjectInspector();
+ break;
+ case FLOAT:
+ oi = new PhoenixFloatObjectInspector();
+ break;
+ case DOUBLE:
+ oi = new PhoenixDoubleObjectInspector();
+ break;
+ case VARCHAR:
+ // same string
+ case STRING:
+ oi = new PhoenixStringObjectInspector(serdeParams.isEscaped(),
+ serdeParams.getEscapeChar());
+ break;
+ case CHAR:
+ oi = new PhoenixCharObjectInspector((PrimitiveTypeInfo)type);
+ break;
+ case DATE:
+ oi = new PhoenixDateObjectInspector();
+ break;
+ case TIMESTAMP:
+ oi = new PhoenixTimestampObjectInspector();
+ break;
+ case DECIMAL:
+ oi = new PhoenixDecimalObjectInspector((PrimitiveTypeInfo) type);
+ break;
+ case BINARY:
+ oi = new PhoenixBinaryObjectInspector();
+ break;
+ default:
+ throw new RuntimeException("Hive internal error. not supported data type " +
+ ": " + type);
+ }
+
+ break;
+ case LIST:
+ if (LOG.isDebugEnabled()) {
+ LOG.debug("List type started");
+ }
+
+ ObjectInspector listElementObjectInspector = createObjectInspector((
+ (ListTypeInfo) type).getListElementTypeInfo(), serdeParams);
+
+ if (LOG.isDebugEnabled()) {
+ LOG.debug("List type ended");
+ }
+
+ oi = new PhoenixListObjectInspector(listElementObjectInspector, serdeParams
+ .getSeparators()[0], serdeParams);
+
+ break;
+ default:
+ throw new RuntimeException("Hive internal error. not supported data type : " +
+ type);
+ }
+
+ return oi;
+ }
+}
diff --git a/phoenix-hive/src/main/java/org/apache/phoenix/hive/objectinspector/PhoenixShortObjectInspector.java b/phoenix-hive/src/main/java/org/apache/phoenix/hive/objectinspector/PhoenixShortObjectInspector.java
new file mode 100644
index 0000000..84529b0
--- /dev/null
+++ b/phoenix-hive/src/main/java/org/apache/phoenix/hive/objectinspector/PhoenixShortObjectInspector.java
@@ -0,0 +1,56 @@
+/*
+ * 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.phoenix.hive.objectinspector;
+
+import org.apache.hadoop.hive.serde2.io.ShortWritable;
+import org.apache.hadoop.hive.serde2.objectinspector.primitive.ShortObjectInspector;
+import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoFactory;
+
+public class PhoenixShortObjectInspector extends AbstractPhoenixObjectInspector<ShortWritable>
+ implements ShortObjectInspector {
+
+ public PhoenixShortObjectInspector() {
+ super(TypeInfoFactory.shortTypeInfo);
+ }
+
+ @Override
+ public Object copyObject(Object o) {
+ return o == null ? null : new Short((Short) o);
+ }
+
+ @Override
+ public ShortWritable getPrimitiveWritableObject(Object o) {
+ return new ShortWritable(get(o));
+ }
+
+ @Override
+ public short get(Object o) {
+ Short value = null;
+
+ if (o != null) {
+ try {
+ value = ((Short) o).shortValue();
+ } catch (Exception e) {
+ logExceptionMessage(o, "SHORT");
+ }
+ }
+
+ return value;
+ }
+
+}
diff --git a/phoenix-hive/src/main/java/org/apache/phoenix/hive/objectinspector/PhoenixStringObjectInspector.java b/phoenix-hive/src/main/java/org/apache/phoenix/hive/objectinspector/PhoenixStringObjectInspector.java
new file mode 100644
index 0000000..e409e1d
--- /dev/null
+++ b/phoenix-hive/src/main/java/org/apache/phoenix/hive/objectinspector/PhoenixStringObjectInspector.java
@@ -0,0 +1,72 @@
+/*
+ * 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.phoenix.hive.objectinspector;
+
+import org.apache.hadoop.hive.serde2.objectinspector.primitive.StringObjectInspector;
+import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoFactory;
+import org.apache.hadoop.io.Text;
+
+/**
+ * ObjectInspector for string type
+ */
+public class PhoenixStringObjectInspector extends AbstractPhoenixObjectInspector<Text>
+ implements StringObjectInspector {
+
+ private boolean escaped;
... 17288 lines suppressed ...