mirror of https://github.com/apache/nifi.git
NIFI-4963: Added Hive3 bundle
- Incorporated review comments - Added more defensive code for PutHive3Streaming error handling This closes #2755. Signed-off-by: Bryan Bende <bbende@apache.org>
This commit is contained in:
parent
8feac9ae54
commit
da99f873a7
|
@ -55,4 +55,4 @@ script:
|
|||
# Note: The reason the sed is done as part of script is to ensure the pom hack
|
||||
# won't affect the 'clean install' above
|
||||
- bash .travis.sh
|
||||
- mvn -T 2 clean install -Pcontrib-check,include-grpc,include-atlas -Ddir-only | grep -v -F -f .travis-output-filters && exit ${PIPESTATUS[0]}
|
||||
- mvn -T 2 clean install -Pcontrib-check,include-grpc,include-atlas,include-hive3 -Ddir-only | grep -v -F -f .travis-output-filters && exit ${PIPESTATUS[0]}
|
||||
|
|
|
@ -547,6 +547,12 @@ language governing permissions and limitations under the License. -->
|
|||
<version>1.7.0-SNAPSHOT</version>
|
||||
<type>nar</type>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.apache.nifi</groupId>
|
||||
<artifactId>nifi-hive3-nar</artifactId>
|
||||
<version>1.7.0-SNAPSHOT</version>
|
||||
<type>nar</type>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.apache.nifi</groupId>
|
||||
<artifactId>nifi-site-to-site-reporting-nar</artifactId>
|
||||
|
@ -745,6 +751,23 @@ language governing permissions and limitations under the License. -->
|
|||
</dependency>
|
||||
</dependencies>
|
||||
</profile>
|
||||
<profile>
|
||||
<id>include-hive3</id>
|
||||
<!-- This profile handles the inclusion of Hive 3 artifacts. The NAR
|
||||
is quite large and makes the resultant binary distribution significantly
|
||||
larger (275+ MB). -->
|
||||
<activation>
|
||||
<activeByDefault>false</activeByDefault>
|
||||
</activation>
|
||||
<dependencies>
|
||||
<dependency>
|
||||
<groupId>org.apache.nifi</groupId>
|
||||
<artifactId>nifi-hive3-nar</artifactId>
|
||||
<version>1.7.0-SNAPSHOT</version>
|
||||
<type>nar</type>
|
||||
</dependency>
|
||||
</dependencies>
|
||||
</profile>
|
||||
<profile>
|
||||
<id>rpm</id>
|
||||
<activation>
|
||||
|
|
|
@ -28,6 +28,8 @@
|
|||
<properties>
|
||||
<maven.javadoc.skip>true</maven.javadoc.skip>
|
||||
<source.skip>true</source.skip>
|
||||
<!-- Need to override hadoop.version here, for Hive and hadoop-client transitive dependencies -->
|
||||
<hadoop.version>${hive.hadoop.version}</hadoop.version>
|
||||
</properties>
|
||||
|
||||
<dependencies>
|
||||
|
|
|
@ -24,13 +24,17 @@
|
|||
|
||||
<artifactId>nifi-hive-processors</artifactId>
|
||||
<packaging>jar</packaging>
|
||||
|
||||
<properties>
|
||||
<hive.version>1.2.1</hive.version>
|
||||
<!-- Need to override hadoop.version here, for Hive and hadoop-client transitive dependencies -->
|
||||
<hadoop.version>${hive.hadoop.version}</hadoop.version>
|
||||
</properties>
|
||||
|
||||
<dependencies>
|
||||
<dependency>
|
||||
<groupId>org.apache.nifi</groupId>
|
||||
<artifactId>nifi-api</artifactId>
|
||||
<version>1.7.0-SNAPSHOT</version>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.apache.nifi</groupId>
|
||||
|
@ -84,6 +88,7 @@
|
|||
<dependency>
|
||||
<groupId>org.apache.hadoop</groupId>
|
||||
<artifactId>hadoop-client</artifactId>
|
||||
<version>${hadoop.version}</version>
|
||||
<exclusions>
|
||||
<exclusion>
|
||||
<groupId>com.google.code.findbugs</groupId>
|
||||
|
|
|
@ -28,6 +28,8 @@
|
|||
<properties>
|
||||
<maven.javadoc.skip>true</maven.javadoc.skip>
|
||||
<source.skip>true</source.skip>
|
||||
<!-- Need to override hadoop.version here, for Hive and hadoop-client transitive dependencies -->
|
||||
<hadoop.version>${hive.hadoop.version}</hadoop.version>
|
||||
</properties>
|
||||
|
||||
<dependencies>
|
||||
|
|
|
@ -25,10 +25,16 @@
|
|||
<artifactId>nifi-hive-services-api</artifactId>
|
||||
<packaging>jar</packaging>
|
||||
|
||||
<properties>
|
||||
<!-- Need to override hadoop.version here, for Hive and hadoop-client transitive dependencies -->
|
||||
<hadoop.version>${hive.hadoop.version}</hadoop.version>
|
||||
</properties>
|
||||
|
||||
<dependencies>
|
||||
<dependency>
|
||||
<groupId>org.apache.nifi</groupId>
|
||||
<artifactId>nifi-api</artifactId>
|
||||
<version>1.7.0-SNAPSHOT</version>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.apache.nifi</groupId>
|
||||
|
|
|
@ -0,0 +1,30 @@
|
|||
/*
|
||||
* 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.nifi.dbcp.hive;
|
||||
|
||||
|
||||
import org.apache.nifi.annotation.documentation.CapabilityDescription;
|
||||
import org.apache.nifi.annotation.documentation.Tags;
|
||||
|
||||
/**
|
||||
* Definition for Database Connection Pooling Service.
|
||||
*
|
||||
*/
|
||||
@Tags({"hive", "dbcp", "jdbc", "database", "connection", "pooling", "store"})
|
||||
@CapabilityDescription("Provides Database Connection Pooling Service for Apache Hive. Connections can be asked from pool and returned after usage.")
|
||||
public interface Hive3DBCPService extends HiveDBCPService {
|
||||
}
|
|
@ -0,0 +1,49 @@
|
|||
<?xml version="1.0" encoding="UTF-8"?>
|
||||
<!--
|
||||
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.nifi</groupId>
|
||||
<artifactId>nifi-hive-bundle</artifactId>
|
||||
<version>1.7.0-SNAPSHOT</version>
|
||||
</parent>
|
||||
|
||||
<artifactId>nifi-hive3-nar</artifactId>
|
||||
<version>1.7.0-SNAPSHOT</version>
|
||||
<packaging>nar</packaging>
|
||||
<properties>
|
||||
<maven.javadoc.skip>true</maven.javadoc.skip>
|
||||
<source.skip>true</source.skip>
|
||||
<!-- Need to override hadoop.version here, for Hive and hadoop-client transitive dependencies -->
|
||||
<hadoop.version>${hive3.hadoop.version}</hadoop.version>
|
||||
</properties>
|
||||
|
||||
<dependencies>
|
||||
<dependency>
|
||||
<groupId>org.apache.nifi</groupId>
|
||||
<artifactId>nifi-hive-services-api-nar</artifactId>
|
||||
<version>1.7.0-SNAPSHOT</version>
|
||||
<type>nar</type>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.apache.nifi</groupId>
|
||||
<artifactId>nifi-hive3-processors</artifactId>
|
||||
<version>1.7.0-SNAPSHOT</version>
|
||||
</dependency>
|
||||
</dependencies>
|
||||
|
||||
</project>
|
|
@ -0,0 +1,349 @@
|
|||
nifi-hive-nar
|
||||
Copyright 2014-2017 The Apache Software Foundation
|
||||
|
||||
This product includes software developed at
|
||||
The Apache Software Foundation (http://www.apache.org/).
|
||||
|
||||
This includes derived works from the Apache Storm (ASLv2 licensed) project (https://github.com/apache/storm):
|
||||
Copyright 2015 The Apache Software Foundation
|
||||
The derived work is adapted from
|
||||
org/apache/storm/hive/common/HiveWriter.java
|
||||
org/apache/storm/hive/common/HiveOptions.java
|
||||
and can be found in the org.apache.nifi.util.hive package
|
||||
|
||||
This includes derived works from the Apache Hive (ASLv2 licensed) project (https://github.com/apache/hive):
|
||||
Copyright 2008-2016 The Apache Software Foundation
|
||||
The derived work is adapted from
|
||||
release-1.2.1/ql/src/java/org/apache/hadoop/hive/ql/io/orc/WriterImpl.java
|
||||
and can be found in the org.apache.hadoop.hive.ql.io.orc package
|
||||
The derived work is adapted from
|
||||
branch-3.0/streaming/src/java/org/apache/hive/streaming/StrictJsonWriter.java
|
||||
and can be found in the org.apache.hive.streaming.HiveRecordWriter class
|
||||
The derived work is adapted from
|
||||
branch-3.0/serde/src/java/org/apache/hadoop/hive/serde2/JsonSerDe.java
|
||||
and can be found in the org.apache.hive.streaming.NiFiRecordSerDe class
|
||||
|
||||
===========================================
|
||||
Apache Software License v2
|
||||
===========================================
|
||||
|
||||
The following binary components are provided under the Apache Software License v2
|
||||
|
||||
(ASLv2) Apache Ant
|
||||
The following NOTICE information applies:
|
||||
Apache Ant
|
||||
Copyright 1999-2016 The Apache Software Foundation
|
||||
|
||||
(ASLv2) Apache Commons Codec
|
||||
The following NOTICE information applies:
|
||||
Apache Commons Codec
|
||||
Copyright 2002-2014 The Apache Software Foundation
|
||||
|
||||
src/test/org/apache/commons/codec/language/DoubleMetaphoneTest.java
|
||||
contains test data from http://aspell.net/test/orig/batch0.tab.
|
||||
Copyright (C) 2002 Kevin Atkinson (kevina@gnu.org)
|
||||
|
||||
===============================================================================
|
||||
|
||||
The content of package org.apache.commons.codec.language.bm has been translated
|
||||
from the original php source code available at http://stevemorse.org/phoneticinfo.htm
|
||||
with permission from the original authors.
|
||||
Original source copyright:
|
||||
Copyright (c) 2008 Alexander Beider & Stephen P. Morse.
|
||||
|
||||
(ASLv2) Apache Commons DBCP
|
||||
The following NOTICE information applies:
|
||||
Apache Commons DBCP
|
||||
Copyright 2001-2015 The Apache Software Foundation.
|
||||
|
||||
(ASLv2) Apache Commons EL
|
||||
The following NOTICE information applies:
|
||||
Apache Commons EL
|
||||
Copyright 1999-2016 The Apache Software Foundation
|
||||
|
||||
EL-8 patch - Copyright 2004-2007 Jamie Taylor
|
||||
http://issues.apache.org/jira/browse/EL-8
|
||||
|
||||
(ASLv2) Apache HttpComponents
|
||||
The following NOTICE information applies:
|
||||
Apache HttpComponents Client
|
||||
Copyright 1999-2016 The Apache Software Foundation
|
||||
Apache HttpComponents Core - HttpCore
|
||||
Copyright 2006-2009 The Apache Software Foundation
|
||||
|
||||
(ASLv2) Apache Commons Logging
|
||||
The following NOTICE information applies:
|
||||
Apache Commons Logging
|
||||
Copyright 2003-2014 The Apache Software Foundation
|
||||
|
||||
(ASLv2) Apache Commons Pool
|
||||
The following NOTICE information applies:
|
||||
Apache Commons Pool
|
||||
Copyright 1999-2009 The Apache Software Foundation.
|
||||
|
||||
(ASLv2) Apache Commons IO
|
||||
The following NOTICE information applies:
|
||||
Apache Commons IO
|
||||
Copyright 2002-2016 The Apache Software Foundation
|
||||
|
||||
(ASLv2) Apache Hive
|
||||
The following NOTICE information applies:
|
||||
Apache Hive
|
||||
Copyright 2008-2015 The Apache Software Foundation
|
||||
|
||||
This product includes software developed by The Apache Software
|
||||
Foundation (http://www.apache.org/).
|
||||
|
||||
This product includes Jersey (https://jersey.java.net/)
|
||||
Copyright (c) 2010-2014 Oracle and/or its affiliates.
|
||||
|
||||
This project includes software copyrighted by Microsoft Corporation and
|
||||
licensed under the Apache License, Version 2.0.
|
||||
|
||||
This project includes software copyrighted by Dell SecureWorks and
|
||||
licensed under the Apache License, Version 2.0.
|
||||
|
||||
(ASLv2) Jackson JSON processor
|
||||
The following NOTICE information applies:
|
||||
# Jackson JSON processor
|
||||
|
||||
Jackson is a high-performance, Free/Open Source JSON processing library.
|
||||
It was originally written by Tatu Saloranta (tatu.saloranta@iki.fi), and has
|
||||
been in development since 2007.
|
||||
It is currently developed by a community of developers, as well as supported
|
||||
commercially by FasterXML.com.
|
||||
|
||||
## Licensing
|
||||
|
||||
Jackson core and extension components may licensed under different licenses.
|
||||
To find the details that apply to this artifact see the accompanying LICENSE file.
|
||||
For more information, including possible other licensing options, contact
|
||||
FasterXML.com (http://fasterxml.com).
|
||||
|
||||
## Credits
|
||||
|
||||
A list of contributors may be found from CREDITS file, which is included
|
||||
in some artifacts (usually source distributions); but is always available
|
||||
from the source code management (SCM) system project uses.
|
||||
|
||||
(ASLv2) BoneCP
|
||||
The following NOTICE information applies:
|
||||
BoneCP
|
||||
Copyright 2010 Wallace Wadge
|
||||
|
||||
(ASLv2) Apache Hadoop
|
||||
The following NOTICE information applies:
|
||||
The binary distribution of this product bundles binaries of
|
||||
org.iq80.leveldb:leveldb-api (https://github.com/dain/leveldb), which has the
|
||||
following notices:
|
||||
* Copyright 2011 Dain Sundstrom <dain@iq80.com>
|
||||
* Copyright 2011 FuseSource Corp. http://fusesource.com
|
||||
|
||||
The binary distribution of this product bundles binaries of
|
||||
org.fusesource.hawtjni:hawtjni-runtime (https://github.com/fusesource/hawtjni),
|
||||
which has the following notices:
|
||||
* This product includes software developed by FuseSource Corp.
|
||||
http://fusesource.com
|
||||
* This product includes software developed at
|
||||
Progress Software Corporation and/or its subsidiaries or affiliates.
|
||||
* This product includes software developed by IBM Corporation and others.
|
||||
|
||||
(ASLv2) Apache HBase
|
||||
The following NOTICE information applies:
|
||||
Apache HBase
|
||||
Copyright 2007-2015 The Apache Software Foundation
|
||||
|
||||
--
|
||||
This product incorporates portions of the 'Hadoop' project
|
||||
|
||||
Copyright 2007-2009 The Apache Software Foundation
|
||||
|
||||
Licensed under the Apache License v2.0
|
||||
--
|
||||
Our Orca logo we got here: http://www.vectorfree.com/jumping-orca
|
||||
It is licensed Creative Commons Attribution 3.0.
|
||||
See https://creativecommons.org/licenses/by/3.0/us/
|
||||
We changed the logo by stripping the colored background, inverting
|
||||
it and then rotating it some.
|
||||
|
||||
Later we found that vectorfree.com image is not properly licensed.
|
||||
The original is owned by vectorportal.com. The original was
|
||||
relicensed so we could use it as Creative Commons Attribution 3.0.
|
||||
The license is bundled with the download available here:
|
||||
http://www.vectorportal.com/subcategory/205/KILLER-WHALE-FREE-VECTOR.eps/ifile/9136/detailtest.asp
|
||||
--
|
||||
This product includes portions of the Bootstrap project v3.0.0
|
||||
|
||||
Copyright 2013 Twitter, Inc.
|
||||
|
||||
Licensed under the Apache License v2.0
|
||||
|
||||
This product uses the Glyphicons Halflings icon set.
|
||||
|
||||
http://glyphicons.com/
|
||||
|
||||
Copyright Jan Kovařík
|
||||
|
||||
Licensed under the Apache License v2.0 as a part of the Bootstrap project.
|
||||
|
||||
--
|
||||
This product includes portions of the Guava project v14, specifically
|
||||
'hbase-common/src/main/java/org/apache/hadoop/hbase/io/LimitInputStream.java'
|
||||
|
||||
Copyright (C) 2007 The Guava Authors
|
||||
|
||||
Licensed under the Apache License, Version 2.0
|
||||
|
||||
(ASLv2) Apache Commons Lang
|
||||
The following NOTICE information applies:
|
||||
Apache Commons Lang
|
||||
Copyright 2001-2015 The Apache Software Foundation
|
||||
|
||||
(ASLv2) Apache Curator
|
||||
The following NOTICE information applies:
|
||||
Apache Curator
|
||||
Copyright 2013-2014 The Apache Software Foundation
|
||||
|
||||
(ASLv2) Apache Derby
|
||||
The following NOTICE information applies:
|
||||
Apache Derby
|
||||
Copyright 2004-2014 Apache, Apache DB, Apache Derby, Apache Torque, Apache JDO, Apache DDLUtils,
|
||||
the Derby hat logo, the Apache JDO logo, and the Apache feather logo are trademarks of The Apache Software Foundation.
|
||||
|
||||
(ASLv2) Apache DS
|
||||
The following NOTICE information applies:
|
||||
ApacheDS
|
||||
Copyright 2003-2015 The Apache Software Foundation
|
||||
|
||||
(ASLv2) Apache Geronimo
|
||||
The following NOTICE information applies:
|
||||
Apache Geronimo
|
||||
Copyright 2003-2008 The Apache Software Foundation
|
||||
|
||||
(ASLv2) HTrace Core
|
||||
The following NOTICE information applies:
|
||||
In addition, this product includes software dependencies. See
|
||||
the accompanying LICENSE.txt for a listing of dependencies
|
||||
that are NOT Apache licensed (with pointers to their licensing)
|
||||
|
||||
Apache HTrace includes an Apache Thrift connector to Zipkin. Zipkin
|
||||
is a distributed tracing system that is Apache 2.0 Licensed.
|
||||
Copyright 2012 Twitter, Inc.
|
||||
|
||||
(ASLv2) Jettison
|
||||
The following NOTICE information applies:
|
||||
Copyright 2006 Envoi Solutions LLC
|
||||
|
||||
(ASLv2) Jetty
|
||||
The following NOTICE information applies:
|
||||
Jetty Web Container
|
||||
Copyright 1995-2017 Mort Bay Consulting Pty Ltd.
|
||||
|
||||
(ASLv2) Apache log4j
|
||||
The following NOTICE information applies:
|
||||
Apache log4j
|
||||
Copyright 2007 The Apache Software Foundation
|
||||
|
||||
(ASLv2) Parquet MR
|
||||
The following NOTICE information applies:
|
||||
Parquet MR
|
||||
Copyright 2012 Twitter, Inc.
|
||||
|
||||
This project includes code from https://github.com/lemire/JavaFastPFOR
|
||||
parquet-column/src/main/java/parquet/column/values/bitpacking/LemireBitPacking.java
|
||||
Apache License Version 2.0 http://www.apache.org/licenses/.
|
||||
(c) Daniel Lemire, http://lemire.me/en/
|
||||
|
||||
(ASLv2) Apache Thrift
|
||||
The following NOTICE information applies:
|
||||
Apache Thrift
|
||||
Copyright 2006-2010 The Apache Software Foundation.
|
||||
|
||||
(ASLv2) Apache Twill
|
||||
The following NOTICE information applies:
|
||||
Apache Twill
|
||||
Copyright 2013-2016 The Apache Software Foundation
|
||||
|
||||
(ASLv2) Dropwizard Metrics
|
||||
The following NOTICE information applies:
|
||||
Metrics
|
||||
Copyright 2010-2013 Coda Hale and Yammer, Inc.
|
||||
|
||||
This product includes code derived from the JSR-166 project (ThreadLocalRandom, Striped64,
|
||||
LongAdder), which was released with the following comments:
|
||||
|
||||
Written by Doug Lea with assistance from members of JCP JSR-166
|
||||
Expert Group and released to the public domain, as explained at
|
||||
http://creativecommons.org/publicdomain/zero/1.0/
|
||||
|
||||
(ASLv2) Joda Time
|
||||
The following NOTICE information applies:
|
||||
This product includes software developed by
|
||||
Joda.org (http://www.joda.org/).
|
||||
|
||||
(ASLv2) The Netty Project
|
||||
The following NOTICE information applies:
|
||||
The Netty Project
|
||||
Copyright 2011 The Netty Project
|
||||
|
||||
(ASLv2) Apache Tomcat
|
||||
The following NOTICE information applies:
|
||||
Apache Tomcat
|
||||
Copyright 2007 The Apache Software Foundation
|
||||
|
||||
Java Management Extensions (JMX) support is provided by
|
||||
the MX4J package, which is open source software. The
|
||||
original software and related information is available
|
||||
at http://mx4j.sourceforge.net.
|
||||
|
||||
Java compilation software for JSP pages is provided by Eclipse,
|
||||
which is open source software. The orginal software and
|
||||
related infomation is available at
|
||||
http://www.eclipse.org.
|
||||
|
||||
(ASLv2) Apache ZooKeeper
|
||||
The following NOTICE information applies:
|
||||
Apache ZooKeeper
|
||||
Copyright 2009-2012 The Apache Software Foundation
|
||||
|
||||
(ASLv2) Google GSON
|
||||
The following NOTICE information applies:
|
||||
Copyright 2008 Google Inc.
|
||||
|
||||
(ASLv2) JPam
|
||||
The following NOTICE information applies:
|
||||
Copyright 2003-2006 Greg Luck
|
||||
|
||||
************************
|
||||
Common Development and Distribution License 1.1
|
||||
************************
|
||||
|
||||
The following binary components are provided under the Common Development and Distribution License 1.1. See project link for details.
|
||||
|
||||
(CDDL 1.1) (GPL2 w/ CPE) jersey-client (com.sun.jersey:jersey-client:jar:1.9 - https://jersey.java.net)
|
||||
(CDDL 1.1) (GPL2 w/ CPE) jersey-core (com.sun.jersey:jersey-core:jar:1.9 - https://jersey.java.net/)
|
||||
(CDDL 1.1) (GPL2 w/ CPE) jersey-json (com.sun.jersey:jersey-json:jar:1.9 - https://jersey.java.net/)
|
||||
(CDDL 1.1) (GPL2 w/ CPE) jersey-server (com.sun.jersey:jersey-server:jar:1.9 - https://jersey.java.net/)
|
||||
(CDDL 1.1) (GPL2 w/ CPE) jersey-guice (com.sun.jersey.contribs:jersey-guice:jar:1.9 - https://jersey.java.net/)
|
||||
(CDDL 1.1) (GPL2 w/ CPE) Java Architecture For XML Binding (javax.xml.bind:jaxb-api:jar:2.2.2 - https://jaxb.dev.java.net/)
|
||||
(CDDL 1.1) (GPL2 w/ CPE) JavaMail API (compat) (javax.mail:mail:jar:1.4.7 - http://kenai.com/projects/javamail/mail)
|
||||
|
||||
|
||||
************************
|
||||
Common Development and Distribution License 1.0
|
||||
************************
|
||||
|
||||
The following binary components are provided under the Common Development and Distribution License 1.0. See project link for details.
|
||||
|
||||
(CDDL 1.0) JavaServlet(TM) Specification (javax.servlet:servlet-api:jar:2.5 - no url available)
|
||||
(CDDL 1.0) (GPL3) Streaming API For XML (javax.xml.stream:stax-api:jar:1.0-2 - no url provided)
|
||||
(CDDL 1.0) JavaBeans Activation Framework (JAF) (javax.activation:activation:jar:1.1 - http://java.sun.com/products/javabeans/jaf/index.jsp)
|
||||
(CDDL 1.0) JavaServer Pages(TM) API (javax.servlet.jsp:jsp-api:jar:2.1 - http://jsp.java.net)
|
||||
|
||||
*****************
|
||||
Public Domain
|
||||
*****************
|
||||
|
||||
The following binary components are provided to the 'Public Domain'. See project link for details.
|
||||
|
||||
(Public Domain) AOP Alliance 1.0 (http://aopalliance.sourceforge.net/)
|
|
@ -0,0 +1,140 @@
|
|||
<?xml version="1.0" encoding="UTF-8"?>
|
||||
<!--
|
||||
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.nifi</groupId>
|
||||
<artifactId>nifi-hive-bundle</artifactId>
|
||||
<version>1.7.0-SNAPSHOT</version>
|
||||
</parent>
|
||||
|
||||
<artifactId>nifi-hive3-processors</artifactId>
|
||||
<packaging>jar</packaging>
|
||||
|
||||
<properties>
|
||||
<!-- Need to override hadoop.version here, for Hive and hadoop-client transitive dependencies -->
|
||||
<hadoop.version>${hive3.hadoop.version}</hadoop.version>
|
||||
</properties>
|
||||
|
||||
<dependencies>
|
||||
<dependency>
|
||||
<groupId>org.apache.nifi</groupId>
|
||||
<artifactId>nifi-api</artifactId>
|
||||
<version>1.7.0-SNAPSHOT</version>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.apache.nifi</groupId>
|
||||
<artifactId>nifi-processor-utils</artifactId>
|
||||
<version>1.7.0-SNAPSHOT</version>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.apache.nifi</groupId>
|
||||
<artifactId>nifi-dbcp-service-api</artifactId>
|
||||
<version>1.7.0-SNAPSHOT</version>
|
||||
<scope>provided</scope>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.apache.nifi</groupId>
|
||||
<artifactId>nifi-hive-services-api</artifactId>
|
||||
<version>1.7.0-SNAPSHOT</version>
|
||||
<scope>provided</scope>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.apache.nifi</groupId>
|
||||
<artifactId>nifi-record</artifactId>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.apache.nifi</groupId>
|
||||
<artifactId>nifi-hadoop-record-utils</artifactId>
|
||||
<version>1.7.0-SNAPSHOT</version>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.apache.nifi</groupId>
|
||||
<artifactId>nifi-record-serialization-service-api</artifactId>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.apache.nifi</groupId>
|
||||
<artifactId>nifi-kerberos-credentials-service-api</artifactId>
|
||||
<scope>provided</scope>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.apache.hive</groupId>
|
||||
<artifactId>hive-jdbc</artifactId>
|
||||
<version>${hive3.version}</version>
|
||||
<exclusions>
|
||||
<exclusion>
|
||||
<groupId>org.json</groupId>
|
||||
<artifactId>json</artifactId>
|
||||
</exclusion>
|
||||
</exclusions>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.apache.hive</groupId>
|
||||
<artifactId>hive-streaming</artifactId>
|
||||
<version>${hive3.version}</version>
|
||||
<exclusions>
|
||||
<exclusion>
|
||||
<groupId>org.slf4j</groupId>
|
||||
<artifactId>slf4j-log4j12</artifactId>
|
||||
</exclusion>
|
||||
</exclusions>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.apache.hive.hcatalog</groupId>
|
||||
<artifactId>hive-hcatalog-core</artifactId>
|
||||
<version>${hive3.version}</version>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.apache.hadoop</groupId>
|
||||
<artifactId>hadoop-client</artifactId>
|
||||
<version>${hadoop.version}</version>
|
||||
<exclusions>
|
||||
<exclusion>
|
||||
<groupId>com.google.code.findbugs</groupId>
|
||||
<artifactId>jsr305</artifactId>
|
||||
</exclusion>
|
||||
</exclusions>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.apache.nifi</groupId>
|
||||
<artifactId>nifi-hadoop-utils</artifactId>
|
||||
<version>1.7.0-SNAPSHOT</version>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>com.github.stephenc.findbugs</groupId>
|
||||
<artifactId>findbugs-annotations</artifactId>
|
||||
<version>1.3.9-1</version>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.apache.nifi</groupId>
|
||||
<artifactId>nifi-mock-record-utils</artifactId>
|
||||
<version>1.7.0-SNAPSHOT</version>
|
||||
<scope>test</scope>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.apache.nifi</groupId>
|
||||
<artifactId>nifi-mock</artifactId>
|
||||
<version>1.7.0-SNAPSHOT</version>
|
||||
<scope>test</scope>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>junit</groupId>
|
||||
<artifactId>junit</artifactId>
|
||||
<scope>test</scope>
|
||||
</dependency>
|
||||
</dependencies>
|
||||
</project>
|
|
@ -0,0 +1,533 @@
|
|||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
* contributor license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright ownership.
|
||||
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||
* (the "License"); you may not use this file except in compliance with
|
||||
* the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
package org.apache.hadoop.hive.ql.io.orc;
|
||||
|
||||
import org.apache.avro.LogicalType;
|
||||
import org.apache.avro.LogicalTypes;
|
||||
import org.apache.avro.Schema;
|
||||
import org.apache.avro.generic.GenericData;
|
||||
import org.apache.avro.util.Utf8;
|
||||
import org.apache.commons.lang3.StringUtils;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.hive.serde2.io.DateWritable;
|
||||
import org.apache.hadoop.hive.serde2.io.TimestampWritable;
|
||||
import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector;
|
||||
import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorFactory;
|
||||
import org.apache.hadoop.hive.serde2.objectinspector.SettableStructObjectInspector;
|
||||
import org.apache.hadoop.hive.serde2.objectinspector.StructField;
|
||||
import org.apache.hadoop.hive.serde2.typeinfo.ListTypeInfo;
|
||||
import org.apache.hadoop.hive.serde2.typeinfo.MapTypeInfo;
|
||||
import org.apache.hadoop.hive.serde2.typeinfo.TypeInfo;
|
||||
import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoFactory;
|
||||
import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoUtils;
|
||||
import org.apache.hadoop.hive.serde2.typeinfo.UnionTypeInfo;
|
||||
import org.apache.hadoop.io.BooleanWritable;
|
||||
import org.apache.hadoop.io.BytesWritable;
|
||||
import org.apache.hadoop.io.DoubleWritable;
|
||||
import org.apache.hadoop.io.FloatWritable;
|
||||
import org.apache.hadoop.io.IntWritable;
|
||||
import org.apache.hadoop.io.LongWritable;
|
||||
import org.apache.hadoop.io.Text;
|
||||
import org.apache.orc.MemoryManager;
|
||||
import org.apache.orc.OrcConf;
|
||||
import org.apache.orc.impl.MemoryManagerImpl;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.nio.ByteBuffer;
|
||||
import java.sql.Date;
|
||||
import java.sql.Timestamp;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Arrays;
|
||||
import java.util.HashMap;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.stream.Collectors;
|
||||
import java.util.stream.IntStream;
|
||||
|
||||
|
||||
/**
|
||||
* Utility methods for ORC support (conversion from Avro, conversion to Hive types, e.g.
|
||||
*/
|
||||
public class NiFiOrcUtils {
|
||||
|
||||
public static Object convertToORCObject(TypeInfo typeInfo, Object o, final boolean hiveFieldNames) {
|
||||
if (o != null) {
|
||||
if (typeInfo instanceof UnionTypeInfo) {
|
||||
OrcUnion union = new OrcUnion();
|
||||
// Avro uses Utf8 and GenericData.EnumSymbol objects instead of Strings. This is handled in other places in the method, but here
|
||||
// we need to determine the union types from the objects, so choose String.class if the object is one of those Avro classes
|
||||
Class clazzToCompareTo = o.getClass();
|
||||
if (o instanceof org.apache.avro.util.Utf8 || o instanceof GenericData.EnumSymbol) {
|
||||
clazzToCompareTo = String.class;
|
||||
}
|
||||
// Need to find which of the union types correspond to the primitive object
|
||||
TypeInfo objectTypeInfo = TypeInfoUtils.getTypeInfoFromObjectInspector(
|
||||
ObjectInspectorFactory.getReflectionObjectInspector(clazzToCompareTo, ObjectInspectorFactory.ObjectInspectorOptions.JAVA));
|
||||
List<TypeInfo> unionTypeInfos = ((UnionTypeInfo) typeInfo).getAllUnionObjectTypeInfos();
|
||||
|
||||
int index = 0;
|
||||
while (index < unionTypeInfos.size() && !unionTypeInfos.get(index).equals(objectTypeInfo)) {
|
||||
index++;
|
||||
}
|
||||
if (index < unionTypeInfos.size()) {
|
||||
union.set((byte) index, convertToORCObject(objectTypeInfo, o, hiveFieldNames));
|
||||
} else {
|
||||
throw new IllegalArgumentException("Object Type for class " + o.getClass().getName() + " not in Union declaration");
|
||||
}
|
||||
return union;
|
||||
}
|
||||
if (o instanceof Integer) {
|
||||
return new IntWritable((int) o);
|
||||
}
|
||||
if (o instanceof Boolean) {
|
||||
return new BooleanWritable((boolean) o);
|
||||
}
|
||||
if (o instanceof Long) {
|
||||
return new LongWritable((long) o);
|
||||
}
|
||||
if (o instanceof Float) {
|
||||
return new FloatWritable((float) o);
|
||||
}
|
||||
if (o instanceof Double) {
|
||||
return new DoubleWritable((double) o);
|
||||
}
|
||||
if (o instanceof String || o instanceof Utf8 || o instanceof GenericData.EnumSymbol) {
|
||||
return new Text(o.toString());
|
||||
}
|
||||
if (o instanceof ByteBuffer) {
|
||||
return new BytesWritable(((ByteBuffer) o).array());
|
||||
}
|
||||
if (o instanceof Timestamp) {
|
||||
return new TimestampWritable((Timestamp) o);
|
||||
}
|
||||
if (o instanceof Date) {
|
||||
return new DateWritable((Date) o);
|
||||
}
|
||||
if (o instanceof Object[]) {
|
||||
Object[] objArray = (Object[]) o;
|
||||
TypeInfo listTypeInfo = ((ListTypeInfo) typeInfo).getListElementTypeInfo();
|
||||
return Arrays.stream(objArray)
|
||||
.map(o1 -> convertToORCObject(listTypeInfo, o1, hiveFieldNames))
|
||||
.collect(Collectors.toList());
|
||||
}
|
||||
if (o instanceof int[]) {
|
||||
int[] intArray = (int[]) o;
|
||||
return Arrays.stream(intArray)
|
||||
.mapToObj((element) -> convertToORCObject(TypeInfoFactory.getPrimitiveTypeInfo("int"), element, hiveFieldNames))
|
||||
.collect(Collectors.toList());
|
||||
}
|
||||
if (o instanceof long[]) {
|
||||
long[] longArray = (long[]) o;
|
||||
return Arrays.stream(longArray)
|
||||
.mapToObj((element) -> convertToORCObject(TypeInfoFactory.getPrimitiveTypeInfo("bigint"), element, hiveFieldNames))
|
||||
.collect(Collectors.toList());
|
||||
}
|
||||
if (o instanceof float[]) {
|
||||
float[] floatArray = (float[]) o;
|
||||
return IntStream.range(0, floatArray.length)
|
||||
.mapToDouble(i -> floatArray[i])
|
||||
.mapToObj((element) -> convertToORCObject(TypeInfoFactory.getPrimitiveTypeInfo("float"), (float) element, hiveFieldNames))
|
||||
.collect(Collectors.toList());
|
||||
}
|
||||
if (o instanceof double[]) {
|
||||
double[] doubleArray = (double[]) o;
|
||||
return Arrays.stream(doubleArray)
|
||||
.mapToObj((element) -> convertToORCObject(TypeInfoFactory.getPrimitiveTypeInfo("double"), element, hiveFieldNames))
|
||||
.collect(Collectors.toList());
|
||||
}
|
||||
if (o instanceof boolean[]) {
|
||||
boolean[] booleanArray = (boolean[]) o;
|
||||
return IntStream.range(0, booleanArray.length)
|
||||
.map(i -> booleanArray[i] ? 1 : 0)
|
||||
.mapToObj((element) -> convertToORCObject(TypeInfoFactory.getPrimitiveTypeInfo("boolean"), element == 1, hiveFieldNames))
|
||||
.collect(Collectors.toList());
|
||||
}
|
||||
if (o instanceof GenericData.Array) {
|
||||
GenericData.Array array = ((GenericData.Array) o);
|
||||
// The type information in this case is interpreted as a List
|
||||
TypeInfo listTypeInfo = ((ListTypeInfo) typeInfo).getListElementTypeInfo();
|
||||
return array.stream().map((element) -> convertToORCObject(listTypeInfo, element, hiveFieldNames)).collect(Collectors.toList());
|
||||
}
|
||||
if (o instanceof List) {
|
||||
return o;
|
||||
}
|
||||
if (o instanceof Map) {
|
||||
Map map = new HashMap();
|
||||
TypeInfo keyInfo = ((MapTypeInfo) typeInfo).getMapKeyTypeInfo();
|
||||
TypeInfo valueInfo = ((MapTypeInfo) typeInfo).getMapValueTypeInfo();
|
||||
// Unions are not allowed as key/value types, so if we convert the key and value objects,
|
||||
// they should return Writable objects
|
||||
((Map) o).forEach((key, value) -> {
|
||||
Object keyObject = convertToORCObject(keyInfo, key, hiveFieldNames);
|
||||
Object valueObject = convertToORCObject(valueInfo, value, hiveFieldNames);
|
||||
if (keyObject == null) {
|
||||
throw new IllegalArgumentException("Maps' key cannot be null");
|
||||
}
|
||||
map.put(keyObject, valueObject);
|
||||
});
|
||||
return map;
|
||||
}
|
||||
if (o instanceof GenericData.Record) {
|
||||
GenericData.Record record = (GenericData.Record) o;
|
||||
TypeInfo recordSchema = NiFiOrcUtils.getOrcField(record.getSchema(), hiveFieldNames);
|
||||
List<Schema.Field> recordFields = record.getSchema().getFields();
|
||||
if (recordFields != null) {
|
||||
Object[] fieldObjects = new Object[recordFields.size()];
|
||||
for (int i = 0; i < recordFields.size(); i++) {
|
||||
Schema.Field field = recordFields.get(i);
|
||||
Schema fieldSchema = field.schema();
|
||||
Object fieldObject = record.get(field.name());
|
||||
fieldObjects[i] = NiFiOrcUtils.convertToORCObject(NiFiOrcUtils.getOrcField(fieldSchema, hiveFieldNames), fieldObject, hiveFieldNames);
|
||||
}
|
||||
return NiFiOrcUtils.createOrcStruct(recordSchema, fieldObjects);
|
||||
}
|
||||
}
|
||||
throw new IllegalArgumentException("Error converting object of type " + o.getClass().getName() + " to ORC type " + typeInfo.getTypeName());
|
||||
} else {
|
||||
return null;
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
/**
|
||||
* Create an object of OrcStruct given a TypeInfo and a list of objects
|
||||
*
|
||||
* @param typeInfo The TypeInfo object representing the ORC record schema
|
||||
* @param objs ORC objects/Writables
|
||||
* @return an OrcStruct containing the specified objects for the specified schema
|
||||
*/
|
||||
@SuppressWarnings("unchecked")
|
||||
public static OrcStruct createOrcStruct(TypeInfo typeInfo, Object... objs) {
|
||||
SettableStructObjectInspector oi = (SettableStructObjectInspector) OrcStruct
|
||||
.createObjectInspector(typeInfo);
|
||||
List<StructField> fields = (List<StructField>) oi.getAllStructFieldRefs();
|
||||
OrcStruct result = (OrcStruct) oi.create();
|
||||
result.setNumFields(fields.size());
|
||||
for (int i = 0; i < fields.size(); i++) {
|
||||
oi.setStructFieldData(result, fields.get(i), objs[i]);
|
||||
}
|
||||
return result;
|
||||
}
|
||||
|
||||
public static String normalizeHiveTableName(String name) {
|
||||
return name.replaceAll("[\\. ]", "_");
|
||||
}
|
||||
|
||||
public static String generateHiveDDL(Schema avroSchema, String tableName, boolean hiveFieldNames) {
|
||||
Schema.Type schemaType = avroSchema.getType();
|
||||
StringBuilder sb = new StringBuilder("CREATE EXTERNAL TABLE IF NOT EXISTS ");
|
||||
sb.append(tableName);
|
||||
sb.append(" (");
|
||||
if (Schema.Type.RECORD.equals(schemaType)) {
|
||||
List<String> hiveColumns = new ArrayList<>();
|
||||
List<Schema.Field> fields = avroSchema.getFields();
|
||||
if (fields != null) {
|
||||
hiveColumns.addAll(
|
||||
fields.stream().map(field -> (hiveFieldNames ? field.name().toLowerCase() : field.name()) + " "
|
||||
+ getHiveTypeFromAvroType(field.schema(), hiveFieldNames)).collect(Collectors.toList()));
|
||||
}
|
||||
sb.append(StringUtils.join(hiveColumns, ", "));
|
||||
sb.append(") STORED AS ORC");
|
||||
return sb.toString();
|
||||
} else {
|
||||
throw new IllegalArgumentException("Avro schema is of type " + schemaType.getName() + ", not RECORD");
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
public static TypeInfo getOrcField(Schema fieldSchema, boolean hiveFieldNames) throws IllegalArgumentException {
|
||||
Schema.Type fieldType = fieldSchema.getType();
|
||||
LogicalType logicalType = fieldSchema.getLogicalType();
|
||||
|
||||
switch (fieldType) {
|
||||
case INT:
|
||||
case LONG:
|
||||
// Handle logical types
|
||||
if (logicalType != null) {
|
||||
if (LogicalTypes.date().equals(logicalType)) {
|
||||
return TypeInfoFactory.dateTypeInfo;
|
||||
} else if (LogicalTypes.timeMicros().equals(logicalType)) {
|
||||
// Time micros isn't supported by our Record Field types (see AvroTypeUtil)
|
||||
throw new IllegalArgumentException("time-micros is not a supported field type");
|
||||
} else if (LogicalTypes.timeMillis().equals(logicalType)) {
|
||||
return TypeInfoFactory.intTypeInfo;
|
||||
} else if (LogicalTypes.timestampMicros().equals(logicalType)) {
|
||||
// Timestamp micros isn't supported by our Record Field types (see AvroTypeUtil)
|
||||
throw new IllegalArgumentException("timestamp-micros is not a supported field type");
|
||||
} else if (LogicalTypes.timestampMillis().equals(logicalType)) {
|
||||
return TypeInfoFactory.timestampTypeInfo;
|
||||
}
|
||||
}
|
||||
return getPrimitiveOrcTypeFromPrimitiveAvroType(fieldType);
|
||||
case BYTES:
|
||||
// Handle logical types
|
||||
if (logicalType != null) {
|
||||
if (logicalType instanceof LogicalTypes.Decimal) {
|
||||
return TypeInfoFactory.doubleTypeInfo;
|
||||
}
|
||||
}
|
||||
return getPrimitiveOrcTypeFromPrimitiveAvroType(fieldType);
|
||||
|
||||
case BOOLEAN:
|
||||
case DOUBLE:
|
||||
case FLOAT:
|
||||
case STRING:
|
||||
return getPrimitiveOrcTypeFromPrimitiveAvroType(fieldType);
|
||||
|
||||
case UNION:
|
||||
List<Schema> unionFieldSchemas = fieldSchema.getTypes();
|
||||
|
||||
if (unionFieldSchemas != null) {
|
||||
// Ignore null types in union
|
||||
List<TypeInfo> orcFields = unionFieldSchemas.stream().filter(
|
||||
unionFieldSchema -> !Schema.Type.NULL.equals(unionFieldSchema.getType()))
|
||||
.map((it) -> NiFiOrcUtils.getOrcField(it, hiveFieldNames))
|
||||
.collect(Collectors.toList());
|
||||
|
||||
// Flatten the field if the union only has one non-null element
|
||||
if (orcFields.size() == 1) {
|
||||
return orcFields.get(0);
|
||||
} else {
|
||||
return TypeInfoFactory.getUnionTypeInfo(orcFields);
|
||||
}
|
||||
}
|
||||
return null;
|
||||
|
||||
case ARRAY:
|
||||
return TypeInfoFactory.getListTypeInfo(getOrcField(fieldSchema.getElementType(), hiveFieldNames));
|
||||
|
||||
case MAP:
|
||||
return TypeInfoFactory.getMapTypeInfo(
|
||||
getPrimitiveOrcTypeFromPrimitiveAvroType(Schema.Type.STRING),
|
||||
getOrcField(fieldSchema.getValueType(), hiveFieldNames));
|
||||
|
||||
case RECORD:
|
||||
List<Schema.Field> avroFields = fieldSchema.getFields();
|
||||
if (avroFields != null) {
|
||||
List<String> orcFieldNames = new ArrayList<>(avroFields.size());
|
||||
List<TypeInfo> orcFields = new ArrayList<>(avroFields.size());
|
||||
avroFields.forEach(avroField -> {
|
||||
String fieldName = hiveFieldNames ? avroField.name().toLowerCase() : avroField.name();
|
||||
orcFieldNames.add(fieldName);
|
||||
orcFields.add(getOrcField(avroField.schema(), hiveFieldNames));
|
||||
});
|
||||
return TypeInfoFactory.getStructTypeInfo(orcFieldNames, orcFields);
|
||||
}
|
||||
return null;
|
||||
|
||||
case ENUM:
|
||||
// An enum value is just a String for ORC/Hive
|
||||
return getPrimitiveOrcTypeFromPrimitiveAvroType(Schema.Type.STRING);
|
||||
|
||||
default:
|
||||
throw new IllegalArgumentException("Did not recognize Avro type " + fieldType.getName());
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
public static Schema.Type getAvroSchemaTypeOfObject(Object o) {
|
||||
if (o == null) {
|
||||
return Schema.Type.NULL;
|
||||
} else if (o instanceof Integer) {
|
||||
return Schema.Type.INT;
|
||||
} else if (o instanceof Long) {
|
||||
return Schema.Type.LONG;
|
||||
} else if (o instanceof Boolean) {
|
||||
return Schema.Type.BOOLEAN;
|
||||
} else if (o instanceof byte[]) {
|
||||
return Schema.Type.BYTES;
|
||||
} else if (o instanceof Float) {
|
||||
return Schema.Type.FLOAT;
|
||||
} else if (o instanceof Double) {
|
||||
return Schema.Type.DOUBLE;
|
||||
} else if (o instanceof Enum) {
|
||||
return Schema.Type.ENUM;
|
||||
} else if (o instanceof Object[]) {
|
||||
return Schema.Type.ARRAY;
|
||||
} else if (o instanceof List) {
|
||||
return Schema.Type.ARRAY;
|
||||
} else if (o instanceof Map) {
|
||||
return Schema.Type.MAP;
|
||||
} else {
|
||||
throw new IllegalArgumentException("Object of class " + o.getClass() + " is not a supported Avro Type");
|
||||
}
|
||||
}
|
||||
|
||||
public static TypeInfo getPrimitiveOrcTypeFromPrimitiveAvroType(Schema.Type avroType) throws IllegalArgumentException {
|
||||
if (avroType == null) {
|
||||
throw new IllegalArgumentException("Avro type is null");
|
||||
}
|
||||
switch (avroType) {
|
||||
case INT:
|
||||
return TypeInfoFactory.getPrimitiveTypeInfo("int");
|
||||
case LONG:
|
||||
return TypeInfoFactory.getPrimitiveTypeInfo("bigint");
|
||||
case BOOLEAN:
|
||||
return TypeInfoFactory.getPrimitiveTypeInfo("boolean");
|
||||
case BYTES:
|
||||
return TypeInfoFactory.getPrimitiveTypeInfo("binary");
|
||||
case DOUBLE:
|
||||
return TypeInfoFactory.getPrimitiveTypeInfo("double");
|
||||
case FLOAT:
|
||||
return TypeInfoFactory.getPrimitiveTypeInfo("float");
|
||||
case STRING:
|
||||
return TypeInfoFactory.getPrimitiveTypeInfo("string");
|
||||
default:
|
||||
throw new IllegalArgumentException("Avro type " + avroType.getName() + " is not a primitive type");
|
||||
}
|
||||
}
|
||||
|
||||
public static String getHiveTypeFromAvroType(Schema avroSchema, boolean hiveFieldNames) {
|
||||
if (avroSchema == null) {
|
||||
throw new IllegalArgumentException("Avro schema is null");
|
||||
}
|
||||
|
||||
Schema.Type avroType = avroSchema.getType();
|
||||
LogicalType logicalType = avroSchema.getLogicalType();
|
||||
|
||||
switch (avroType) {
|
||||
case INT:
|
||||
if (logicalType != null) {
|
||||
if (LogicalTypes.date().equals(logicalType)) {
|
||||
return "DATE";
|
||||
}
|
||||
// Time-millis has no current corresponding Hive type, perhaps an INTERVAL type when that is fully supported.
|
||||
}
|
||||
return "INT";
|
||||
case LONG:
|
||||
if (logicalType != null) {
|
||||
if (LogicalTypes.timestampMillis().equals(logicalType)) {
|
||||
return "TIMESTAMP";
|
||||
}
|
||||
// Timestamp-micros and time-micros are not supported by our Record Field type system
|
||||
}
|
||||
return "BIGINT";
|
||||
case BOOLEAN:
|
||||
return "BOOLEAN";
|
||||
case BYTES:
|
||||
if (logicalType != null) {
|
||||
if (logicalType instanceof LogicalTypes.Decimal) {
|
||||
return "DOUBLE";
|
||||
}
|
||||
}
|
||||
return "BINARY";
|
||||
case DOUBLE:
|
||||
return "DOUBLE";
|
||||
case FLOAT:
|
||||
return "FLOAT";
|
||||
case STRING:
|
||||
case ENUM:
|
||||
return "STRING";
|
||||
case UNION:
|
||||
List<Schema> unionFieldSchemas = avroSchema.getTypes();
|
||||
if (unionFieldSchemas != null) {
|
||||
List<String> hiveFields = new ArrayList<>();
|
||||
for (Schema unionFieldSchema : unionFieldSchemas) {
|
||||
Schema.Type unionFieldSchemaType = unionFieldSchema.getType();
|
||||
// Ignore null types in union
|
||||
if (!Schema.Type.NULL.equals(unionFieldSchemaType)) {
|
||||
hiveFields.add(getHiveTypeFromAvroType(unionFieldSchema, hiveFieldNames));
|
||||
}
|
||||
}
|
||||
// Flatten the field if the union only has one non-null element
|
||||
return (hiveFields.size() == 1)
|
||||
? hiveFields.get(0)
|
||||
: "UNIONTYPE<" + StringUtils.join(hiveFields, ", ") + ">";
|
||||
|
||||
}
|
||||
break;
|
||||
case MAP:
|
||||
return "MAP<STRING, " + getHiveTypeFromAvroType(avroSchema.getValueType(), hiveFieldNames) + ">";
|
||||
case ARRAY:
|
||||
return "ARRAY<" + getHiveTypeFromAvroType(avroSchema.getElementType(), hiveFieldNames) + ">";
|
||||
case RECORD:
|
||||
List<Schema.Field> recordFields = avroSchema.getFields();
|
||||
if (recordFields != null) {
|
||||
List<String> hiveFields = recordFields.stream().map(
|
||||
recordField -> (hiveFieldNames ? recordField.name().toLowerCase() : recordField.name()) + ":"
|
||||
+ getHiveTypeFromAvroType(recordField.schema(), hiveFieldNames)).collect(Collectors.toList());
|
||||
return "STRUCT<" + StringUtils.join(hiveFields, ", ") + ">";
|
||||
}
|
||||
break;
|
||||
default:
|
||||
break;
|
||||
}
|
||||
|
||||
throw new IllegalArgumentException("Error converting Avro type " + avroType.getName() + " to Hive type");
|
||||
}
|
||||
|
||||
|
||||
public static Writer createWriter(
|
||||
Path path,
|
||||
Configuration conf,
|
||||
TypeInfo orcSchema,
|
||||
long stripeSize,
|
||||
CompressionKind compress,
|
||||
int bufferSize) throws IOException {
|
||||
|
||||
int rowIndexStride = (int) OrcConf.ROW_INDEX_STRIDE.getLong(conf);
|
||||
|
||||
boolean addBlockPadding = OrcConf.BLOCK_PADDING.getBoolean(conf);
|
||||
|
||||
String versionName = OrcConf.WRITE_FORMAT.getString(conf);
|
||||
OrcFile.Version versionValue = (versionName == null)
|
||||
? OrcFile.Version.CURRENT
|
||||
: OrcFile.Version.byName(versionName);
|
||||
|
||||
OrcFile.EncodingStrategy encodingStrategy;
|
||||
String enString = OrcConf.ENCODING_STRATEGY.getString(conf);
|
||||
if (enString == null) {
|
||||
encodingStrategy = OrcFile.EncodingStrategy.SPEED;
|
||||
} else {
|
||||
encodingStrategy = OrcFile.EncodingStrategy.valueOf(enString);
|
||||
}
|
||||
|
||||
final double paddingTolerance = OrcConf.BLOCK_PADDING_TOLERANCE.getDouble(conf);
|
||||
|
||||
long blockSizeValue = OrcConf.BLOCK_SIZE.getLong(conf);
|
||||
|
||||
double bloomFilterFpp = OrcConf.BLOOM_FILTER_FPP.getDouble(conf);
|
||||
|
||||
ObjectInspector inspector = OrcStruct.createObjectInspector(orcSchema);
|
||||
|
||||
OrcFile.WriterOptions writerOptions = OrcFile.writerOptions(conf)
|
||||
.rowIndexStride(rowIndexStride)
|
||||
.blockPadding(addBlockPadding)
|
||||
.version(versionValue)
|
||||
.encodingStrategy(encodingStrategy)
|
||||
.paddingTolerance(paddingTolerance)
|
||||
.blockSize(blockSizeValue)
|
||||
.bloomFilterFpp(bloomFilterFpp)
|
||||
.memory(getMemoryManager(conf))
|
||||
.inspector(inspector)
|
||||
.stripeSize(stripeSize)
|
||||
.bufferSize(bufferSize)
|
||||
.compress(compress);
|
||||
|
||||
return OrcFile.createWriter(path, writerOptions);
|
||||
}
|
||||
|
||||
private static MemoryManager memoryManager = null;
|
||||
|
||||
private static synchronized MemoryManager getMemoryManager(Configuration conf) {
|
||||
if (memoryManager == null) {
|
||||
memoryManager = new MemoryManagerImpl(conf);
|
||||
}
|
||||
return memoryManager;
|
||||
}
|
||||
}
|
|
@ -0,0 +1,106 @@
|
|||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
* contributor license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright ownership.
|
||||
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||
* (the "License"); you may not use this file except in compliance with
|
||||
* the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
package org.apache.hive.streaming;
|
||||
|
||||
import com.google.common.base.Joiner;
|
||||
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.SerDeUtils;
|
||||
import org.apache.hadoop.io.ObjectWritable;
|
||||
import org.apache.nifi.logging.ComponentLog;
|
||||
import org.apache.nifi.serialization.MalformedRecordException;
|
||||
import org.apache.nifi.serialization.RecordReader;
|
||||
import org.apache.nifi.serialization.record.Record;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.io.InputStream;
|
||||
import java.util.List;
|
||||
import java.util.Properties;
|
||||
|
||||
public class HiveRecordWriter extends AbstractRecordWriter {
|
||||
|
||||
private RecordReader recordReader;
|
||||
private NiFiRecordSerDe serde;
|
||||
private ComponentLog log;
|
||||
|
||||
public HiveRecordWriter(RecordReader recordReader, ComponentLog log) {
|
||||
super(null);
|
||||
this.recordReader = recordReader;
|
||||
this.log = log;
|
||||
}
|
||||
|
||||
@Override
|
||||
public AbstractSerDe createSerde() throws SerializationError {
|
||||
try {
|
||||
Properties tableProps = table.getMetadata();
|
||||
tableProps.setProperty(serdeConstants.LIST_COLUMNS, Joiner.on(",").join(inputColumns));
|
||||
tableProps.setProperty(serdeConstants.LIST_COLUMN_TYPES, Joiner.on(":").join(inputTypes));
|
||||
NiFiRecordSerDe serde = new NiFiRecordSerDe(recordReader, log);
|
||||
SerDeUtils.initializeSerDe(serde, conf, tableProps, null);
|
||||
this.serde = serde;
|
||||
return serde;
|
||||
} catch (SerDeException e) {
|
||||
throw new SerializationError("Error initializing serde " + NiFiRecordSerDe.class.getName(), e);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public Object encode(byte[] bytes) {
|
||||
throw new UnsupportedOperationException(this.getClass().getName() + " does not support encoding of records via bytes, only via an InputStream");
|
||||
}
|
||||
|
||||
@Override
|
||||
public void write(long writeId, byte[] record) {
|
||||
throw new UnsupportedOperationException(this.getClass().getName() + " does not support writing of records via bytes, only via an InputStream");
|
||||
}
|
||||
|
||||
@Override
|
||||
public void write(long writeId, InputStream inputStream) throws StreamingException {
|
||||
// The inputStream is already available to the recordReader, so just iterate through the records
|
||||
try {
|
||||
Record record;
|
||||
while ((record = recordReader.nextRecord()) != null) {
|
||||
write(writeId, record);
|
||||
}
|
||||
} catch (MalformedRecordException | IOException e) {
|
||||
throw new StreamingException(e.getLocalizedMessage(), e);
|
||||
}
|
||||
}
|
||||
|
||||
public Object encode(Record record) throws SerializationError {
|
||||
try {
|
||||
ObjectWritable blob = new ObjectWritable(record);
|
||||
return serde.deserialize(blob);
|
||||
} catch (SerDeException e) {
|
||||
throw new SerializationError("Unable to convert Record into Object", e);
|
||||
}
|
||||
}
|
||||
|
||||
private void write(long writeId, Record record) throws StreamingException {
|
||||
checkAutoFlush();
|
||||
try {
|
||||
Object encodedRow = encode(record);
|
||||
int bucket = getBucket(encodedRow);
|
||||
List<String> partitionValues = getPartitionValues(encodedRow);
|
||||
getRecordUpdater(partitionValues, bucket).insert(writeId, encodedRow);
|
||||
conn.getConnectionStats().incrementRecordsWritten();
|
||||
} catch (IOException e) {
|
||||
throw new StreamingIOFailure("Error writing record in transaction write id (" + writeId + ")", e);
|
||||
}
|
||||
}
|
||||
}
|
|
@ -0,0 +1,282 @@
|
|||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
* contributor license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright ownership.
|
||||
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||
* (the "License"); you may not use this file except in compliance with
|
||||
* the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
package org.apache.hive.streaming;
|
||||
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.hive.conf.HiveConf;
|
||||
import org.apache.hadoop.hive.serde.serdeConstants;
|
||||
import org.apache.hadoop.hive.serde2.AbstractSerDe;
|
||||
import org.apache.hadoop.hive.serde2.SerDeException;
|
||||
import org.apache.hadoop.hive.serde2.SerDeStats;
|
||||
import org.apache.hadoop.hive.serde2.SerDeUtils;
|
||||
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.typeinfo.PrimitiveTypeInfo;
|
||||
import org.apache.hadoop.hive.serde2.typeinfo.StructTypeInfo;
|
||||
import org.apache.hadoop.hive.serde2.typeinfo.TypeInfo;
|
||||
import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoFactory;
|
||||
import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoUtils;
|
||||
import org.apache.hadoop.io.ObjectWritable;
|
||||
import org.apache.hadoop.io.Writable;
|
||||
import org.apache.hive.common.util.HiveStringUtils;
|
||||
import org.apache.hive.common.util.TimestampParser;
|
||||
import org.apache.nifi.avro.AvroTypeUtil;
|
||||
import org.apache.nifi.logging.ComponentLog;
|
||||
import org.apache.nifi.serialization.MalformedRecordException;
|
||||
import org.apache.nifi.serialization.RecordReader;
|
||||
import org.apache.nifi.serialization.record.Record;
|
||||
import org.apache.nifi.serialization.record.RecordField;
|
||||
import org.apache.nifi.serialization.record.RecordSchema;
|
||||
import org.apache.nifi.serialization.record.util.DataTypeUtils;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Arrays;
|
||||
import java.util.Collections;
|
||||
import java.util.HashMap;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Properties;
|
||||
import java.util.regex.Matcher;
|
||||
import java.util.regex.Pattern;
|
||||
|
||||
public class NiFiRecordSerDe extends AbstractSerDe {
|
||||
|
||||
protected RecordReader recordReader;
|
||||
protected ComponentLog log;
|
||||
protected List<String> columnNames;
|
||||
protected StructTypeInfo schema;
|
||||
protected SerDeStats stats;
|
||||
|
||||
protected StandardStructObjectInspector cachedObjectInspector;
|
||||
protected TimestampParser tsParser;
|
||||
|
||||
private final static Pattern INTERNAL_PATTERN = Pattern.compile("_col([0-9]+)");
|
||||
|
||||
private Map<String, Integer> fieldPositionMap;
|
||||
|
||||
public NiFiRecordSerDe(RecordReader recordReader, ComponentLog log) {
|
||||
this.recordReader = recordReader;
|
||||
this.log = log;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void initialize(Configuration conf, Properties tbl) throws SerDeException {
|
||||
List<TypeInfo> columnTypes;
|
||||
StructTypeInfo rowTypeInfo;
|
||||
|
||||
log.debug("Initializing NiFiRecordSerDe: {}", tbl.entrySet().toArray());
|
||||
|
||||
// Get column names and types
|
||||
String columnNameProperty = tbl.getProperty(serdeConstants.LIST_COLUMNS);
|
||||
String columnTypeProperty = tbl.getProperty(serdeConstants.LIST_COLUMN_TYPES);
|
||||
final String columnNameDelimiter = tbl.containsKey(serdeConstants.COLUMN_NAME_DELIMITER) ? tbl
|
||||
.getProperty(serdeConstants.COLUMN_NAME_DELIMITER) : String.valueOf(SerDeUtils.COMMA);
|
||||
// all table column names
|
||||
if (columnNameProperty.isEmpty()) {
|
||||
columnNames = new ArrayList<>(0);
|
||||
} else {
|
||||
columnNames = new ArrayList<>(Arrays.asList(columnNameProperty.split(columnNameDelimiter)));
|
||||
}
|
||||
|
||||
// all column types
|
||||
if (columnTypeProperty.isEmpty()) {
|
||||
columnTypes = new ArrayList<>(0);
|
||||
} else {
|
||||
columnTypes = TypeInfoUtils.getTypeInfosFromTypeString(columnTypeProperty);
|
||||
}
|
||||
|
||||
log.debug("columns: {}, {}", new Object[]{columnNameProperty, columnNames});
|
||||
log.debug("types: {}, {} ", new Object[]{columnTypeProperty, columnTypes});
|
||||
|
||||
assert (columnNames.size() == columnTypes.size());
|
||||
|
||||
rowTypeInfo = (StructTypeInfo) TypeInfoFactory.getStructTypeInfo(columnNames, columnTypes);
|
||||
schema = rowTypeInfo;
|
||||
log.debug("schema : {}", new Object[]{schema});
|
||||
cachedObjectInspector = (StandardStructObjectInspector) TypeInfoUtils.getStandardJavaObjectInspectorFromTypeInfo(rowTypeInfo);
|
||||
tsParser = new TimestampParser(HiveStringUtils.splitAndUnEscape(tbl.getProperty(serdeConstants.TIMESTAMP_FORMATS)));
|
||||
// Populate mapping of field names to column positions
|
||||
try {
|
||||
populateFieldPositionMap();
|
||||
} catch (MalformedRecordException | IOException e) {
|
||||
throw new SerDeException(e);
|
||||
}
|
||||
stats = new SerDeStats();
|
||||
}
|
||||
|
||||
@Override
|
||||
public Class<? extends Writable> getSerializedClass() {
|
||||
return ObjectWritable.class;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Writable serialize(Object o, ObjectInspector objectInspector) throws SerDeException {
|
||||
throw new UnsupportedOperationException("This SerDe only supports deserialization");
|
||||
}
|
||||
|
||||
@Override
|
||||
public SerDeStats getSerDeStats() {
|
||||
return stats;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Object deserialize(Writable writable) throws SerDeException {
|
||||
ObjectWritable t = (ObjectWritable) writable;
|
||||
Record record = (Record) t.get();
|
||||
List<Object> r = new ArrayList<>(Collections.nCopies(columnNames.size(), null));
|
||||
try {
|
||||
RecordSchema recordSchema = record.getSchema();
|
||||
for (RecordField field : recordSchema.getFields()) {
|
||||
String fieldName = field.getFieldName();
|
||||
String normalizedFieldName = fieldName.toLowerCase();
|
||||
|
||||
// Get column position of field name, and set field value there
|
||||
Integer fpos = fieldPositionMap.get(normalizedFieldName);
|
||||
if(fpos == null || fpos == -1) {
|
||||
// This is either a partition column or not a column in the target table, ignore either way
|
||||
continue;
|
||||
}
|
||||
Object currField = extractCurrentField(record, field, schema.getStructFieldTypeInfo(normalizedFieldName));
|
||||
r.set(fpos, currField);
|
||||
}
|
||||
stats.setRowCount(stats.getRowCount() + 1);
|
||||
|
||||
} catch (Exception e) {
|
||||
log.warn("Error [{}] parsing Record [{}].", new Object[]{e.getLocalizedMessage(), t}, e);
|
||||
throw new SerDeException(e);
|
||||
}
|
||||
|
||||
return r;
|
||||
}
|
||||
|
||||
/**
|
||||
* Utility method to extract current expected field from given JsonParser
|
||||
* isTokenCurrent is a boolean variable also passed in, which determines
|
||||
* if the JsonParser is already at the token we expect to read next, or
|
||||
* needs advancing to the next before we read.
|
||||
*/
|
||||
private Object extractCurrentField(Record record, RecordField field, TypeInfo fieldTypeInfo) {
|
||||
Object val;
|
||||
String fieldName = (field != null) ? field.getFieldName() : null;
|
||||
|
||||
switch (fieldTypeInfo.getCategory()) {
|
||||
case PRIMITIVE:
|
||||
PrimitiveObjectInspector.PrimitiveCategory primitiveCategory = PrimitiveObjectInspector.PrimitiveCategory.UNKNOWN;
|
||||
if (fieldTypeInfo instanceof PrimitiveTypeInfo) {
|
||||
primitiveCategory = ((PrimitiveTypeInfo) fieldTypeInfo).getPrimitiveCategory();
|
||||
}
|
||||
switch (primitiveCategory) {
|
||||
case INT:
|
||||
case BYTE:
|
||||
case SHORT:
|
||||
val = record.getAsInt(fieldName);
|
||||
break;
|
||||
case LONG:
|
||||
val = record.getAsLong(fieldName);
|
||||
break;
|
||||
case BOOLEAN:
|
||||
val = record.getAsBoolean(fieldName);
|
||||
break;
|
||||
case FLOAT:
|
||||
val = record.getAsFloat(fieldName);
|
||||
break;
|
||||
case DOUBLE:
|
||||
val = record.getAsDouble(fieldName);
|
||||
break;
|
||||
case STRING:
|
||||
case VARCHAR:
|
||||
case CHAR:
|
||||
val = record.getAsString(fieldName);
|
||||
break;
|
||||
case BINARY:
|
||||
val = AvroTypeUtil.convertByteArray(record.getAsArray(fieldName)).array();
|
||||
break;
|
||||
case DATE:
|
||||
val = record.getAsDate(fieldName, field.getDataType().getFormat());
|
||||
break;
|
||||
case TIMESTAMP:
|
||||
val = DataTypeUtils.toTimestamp(record.getValue(fieldName), () -> DataTypeUtils.getDateFormat(field.getDataType().getFormat()), fieldName);
|
||||
break;
|
||||
case DECIMAL:
|
||||
val = record.getAsDouble(fieldName);
|
||||
break;
|
||||
default:
|
||||
throw new IllegalArgumentException("Field " + fieldName + " cannot be converted to unknown type: " + primitiveCategory.name());
|
||||
}
|
||||
break;
|
||||
case LIST:
|
||||
val = Arrays.asList(record.getAsArray(fieldName));
|
||||
break;
|
||||
case MAP:
|
||||
val = DataTypeUtils.convertRecordFieldtoObject(record.getValue(fieldName), field.getDataType());
|
||||
break;
|
||||
case STRUCT:
|
||||
val = DataTypeUtils.convertRecordFieldtoObject(record.getValue(fieldName), field.getDataType());
|
||||
break;
|
||||
default:
|
||||
log.error("Unknown type found: " + fieldTypeInfo + "for field of type: " + field.getDataType().toString());
|
||||
return null;
|
||||
}
|
||||
return val;
|
||||
}
|
||||
|
||||
@Override
|
||||
public ObjectInspector getObjectInspector() {
|
||||
return cachedObjectInspector;
|
||||
}
|
||||
|
||||
private void populateFieldPositionMap() throws MalformedRecordException, IOException {
|
||||
// Populate the mapping of field names to column positions only once
|
||||
fieldPositionMap = new HashMap<>(columnNames.size());
|
||||
|
||||
RecordSchema recordSchema = recordReader.getSchema();
|
||||
for (RecordField field : recordSchema.getFields()) {
|
||||
String fieldName = field.getFieldName();
|
||||
String normalizedFieldName = fieldName.toLowerCase();
|
||||
|
||||
int fpos = schema.getAllStructFieldNames().indexOf(fieldName.toLowerCase());
|
||||
if (fpos == -1) {
|
||||
Matcher m = INTERNAL_PATTERN.matcher(fieldName);
|
||||
fpos = m.matches() ? Integer.parseInt(m.group(1)) : -1;
|
||||
|
||||
log.debug("NPE finding position for field [{}] in schema [{}],"
|
||||
+ " attempting to check if it is an internal column name like _col0", new Object[]{fieldName, schema});
|
||||
if (fpos == -1) {
|
||||
// unknown field, we return. We'll continue from the next field onwards. Log at debug level because partition columns will be "unknown fields"
|
||||
log.debug("Field {} is not found in the target table, ignoring...", new Object[]{field.getFieldName()});
|
||||
continue;
|
||||
}
|
||||
// If we get past this, then the column name did match the hive pattern for an internal
|
||||
// column name, such as _col0, etc, so it *MUST* match the schema for the appropriate column.
|
||||
// This means people can't use arbitrary column names such as _col0, and expect us to ignore it
|
||||
// if we find it.
|
||||
if (!fieldName.equalsIgnoreCase(HiveConf.getColumnInternalName(fpos))) {
|
||||
log.error("Hive internal column name {} and position "
|
||||
+ "encoding {} for the column name are at odds", new Object[]{fieldName, fpos});
|
||||
throw new IOException("Hive internal column name (" + fieldName
|
||||
+ ") and position encoding (" + fpos
|
||||
+ ") for the column name are at odds");
|
||||
}
|
||||
// If we reached here, then we were successful at finding an alternate internal
|
||||
// column mapping, and we're about to proceed.
|
||||
}
|
||||
fieldPositionMap.put(normalizedFieldName, fpos);
|
||||
}
|
||||
}
|
||||
}
|
|
@ -0,0 +1,385 @@
|
|||
/*
|
||||
* 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.nifi.dbcp.hive;
|
||||
|
||||
import java.io.File;
|
||||
|
||||
import org.apache.commons.dbcp.BasicDataSource;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.security.UserGroupInformation;
|
||||
import org.apache.hive.jdbc.HiveDriver;
|
||||
import org.apache.nifi.annotation.behavior.RequiresInstanceClassLoading;
|
||||
import org.apache.nifi.annotation.documentation.CapabilityDescription;
|
||||
import org.apache.nifi.annotation.documentation.Tags;
|
||||
import org.apache.nifi.annotation.lifecycle.OnDisabled;
|
||||
import org.apache.nifi.annotation.lifecycle.OnEnabled;
|
||||
import org.apache.nifi.components.PropertyDescriptor;
|
||||
import org.apache.nifi.components.ValidationContext;
|
||||
import org.apache.nifi.components.ValidationResult;
|
||||
import org.apache.nifi.controller.AbstractControllerService;
|
||||
import org.apache.nifi.controller.ConfigurationContext;
|
||||
import org.apache.nifi.expression.ExpressionLanguageScope;
|
||||
import org.apache.nifi.hadoop.KerberosProperties;
|
||||
import org.apache.nifi.hadoop.SecurityUtil;
|
||||
import org.apache.nifi.kerberos.KerberosCredentialsService;
|
||||
import org.apache.nifi.logging.ComponentLog;
|
||||
import org.apache.nifi.processor.exception.ProcessException;
|
||||
import org.apache.nifi.processor.util.StandardValidators;
|
||||
import org.apache.nifi.reporting.InitializationException;
|
||||
import org.apache.nifi.util.hive.AuthenticationFailedException;
|
||||
import org.apache.nifi.util.hive.HiveConfigurator;
|
||||
import org.apache.nifi.util.hive.HiveUtils;
|
||||
import org.apache.nifi.util.hive.ValidationResources;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.lang.reflect.UndeclaredThrowableException;
|
||||
import java.security.PrivilegedExceptionAction;
|
||||
import java.sql.Connection;
|
||||
import java.sql.SQLException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Collection;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
import java.util.concurrent.atomic.AtomicReference;
|
||||
|
||||
import org.apache.nifi.controller.ControllerServiceInitializationContext;
|
||||
|
||||
/**
|
||||
* Implementation for Database Connection Pooling Service used for Apache Hive
|
||||
* connections. Apache DBCP is used for connection pooling functionality.
|
||||
*/
|
||||
@RequiresInstanceClassLoading
|
||||
@Tags({"hive", "dbcp", "jdbc", "database", "connection", "pooling", "store"})
|
||||
@CapabilityDescription("Provides Database Connection Pooling Service for Apache Hive 3.x. Connections can be asked from pool and returned after usage.")
|
||||
public class Hive3ConnectionPool extends AbstractControllerService implements Hive3DBCPService {
|
||||
private static final String ALLOW_EXPLICIT_KEYTAB = "NIFI_ALLOW_EXPLICIT_KEYTAB";
|
||||
|
||||
static final PropertyDescriptor DATABASE_URL = new PropertyDescriptor.Builder()
|
||||
.name("hive-db-connect-url")
|
||||
.displayName("Database Connection URL")
|
||||
.description("A database connection URL used to connect to a database. May contain database system name, host, port, database name and some parameters."
|
||||
+ " The exact syntax of a database connection URL is specified by the Hive documentation. For example, the server principal is often included "
|
||||
+ "as a connection parameter when connecting to a secure Hive server.")
|
||||
.defaultValue(null)
|
||||
.addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
|
||||
.required(true)
|
||||
.expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
|
||||
.build();
|
||||
|
||||
static final PropertyDescriptor HIVE_CONFIGURATION_RESOURCES = new PropertyDescriptor.Builder()
|
||||
.name("hive-config-resources")
|
||||
.displayName("Hive Configuration Resources")
|
||||
.description("A file or comma separated list of files which contains the Hive configuration (hive-site.xml, e.g.). Without this, Hadoop "
|
||||
+ "will search the classpath for a 'hive-site.xml' file or will revert to a default configuration. Note that to enable authentication "
|
||||
+ "with Kerberos e.g., the appropriate properties must be set in the configuration files. Please see the Hive documentation for more details.")
|
||||
.required(false)
|
||||
.addValidator(HiveUtils.createMultipleFilesExistValidator())
|
||||
.expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
|
||||
.build();
|
||||
|
||||
static final PropertyDescriptor DB_USER = new PropertyDescriptor.Builder()
|
||||
.name("hive-db-user")
|
||||
.displayName("Database User")
|
||||
.description("Database user name")
|
||||
.defaultValue(null)
|
||||
.addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
|
||||
.expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
|
||||
.build();
|
||||
|
||||
static final PropertyDescriptor DB_PASSWORD = new PropertyDescriptor.Builder()
|
||||
.name("hive-db-password")
|
||||
.displayName("Password")
|
||||
.description("The password for the database user")
|
||||
.defaultValue(null)
|
||||
.required(false)
|
||||
.sensitive(true)
|
||||
.addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
|
||||
.expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
|
||||
.build();
|
||||
|
||||
static final PropertyDescriptor MAX_WAIT_TIME = new PropertyDescriptor.Builder()
|
||||
.name("hive-max-wait-time")
|
||||
.displayName("Max Wait Time")
|
||||
.description("The maximum amount of time that the pool will wait (when there are no available connections) "
|
||||
+ " for a connection to be returned before failing, or -1 to wait indefinitely. ")
|
||||
.defaultValue("500 millis")
|
||||
.required(true)
|
||||
.addValidator(StandardValidators.TIME_PERIOD_VALIDATOR)
|
||||
.expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
|
||||
.build();
|
||||
|
||||
static final PropertyDescriptor MAX_TOTAL_CONNECTIONS = new PropertyDescriptor.Builder()
|
||||
.name("hive-max-total-connections")
|
||||
.displayName("Max Total Connections")
|
||||
.description("The maximum number of active connections that can be allocated from this pool at the same time, "
|
||||
+ "or negative for no limit.")
|
||||
.defaultValue("8")
|
||||
.required(true)
|
||||
.addValidator(StandardValidators.INTEGER_VALIDATOR)
|
||||
.expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
|
||||
.build();
|
||||
|
||||
static final PropertyDescriptor VALIDATION_QUERY = new PropertyDescriptor.Builder()
|
||||
.name("Validation-query")
|
||||
.displayName("Validation query")
|
||||
.description("Validation query used to validate connections before returning them. "
|
||||
+ "When a borrowed connection is invalid, it gets dropped and a new valid connection will be returned. "
|
||||
+ "NOTE: Using validation may have a performance penalty.")
|
||||
.required(false)
|
||||
.addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
|
||||
.expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
|
||||
.build();
|
||||
|
||||
private static final PropertyDescriptor KERBEROS_CREDENTIALS_SERVICE = new PropertyDescriptor.Builder()
|
||||
.name("kerberos-credentials-service")
|
||||
.displayName("Kerberos Credentials Service")
|
||||
.description("Specifies the Kerberos Credentials Controller Service that should be used for authenticating with Kerberos")
|
||||
.identifiesControllerService(KerberosCredentialsService.class)
|
||||
.required(false)
|
||||
.build();
|
||||
|
||||
|
||||
private List<PropertyDescriptor> properties;
|
||||
|
||||
private String connectionUrl = "unknown";
|
||||
|
||||
// Holder of cached Configuration information so validation does not reload the same config over and over
|
||||
private final AtomicReference<ValidationResources> validationResourceHolder = new AtomicReference<>();
|
||||
|
||||
private volatile BasicDataSource dataSource;
|
||||
|
||||
private volatile HiveConfigurator hiveConfigurator = new HiveConfigurator();
|
||||
private volatile UserGroupInformation ugi;
|
||||
private volatile File kerberosConfigFile = null;
|
||||
private volatile KerberosProperties kerberosProperties;
|
||||
|
||||
@Override
|
||||
protected void init(final ControllerServiceInitializationContext context) {
|
||||
List<PropertyDescriptor> props = new ArrayList<>();
|
||||
props.add(DATABASE_URL);
|
||||
props.add(HIVE_CONFIGURATION_RESOURCES);
|
||||
props.add(DB_USER);
|
||||
props.add(DB_PASSWORD);
|
||||
props.add(MAX_WAIT_TIME);
|
||||
props.add(MAX_TOTAL_CONNECTIONS);
|
||||
props.add(VALIDATION_QUERY);
|
||||
props.add(KERBEROS_CREDENTIALS_SERVICE);
|
||||
|
||||
kerberosConfigFile = context.getKerberosConfigurationFile();
|
||||
kerberosProperties = new KerberosProperties(kerberosConfigFile);
|
||||
props.add(kerberosProperties.getKerberosPrincipal());
|
||||
props.add(kerberosProperties.getKerberosKeytab());
|
||||
properties = props;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
|
||||
return properties;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected Collection<ValidationResult> customValidate(ValidationContext validationContext) {
|
||||
boolean confFileProvided = validationContext.getProperty(HIVE_CONFIGURATION_RESOURCES).isSet();
|
||||
|
||||
final List<ValidationResult> problems = new ArrayList<>();
|
||||
|
||||
if (confFileProvided) {
|
||||
final String explicitPrincipal = validationContext.getProperty(kerberosProperties.getKerberosPrincipal()).evaluateAttributeExpressions().getValue();
|
||||
final String explicitKeytab = validationContext.getProperty(kerberosProperties.getKerberosKeytab()).evaluateAttributeExpressions().getValue();
|
||||
final KerberosCredentialsService credentialsService = validationContext.getProperty(KERBEROS_CREDENTIALS_SERVICE).asControllerService(KerberosCredentialsService.class);
|
||||
|
||||
final String resolvedPrincipal;
|
||||
final String resolvedKeytab;
|
||||
if (credentialsService == null) {
|
||||
resolvedPrincipal = explicitPrincipal;
|
||||
resolvedKeytab = explicitKeytab;
|
||||
} else {
|
||||
resolvedPrincipal = credentialsService.getPrincipal();
|
||||
resolvedKeytab = credentialsService.getKeytab();
|
||||
}
|
||||
|
||||
|
||||
final String configFiles = validationContext.getProperty(HIVE_CONFIGURATION_RESOURCES).evaluateAttributeExpressions().getValue();
|
||||
problems.addAll(hiveConfigurator.validate(configFiles, resolvedPrincipal, resolvedKeytab, validationResourceHolder, getLogger()));
|
||||
|
||||
if (credentialsService != null && (explicitPrincipal != null || explicitKeytab != null)) {
|
||||
problems.add(new ValidationResult.Builder()
|
||||
.subject("Kerberos Credentials")
|
||||
.valid(false)
|
||||
.explanation("Cannot specify both a Kerberos Credentials Service and a principal/keytab")
|
||||
.build());
|
||||
}
|
||||
|
||||
final String allowExplicitKeytabVariable = System.getenv(ALLOW_EXPLICIT_KEYTAB);
|
||||
if ("false".equalsIgnoreCase(allowExplicitKeytabVariable) && (explicitPrincipal != null || explicitKeytab != null)) {
|
||||
problems.add(new ValidationResult.Builder()
|
||||
.subject("Kerberos Credentials")
|
||||
.valid(false)
|
||||
.explanation("The '" + ALLOW_EXPLICIT_KEYTAB + "' system environment variable is configured to forbid explicitly configuring principal/keytab in processors. "
|
||||
+ "The Kerberos Credentials Service should be used instead of setting the Kerberos Keytab or Kerberos Principal property.")
|
||||
.build());
|
||||
}
|
||||
}
|
||||
|
||||
return problems;
|
||||
}
|
||||
|
||||
/**
|
||||
* Configures connection pool by creating an instance of the
|
||||
* {@link BasicDataSource} based on configuration provided with
|
||||
* {@link ConfigurationContext}.
|
||||
* <p>
|
||||
* This operation makes no guarantees that the actual connection could be
|
||||
* made since the underlying system may still go off-line during normal
|
||||
* operation of the connection pool.
|
||||
* <p/>
|
||||
* As of Apache NiFi 1.5.0, due to changes made to
|
||||
* {@link SecurityUtil#loginKerberos(Configuration, String, String)}, which is used by this class invoking
|
||||
* {@link HiveConfigurator#authenticate(Configuration, String, String)}
|
||||
* to authenticate a principal with Kerberos, Hive controller services no longer
|
||||
* attempt relogins explicitly. For more information, please read the documentation for
|
||||
* {@link SecurityUtil#loginKerberos(Configuration, String, String)}.
|
||||
* <p/>
|
||||
* In previous versions of NiFi, a {@link org.apache.nifi.hadoop.KerberosTicketRenewer} was started by
|
||||
* {@link HiveConfigurator#authenticate(Configuration, String, String, long)} when the Hive
|
||||
* controller service was enabled. The use of a separate thread to explicitly relogin could cause race conditions
|
||||
* with the implicit relogin attempts made by hadoop/Hive code on a thread that references the same
|
||||
* {@link UserGroupInformation} instance. One of these threads could leave the
|
||||
* {@link javax.security.auth.Subject} in {@link UserGroupInformation} to be cleared or in an unexpected state
|
||||
* while the other thread is attempting to use the {@link javax.security.auth.Subject}, resulting in failed
|
||||
* authentication attempts that would leave the Hive controller service in an unrecoverable state.
|
||||
*
|
||||
* @see SecurityUtil#loginKerberos(Configuration, String, String)
|
||||
* @see HiveConfigurator#authenticate(Configuration, String, String)
|
||||
* @see HiveConfigurator#authenticate(Configuration, String, String, long)
|
||||
* @param context the configuration context
|
||||
* @throws InitializationException if unable to create a database connection
|
||||
*/
|
||||
@OnEnabled
|
||||
public void onConfigured(final ConfigurationContext context) throws InitializationException {
|
||||
|
||||
ComponentLog log = getLogger();
|
||||
|
||||
final String configFiles = context.getProperty(HIVE_CONFIGURATION_RESOURCES).evaluateAttributeExpressions().getValue();
|
||||
final Configuration hiveConfig = hiveConfigurator.getConfigurationFromFiles(configFiles);
|
||||
final String validationQuery = context.getProperty(VALIDATION_QUERY).evaluateAttributeExpressions().getValue();
|
||||
|
||||
// add any dynamic properties to the Hive configuration
|
||||
for (final Map.Entry<PropertyDescriptor, String> entry : context.getProperties().entrySet()) {
|
||||
final PropertyDescriptor descriptor = entry.getKey();
|
||||
if (descriptor.isDynamic()) {
|
||||
hiveConfig.set(descriptor.getName(), context.getProperty(descriptor).evaluateAttributeExpressions().getValue());
|
||||
}
|
||||
}
|
||||
|
||||
final String drv = HiveDriver.class.getName();
|
||||
if (SecurityUtil.isSecurityEnabled(hiveConfig)) {
|
||||
final String explicitPrincipal = context.getProperty(kerberosProperties.getKerberosPrincipal()).evaluateAttributeExpressions().getValue();
|
||||
final String explicitKeytab = context.getProperty(kerberosProperties.getKerberosKeytab()).evaluateAttributeExpressions().getValue();
|
||||
final KerberosCredentialsService credentialsService = context.getProperty(KERBEROS_CREDENTIALS_SERVICE).asControllerService(KerberosCredentialsService.class);
|
||||
|
||||
final String resolvedPrincipal;
|
||||
final String resolvedKeytab;
|
||||
if (credentialsService == null) {
|
||||
resolvedPrincipal = explicitPrincipal;
|
||||
resolvedKeytab = explicitKeytab;
|
||||
} else {
|
||||
resolvedPrincipal = credentialsService.getPrincipal();
|
||||
resolvedKeytab = credentialsService.getKeytab();
|
||||
}
|
||||
|
||||
log.info("Hive Security Enabled, logging in as principal {} with keytab {}", new Object[] {resolvedPrincipal, resolvedKeytab});
|
||||
try {
|
||||
ugi = hiveConfigurator.authenticate(hiveConfig, resolvedPrincipal, resolvedKeytab);
|
||||
} catch (AuthenticationFailedException ae) {
|
||||
log.error(ae.getMessage(), ae);
|
||||
}
|
||||
|
||||
getLogger().info("Successfully logged in as principal {} with keytab {}", new Object[] {resolvedPrincipal, resolvedKeytab});
|
||||
}
|
||||
|
||||
final String user = context.getProperty(DB_USER).evaluateAttributeExpressions().getValue();
|
||||
final String passw = context.getProperty(DB_PASSWORD).evaluateAttributeExpressions().getValue();
|
||||
final Long maxWaitMillis = context.getProperty(MAX_WAIT_TIME).evaluateAttributeExpressions().asTimePeriod(TimeUnit.MILLISECONDS);
|
||||
final Integer maxTotal = context.getProperty(MAX_TOTAL_CONNECTIONS).evaluateAttributeExpressions().asInteger();
|
||||
|
||||
dataSource = new BasicDataSource();
|
||||
dataSource.setDriverClassName(drv);
|
||||
|
||||
connectionUrl = context.getProperty(DATABASE_URL).evaluateAttributeExpressions().getValue();
|
||||
|
||||
dataSource.setMaxWait(maxWaitMillis);
|
||||
dataSource.setMaxActive(maxTotal);
|
||||
|
||||
if (validationQuery != null && !validationQuery.isEmpty()) {
|
||||
dataSource.setValidationQuery(validationQuery);
|
||||
dataSource.setTestOnBorrow(true);
|
||||
}
|
||||
|
||||
dataSource.setUrl(connectionUrl);
|
||||
dataSource.setUsername(user);
|
||||
dataSource.setPassword(passw);
|
||||
}
|
||||
|
||||
/**
|
||||
* Shutdown pool, close all open connections.
|
||||
*/
|
||||
@OnDisabled
|
||||
public void shutdown() {
|
||||
try {
|
||||
dataSource.close();
|
||||
} catch (final SQLException e) {
|
||||
throw new ProcessException(e);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public Connection getConnection() throws ProcessException {
|
||||
try {
|
||||
if (ugi != null) {
|
||||
try {
|
||||
return ugi.doAs((PrivilegedExceptionAction<Connection>) () -> dataSource.getConnection());
|
||||
} catch (UndeclaredThrowableException e) {
|
||||
Throwable cause = e.getCause();
|
||||
if (cause instanceof SQLException) {
|
||||
throw (SQLException) cause;
|
||||
} else {
|
||||
throw e;
|
||||
}
|
||||
}
|
||||
} else {
|
||||
getLogger().info("Simple Authentication");
|
||||
return dataSource.getConnection();
|
||||
}
|
||||
} catch (SQLException | IOException | InterruptedException e) {
|
||||
getLogger().error("Error getting Hive connection", e);
|
||||
throw new ProcessException(e);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
return "Hive3ConnectionPool[id=" + getIdentifier() + "]";
|
||||
}
|
||||
|
||||
@Override
|
||||
public String getConnectionURL() {
|
||||
return connectionUrl;
|
||||
}
|
||||
|
||||
}
|
|
@ -0,0 +1,348 @@
|
|||
/*
|
||||
* 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.nifi.processors.hive;
|
||||
|
||||
import org.antlr.runtime.tree.CommonTree;
|
||||
import org.apache.hadoop.hive.ql.parse.ASTNode;
|
||||
import org.apache.hadoop.hive.ql.parse.ParseDriver;
|
||||
import org.apache.hadoop.hive.ql.parse.ParseException;
|
||||
import org.apache.nifi.components.PropertyDescriptor;
|
||||
import org.apache.nifi.dbcp.hive.Hive3DBCPService;
|
||||
import org.apache.nifi.expression.ExpressionLanguageScope;
|
||||
import org.apache.nifi.flowfile.FlowFile;
|
||||
import org.apache.nifi.processor.AbstractSessionFactoryProcessor;
|
||||
import org.apache.nifi.processor.ProcessSession;
|
||||
import org.apache.nifi.processor.util.StandardValidators;
|
||||
import org.apache.nifi.stream.io.StreamUtils;
|
||||
|
||||
import java.math.BigDecimal;
|
||||
import java.nio.charset.Charset;
|
||||
import java.sql.Date;
|
||||
import java.sql.PreparedStatement;
|
||||
import java.sql.SQLDataException;
|
||||
import java.sql.SQLException;
|
||||
import java.sql.Time;
|
||||
import java.sql.Timestamp;
|
||||
import java.sql.Types;
|
||||
import java.util.Collections;
|
||||
import java.util.HashMap;
|
||||
import java.util.HashSet;
|
||||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
import java.util.TreeMap;
|
||||
import java.util.regex.Matcher;
|
||||
import java.util.regex.Pattern;
|
||||
|
||||
/**
|
||||
* An abstract base class for HiveQL processors to share common data, methods, etc.
|
||||
*/
|
||||
public abstract class AbstractHive3QLProcessor extends AbstractSessionFactoryProcessor {
|
||||
|
||||
protected static final Pattern HIVEQL_TYPE_ATTRIBUTE_PATTERN = Pattern.compile("hiveql\\.args\\.(\\d+)\\.type");
|
||||
protected static final Pattern NUMBER_PATTERN = Pattern.compile("-?\\d+");
|
||||
static String ATTR_INPUT_TABLES = "query.input.tables";
|
||||
static String ATTR_OUTPUT_TABLES = "query.output.tables";
|
||||
|
||||
|
||||
public static final PropertyDescriptor HIVE_DBCP_SERVICE = new PropertyDescriptor.Builder()
|
||||
.name("hive3-dbcp-service")
|
||||
.displayName("Hive Database Connection Pooling Service")
|
||||
.description("The Hive Controller Service that is used to obtain connection(s) to the Hive database")
|
||||
.required(true)
|
||||
.identifiesControllerService(Hive3DBCPService.class)
|
||||
.build();
|
||||
|
||||
public static final PropertyDescriptor CHARSET = new PropertyDescriptor.Builder()
|
||||
.name("hive3-charset")
|
||||
.displayName("Character Set")
|
||||
.description("Specifies the character set of the record data.")
|
||||
.required(true)
|
||||
.defaultValue("UTF-8")
|
||||
.addValidator(StandardValidators.CHARACTER_SET_VALIDATOR)
|
||||
.build();
|
||||
|
||||
public static final PropertyDescriptor QUERY_TIMEOUT = new PropertyDescriptor.Builder()
|
||||
.name("hive3-query-timeout")
|
||||
.displayName("Query timeout")
|
||||
.description("Sets the number of seconds the driver will wait for a query to execute. "
|
||||
+ "A value of 0 means no timeout. NOTE: Non-zero values may not be supported by the driver.")
|
||||
.defaultValue("0")
|
||||
.required(true)
|
||||
.addValidator(StandardValidators.INTEGER_VALIDATOR)
|
||||
.expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
|
||||
.build();
|
||||
|
||||
/**
|
||||
* Determines the HiveQL statement that should be executed for the given FlowFile
|
||||
*
|
||||
* @param session the session that can be used to access the given FlowFile
|
||||
* @param flowFile the FlowFile whose HiveQL statement should be executed
|
||||
* @return the HiveQL that is associated with the given FlowFile
|
||||
*/
|
||||
protected String getHiveQL(final ProcessSession session, final FlowFile flowFile, final Charset charset) {
|
||||
// Read the HiveQL from the FlowFile's content
|
||||
final byte[] buffer = new byte[(int) flowFile.getSize()];
|
||||
session.read(flowFile, in -> StreamUtils.fillBuffer(in, buffer));
|
||||
|
||||
// Create the PreparedStatement to use for this FlowFile.
|
||||
return new String(buffer, charset);
|
||||
}
|
||||
|
||||
private class ParameterHolder {
|
||||
String attributeName;
|
||||
int jdbcType;
|
||||
String value;
|
||||
}
|
||||
|
||||
/**
|
||||
* Sets all of the appropriate parameters on the given PreparedStatement, based on the given FlowFile attributes.
|
||||
*
|
||||
* @param stmt the statement to set the parameters on
|
||||
* @param attributes the attributes from which to derive parameter indices, values, and types
|
||||
* @throws SQLException if the PreparedStatement throws a SQLException when the appropriate setter is called
|
||||
*/
|
||||
protected int setParameters(int base, final PreparedStatement stmt, int paramCount, final Map<String, String> attributes) throws SQLException {
|
||||
|
||||
Map<Integer, ParameterHolder> parmMap = new TreeMap<Integer, ParameterHolder>();
|
||||
|
||||
for (final Map.Entry<String, String> entry : attributes.entrySet()) {
|
||||
final String key = entry.getKey();
|
||||
final Matcher matcher = HIVEQL_TYPE_ATTRIBUTE_PATTERN.matcher(key);
|
||||
if (matcher.matches()) {
|
||||
final int parameterIndex = Integer.parseInt(matcher.group(1));
|
||||
if (parameterIndex >= base && parameterIndex < base + paramCount) {
|
||||
final boolean isNumeric = NUMBER_PATTERN.matcher(entry.getValue()).matches();
|
||||
if (!isNumeric) {
|
||||
throw new SQLDataException("Value of the " + key + " attribute is '" + entry.getValue() + "', which is not a valid JDBC numeral jdbcType");
|
||||
}
|
||||
|
||||
final String valueAttrName = "hiveql.args." + parameterIndex + ".value";
|
||||
|
||||
ParameterHolder ph = new ParameterHolder();
|
||||
int realIndexLoc = parameterIndex - base +1;
|
||||
|
||||
ph.jdbcType = Integer.parseInt(entry.getValue());
|
||||
ph.value = attributes.get(valueAttrName);
|
||||
ph.attributeName = valueAttrName;
|
||||
|
||||
parmMap.put(realIndexLoc, ph);
|
||||
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
// Now that's we've retrieved the correct number of parameters and it's sorted, let's set them.
|
||||
for (final Map.Entry<Integer, ParameterHolder> entry : parmMap.entrySet()) {
|
||||
final Integer index = entry.getKey();
|
||||
final ParameterHolder ph = entry.getValue();
|
||||
|
||||
try {
|
||||
setParameter(stmt, ph.attributeName, index, ph.value, ph.jdbcType);
|
||||
} catch (final NumberFormatException nfe) {
|
||||
throw new SQLDataException("The value of the " + ph.attributeName + " is '" + ph.value + "', which cannot be converted into the necessary data jdbcType", nfe);
|
||||
}
|
||||
}
|
||||
return base + paramCount;
|
||||
}
|
||||
|
||||
/**
|
||||
* Determines how to map the given value to the appropriate JDBC data jdbcType and sets the parameter on the
|
||||
* provided PreparedStatement
|
||||
*
|
||||
* @param stmt the PreparedStatement to set the parameter on
|
||||
* @param attrName the name of the attribute that the parameter is coming from - for logging purposes
|
||||
* @param parameterIndex the index of the HiveQL parameter to set
|
||||
* @param parameterValue the value of the HiveQL parameter to set
|
||||
* @param jdbcType the JDBC Type of the HiveQL parameter to set
|
||||
* @throws SQLException if the PreparedStatement throws a SQLException when calling the appropriate setter
|
||||
*/
|
||||
protected void setParameter(final PreparedStatement stmt, final String attrName, final int parameterIndex, final String parameterValue, final int jdbcType) throws SQLException {
|
||||
if (parameterValue == null) {
|
||||
stmt.setNull(parameterIndex, jdbcType);
|
||||
} else {
|
||||
try {
|
||||
switch (jdbcType) {
|
||||
case Types.BIT:
|
||||
case Types.BOOLEAN:
|
||||
stmt.setBoolean(parameterIndex, Boolean.parseBoolean(parameterValue));
|
||||
break;
|
||||
case Types.TINYINT:
|
||||
stmt.setByte(parameterIndex, Byte.parseByte(parameterValue));
|
||||
break;
|
||||
case Types.SMALLINT:
|
||||
stmt.setShort(parameterIndex, Short.parseShort(parameterValue));
|
||||
break;
|
||||
case Types.INTEGER:
|
||||
stmt.setInt(parameterIndex, Integer.parseInt(parameterValue));
|
||||
break;
|
||||
case Types.BIGINT:
|
||||
stmt.setLong(parameterIndex, Long.parseLong(parameterValue));
|
||||
break;
|
||||
case Types.REAL:
|
||||
stmt.setFloat(parameterIndex, Float.parseFloat(parameterValue));
|
||||
break;
|
||||
case Types.FLOAT:
|
||||
case Types.DOUBLE:
|
||||
stmt.setDouble(parameterIndex, Double.parseDouble(parameterValue));
|
||||
break;
|
||||
case Types.DECIMAL:
|
||||
case Types.NUMERIC:
|
||||
stmt.setBigDecimal(parameterIndex, new BigDecimal(parameterValue));
|
||||
break;
|
||||
case Types.DATE:
|
||||
stmt.setDate(parameterIndex, new Date(Long.parseLong(parameterValue)));
|
||||
break;
|
||||
case Types.TIME:
|
||||
stmt.setTime(parameterIndex, new Time(Long.parseLong(parameterValue)));
|
||||
break;
|
||||
case Types.TIMESTAMP:
|
||||
stmt.setTimestamp(parameterIndex, new Timestamp(Long.parseLong(parameterValue)));
|
||||
break;
|
||||
case Types.CHAR:
|
||||
case Types.VARCHAR:
|
||||
case Types.LONGNVARCHAR:
|
||||
case Types.LONGVARCHAR:
|
||||
stmt.setString(parameterIndex, parameterValue);
|
||||
break;
|
||||
default:
|
||||
stmt.setObject(parameterIndex, parameterValue, jdbcType);
|
||||
break;
|
||||
}
|
||||
} catch (SQLException e) {
|
||||
// Log which attribute/parameter had an error, then rethrow to be handled at the top level
|
||||
getLogger().error("Error setting parameter {} to value from {} ({})", new Object[]{parameterIndex, attrName, parameterValue}, e);
|
||||
throw e;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
protected static class TableName {
|
||||
private final String database;
|
||||
private final String table;
|
||||
private final boolean input;
|
||||
|
||||
TableName(String database, String table, boolean input) {
|
||||
this.database = database;
|
||||
this.table = table;
|
||||
this.input = input;
|
||||
}
|
||||
|
||||
public String getDatabase() {
|
||||
return database;
|
||||
}
|
||||
|
||||
public String getTable() {
|
||||
return table;
|
||||
}
|
||||
|
||||
public boolean isInput() {
|
||||
return input;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
return database == null || database.isEmpty() ? table : database + '.' + table;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean equals(Object o) {
|
||||
if (this == o) return true;
|
||||
if (o == null || getClass() != o.getClass()) return false;
|
||||
|
||||
TableName tableName = (TableName) o;
|
||||
|
||||
if (input != tableName.input) return false;
|
||||
if (database != null ? !database.equals(tableName.database) : tableName.database != null) return false;
|
||||
return table.equals(tableName.table);
|
||||
}
|
||||
|
||||
@Override
|
||||
public int hashCode() {
|
||||
int result = database != null ? database.hashCode() : 0;
|
||||
result = 31 * result + table.hashCode();
|
||||
result = 31 * result + (input ? 1 : 0);
|
||||
return result;
|
||||
}
|
||||
}
|
||||
|
||||
protected Set<TableName> findTableNames(final String query) {
|
||||
final ASTNode node;
|
||||
try {
|
||||
node = new ParseDriver().parse(normalize(query));
|
||||
} catch (ParseException e) {
|
||||
// If failed to parse the query, just log a message, but continue.
|
||||
getLogger().debug("Failed to parse query: {} due to {}", new Object[]{query, e}, e);
|
||||
return Collections.emptySet();
|
||||
}
|
||||
final HashSet<TableName> tableNames = new HashSet<>();
|
||||
findTableNames(node, tableNames);
|
||||
return tableNames;
|
||||
}
|
||||
|
||||
/**
|
||||
* Normalize query.
|
||||
* Hive resolves prepared statement parameters before executing a query,
|
||||
* see {@link org.apache.hive.jdbc.HivePreparedStatement#updateSql(String, HashMap)} for detail.
|
||||
* HiveParser does not expect '?' to be in a query string, and throws an Exception if there is one.
|
||||
* In this normalize method, '?' is replaced to 'x' to avoid that.
|
||||
*/
|
||||
private String normalize(String query) {
|
||||
return query.replace('?', 'x');
|
||||
}
|
||||
|
||||
private void findTableNames(final Object obj, final Set<TableName> tableNames) {
|
||||
if (!(obj instanceof CommonTree)) {
|
||||
return;
|
||||
}
|
||||
final CommonTree tree = (CommonTree) obj;
|
||||
final int childCount = tree.getChildCount();
|
||||
if ("TOK_TABNAME".equals(tree.getText())) {
|
||||
final TableName tableName;
|
||||
final boolean isInput = "TOK_TABREF".equals(tree.getParent().getText());
|
||||
switch (childCount) {
|
||||
case 1 :
|
||||
tableName = new TableName(null, tree.getChild(0).getText(), isInput);
|
||||
break;
|
||||
case 2:
|
||||
tableName = new TableName(tree.getChild(0).getText(), tree.getChild(1).getText(), isInput);
|
||||
break;
|
||||
default:
|
||||
throw new IllegalStateException("TOK_TABNAME does not have expected children, childCount=" + childCount);
|
||||
}
|
||||
// If parent is TOK_TABREF, then it is an input table.
|
||||
tableNames.add(tableName);
|
||||
return;
|
||||
}
|
||||
for (int i = 0; i < childCount; i++) {
|
||||
findTableNames(tree.getChild(i), tableNames);
|
||||
}
|
||||
}
|
||||
|
||||
protected Map<String, String> toQueryTableAttributes(Set<TableName> tableNames) {
|
||||
final Map<String, String> attributes = new HashMap<>();
|
||||
for (TableName tableName : tableNames) {
|
||||
final String attributeName = tableName.isInput() ? ATTR_INPUT_TABLES : ATTR_OUTPUT_TABLES;
|
||||
if (attributes.containsKey(attributeName)) {
|
||||
attributes.put(attributeName, attributes.get(attributeName) + "," + tableName);
|
||||
} else {
|
||||
attributes.put(attributeName, tableName.toString());
|
||||
}
|
||||
}
|
||||
return attributes;
|
||||
}
|
||||
}
|
|
@ -0,0 +1,280 @@
|
|||
/*
|
||||
* 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.nifi.processors.hive;
|
||||
|
||||
import org.apache.commons.lang3.StringUtils;
|
||||
import org.apache.nifi.annotation.behavior.InputRequirement;
|
||||
import org.apache.nifi.annotation.behavior.InputRequirement.Requirement;
|
||||
import org.apache.nifi.annotation.behavior.ReadsAttribute;
|
||||
import org.apache.nifi.annotation.behavior.ReadsAttributes;
|
||||
import org.apache.nifi.annotation.behavior.WritesAttribute;
|
||||
import org.apache.nifi.annotation.behavior.WritesAttributes;
|
||||
import org.apache.nifi.annotation.documentation.CapabilityDescription;
|
||||
import org.apache.nifi.annotation.documentation.SeeAlso;
|
||||
import org.apache.nifi.annotation.documentation.Tags;
|
||||
import org.apache.nifi.annotation.lifecycle.OnScheduled;
|
||||
import org.apache.nifi.components.PropertyDescriptor;
|
||||
import org.apache.nifi.dbcp.hive.Hive3DBCPService;
|
||||
import org.apache.nifi.expression.ExpressionLanguageScope;
|
||||
import org.apache.nifi.flowfile.FlowFile;
|
||||
import org.apache.nifi.processor.ProcessContext;
|
||||
import org.apache.nifi.processor.ProcessSession;
|
||||
import org.apache.nifi.processor.ProcessSessionFactory;
|
||||
import org.apache.nifi.processor.Relationship;
|
||||
import org.apache.nifi.processor.exception.ProcessException;
|
||||
import org.apache.nifi.processor.util.StandardValidators;
|
||||
import org.apache.nifi.processor.util.pattern.ErrorTypes;
|
||||
import org.apache.nifi.processor.util.pattern.ExceptionHandler;
|
||||
import org.apache.nifi.processor.util.pattern.ExceptionHandler.OnError;
|
||||
import org.apache.nifi.processor.util.pattern.PartialFunctions.FetchFlowFiles;
|
||||
import org.apache.nifi.processor.util.pattern.PartialFunctions.InitConnection;
|
||||
import org.apache.nifi.processor.util.pattern.Put;
|
||||
import org.apache.nifi.processor.util.pattern.RollbackOnFailure;
|
||||
import org.apache.nifi.processor.util.pattern.RoutingResult;
|
||||
|
||||
import java.nio.charset.Charset;
|
||||
import java.sql.Connection;
|
||||
import java.sql.PreparedStatement;
|
||||
import java.sql.SQLException;
|
||||
import java.sql.SQLNonTransientException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Collections;
|
||||
import java.util.HashSet;
|
||||
import java.util.List;
|
||||
import java.util.Set;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
import java.util.regex.Pattern;
|
||||
|
||||
@SeeAlso(SelectHive3QL.class)
|
||||
@InputRequirement(Requirement.INPUT_REQUIRED)
|
||||
@Tags({"sql", "hive", "put", "database", "update", "insert"})
|
||||
@CapabilityDescription("Executes a HiveQL DDL/DML command (UPDATE, INSERT, e.g.). The content of an incoming FlowFile is expected to be the HiveQL command "
|
||||
+ "to execute. The HiveQL command may use the ? to escape parameters. In this case, the parameters to use must exist as FlowFile attributes "
|
||||
+ "with the naming convention hiveql.args.N.type and hiveql.args.N.value, where N is a positive integer. The hiveql.args.N.type is expected to be "
|
||||
+ "a number indicating the JDBC Type. The content of the FlowFile is expected to be in UTF-8 format.")
|
||||
@ReadsAttributes({
|
||||
@ReadsAttribute(attribute = "hiveql.args.N.type", description = "Incoming FlowFiles are expected to be parametrized HiveQL statements. The type of each Parameter is specified as an integer "
|
||||
+ "that represents the JDBC Type of the parameter."),
|
||||
@ReadsAttribute(attribute = "hiveql.args.N.value", description = "Incoming FlowFiles are expected to be parametrized HiveQL statements. The value of the Parameters are specified as "
|
||||
+ "hiveql.args.1.value, hiveql.args.2.value, hiveql.args.3.value, and so on. The type of the hiveql.args.1.value Parameter is specified by the hiveql.args.1.type attribute.")
|
||||
})
|
||||
@WritesAttributes({
|
||||
@WritesAttribute(attribute = "query.input.tables", description = "This attribute is written on the flow files routed to the 'success' relationships, "
|
||||
+ "and contains input table names (if any) in comma delimited 'databaseName.tableName' format."),
|
||||
@WritesAttribute(attribute = "query.output.tables", description = "This attribute is written on the flow files routed to the 'success' relationships, "
|
||||
+ "and contains the target table names in 'databaseName.tableName' format.")
|
||||
})
|
||||
public class PutHive3QL extends AbstractHive3QLProcessor {
|
||||
|
||||
public static final PropertyDescriptor BATCH_SIZE = new PropertyDescriptor.Builder()
|
||||
.name("hive-batch-size")
|
||||
.displayName("Batch Size")
|
||||
.description("The preferred number of FlowFiles to put to the database in a single transaction")
|
||||
.required(true)
|
||||
.addValidator(StandardValidators.POSITIVE_INTEGER_VALIDATOR)
|
||||
.defaultValue("100")
|
||||
.build();
|
||||
|
||||
public static final PropertyDescriptor STATEMENT_DELIMITER = new PropertyDescriptor.Builder()
|
||||
.name("statement-delimiter")
|
||||
.displayName("Statement Delimiter")
|
||||
.description("Statement Delimiter used to separate SQL statements in a multiple statement script")
|
||||
.required(true)
|
||||
.defaultValue(";")
|
||||
.addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
|
||||
.expressionLanguageSupported(ExpressionLanguageScope.NONE)
|
||||
.build();
|
||||
|
||||
public static final Relationship REL_SUCCESS = new Relationship.Builder()
|
||||
.name("success")
|
||||
.description("A FlowFile is routed to this relationship after the database is successfully updated")
|
||||
.build();
|
||||
public static final Relationship REL_RETRY = new Relationship.Builder()
|
||||
.name("retry")
|
||||
.description("A FlowFile is routed to this relationship if the database cannot be updated but attempting the operation again may succeed")
|
||||
.build();
|
||||
public static final Relationship REL_FAILURE = new Relationship.Builder()
|
||||
.name("failure")
|
||||
.description("A FlowFile is routed to this relationship if the database cannot be updated and retrying the operation will also fail, "
|
||||
+ "such as an invalid query or an integrity constraint violation")
|
||||
.build();
|
||||
|
||||
|
||||
private final static List<PropertyDescriptor> propertyDescriptors;
|
||||
private final static Set<Relationship> relationships;
|
||||
|
||||
/*
|
||||
* Will ensure that the list of property descriptors is built only once.
|
||||
* Will also create a Set of relationships
|
||||
*/
|
||||
static {
|
||||
List<PropertyDescriptor> _propertyDescriptors = new ArrayList<>();
|
||||
_propertyDescriptors.add(HIVE_DBCP_SERVICE);
|
||||
_propertyDescriptors.add(BATCH_SIZE);
|
||||
_propertyDescriptors.add(QUERY_TIMEOUT);
|
||||
_propertyDescriptors.add(CHARSET);
|
||||
_propertyDescriptors.add(STATEMENT_DELIMITER);
|
||||
_propertyDescriptors.add(RollbackOnFailure.ROLLBACK_ON_FAILURE);
|
||||
propertyDescriptors = Collections.unmodifiableList(_propertyDescriptors);
|
||||
|
||||
Set<Relationship> _relationships = new HashSet<>();
|
||||
_relationships.add(REL_SUCCESS);
|
||||
_relationships.add(REL_FAILURE);
|
||||
_relationships.add(REL_RETRY);
|
||||
relationships = Collections.unmodifiableSet(_relationships);
|
||||
}
|
||||
|
||||
private Put<FunctionContext, Connection> process;
|
||||
private ExceptionHandler<FunctionContext> exceptionHandler;
|
||||
|
||||
@OnScheduled
|
||||
public void constructProcess() {
|
||||
exceptionHandler = new ExceptionHandler<>();
|
||||
exceptionHandler.mapException(e -> {
|
||||
if (e instanceof SQLNonTransientException) {
|
||||
return ErrorTypes.InvalidInput;
|
||||
} else if (e instanceof SQLException) {
|
||||
return ErrorTypes.TemporalFailure;
|
||||
} else {
|
||||
return ErrorTypes.UnknownFailure;
|
||||
}
|
||||
});
|
||||
exceptionHandler.adjustError(RollbackOnFailure.createAdjustError(getLogger()));
|
||||
|
||||
process = new Put<>();
|
||||
process.setLogger(getLogger());
|
||||
process.initConnection(initConnection);
|
||||
process.fetchFlowFiles(fetchFlowFiles);
|
||||
process.putFlowFile(putFlowFile);
|
||||
process.adjustRoute(RollbackOnFailure.createAdjustRoute(REL_FAILURE, REL_RETRY));
|
||||
}
|
||||
|
||||
@Override
|
||||
protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
|
||||
return propertyDescriptors;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Set<Relationship> getRelationships() {
|
||||
return relationships;
|
||||
}
|
||||
|
||||
private class FunctionContext extends RollbackOnFailure {
|
||||
final Charset charset;
|
||||
final String statementDelimiter;
|
||||
final long startNanos = System.nanoTime();
|
||||
|
||||
String connectionUrl;
|
||||
|
||||
|
||||
private FunctionContext(boolean rollbackOnFailure, Charset charset, String statementDelimiter) {
|
||||
super(rollbackOnFailure, false);
|
||||
this.charset = charset;
|
||||
this.statementDelimiter = statementDelimiter;
|
||||
}
|
||||
}
|
||||
|
||||
private InitConnection<FunctionContext, Connection> initConnection = (context, session, fc, ff) -> {
|
||||
final Hive3DBCPService dbcpService = context.getProperty(HIVE_DBCP_SERVICE).asControllerService(Hive3DBCPService.class);
|
||||
final Connection connection = dbcpService.getConnection();
|
||||
fc.connectionUrl = dbcpService.getConnectionURL();
|
||||
return connection;
|
||||
};
|
||||
|
||||
private FetchFlowFiles<FunctionContext> fetchFlowFiles = (context, session, functionContext, result) -> {
|
||||
final int batchSize = context.getProperty(BATCH_SIZE).asInteger();
|
||||
return session.get(batchSize);
|
||||
};
|
||||
|
||||
private Put.PutFlowFile<FunctionContext, Connection> putFlowFile = (context, session, fc, conn, flowFile, result) -> {
|
||||
final String script = getHiveQL(session, flowFile, fc.charset);
|
||||
String regex = "(?<!\\\\)" + Pattern.quote(fc.statementDelimiter);
|
||||
|
||||
String[] hiveQLs = script.split(regex);
|
||||
|
||||
final Set<TableName> tableNames = new HashSet<>();
|
||||
exceptionHandler.execute(fc, flowFile, input -> {
|
||||
int loc = 1;
|
||||
for (String hiveQLStr: hiveQLs) {
|
||||
getLogger().debug("HiveQL: {}", new Object[]{hiveQLStr});
|
||||
|
||||
final String hiveQL = hiveQLStr.trim();
|
||||
if (!StringUtils.isEmpty(hiveQL)) {
|
||||
final PreparedStatement stmt = conn.prepareStatement(hiveQL);
|
||||
|
||||
// Get ParameterMetadata
|
||||
// Hive JDBC Doesn't support this yet:
|
||||
// ParameterMetaData pmd = stmt.getParameterMetaData();
|
||||
// int paramCount = pmd.getParameterCount();
|
||||
int paramCount = StringUtils.countMatches(hiveQL, "?");
|
||||
|
||||
if (paramCount > 0) {
|
||||
loc = setParameters(loc, stmt, paramCount, flowFile.getAttributes());
|
||||
}
|
||||
|
||||
// Parse hiveQL and extract input/output tables
|
||||
try {
|
||||
tableNames.addAll(findTableNames(hiveQL));
|
||||
} catch (Exception e) {
|
||||
// If failed to parse the query, just log a warning message, but continue.
|
||||
getLogger().warn("Failed to parse hiveQL: {} due to {}", new Object[]{hiveQL, e}, e);
|
||||
}
|
||||
|
||||
stmt.setQueryTimeout(context.getProperty(QUERY_TIMEOUT).evaluateAttributeExpressions(flowFile).asInteger());
|
||||
|
||||
// Execute the statement
|
||||
stmt.execute();
|
||||
fc.proceed();
|
||||
}
|
||||
}
|
||||
|
||||
// Emit a Provenance SEND event
|
||||
final long transmissionMillis = TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - fc.startNanos);
|
||||
|
||||
final FlowFile updatedFlowFile = session.putAllAttributes(flowFile, toQueryTableAttributes(tableNames));
|
||||
session.getProvenanceReporter().send(updatedFlowFile, fc.connectionUrl, transmissionMillis, true);
|
||||
result.routeTo(flowFile, REL_SUCCESS);
|
||||
|
||||
}, onFlowFileError(context, session, result));
|
||||
|
||||
};
|
||||
|
||||
private OnError<FunctionContext, FlowFile> onFlowFileError(final ProcessContext context, final ProcessSession session, final RoutingResult result) {
|
||||
OnError<FunctionContext, FlowFile> onFlowFileError = ExceptionHandler.createOnError(context, session, result, REL_FAILURE, REL_RETRY);
|
||||
onFlowFileError = onFlowFileError.andThen((c, i, r, e) -> {
|
||||
switch (r.destination()) {
|
||||
case Failure:
|
||||
getLogger().error("Failed to update Hive for {} due to {}; routing to failure", new Object[] {i, e}, e);
|
||||
break;
|
||||
case Retry:
|
||||
getLogger().error("Failed to update Hive for {} due to {}; it is possible that retrying the operation will succeed, so routing to retry",
|
||||
new Object[] {i, e}, e);
|
||||
break;
|
||||
}
|
||||
});
|
||||
return RollbackOnFailure.createOnError(onFlowFileError);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void onTrigger(ProcessContext context, ProcessSessionFactory sessionFactory) throws ProcessException {
|
||||
final Boolean rollbackOnFailure = context.getProperty(RollbackOnFailure.ROLLBACK_ON_FAILURE).asBoolean();
|
||||
final Charset charset = Charset.forName(context.getProperty(CHARSET).getValue());
|
||||
final String statementDelimiter = context.getProperty(STATEMENT_DELIMITER).getValue();
|
||||
final FunctionContext functionContext = new FunctionContext(rollbackOnFailure, charset, statementDelimiter);
|
||||
RollbackOnFailure.onTrigger(context, sessionFactory, functionContext, getLogger(), session -> process.onTrigger(context, session, functionContext));
|
||||
}
|
||||
}
|
|
@ -0,0 +1,560 @@
|
|||
/*
|
||||
* 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.nifi.processors.hive;
|
||||
|
||||
import com.google.common.util.concurrent.ThreadFactoryBuilder;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.hive.conf.HiveConf;
|
||||
import org.apache.hadoop.security.UserGroupInformation;
|
||||
import org.apache.hive.common.util.ShutdownHookManager;
|
||||
import org.apache.hive.streaming.ConnectionError;
|
||||
import org.apache.hive.streaming.HiveStreamingConnection;
|
||||
import org.apache.hive.streaming.InvalidTable;
|
||||
import org.apache.hive.streaming.SerializationError;
|
||||
import org.apache.hive.streaming.StreamingConnection;
|
||||
import org.apache.hive.streaming.StreamingException;
|
||||
import org.apache.hive.streaming.StreamingIOFailure;
|
||||
import org.apache.hive.streaming.TransactionError;
|
||||
import org.apache.nifi.annotation.behavior.RequiresInstanceClassLoading;
|
||||
import org.apache.nifi.annotation.behavior.WritesAttribute;
|
||||
import org.apache.nifi.annotation.behavior.WritesAttributes;
|
||||
import org.apache.nifi.annotation.documentation.CapabilityDescription;
|
||||
import org.apache.nifi.annotation.documentation.Tags;
|
||||
import org.apache.nifi.annotation.lifecycle.OnScheduled;
|
||||
import org.apache.nifi.annotation.lifecycle.OnStopped;
|
||||
import org.apache.nifi.components.PropertyDescriptor;
|
||||
import org.apache.nifi.components.ValidationContext;
|
||||
import org.apache.nifi.components.ValidationResult;
|
||||
import org.apache.nifi.expression.ExpressionLanguageScope;
|
||||
import org.apache.nifi.flowfile.FlowFile;
|
||||
import org.apache.nifi.hadoop.SecurityUtil;
|
||||
import org.apache.nifi.kerberos.KerberosCredentialsService;
|
||||
import org.apache.nifi.logging.ComponentLog;
|
||||
import org.apache.nifi.processor.AbstractProcessor;
|
||||
import org.apache.nifi.processor.ProcessContext;
|
||||
import org.apache.nifi.processor.ProcessSession;
|
||||
import org.apache.nifi.processor.ProcessorInitializationContext;
|
||||
import org.apache.nifi.processor.Relationship;
|
||||
import org.apache.nifi.processor.exception.ProcessException;
|
||||
import org.apache.nifi.processor.util.StandardValidators;
|
||||
import org.apache.nifi.processor.util.pattern.DiscontinuedException;
|
||||
import org.apache.nifi.processor.util.pattern.RollbackOnFailure;
|
||||
import org.apache.nifi.processors.hadoop.exception.RecordReaderFactoryException;
|
||||
import org.apache.nifi.serialization.RecordReader;
|
||||
import org.apache.nifi.serialization.RecordReaderFactory;
|
||||
import org.apache.nifi.util.StringUtils;
|
||||
import org.apache.nifi.util.hive.AuthenticationFailedException;
|
||||
import org.apache.nifi.util.hive.HiveConfigurator;
|
||||
import org.apache.nifi.util.hive.HiveOptions;
|
||||
import org.apache.hive.streaming.HiveRecordWriter;
|
||||
import org.apache.nifi.util.hive.HiveUtils;
|
||||
import org.apache.nifi.util.hive.ValidationResources;
|
||||
|
||||
import java.io.BufferedInputStream;
|
||||
import java.io.IOException;
|
||||
import java.io.InputStream;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Arrays;
|
||||
import java.util.Collection;
|
||||
import java.util.Collections;
|
||||
import java.util.HashMap;
|
||||
import java.util.HashSet;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Objects;
|
||||
import java.util.Set;
|
||||
import java.util.concurrent.ExecutorService;
|
||||
import java.util.concurrent.Executors;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
import java.util.concurrent.atomic.AtomicReference;
|
||||
import java.util.regex.Pattern;
|
||||
import java.util.stream.Collectors;
|
||||
|
||||
import static org.apache.nifi.processors.hive.AbstractHive3QLProcessor.ATTR_OUTPUT_TABLES;
|
||||
|
||||
@Tags({"hive", "streaming", "put", "database", "store"})
|
||||
@CapabilityDescription("This processor uses Hive Streaming to send flow file records to an Apache Hive 3.0+ table. "
|
||||
+ "The partition values are expected to be the 'last' fields of each record, so if the table is partitioned on column A for example, then the last field in "
|
||||
+ "each record should be field A.")
|
||||
@WritesAttributes({
|
||||
@WritesAttribute(attribute = "hivestreaming.record.count", description = "This attribute is written on the flow files routed to the 'success' "
|
||||
+ "and 'failure' relationships, and contains the number of records from the incoming flow file. All records in a flow file are committed as a single transaction."),
|
||||
@WritesAttribute(attribute = "query.output.tables", description = "This attribute is written on the flow files routed to the 'success' "
|
||||
+ "and 'failure' relationships, and contains the target table name in 'databaseName.tableName' format.")
|
||||
})
|
||||
@RequiresInstanceClassLoading
|
||||
public class PutHive3Streaming extends AbstractProcessor {
|
||||
// Attributes
|
||||
public static final String HIVE_STREAMING_RECORD_COUNT_ATTR = "hivestreaming.record.count";
|
||||
|
||||
private static final String CLIENT_CACHE_DISABLED_PROPERTY = "hcatalog.hive.client.cache.disabled";
|
||||
|
||||
// Properties
|
||||
static final PropertyDescriptor RECORD_READER = new PropertyDescriptor.Builder()
|
||||
.name("record-reader")
|
||||
.displayName("Record Reader")
|
||||
.description("The service for reading records from incoming flow files.")
|
||||
.identifiesControllerService(RecordReaderFactory.class)
|
||||
.required(true)
|
||||
.build();
|
||||
|
||||
static final PropertyDescriptor METASTORE_URI = new PropertyDescriptor.Builder()
|
||||
.name("hive3-stream-metastore-uri")
|
||||
.displayName("Hive Metastore URI")
|
||||
.description("The URI location for the Hive Metastore. Note that this is not the location of the Hive Server. The default port for the "
|
||||
+ "Hive metastore is 9043.")
|
||||
.required(true)
|
||||
.expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
|
||||
.addValidator(StandardValidators.URI_VALIDATOR)
|
||||
.addValidator(StandardValidators.createRegexMatchingValidator(Pattern.compile("(^[^/]+.*[^/]+$|^[^/]+$|^$)"))) // no start with / or end with /
|
||||
.build();
|
||||
|
||||
static final PropertyDescriptor HIVE_CONFIGURATION_RESOURCES = new PropertyDescriptor.Builder()
|
||||
.name("hive3-config-resources")
|
||||
.displayName("Hive Configuration Resources")
|
||||
.description("A file or comma separated list of files which contains the Hive configuration (hive-site.xml, e.g.). Without this, Hadoop "
|
||||
+ "will search the classpath for a 'hive-site.xml' file or will revert to a default configuration. Note that to enable authentication "
|
||||
+ "with Kerberos e.g., the appropriate properties must be set in the configuration files. Also note that if Max Concurrent Tasks is set "
|
||||
+ "to a number greater than one, the 'hcatalog.hive.client.cache.disabled' property will be forced to 'true' to avoid concurrency issues. "
|
||||
+ "Please see the Hive documentation for more details.")
|
||||
.required(false)
|
||||
.addValidator(HiveUtils.createMultipleFilesExistValidator())
|
||||
.build();
|
||||
|
||||
static final PropertyDescriptor DB_NAME = new PropertyDescriptor.Builder()
|
||||
.name("hive3-stream-database-name")
|
||||
.displayName("Database Name")
|
||||
.description("The name of the database in which to put the data.")
|
||||
.required(true)
|
||||
.expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
|
||||
.addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
|
||||
.build();
|
||||
|
||||
static final PropertyDescriptor TABLE_NAME = new PropertyDescriptor.Builder()
|
||||
.name("hive3-stream-table-name")
|
||||
.displayName("Table Name")
|
||||
.description("The name of the database table in which to put the data.")
|
||||
.required(true)
|
||||
.expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
|
||||
.addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
|
||||
.build();
|
||||
|
||||
static final PropertyDescriptor PARTITION_VALUES = new PropertyDescriptor.Builder()
|
||||
.name("hive3-stream-part-vals")
|
||||
.displayName("Partition Values")
|
||||
.description("Specifies a comma-separated list of the values for the partition columns of the target table. If the incoming records all have the same values "
|
||||
+ "for the partition columns, those values can be entered here, resulting in a performance gain. If specified, this property will often contain "
|
||||
+ "Expression Language, for example if PartitionRecord is upstream and two partitions 'name' and 'age' are used, then this property can be set to "
|
||||
+ "${name},${age}.")
|
||||
.required(false)
|
||||
.expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
|
||||
.addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
|
||||
.build();
|
||||
|
||||
static final PropertyDescriptor AUTOCREATE_PARTITIONS = new PropertyDescriptor.Builder()
|
||||
.name("hive3-stream-autocreate-partition")
|
||||
.displayName("Auto-Create Partitions")
|
||||
.description("Flag indicating whether partitions should be automatically created")
|
||||
.required(true)
|
||||
.addValidator(StandardValidators.BOOLEAN_VALIDATOR)
|
||||
.allowableValues("true", "false")
|
||||
.defaultValue("true")
|
||||
.build();
|
||||
|
||||
static final PropertyDescriptor CALL_TIMEOUT = new PropertyDescriptor.Builder()
|
||||
.name("hive3-stream-call-timeout")
|
||||
.displayName("Call Timeout")
|
||||
.description("The number of seconds allowed for a Hive Streaming operation to complete. A value of 0 indicates the processor should wait indefinitely on operations. "
|
||||
+ "Note that although this property supports Expression Language, it will not be evaluated against incoming FlowFile attributes.")
|
||||
.defaultValue("0")
|
||||
.required(true)
|
||||
.addValidator(StandardValidators.NON_NEGATIVE_INTEGER_VALIDATOR)
|
||||
.expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
|
||||
.build();
|
||||
|
||||
static final PropertyDescriptor DISABLE_STREAMING_OPTIMIZATIONS = new PropertyDescriptor.Builder()
|
||||
.name("hive3-stream-disable-optimizations")
|
||||
.displayName("Disable Streaming Optimizations")
|
||||
.description("Whether to disable streaming optimizations. Disabling streaming optimizations will have significant impact to performance and memory consumption.")
|
||||
.required(true)
|
||||
.addValidator(StandardValidators.BOOLEAN_VALIDATOR)
|
||||
.allowableValues("true", "false")
|
||||
.defaultValue("false")
|
||||
.build();
|
||||
|
||||
|
||||
static final PropertyDescriptor ROLLBACK_ON_FAILURE = RollbackOnFailure.createRollbackOnFailureProperty(
|
||||
"NOTE: When an error occurred after a Hive streaming transaction which is derived from the same input FlowFile is already committed," +
|
||||
" (i.e. a FlowFile contains more records than 'Records per Transaction' and a failure occurred at the 2nd transaction or later)" +
|
||||
" then the succeeded records will be transferred to 'success' relationship while the original input FlowFile stays in incoming queue." +
|
||||
" Duplicated records can be created for the succeeded ones when the same FlowFile is processed again.");
|
||||
|
||||
static final PropertyDescriptor KERBEROS_CREDENTIALS_SERVICE = new PropertyDescriptor.Builder()
|
||||
.name("kerberos-credentials-service")
|
||||
.displayName("Kerberos Credentials Service")
|
||||
.description("Specifies the Kerberos Credentials Controller Service that should be used for authenticating with Kerberos")
|
||||
.identifiesControllerService(KerberosCredentialsService.class)
|
||||
.required(false)
|
||||
.build();
|
||||
|
||||
// Relationships
|
||||
public static final Relationship REL_SUCCESS = new Relationship.Builder()
|
||||
.name("success")
|
||||
.description("A FlowFile containing Avro records routed to this relationship after the record has been successfully transmitted to Hive.")
|
||||
.build();
|
||||
|
||||
public static final Relationship REL_FAILURE = new Relationship.Builder()
|
||||
.name("failure")
|
||||
.description("A FlowFile containing Avro records routed to this relationship if the record could not be transmitted to Hive.")
|
||||
.build();
|
||||
|
||||
public static final Relationship REL_RETRY = new Relationship.Builder()
|
||||
.name("retry")
|
||||
.description("The incoming FlowFile is routed to this relationship if its records cannot be transmitted to Hive. Note that "
|
||||
+ "some records may have been processed successfully, they will be routed (as Avro flow files) to the success relationship. "
|
||||
+ "The combination of the retry, success, and failure relationships indicate how many records succeeded and/or failed. This "
|
||||
+ "can be used to provide a retry capability since full rollback is not possible.")
|
||||
.build();
|
||||
|
||||
private List<PropertyDescriptor> propertyDescriptors;
|
||||
private Set<Relationship> relationships;
|
||||
|
||||
protected volatile HiveConfigurator hiveConfigurator = new HiveConfigurator();
|
||||
protected volatile UserGroupInformation ugi;
|
||||
protected volatile HiveConf hiveConfig;
|
||||
|
||||
protected volatile int callTimeout;
|
||||
protected ExecutorService callTimeoutPool;
|
||||
protected volatile boolean rollbackOnFailure;
|
||||
|
||||
// Holder of cached Configuration information so validation does not reload the same config over and over
|
||||
private final AtomicReference<ValidationResources> validationResourceHolder = new AtomicReference<>();
|
||||
|
||||
@Override
|
||||
protected void init(ProcessorInitializationContext context) {
|
||||
List<PropertyDescriptor> props = new ArrayList<>();
|
||||
props.add(RECORD_READER);
|
||||
props.add(METASTORE_URI);
|
||||
props.add(HIVE_CONFIGURATION_RESOURCES);
|
||||
props.add(DB_NAME);
|
||||
props.add(TABLE_NAME);
|
||||
props.add(PARTITION_VALUES);
|
||||
props.add(AUTOCREATE_PARTITIONS);
|
||||
props.add(CALL_TIMEOUT);
|
||||
props.add(DISABLE_STREAMING_OPTIMIZATIONS);
|
||||
props.add(ROLLBACK_ON_FAILURE);
|
||||
props.add(KERBEROS_CREDENTIALS_SERVICE);
|
||||
|
||||
propertyDescriptors = Collections.unmodifiableList(props);
|
||||
|
||||
Set<Relationship> _relationships = new HashSet<>();
|
||||
_relationships.add(REL_SUCCESS);
|
||||
_relationships.add(REL_FAILURE);
|
||||
_relationships.add(REL_RETRY);
|
||||
relationships = Collections.unmodifiableSet(_relationships);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
|
||||
return propertyDescriptors;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Set<Relationship> getRelationships() {
|
||||
return relationships;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected Collection<ValidationResult> customValidate(final ValidationContext validationContext) {
|
||||
boolean confFileProvided = validationContext.getProperty(HIVE_CONFIGURATION_RESOURCES).isSet();
|
||||
|
||||
final List<ValidationResult> problems = new ArrayList<>();
|
||||
|
||||
final KerberosCredentialsService credentialsService = validationContext.getProperty(KERBEROS_CREDENTIALS_SERVICE).asControllerService(KerberosCredentialsService.class);
|
||||
|
||||
final String resolvedPrincipal = credentialsService != null ? credentialsService.getPrincipal() : null;
|
||||
final String resolvedKeytab = credentialsService != null ? credentialsService.getKeytab() : null;
|
||||
if (confFileProvided) {
|
||||
final String configFiles = validationContext.getProperty(HIVE_CONFIGURATION_RESOURCES).evaluateAttributeExpressions().getValue();
|
||||
problems.addAll(hiveConfigurator.validate(configFiles, resolvedPrincipal, resolvedKeytab, validationResourceHolder, getLogger()));
|
||||
}
|
||||
|
||||
return problems;
|
||||
}
|
||||
|
||||
@OnScheduled
|
||||
public void setup(final ProcessContext context) {
|
||||
ComponentLog log = getLogger();
|
||||
rollbackOnFailure = context.getProperty(ROLLBACK_ON_FAILURE).asBoolean();
|
||||
|
||||
final String configFiles = context.getProperty(HIVE_CONFIGURATION_RESOURCES).getValue();
|
||||
hiveConfig = hiveConfigurator.getConfigurationFromFiles(configFiles);
|
||||
|
||||
// If more than one concurrent task, force 'hcatalog.hive.client.cache.disabled' to true
|
||||
if (context.getMaxConcurrentTasks() > 1) {
|
||||
hiveConfig.setBoolean(CLIENT_CACHE_DISABLED_PROPERTY, true);
|
||||
}
|
||||
|
||||
// add any dynamic properties to the Hive configuration
|
||||
for (final Map.Entry<PropertyDescriptor, String> entry : context.getProperties().entrySet()) {
|
||||
final PropertyDescriptor descriptor = entry.getKey();
|
||||
if (descriptor.isDynamic()) {
|
||||
hiveConfig.set(descriptor.getName(), entry.getValue());
|
||||
}
|
||||
}
|
||||
|
||||
hiveConfigurator.preload(hiveConfig);
|
||||
|
||||
if (SecurityUtil.isSecurityEnabled(hiveConfig)) {
|
||||
final KerberosCredentialsService credentialsService = context.getProperty(KERBEROS_CREDENTIALS_SERVICE).asControllerService(KerberosCredentialsService.class);
|
||||
|
||||
final String resolvedPrincipal = credentialsService.getPrincipal();
|
||||
final String resolvedKeytab = credentialsService.getKeytab();
|
||||
|
||||
log.info("Hive Security Enabled, logging in as principal {} with keytab {}", new Object[]{resolvedPrincipal, resolvedKeytab});
|
||||
try {
|
||||
ugi = hiveConfigurator.authenticate(hiveConfig, resolvedPrincipal, resolvedKeytab);
|
||||
} catch (AuthenticationFailedException ae) {
|
||||
throw new ProcessException("Kerberos authentication failed for Hive Streaming", ae);
|
||||
}
|
||||
|
||||
log.info("Successfully logged in as principal {} with keytab {}", new Object[]{resolvedPrincipal, resolvedKeytab});
|
||||
} else {
|
||||
ugi = null;
|
||||
}
|
||||
|
||||
callTimeout = context.getProperty(CALL_TIMEOUT).evaluateAttributeExpressions().asInteger() * 1000; // milliseconds
|
||||
String timeoutName = "put-hive3-streaming-%d";
|
||||
this.callTimeoutPool = Executors.newFixedThreadPool(1,
|
||||
new ThreadFactoryBuilder().setNameFormat(timeoutName).build());
|
||||
}
|
||||
|
||||
public void onTrigger(ProcessContext context, ProcessSession session) throws ProcessException {
|
||||
FlowFile flowFile = session.get();
|
||||
if (flowFile == null) {
|
||||
return;
|
||||
}
|
||||
|
||||
final RecordReaderFactory recordReaderFactory = context.getProperty(RECORD_READER).asControllerService(RecordReaderFactory.class);
|
||||
final String dbName = context.getProperty(DB_NAME).evaluateAttributeExpressions(flowFile).getValue();
|
||||
final String tableName = context.getProperty(TABLE_NAME).evaluateAttributeExpressions(flowFile).getValue();
|
||||
|
||||
final ComponentLog log = getLogger();
|
||||
final String metastoreUri = context.getProperty(METASTORE_URI).evaluateAttributeExpressions(flowFile).getValue();
|
||||
|
||||
final String partitionValuesString = context.getProperty(PARTITION_VALUES).evaluateAttributeExpressions(flowFile).getValue();
|
||||
final boolean autoCreatePartitions = context.getProperty(AUTOCREATE_PARTITIONS).asBoolean();
|
||||
final boolean disableStreamingOptimizations = context.getProperty(DISABLE_STREAMING_OPTIMIZATIONS).asBoolean();
|
||||
|
||||
HiveOptions o = new HiveOptions(metastoreUri, dbName, tableName)
|
||||
.withHiveConf(hiveConfig)
|
||||
.withAutoCreatePartitions(autoCreatePartitions)
|
||||
.withCallTimeout(callTimeout)
|
||||
.withStreamingOptimizations(!disableStreamingOptimizations);
|
||||
|
||||
if (!StringUtils.isEmpty(partitionValuesString)) {
|
||||
List<String> staticPartitionValues = Arrays.stream(partitionValuesString.split(",")).filter(Objects::nonNull).map(String::trim).collect(Collectors.toList());
|
||||
o = o.withStaticPartitionValues(staticPartitionValues);
|
||||
}
|
||||
|
||||
if (SecurityUtil.isSecurityEnabled(hiveConfig)) {
|
||||
final KerberosCredentialsService credentialsService = context.getProperty(KERBEROS_CREDENTIALS_SERVICE).asControllerService(KerberosCredentialsService.class);
|
||||
o = o.withKerberosPrincipal(credentialsService.getPrincipal()).withKerberosKeytab(credentialsService.getKeytab());
|
||||
}
|
||||
|
||||
final HiveOptions options = o;
|
||||
|
||||
// Store the original class loader, then explicitly set it to this class's classloader (for use by the Hive Metastore)
|
||||
ClassLoader originalClassloader = Thread.currentThread().getContextClassLoader();
|
||||
Thread.currentThread().setContextClassLoader(this.getClass().getClassLoader());
|
||||
|
||||
StreamingConnection hiveStreamingConnection = null;
|
||||
|
||||
try (final InputStream rawIn = session.read(flowFile)) {
|
||||
final RecordReader reader;
|
||||
|
||||
try (final BufferedInputStream in = new BufferedInputStream(rawIn)) {
|
||||
|
||||
// if we fail to create the RecordReader then we want to route to failure, so we need to
|
||||
// handle this separately from the other IOExceptions which normally route to retry
|
||||
try {
|
||||
reader = recordReaderFactory.createRecordReader(flowFile, in, getLogger());
|
||||
} catch (Exception e) {
|
||||
throw new RecordReaderFactoryException("Unable to create RecordReader", e);
|
||||
}
|
||||
|
||||
hiveStreamingConnection = makeStreamingConnection(options, reader);
|
||||
// Add shutdown handler with higher priority than FileSystem shutdown hook so that streaming connection gets closed first before
|
||||
// filesystem close (to avoid ClosedChannelException)
|
||||
ShutdownHookManager.addShutdownHook(hiveStreamingConnection::close, FileSystem.SHUTDOWN_HOOK_PRIORITY + 1);
|
||||
|
||||
// Write records to Hive streaming, then commit and close
|
||||
hiveStreamingConnection.beginTransaction();
|
||||
hiveStreamingConnection.write(in);
|
||||
hiveStreamingConnection.commitTransaction();
|
||||
rawIn.close();
|
||||
|
||||
Map<String, String> updateAttributes = new HashMap<>();
|
||||
updateAttributes.put(HIVE_STREAMING_RECORD_COUNT_ATTR, Long.toString(hiveStreamingConnection.getConnectionStats().getRecordsWritten()));
|
||||
updateAttributes.put(ATTR_OUTPUT_TABLES, options.getQualifiedTableName());
|
||||
flowFile = session.putAllAttributes(flowFile, updateAttributes);
|
||||
session.getProvenanceReporter().send(flowFile, hiveStreamingConnection.getMetastoreUri());
|
||||
session.transfer(flowFile, REL_SUCCESS);
|
||||
} catch (TransactionError te) {
|
||||
if (rollbackOnFailure) {
|
||||
throw new ProcessException(te.getLocalizedMessage(), te);
|
||||
} else {
|
||||
throw new ShouldRetryException(te.getLocalizedMessage(), te);
|
||||
}
|
||||
} catch (RecordReaderFactoryException rrfe) {
|
||||
throw new ProcessException(rrfe);
|
||||
}
|
||||
} catch (InvalidTable | SerializationError | StreamingIOFailure | IOException e) {
|
||||
if (rollbackOnFailure) {
|
||||
if (hiveStreamingConnection != null) {
|
||||
abortConnection(hiveStreamingConnection);
|
||||
}
|
||||
throw new ProcessException(e.getLocalizedMessage(), e);
|
||||
} else {
|
||||
Map<String, String> updateAttributes = new HashMap<>();
|
||||
updateAttributes.put(HIVE_STREAMING_RECORD_COUNT_ATTR, Long.toString(hiveStreamingConnection.getConnectionStats().getRecordsWritten()));
|
||||
updateAttributes.put(ATTR_OUTPUT_TABLES, options.getQualifiedTableName());
|
||||
flowFile = session.putAllAttributes(flowFile, updateAttributes);
|
||||
session.transfer(flowFile, REL_FAILURE);
|
||||
}
|
||||
} catch (DiscontinuedException e) {
|
||||
// The input FlowFile processing is discontinued. Keep it in the input queue.
|
||||
getLogger().warn("Discontinued processing for {} due to {}", new Object[]{flowFile, e}, e);
|
||||
session.transfer(flowFile, Relationship.SELF);
|
||||
} catch (ConnectionError ce) {
|
||||
// If we can't connect to the metastore, yield the processor
|
||||
context.yield();
|
||||
throw new ProcessException("A connection to metastore cannot be established", ce);
|
||||
} catch (ShouldRetryException e) {
|
||||
// This exception is already a result of adjusting an error, so simply transfer the FlowFile to retry. Still need to abort the txn
|
||||
getLogger().error(e.getLocalizedMessage(), e);
|
||||
if (hiveStreamingConnection != null) {
|
||||
abortConnection(hiveStreamingConnection);
|
||||
}
|
||||
flowFile = session.penalize(flowFile);
|
||||
session.transfer(flowFile, REL_RETRY);
|
||||
} catch (StreamingException se) {
|
||||
// Handle all other exceptions. These are often record-based exceptions (since Hive will throw a subclass of the exception caught above)
|
||||
Throwable cause = se.getCause();
|
||||
if (cause == null) cause = se;
|
||||
// This is a failure on the incoming data, rollback on failure if specified; otherwise route to failure after penalizing (and abort txn in any case)
|
||||
if (rollbackOnFailure) {
|
||||
if (hiveStreamingConnection != null) {
|
||||
abortConnection(hiveStreamingConnection);
|
||||
}
|
||||
throw new ProcessException(cause.getLocalizedMessage(), cause);
|
||||
} else {
|
||||
flowFile = session.penalize(flowFile);
|
||||
Map<String, String> updateAttributes = new HashMap<>();
|
||||
updateAttributes.put(HIVE_STREAMING_RECORD_COUNT_ATTR, Long.toString(hiveStreamingConnection.getConnectionStats().getRecordsWritten()));
|
||||
updateAttributes.put(ATTR_OUTPUT_TABLES, options.getQualifiedTableName());
|
||||
flowFile = session.putAllAttributes(flowFile, updateAttributes);
|
||||
session.transfer(flowFile, REL_FAILURE);
|
||||
}
|
||||
|
||||
} catch (Throwable t) {
|
||||
if (hiveStreamingConnection != null) {
|
||||
abortConnection(hiveStreamingConnection);
|
||||
}
|
||||
throw (t instanceof ProcessException) ? (ProcessException) t : new ProcessException(t);
|
||||
} finally {
|
||||
closeConnection(hiveStreamingConnection);
|
||||
// Restore original class loader, might not be necessary but is good practice since the processor task changed it
|
||||
Thread.currentThread().setContextClassLoader(originalClassloader);
|
||||
}
|
||||
}
|
||||
|
||||
StreamingConnection makeStreamingConnection(HiveOptions options, RecordReader reader) throws StreamingException {
|
||||
return HiveStreamingConnection.newBuilder()
|
||||
.withDatabase(options.getDatabaseName())
|
||||
.withTable(options.getTableName())
|
||||
.withStaticPartitionValues(options.getStaticPartitionValues())
|
||||
.withHiveConf(options.getHiveConf())
|
||||
.withRecordWriter(new HiveRecordWriter(reader, getLogger()))
|
||||
.withAgentInfo("NiFi " + this.getClass().getSimpleName() + " [" + this.getIdentifier()
|
||||
+ "] thread " + Thread.currentThread().getId() + "[" + Thread.currentThread().getName() + "]")
|
||||
.connect();
|
||||
}
|
||||
|
||||
@OnStopped
|
||||
public void cleanup() {
|
||||
validationResourceHolder.set(null); // trigger re-validation of resources
|
||||
|
||||
ComponentLog log = getLogger();
|
||||
|
||||
if (callTimeoutPool != null) {
|
||||
callTimeoutPool.shutdown();
|
||||
try {
|
||||
while (!callTimeoutPool.isTerminated()) {
|
||||
callTimeoutPool.awaitTermination(callTimeout, TimeUnit.MILLISECONDS);
|
||||
}
|
||||
} catch (Throwable t) {
|
||||
log.warn("shutdown interrupted on " + callTimeoutPool, t);
|
||||
}
|
||||
callTimeoutPool = null;
|
||||
}
|
||||
|
||||
ugi = null;
|
||||
}
|
||||
|
||||
private void abortAndCloseConnection(StreamingConnection connection) {
|
||||
try {
|
||||
abortConnection(connection);
|
||||
closeConnection(connection);
|
||||
} catch (Exception ie) {
|
||||
getLogger().warn("unable to close hive connections. ", ie);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Abort current Txn on the connection
|
||||
*/
|
||||
private void abortConnection(StreamingConnection connection) {
|
||||
if (connection != null) {
|
||||
try {
|
||||
connection.abortTransaction();
|
||||
} catch (Exception e) {
|
||||
getLogger().error("Failed to abort Hive Streaming transaction " + connection + " due to exception ", e);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Close the streaming connection
|
||||
*/
|
||||
private void closeConnection(StreamingConnection connection) {
|
||||
if (connection != null) {
|
||||
try {
|
||||
connection.close();
|
||||
} catch (Exception e) {
|
||||
getLogger().error("Failed to close Hive Streaming connection " + connection + " due to exception ", e);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
private static class ShouldRetryException extends RuntimeException {
|
||||
private ShouldRetryException(String message, Throwable cause) {
|
||||
super(message, cause);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
@ -0,0 +1,477 @@
|
|||
/*
|
||||
* 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.nifi.processors.hive;
|
||||
|
||||
import java.nio.charset.Charset;
|
||||
import java.sql.Connection;
|
||||
import java.sql.PreparedStatement;
|
||||
import java.sql.ResultSet;
|
||||
import java.sql.Statement;
|
||||
import java.sql.SQLException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Collections;
|
||||
import java.util.HashMap;
|
||||
import java.util.HashSet;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
import java.util.UUID;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
import java.util.concurrent.atomic.AtomicLong;
|
||||
|
||||
import org.apache.commons.io.IOUtils;
|
||||
import org.apache.commons.lang.StringUtils;
|
||||
import org.apache.nifi.annotation.behavior.EventDriven;
|
||||
import org.apache.nifi.annotation.behavior.InputRequirement;
|
||||
import org.apache.nifi.annotation.behavior.InputRequirement.Requirement;
|
||||
import org.apache.nifi.annotation.behavior.WritesAttribute;
|
||||
import org.apache.nifi.annotation.behavior.WritesAttributes;
|
||||
import org.apache.nifi.annotation.documentation.CapabilityDescription;
|
||||
import org.apache.nifi.annotation.documentation.Tags;
|
||||
import org.apache.nifi.annotation.lifecycle.OnScheduled;
|
||||
import org.apache.nifi.components.PropertyDescriptor;
|
||||
import org.apache.nifi.dbcp.hive.Hive3DBCPService;
|
||||
import org.apache.nifi.expression.ExpressionLanguageScope;
|
||||
import org.apache.nifi.flowfile.FlowFile;
|
||||
import org.apache.nifi.flowfile.attributes.CoreAttributes;
|
||||
import org.apache.nifi.logging.ComponentLog;
|
||||
import org.apache.nifi.processor.ProcessContext;
|
||||
import org.apache.nifi.processor.ProcessSession;
|
||||
import org.apache.nifi.processor.ProcessSessionFactory;
|
||||
import org.apache.nifi.processor.Relationship;
|
||||
import org.apache.nifi.processor.exception.ProcessException;
|
||||
import org.apache.nifi.processor.util.StandardValidators;
|
||||
import org.apache.nifi.processor.util.pattern.PartialFunctions;
|
||||
import org.apache.nifi.util.StopWatch;
|
||||
import org.apache.nifi.util.hive.CsvOutputOptions;
|
||||
import org.apache.nifi.util.hive.HiveJdbcCommon;
|
||||
|
||||
import static org.apache.nifi.util.hive.HiveJdbcCommon.AVRO;
|
||||
import static org.apache.nifi.util.hive.HiveJdbcCommon.CSV;
|
||||
import static org.apache.nifi.util.hive.HiveJdbcCommon.CSV_MIME_TYPE;
|
||||
import static org.apache.nifi.util.hive.HiveJdbcCommon.MIME_TYPE_AVRO_BINARY;
|
||||
import static org.apache.nifi.util.hive.HiveJdbcCommon.NORMALIZE_NAMES_FOR_AVRO;
|
||||
|
||||
@EventDriven
|
||||
@InputRequirement(Requirement.INPUT_ALLOWED)
|
||||
@Tags({"hive", "sql", "select", "jdbc", "query", "database"})
|
||||
@CapabilityDescription("Execute provided HiveQL SELECT query against a Hive database connection. Query result will be converted to Avro or CSV format."
|
||||
+ " Streaming is used so arbitrarily large result sets are supported. This processor can be scheduled to run on "
|
||||
+ "a timer, or cron expression, using the standard scheduling methods, or it can be triggered by an incoming FlowFile. "
|
||||
+ "If it is triggered by an incoming FlowFile, then attributes of that FlowFile will be available when evaluating the "
|
||||
+ "select query. FlowFile attribute 'selecthiveql.row.count' indicates how many rows were selected.")
|
||||
@WritesAttributes({
|
||||
@WritesAttribute(attribute = "mime.type", description = "Sets the MIME type for the outgoing flowfile to application/avro-binary for Avro or text/csv for CSV."),
|
||||
@WritesAttribute(attribute = "filename", description = "Adds .avro or .csv to the filename attribute depending on which output format is selected."),
|
||||
@WritesAttribute(attribute = "selecthiveql.row.count", description = "Indicates how many rows were selected/returned by the query."),
|
||||
@WritesAttribute(attribute = "fragment.identifier", description = "If 'Max Rows Per Flow File' is set then all FlowFiles from the same query result set "
|
||||
+ "will have the same value for the fragment.identifier attribute. This can then be used to correlate the results."),
|
||||
@WritesAttribute(attribute = "fragment.count", description = "If 'Max Rows Per Flow File' is set then this is the total number of "
|
||||
+ "FlowFiles produced by a single ResultSet. This can be used in conjunction with the "
|
||||
+ "fragment.identifier attribute in order to know how many FlowFiles belonged to the same incoming ResultSet."),
|
||||
@WritesAttribute(attribute = "fragment.index", description = "If 'Max Rows Per Flow File' is set then the position of this FlowFile in the list of "
|
||||
+ "outgoing FlowFiles that were all derived from the same result set FlowFile. This can be "
|
||||
+ "used in conjunction with the fragment.identifier attribute to know which FlowFiles originated from the same query result set and in what order "
|
||||
+ "FlowFiles were produced"),
|
||||
@WritesAttribute(attribute = "query.input.tables", description = "Contains input table names in comma delimited 'databaseName.tableName' format.")
|
||||
})
|
||||
public class SelectHive3QL extends AbstractHive3QLProcessor {
|
||||
|
||||
static final String RESULT_ROW_COUNT = "selecthiveql.row.count";
|
||||
|
||||
// Relationships
|
||||
static final Relationship REL_SUCCESS = new Relationship.Builder()
|
||||
.name("success")
|
||||
.description("Successfully created FlowFile from HiveQL query result set.")
|
||||
.build();
|
||||
static final Relationship REL_FAILURE = new Relationship.Builder()
|
||||
.name("failure")
|
||||
.description("HiveQL query execution failed. Incoming FlowFile will be penalized and routed to this relationship")
|
||||
.build();
|
||||
|
||||
|
||||
static final PropertyDescriptor HIVEQL_SELECT_QUERY = new PropertyDescriptor.Builder()
|
||||
.name("hive-query")
|
||||
.displayName("HiveQL Select Query")
|
||||
.description("HiveQL SELECT query to execute. If this is not set, the query is assumed to be in the content of an incoming FlowFile.")
|
||||
.required(false)
|
||||
.addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
|
||||
.expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
|
||||
.build();
|
||||
|
||||
static final PropertyDescriptor FETCH_SIZE = new PropertyDescriptor.Builder()
|
||||
.name("hive-fetch-size")
|
||||
.displayName("Fetch Size")
|
||||
.description("The number of result rows to be fetched from the result set at a time. This is a hint to the driver and may not be "
|
||||
+ "honored and/or exact. If the value specified is zero, then the hint is ignored.")
|
||||
.defaultValue("0")
|
||||
.required(true)
|
||||
.addValidator(StandardValidators.NON_NEGATIVE_INTEGER_VALIDATOR)
|
||||
.expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
|
||||
.build();
|
||||
|
||||
static final PropertyDescriptor MAX_ROWS_PER_FLOW_FILE = new PropertyDescriptor.Builder()
|
||||
.name("hive-max-rows")
|
||||
.displayName("Max Rows Per Flow File")
|
||||
.description("The maximum number of result rows that will be included in a single FlowFile. " +
|
||||
"This will allow you to break up very large result sets into multiple FlowFiles. If the value specified is zero, then all rows are returned in a single FlowFile.")
|
||||
.defaultValue("0")
|
||||
.required(true)
|
||||
.addValidator(StandardValidators.NON_NEGATIVE_INTEGER_VALIDATOR)
|
||||
.expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
|
||||
.build();
|
||||
|
||||
static final PropertyDescriptor MAX_FRAGMENTS = new PropertyDescriptor.Builder()
|
||||
.name("hive-max-frags")
|
||||
.displayName("Maximum Number of Fragments")
|
||||
.description("The maximum number of fragments. If the value specified is zero, then all fragments are returned. " +
|
||||
"This prevents OutOfMemoryError when this processor ingests huge table.")
|
||||
.defaultValue("0")
|
||||
.required(true)
|
||||
.addValidator(StandardValidators.NON_NEGATIVE_INTEGER_VALIDATOR)
|
||||
.expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
|
||||
.build();
|
||||
|
||||
static final PropertyDescriptor HIVEQL_CSV_HEADER = new PropertyDescriptor.Builder()
|
||||
.name("csv-header")
|
||||
.displayName("CSV Header")
|
||||
.description("Include Header in Output")
|
||||
.required(true)
|
||||
.allowableValues("true", "false")
|
||||
.defaultValue("true")
|
||||
.addValidator(StandardValidators.BOOLEAN_VALIDATOR)
|
||||
.build();
|
||||
|
||||
static final PropertyDescriptor HIVEQL_CSV_ALT_HEADER = new PropertyDescriptor.Builder()
|
||||
.name("csv-alt-header")
|
||||
.displayName("Alternate CSV Header")
|
||||
.description("Comma separated list of header fields")
|
||||
.required(false)
|
||||
.addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
|
||||
.expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
|
||||
.build();
|
||||
|
||||
static final PropertyDescriptor HIVEQL_CSV_DELIMITER = new PropertyDescriptor.Builder()
|
||||
.name("csv-delimiter")
|
||||
.displayName("CSV Delimiter")
|
||||
.description("CSV Delimiter used to separate fields")
|
||||
.required(true)
|
||||
.defaultValue(",")
|
||||
.addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
|
||||
.expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
|
||||
.build();
|
||||
|
||||
static final PropertyDescriptor HIVEQL_CSV_QUOTE = new PropertyDescriptor.Builder()
|
||||
.name("csv-quote")
|
||||
.displayName("CSV Quote")
|
||||
.description("Whether to force quoting of CSV fields. Note that this might conflict with the setting for CSV Escape.")
|
||||
.required(true)
|
||||
.allowableValues("true", "false")
|
||||
.defaultValue("true")
|
||||
.addValidator(StandardValidators.BOOLEAN_VALIDATOR)
|
||||
.build();
|
||||
static final PropertyDescriptor HIVEQL_CSV_ESCAPE = new PropertyDescriptor.Builder()
|
||||
.name("csv-escape")
|
||||
.displayName("CSV Escape")
|
||||
.description("Whether to escape CSV strings in output. Note that this might conflict with the setting for CSV Quote.")
|
||||
.required(true)
|
||||
.allowableValues("true", "false")
|
||||
.defaultValue("true")
|
||||
.addValidator(StandardValidators.BOOLEAN_VALIDATOR)
|
||||
.build();
|
||||
|
||||
static final PropertyDescriptor HIVEQL_OUTPUT_FORMAT = new PropertyDescriptor.Builder()
|
||||
.name("hive-output-format")
|
||||
.displayName("Output Format")
|
||||
.description("How to represent the records coming from Hive (Avro, CSV, e.g.)")
|
||||
.required(true)
|
||||
.allowableValues(AVRO, CSV)
|
||||
.defaultValue(AVRO)
|
||||
.expressionLanguageSupported(ExpressionLanguageScope.NONE)
|
||||
.build();
|
||||
|
||||
private final static List<PropertyDescriptor> propertyDescriptors;
|
||||
private final static Set<Relationship> relationships;
|
||||
|
||||
/*
|
||||
* Will ensure that the list of property descriptors is built only once.
|
||||
* Will also create a Set of relationships
|
||||
*/
|
||||
static {
|
||||
List<PropertyDescriptor> _propertyDescriptors = new ArrayList<>();
|
||||
_propertyDescriptors.add(HIVE_DBCP_SERVICE);
|
||||
_propertyDescriptors.add(HIVEQL_SELECT_QUERY);
|
||||
_propertyDescriptors.add(FETCH_SIZE);
|
||||
_propertyDescriptors.add(QUERY_TIMEOUT);
|
||||
_propertyDescriptors.add(MAX_ROWS_PER_FLOW_FILE);
|
||||
_propertyDescriptors.add(MAX_FRAGMENTS);
|
||||
_propertyDescriptors.add(HIVEQL_OUTPUT_FORMAT);
|
||||
_propertyDescriptors.add(NORMALIZE_NAMES_FOR_AVRO);
|
||||
_propertyDescriptors.add(HIVEQL_CSV_HEADER);
|
||||
_propertyDescriptors.add(HIVEQL_CSV_ALT_HEADER);
|
||||
_propertyDescriptors.add(HIVEQL_CSV_DELIMITER);
|
||||
_propertyDescriptors.add(HIVEQL_CSV_QUOTE);
|
||||
_propertyDescriptors.add(HIVEQL_CSV_ESCAPE);
|
||||
_propertyDescriptors.add(CHARSET);
|
||||
propertyDescriptors = Collections.unmodifiableList(_propertyDescriptors);
|
||||
|
||||
Set<Relationship> _relationships = new HashSet<>();
|
||||
_relationships.add(REL_SUCCESS);
|
||||
_relationships.add(REL_FAILURE);
|
||||
relationships = Collections.unmodifiableSet(_relationships);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
|
||||
return propertyDescriptors;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Set<Relationship> getRelationships() {
|
||||
return relationships;
|
||||
}
|
||||
|
||||
@OnScheduled
|
||||
public void setup(ProcessContext context) {
|
||||
// If the query is not set, then an incoming flow file is needed. Otherwise fail the initialization
|
||||
if (!context.getProperty(HIVEQL_SELECT_QUERY).isSet() && !context.hasIncomingConnection()) {
|
||||
final String errorString = "Either the Select Query must be specified or there must be an incoming connection "
|
||||
+ "providing flowfile(s) containing a SQL select query";
|
||||
getLogger().error(errorString);
|
||||
throw new ProcessException(errorString);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void onTrigger(ProcessContext context, ProcessSessionFactory sessionFactory) throws ProcessException {
|
||||
PartialFunctions.onTrigger(context, sessionFactory, getLogger(), session -> onTrigger(context, session));
|
||||
}
|
||||
|
||||
private void onTrigger(final ProcessContext context, final ProcessSession session) throws ProcessException {
|
||||
FlowFile fileToProcess = (context.hasIncomingConnection() ? session.get() : null);
|
||||
FlowFile flowfile = null;
|
||||
|
||||
// If we have no FlowFile, and all incoming connections are self-loops then we can continue on.
|
||||
// However, if we have no FlowFile and we have connections coming from other Processors, then
|
||||
// we know that we should run only if we have a FlowFile.
|
||||
if (context.hasIncomingConnection()) {
|
||||
if (fileToProcess == null && context.hasNonLoopConnection()) {
|
||||
return;
|
||||
}
|
||||
}
|
||||
|
||||
final ComponentLog logger = getLogger();
|
||||
final Hive3DBCPService dbcpService = context.getProperty(HIVE_DBCP_SERVICE).asControllerService(Hive3DBCPService.class);
|
||||
final Charset charset = Charset.forName(context.getProperty(CHARSET).getValue());
|
||||
|
||||
final boolean flowbased = !(context.getProperty(HIVEQL_SELECT_QUERY).isSet());
|
||||
|
||||
// Source the SQL
|
||||
final String selectQuery;
|
||||
|
||||
if (context.getProperty(HIVEQL_SELECT_QUERY).isSet()) {
|
||||
selectQuery = context.getProperty(HIVEQL_SELECT_QUERY).evaluateAttributeExpressions(fileToProcess).getValue();
|
||||
} else {
|
||||
// If the query is not set, then an incoming flow file is required, and expected to contain a valid SQL select query.
|
||||
// If there is no incoming connection, onTrigger will not be called as the processor will fail when scheduled.
|
||||
final StringBuilder queryContents = new StringBuilder();
|
||||
session.read(fileToProcess, in -> queryContents.append(IOUtils.toString(in, charset)));
|
||||
selectQuery = queryContents.toString();
|
||||
}
|
||||
|
||||
|
||||
final Integer fetchSize = context.getProperty(FETCH_SIZE).evaluateAttributeExpressions(fileToProcess).asInteger();
|
||||
final Integer maxRowsPerFlowFile = context.getProperty(MAX_ROWS_PER_FLOW_FILE).evaluateAttributeExpressions(fileToProcess).asInteger();
|
||||
final Integer maxFragments = context.getProperty(MAX_FRAGMENTS).isSet()
|
||||
? context.getProperty(MAX_FRAGMENTS).evaluateAttributeExpressions(fileToProcess).asInteger()
|
||||
: 0;
|
||||
final String outputFormat = context.getProperty(HIVEQL_OUTPUT_FORMAT).getValue();
|
||||
final boolean convertNamesForAvro = context.getProperty(NORMALIZE_NAMES_FOR_AVRO).asBoolean();
|
||||
final StopWatch stopWatch = new StopWatch(true);
|
||||
final boolean header = context.getProperty(HIVEQL_CSV_HEADER).asBoolean();
|
||||
final String altHeader = context.getProperty(HIVEQL_CSV_ALT_HEADER).evaluateAttributeExpressions(fileToProcess).getValue();
|
||||
final String delimiter = context.getProperty(HIVEQL_CSV_DELIMITER).evaluateAttributeExpressions(fileToProcess).getValue();
|
||||
final boolean quote = context.getProperty(HIVEQL_CSV_QUOTE).asBoolean();
|
||||
final boolean escape = context.getProperty(HIVEQL_CSV_HEADER).asBoolean();
|
||||
final String fragmentIdentifier = UUID.randomUUID().toString();
|
||||
|
||||
try (final Connection con = dbcpService.getConnection();
|
||||
final Statement st = (flowbased ? con.prepareStatement(selectQuery) : con.createStatement())
|
||||
) {
|
||||
|
||||
st.setQueryTimeout(context.getProperty(QUERY_TIMEOUT).evaluateAttributeExpressions(fileToProcess).asInteger());
|
||||
|
||||
if (fetchSize != null && fetchSize > 0) {
|
||||
try {
|
||||
st.setFetchSize(fetchSize);
|
||||
} catch (SQLException se) {
|
||||
// Not all drivers support this, just log the error (at debug level) and move on
|
||||
logger.debug("Cannot set fetch size to {} due to {}", new Object[]{fetchSize, se.getLocalizedMessage()}, se);
|
||||
}
|
||||
}
|
||||
|
||||
final List<FlowFile> resultSetFlowFiles = new ArrayList<>();
|
||||
try {
|
||||
logger.debug("Executing query {}", new Object[]{selectQuery});
|
||||
if (flowbased) {
|
||||
// Hive JDBC Doesn't Support this yet:
|
||||
// ParameterMetaData pmd = ((PreparedStatement)st).getParameterMetaData();
|
||||
// int paramCount = pmd.getParameterCount();
|
||||
|
||||
// Alternate way to determine number of params in SQL.
|
||||
int paramCount = StringUtils.countMatches(selectQuery, "?");
|
||||
|
||||
if (paramCount > 0) {
|
||||
setParameters(1, (PreparedStatement) st, paramCount, fileToProcess.getAttributes());
|
||||
}
|
||||
}
|
||||
|
||||
final ResultSet resultSet;
|
||||
|
||||
try {
|
||||
resultSet = (flowbased ? ((PreparedStatement) st).executeQuery() : st.executeQuery(selectQuery));
|
||||
} catch (SQLException se) {
|
||||
// If an error occurs during the query, a flowfile is expected to be routed to failure, so ensure one here
|
||||
flowfile = (fileToProcess == null) ? session.create() : fileToProcess;
|
||||
fileToProcess = null;
|
||||
throw se;
|
||||
}
|
||||
|
||||
int fragmentIndex = 0;
|
||||
String baseFilename = (fileToProcess != null) ? fileToProcess.getAttribute(CoreAttributes.FILENAME.key()) : null;
|
||||
while (true) {
|
||||
final AtomicLong nrOfRows = new AtomicLong(0L);
|
||||
flowfile = (flowfile == null) ? session.create() : session.create(flowfile);
|
||||
if (baseFilename == null) {
|
||||
baseFilename = flowfile.getAttribute(CoreAttributes.FILENAME.key());
|
||||
}
|
||||
try {
|
||||
flowfile = session.write(flowfile, out -> {
|
||||
try {
|
||||
if (AVRO.equals(outputFormat)) {
|
||||
nrOfRows.set(HiveJdbcCommon.convertToAvroStream(resultSet, out, maxRowsPerFlowFile, convertNamesForAvro));
|
||||
} else if (CSV.equals(outputFormat)) {
|
||||
CsvOutputOptions options = new CsvOutputOptions(header, altHeader, delimiter, quote, escape, maxRowsPerFlowFile);
|
||||
nrOfRows.set(HiveJdbcCommon.convertToCsvStream(resultSet, out, options));
|
||||
} else {
|
||||
nrOfRows.set(0L);
|
||||
throw new ProcessException("Unsupported output format: " + outputFormat);
|
||||
}
|
||||
} catch (final SQLException | RuntimeException e) {
|
||||
throw new ProcessException("Error during database query or conversion of records.", e);
|
||||
}
|
||||
});
|
||||
} catch (ProcessException e) {
|
||||
// Add flowfile to results before rethrowing so it will be removed from session in outer catch
|
||||
resultSetFlowFiles.add(flowfile);
|
||||
throw e;
|
||||
}
|
||||
|
||||
if (nrOfRows.get() > 0 || resultSetFlowFiles.isEmpty()) {
|
||||
final Map<String, String> attributes = new HashMap<>();
|
||||
// Set attribute for how many rows were selected
|
||||
attributes.put(RESULT_ROW_COUNT, String.valueOf(nrOfRows.get()));
|
||||
|
||||
try {
|
||||
// Set input/output table names by parsing the query
|
||||
attributes.putAll(toQueryTableAttributes(findTableNames(selectQuery)));
|
||||
} catch (Exception e) {
|
||||
// If failed to parse the query, just log a warning message, but continue.
|
||||
getLogger().warn("Failed to parse query: {} due to {}", new Object[]{selectQuery, e}, e);
|
||||
}
|
||||
|
||||
// Set MIME type on output document and add extension to filename
|
||||
if (AVRO.equals(outputFormat)) {
|
||||
attributes.put(CoreAttributes.MIME_TYPE.key(), MIME_TYPE_AVRO_BINARY);
|
||||
attributes.put(CoreAttributes.FILENAME.key(), baseFilename + "." + fragmentIndex + ".avro");
|
||||
} else if (CSV.equals(outputFormat)) {
|
||||
attributes.put(CoreAttributes.MIME_TYPE.key(), CSV_MIME_TYPE);
|
||||
attributes.put(CoreAttributes.FILENAME.key(), baseFilename + "." + fragmentIndex + ".csv");
|
||||
}
|
||||
|
||||
if (maxRowsPerFlowFile > 0) {
|
||||
attributes.put("fragment.identifier", fragmentIdentifier);
|
||||
attributes.put("fragment.index", String.valueOf(fragmentIndex));
|
||||
}
|
||||
|
||||
flowfile = session.putAllAttributes(flowfile, attributes);
|
||||
|
||||
logger.info("{} contains {} Avro records; transferring to 'success'",
|
||||
new Object[]{flowfile, nrOfRows.get()});
|
||||
|
||||
if (context.hasIncomingConnection()) {
|
||||
// If the flow file came from an incoming connection, issue a Modify Content provenance event
|
||||
|
||||
session.getProvenanceReporter().modifyContent(flowfile, "Retrieved " + nrOfRows.get() + " rows",
|
||||
stopWatch.getElapsed(TimeUnit.MILLISECONDS));
|
||||
} else {
|
||||
// If we created a flow file from rows received from Hive, issue a Receive provenance event
|
||||
session.getProvenanceReporter().receive(flowfile, dbcpService.getConnectionURL(), stopWatch.getElapsed(TimeUnit.MILLISECONDS));
|
||||
}
|
||||
resultSetFlowFiles.add(flowfile);
|
||||
} else {
|
||||
// If there were no rows returned (and the first flow file has been sent, we're done processing, so remove the flowfile and carry on
|
||||
session.remove(flowfile);
|
||||
break;
|
||||
}
|
||||
|
||||
fragmentIndex++;
|
||||
if (maxFragments > 0 && fragmentIndex >= maxFragments) {
|
||||
break;
|
||||
}
|
||||
}
|
||||
|
||||
for (int i = 0; i < resultSetFlowFiles.size(); i++) {
|
||||
// Set count on all FlowFiles
|
||||
if (maxRowsPerFlowFile > 0) {
|
||||
resultSetFlowFiles.set(i,
|
||||
session.putAttribute(resultSetFlowFiles.get(i), "fragment.count", Integer.toString(fragmentIndex)));
|
||||
}
|
||||
}
|
||||
|
||||
} catch (final SQLException e) {
|
||||
throw e;
|
||||
}
|
||||
|
||||
session.transfer(resultSetFlowFiles, REL_SUCCESS);
|
||||
|
||||
} catch (final ProcessException | SQLException e) {
|
||||
logger.error("Issue processing SQL {} due to {}.", new Object[]{selectQuery, e});
|
||||
if (flowfile == null) {
|
||||
// This can happen if any exceptions occur while setting up the connection, statement, etc.
|
||||
logger.error("Unable to execute HiveQL select query {} due to {}. No FlowFile to route to failure",
|
||||
new Object[]{selectQuery, e});
|
||||
context.yield();
|
||||
} else {
|
||||
if (context.hasIncomingConnection()) {
|
||||
logger.error("Unable to execute HiveQL select query {} for {} due to {}; routing to failure",
|
||||
new Object[]{selectQuery, flowfile, e});
|
||||
flowfile = session.penalize(flowfile);
|
||||
} else {
|
||||
logger.error("Unable to execute HiveQL select query {} due to {}; routing to failure",
|
||||
new Object[]{selectQuery, e});
|
||||
context.yield();
|
||||
}
|
||||
session.transfer(flowfile, REL_FAILURE);
|
||||
}
|
||||
} finally {
|
||||
if (fileToProcess != null) {
|
||||
session.remove(fileToProcess);
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
|
@ -0,0 +1,175 @@
|
|||
/*
|
||||
* 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.nifi.processors.orc;
|
||||
|
||||
import org.apache.avro.Schema;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.hive.ql.io.orc.CompressionKind;
|
||||
import org.apache.hadoop.hive.ql.io.orc.NiFiOrcUtils;
|
||||
import org.apache.hadoop.hive.ql.io.orc.Writer;
|
||||
import org.apache.hadoop.hive.serde2.typeinfo.TypeInfo;
|
||||
import org.apache.nifi.annotation.behavior.InputRequirement;
|
||||
import org.apache.nifi.annotation.behavior.ReadsAttribute;
|
||||
import org.apache.nifi.annotation.behavior.Restricted;
|
||||
import org.apache.nifi.annotation.behavior.WritesAttribute;
|
||||
import org.apache.nifi.annotation.behavior.WritesAttributes;
|
||||
import org.apache.nifi.annotation.documentation.CapabilityDescription;
|
||||
import org.apache.nifi.annotation.documentation.Tags;
|
||||
import org.apache.nifi.avro.AvroTypeUtil;
|
||||
import org.apache.nifi.components.AllowableValue;
|
||||
import org.apache.nifi.components.PropertyDescriptor;
|
||||
import org.apache.nifi.expression.ExpressionLanguageScope;
|
||||
import org.apache.nifi.flowfile.FlowFile;
|
||||
import org.apache.nifi.processor.DataUnit;
|
||||
import org.apache.nifi.processor.ProcessContext;
|
||||
import org.apache.nifi.processor.ProcessorInitializationContext;
|
||||
import org.apache.nifi.processor.util.StandardValidators;
|
||||
import org.apache.nifi.processors.hadoop.AbstractPutHDFSRecord;
|
||||
import org.apache.nifi.processors.hadoop.record.HDFSRecordWriter;
|
||||
import org.apache.nifi.processors.orc.record.ORCHDFSRecordWriter;
|
||||
import org.apache.nifi.schema.access.SchemaNotFoundException;
|
||||
import org.apache.nifi.serialization.record.RecordSchema;
|
||||
import org.apache.nifi.util.hive.HiveUtils;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Collections;
|
||||
import java.util.List;
|
||||
|
||||
@InputRequirement(InputRequirement.Requirement.INPUT_REQUIRED)
|
||||
@Tags({"put", "ORC", "hadoop", "HDFS", "filesystem", "restricted", "record"})
|
||||
@CapabilityDescription("Reads records from an incoming FlowFile using the provided Record Reader, and writes those records " +
|
||||
"to a ORC file in the location/filesystem specified in the configuration.")
|
||||
@ReadsAttribute(attribute = "filename", description = "The name of the file to write comes from the value of this attribute.")
|
||||
@WritesAttributes({
|
||||
@WritesAttribute(attribute = "filename", description = "The name of the file is stored in this attribute."),
|
||||
@WritesAttribute(attribute = "absolute.hdfs.path", description = "The absolute path to the file is stored in this attribute."),
|
||||
@WritesAttribute(attribute = "record.count", description = "The number of records written to the ORC file"),
|
||||
@WritesAttribute(attribute = "hive.ddl", description = "Creates a partial Hive DDL statement for creating an external table in Hive from the destination folder. "
|
||||
+ "This can be used in ReplaceText for setting the content to the DDL. To make it valid DDL, add \"LOCATION '<path_to_orc_file_in_hdfs>'\", where "
|
||||
+ "the path is the directory that contains this ORC file on HDFS. For example, this processor can send flow files downstream to ReplaceText to set the content "
|
||||
+ "to this DDL (plus the LOCATION clause as described), then to PutHiveQL processor to create the table if it doesn't exist.")
|
||||
})
|
||||
@Restricted("Provides operator the ability to write to any file that NiFi has access to in HDFS or the local filesystem.")
|
||||
public class PutORC extends AbstractPutHDFSRecord {
|
||||
|
||||
public static final String HIVE_DDL_ATTRIBUTE = "hive.ddl";
|
||||
|
||||
public static final PropertyDescriptor ORC_CONFIGURATION_RESOURCES = new PropertyDescriptor.Builder()
|
||||
.name("putorc-config-resources")
|
||||
.displayName("ORC Configuration Resources")
|
||||
.description("A file or comma separated list of files which contains the ORC configuration (hive-site.xml, e.g.). Without this, Hadoop "
|
||||
+ "will search the classpath for a 'hive-site.xml' file or will revert to a default configuration. Please see the ORC documentation for more details.")
|
||||
.required(false).addValidator(HiveUtils.createMultipleFilesExistValidator()).build();
|
||||
|
||||
public static final PropertyDescriptor STRIPE_SIZE = new PropertyDescriptor.Builder()
|
||||
.name("putorc-stripe-size")
|
||||
.displayName("Stripe Size")
|
||||
.description("The size of the memory buffer (in bytes) for writing stripes to an ORC file")
|
||||
.required(true)
|
||||
.addValidator(StandardValidators.DATA_SIZE_VALIDATOR)
|
||||
.defaultValue("64 MB")
|
||||
.build();
|
||||
|
||||
public static final PropertyDescriptor BUFFER_SIZE = new PropertyDescriptor.Builder()
|
||||
.name("putorc-buffer-size")
|
||||
.displayName("Buffer Size")
|
||||
.description("The maximum size of the memory buffers (in bytes) used for compressing and storing a stripe in memory. This is a hint to the ORC writer, "
|
||||
+ "which may choose to use a smaller buffer size based on stripe size and number of columns for efficient stripe writing and memory utilization.")
|
||||
.required(true)
|
||||
.addValidator(StandardValidators.DATA_SIZE_VALIDATOR)
|
||||
.defaultValue("10 KB")
|
||||
.build();
|
||||
|
||||
static final PropertyDescriptor HIVE_TABLE_NAME = new PropertyDescriptor.Builder()
|
||||
.name("putorc-hive-table-name")
|
||||
.displayName("Hive Table Name")
|
||||
.description("An optional table name to insert into the hive.ddl attribute. The generated DDL can be used by "
|
||||
+ "a PutHive3QL processor (presumably after a PutHDFS processor) to create a table backed by the converted ORC file. "
|
||||
+ "If this property is not provided, the full name (including namespace) of the incoming Avro record will be normalized "
|
||||
+ "and used as the table name.")
|
||||
.required(false)
|
||||
.expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
|
||||
.addValidator(StandardValidators.NON_BLANK_VALIDATOR)
|
||||
.build();
|
||||
|
||||
static final PropertyDescriptor HIVE_FIELD_NAMES = new PropertyDescriptor.Builder()
|
||||
.name("putorc-hive-field-names")
|
||||
.displayName("Normalize Field Names for Hive")
|
||||
.description("Whether to normalize field names for Hive (force lowercase, e.g.). If the ORC file is going to "
|
||||
+ "be part of a Hive table, this property should be set to true. To preserve the original field names from the "
|
||||
+ "schema, this property should be set to false.")
|
||||
.required(true)
|
||||
.addValidator(StandardValidators.BOOLEAN_VALIDATOR)
|
||||
.allowableValues("true", "false")
|
||||
.defaultValue("true")
|
||||
.build();
|
||||
|
||||
|
||||
public static final List<AllowableValue> COMPRESSION_TYPES;
|
||||
|
||||
static {
|
||||
final List<AllowableValue> compressionTypes = new ArrayList<>();
|
||||
compressionTypes.add(new AllowableValue("NONE", "NONE", "No compression"));
|
||||
compressionTypes.add(new AllowableValue("ZLIB", "ZLIB", "ZLIB compression"));
|
||||
compressionTypes.add(new AllowableValue("SNAPPY", "SNAPPY", "Snappy compression"));
|
||||
compressionTypes.add(new AllowableValue("LZO", "LZO", "LZO compression"));
|
||||
COMPRESSION_TYPES = Collections.unmodifiableList(compressionTypes);
|
||||
}
|
||||
|
||||
@Override
|
||||
public List<AllowableValue> getCompressionTypes(final ProcessorInitializationContext context) {
|
||||
return COMPRESSION_TYPES;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String getDefaultCompressionType(final ProcessorInitializationContext context) {
|
||||
return "NONE";
|
||||
}
|
||||
|
||||
@Override
|
||||
public List<PropertyDescriptor> getAdditionalProperties() {
|
||||
final List<PropertyDescriptor> _propertyDescriptors = new ArrayList<>();
|
||||
_propertyDescriptors.add(ORC_CONFIGURATION_RESOURCES);
|
||||
_propertyDescriptors.add(STRIPE_SIZE);
|
||||
_propertyDescriptors.add(BUFFER_SIZE);
|
||||
_propertyDescriptors.add(HIVE_TABLE_NAME);
|
||||
_propertyDescriptors.add(HIVE_FIELD_NAMES);
|
||||
return Collections.unmodifiableList(_propertyDescriptors);
|
||||
}
|
||||
|
||||
@Override
|
||||
public HDFSRecordWriter createHDFSRecordWriter(final ProcessContext context, final FlowFile flowFile, final Configuration conf, final Path path, final RecordSchema schema)
|
||||
throws IOException, SchemaNotFoundException {
|
||||
|
||||
final Schema avroSchema = AvroTypeUtil.extractAvroSchema(schema);
|
||||
|
||||
final long stripeSize = context.getProperty(STRIPE_SIZE).asDataSize(DataUnit.B).longValue();
|
||||
final int bufferSize = context.getProperty(BUFFER_SIZE).asDataSize(DataUnit.B).intValue();
|
||||
final CompressionKind compressionType = CompressionKind.valueOf(context.getProperty(COMPRESSION_TYPE).getValue());
|
||||
final boolean normalizeForHive = context.getProperty(HIVE_FIELD_NAMES).asBoolean();
|
||||
TypeInfo orcSchema = NiFiOrcUtils.getOrcField(avroSchema, normalizeForHive);
|
||||
final Writer orcWriter = NiFiOrcUtils.createWriter(path, conf, orcSchema, stripeSize, compressionType, bufferSize);
|
||||
final String hiveTableName = context.getProperty(HIVE_TABLE_NAME).isSet()
|
||||
? context.getProperty(HIVE_TABLE_NAME).evaluateAttributeExpressions(flowFile).getValue()
|
||||
: NiFiOrcUtils.normalizeHiveTableName(avroSchema.getFullName());
|
||||
final boolean hiveFieldNames = context.getProperty(HIVE_FIELD_NAMES).asBoolean();
|
||||
|
||||
return new ORCHDFSRecordWriter(orcWriter, avroSchema, hiveTableName, hiveFieldNames);
|
||||
}
|
||||
}
|
|
@ -0,0 +1,110 @@
|
|||
/*
|
||||
* 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.nifi.processors.orc.record;
|
||||
|
||||
import org.apache.avro.Schema;
|
||||
import org.apache.hadoop.hive.ql.io.orc.NiFiOrcUtils;
|
||||
import org.apache.hadoop.hive.ql.io.orc.Writer;
|
||||
import org.apache.hadoop.hive.serde2.typeinfo.TypeInfo;
|
||||
import org.apache.nifi.processors.hadoop.record.HDFSRecordWriter;
|
||||
import org.apache.nifi.serialization.WriteResult;
|
||||
import org.apache.nifi.serialization.record.Record;
|
||||
import org.apache.nifi.serialization.record.RecordSet;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.HashMap;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
|
||||
import static org.apache.nifi.processors.orc.PutORC.HIVE_DDL_ATTRIBUTE;
|
||||
|
||||
/**
|
||||
* HDFSRecordWriter that writes ORC files using Avro as the schema representation.
|
||||
*/
|
||||
|
||||
public class ORCHDFSRecordWriter implements HDFSRecordWriter {
|
||||
|
||||
private final Schema avroSchema;
|
||||
private final TypeInfo orcSchema;
|
||||
private final Writer orcWriter;
|
||||
private final String hiveTableName;
|
||||
private final boolean hiveFieldNames;
|
||||
private final List<Schema.Field> recordFields;
|
||||
private final int numRecordFields;
|
||||
private Object[] workingRow;
|
||||
|
||||
public ORCHDFSRecordWriter(final Writer orcWriter, final Schema avroSchema, final String hiveTableName, final boolean hiveFieldNames) {
|
||||
this.avroSchema = avroSchema;
|
||||
this.orcWriter = orcWriter;
|
||||
this.hiveFieldNames = hiveFieldNames;
|
||||
this.orcSchema = NiFiOrcUtils.getOrcField(avroSchema, this.hiveFieldNames);
|
||||
this.hiveTableName = hiveTableName;
|
||||
this.recordFields = avroSchema != null ? avroSchema.getFields() : null;
|
||||
this.numRecordFields = recordFields != null ? recordFields.size() : -1;
|
||||
// Reuse row object
|
||||
this.workingRow = numRecordFields > -1 ? new Object[numRecordFields] : null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void write(final Record record) throws IOException {
|
||||
if (recordFields != null) {
|
||||
for (int i = 0; i < numRecordFields; i++) {
|
||||
final Schema.Field field = recordFields.get(i);
|
||||
final Schema fieldSchema = field.schema();
|
||||
final String fieldName = field.name();
|
||||
Object o = record.getValue(fieldName);
|
||||
try {
|
||||
workingRow[i] = NiFiOrcUtils.convertToORCObject(NiFiOrcUtils.getOrcField(fieldSchema, hiveFieldNames), o, hiveFieldNames);
|
||||
} catch (ArrayIndexOutOfBoundsException aioobe) {
|
||||
final String errorMsg = "Index out of bounds for column " + i + ", type " + fieldName + ", and object " + o.toString();
|
||||
throw new IOException(errorMsg, aioobe);
|
||||
}
|
||||
}
|
||||
orcWriter.addRow(NiFiOrcUtils.createOrcStruct(orcSchema, workingRow));
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* @param recordSet the RecordSet to write
|
||||
* @return the result of writing the record set
|
||||
* @throws IOException if an I/O error happens reading from the RecordSet, or writing a Record
|
||||
*/
|
||||
public WriteResult write(final RecordSet recordSet) throws IOException {
|
||||
int recordCount = 0;
|
||||
|
||||
Record record;
|
||||
while ((record = recordSet.next()) != null) {
|
||||
write(record);
|
||||
recordCount++;
|
||||
}
|
||||
|
||||
// Add Hive DDL Attribute
|
||||
String hiveDDL = NiFiOrcUtils.generateHiveDDL(avroSchema, hiveTableName, hiveFieldNames);
|
||||
Map<String, String> attributes = new HashMap<String, String>() {{
|
||||
put(HIVE_DDL_ATTRIBUTE, hiveDDL);
|
||||
}};
|
||||
|
||||
return WriteResult.of(recordCount, attributes);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void close() throws IOException {
|
||||
orcWriter.close();
|
||||
}
|
||||
|
||||
}
|
||||
|
|
@ -0,0 +1,23 @@
|
|||
/*
|
||||
* 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.nifi.util.hive;
|
||||
|
||||
public class AuthenticationFailedException extends Exception {
|
||||
public AuthenticationFailedException(String reason, Exception cause) {
|
||||
super(reason, cause);
|
||||
}
|
||||
}
|
|
@ -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.nifi.util.hive;
|
||||
|
||||
public class CsvOutputOptions {
|
||||
|
||||
private boolean header = true;
|
||||
private String altHeader = null;
|
||||
private String delimiter = ",";
|
||||
private boolean quote = false;
|
||||
private boolean escape = true;
|
||||
|
||||
private int maxRowsPerFlowFile = 0;
|
||||
|
||||
public boolean isHeader() {
|
||||
return header;
|
||||
}
|
||||
|
||||
public String getAltHeader() {
|
||||
return altHeader;
|
||||
}
|
||||
|
||||
|
||||
public String getDelimiter() {
|
||||
return delimiter;
|
||||
}
|
||||
|
||||
|
||||
public boolean isQuote() {
|
||||
return quote;
|
||||
}
|
||||
|
||||
public boolean isEscape() {
|
||||
return escape;
|
||||
}
|
||||
|
||||
public int getMaxRowsPerFlowFile() {
|
||||
return maxRowsPerFlowFile;
|
||||
}
|
||||
|
||||
public CsvOutputOptions(boolean header, String altHeader, String delimiter, boolean quote, boolean escape, int maxRowsPerFlowFile) {
|
||||
this.header = header;
|
||||
this.altHeader = altHeader;
|
||||
this.delimiter = delimiter;
|
||||
this.quote = quote;
|
||||
this.escape = escape;
|
||||
this.maxRowsPerFlowFile = maxRowsPerFlowFile;
|
||||
}
|
||||
}
|
|
@ -0,0 +1,119 @@
|
|||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
* contributor license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright ownership.
|
||||
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||
* (the "License"); you may not use this file except in compliance with
|
||||
* the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
package org.apache.nifi.util.hive;
|
||||
|
||||
import org.apache.commons.lang3.StringUtils;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.hive.conf.HiveConf;
|
||||
import org.apache.hadoop.security.UserGroupInformation;
|
||||
import org.apache.nifi.components.ValidationResult;
|
||||
import org.apache.nifi.hadoop.KerberosProperties;
|
||||
import org.apache.nifi.hadoop.SecurityUtil;
|
||||
import org.apache.nifi.logging.ComponentLog;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Collection;
|
||||
import java.util.List;
|
||||
import java.util.concurrent.atomic.AtomicReference;
|
||||
|
||||
/**
|
||||
* Created by mburgess on 5/4/16.
|
||||
*/
|
||||
public class HiveConfigurator {
|
||||
|
||||
public Collection<ValidationResult> validate(String configFiles, String principal, String keyTab, AtomicReference<ValidationResources> validationResourceHolder, ComponentLog log) {
|
||||
|
||||
final List<ValidationResult> problems = new ArrayList<>();
|
||||
ValidationResources resources = validationResourceHolder.get();
|
||||
|
||||
// if no resources in the holder, or if the holder has different resources loaded,
|
||||
// then load the Configuration and set the new resources in the holder
|
||||
if (resources == null || !configFiles.equals(resources.getConfigResources())) {
|
||||
log.debug("Reloading validation resources");
|
||||
resources = new ValidationResources(configFiles, getConfigurationFromFiles(configFiles));
|
||||
validationResourceHolder.set(resources);
|
||||
}
|
||||
|
||||
final Configuration hiveConfig = resources.getConfiguration();
|
||||
|
||||
problems.addAll(KerberosProperties.validatePrincipalAndKeytab(this.getClass().getSimpleName(), hiveConfig, principal, keyTab, log));
|
||||
|
||||
return problems;
|
||||
}
|
||||
|
||||
public HiveConf getConfigurationFromFiles(final String configFiles) {
|
||||
final HiveConf hiveConfig = new HiveConf();
|
||||
if (StringUtils.isNotBlank(configFiles)) {
|
||||
for (final String configFile : configFiles.split(",")) {
|
||||
hiveConfig.addResource(new Path(configFile.trim()));
|
||||
}
|
||||
}
|
||||
return hiveConfig;
|
||||
}
|
||||
|
||||
public void preload(Configuration configuration) {
|
||||
try {
|
||||
FileSystem.get(configuration).close();
|
||||
UserGroupInformation.setConfiguration(configuration);
|
||||
} catch (IOException ioe) {
|
||||
// Suppress exception as future uses of this configuration will fail
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* As of Apache NiFi 1.5.0, due to changes made to
|
||||
* {@link SecurityUtil#loginKerberos(Configuration, String, String)}, which is used by this
|
||||
* class to authenticate a principal with Kerberos, Hive controller services no longer
|
||||
* attempt relogins explicitly. For more information, please read the documentation for
|
||||
* {@link SecurityUtil#loginKerberos(Configuration, String, String)}.
|
||||
* <p/>
|
||||
* In previous versions of NiFi, a {@link org.apache.nifi.hadoop.KerberosTicketRenewer} was started by
|
||||
* {@link HiveConfigurator#authenticate(Configuration, String, String, long)} when the Hive
|
||||
* controller service was enabled. The use of a separate thread to explicitly relogin could cause race conditions
|
||||
* with the implicit relogin attempts made by hadoop/Hive code on a thread that references the same
|
||||
* {@link UserGroupInformation} instance. One of these threads could leave the
|
||||
* {@link javax.security.auth.Subject} in {@link UserGroupInformation} to be cleared or in an unexpected state
|
||||
* while the other thread is attempting to use the {@link javax.security.auth.Subject}, resulting in failed
|
||||
* authentication attempts that would leave the Hive controller service in an unrecoverable state.
|
||||
*
|
||||
* @see SecurityUtil#loginKerberos(Configuration, String, String)
|
||||
*/
|
||||
public UserGroupInformation authenticate(final Configuration hiveConfig, String principal, String keyTab) throws AuthenticationFailedException {
|
||||
UserGroupInformation ugi;
|
||||
try {
|
||||
ugi = SecurityUtil.loginKerberos(hiveConfig, principal, keyTab);
|
||||
} catch (IOException ioe) {
|
||||
throw new AuthenticationFailedException("Kerberos Authentication for Hive failed", ioe);
|
||||
}
|
||||
return ugi;
|
||||
}
|
||||
|
||||
/**
|
||||
* As of Apache NiFi 1.5.0, this method has been deprecated and is now a wrapper
|
||||
* method which invokes {@link HiveConfigurator#authenticate(Configuration, String, String)}. It will no longer start a
|
||||
* {@link org.apache.nifi.hadoop.KerberosTicketRenewer} to perform explicit relogins.
|
||||
*
|
||||
* @see HiveConfigurator#authenticate(Configuration, String, String)
|
||||
*/
|
||||
@Deprecated
|
||||
public UserGroupInformation authenticate(final Configuration hiveConfig, String principal, String keyTab, long ticketRenewalPeriod) throws AuthenticationFailedException {
|
||||
return authenticate(hiveConfig, principal, keyTab);
|
||||
}
|
||||
}
|
|
@ -0,0 +1,450 @@
|
|||
/*
|
||||
* 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.nifi.util.hive;
|
||||
|
||||
import org.apache.avro.Schema;
|
||||
import org.apache.avro.SchemaBuilder;
|
||||
import org.apache.avro.SchemaBuilder.FieldAssembler;
|
||||
import org.apache.avro.file.DataFileWriter;
|
||||
import org.apache.avro.generic.GenericData;
|
||||
import org.apache.avro.generic.GenericDatumWriter;
|
||||
import org.apache.avro.generic.GenericRecord;
|
||||
import org.apache.avro.io.DatumWriter;
|
||||
import org.apache.commons.lang3.StringEscapeUtils;
|
||||
import org.apache.commons.lang3.StringUtils;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.hive.conf.HiveConf;
|
||||
import org.apache.nifi.components.PropertyDescriptor;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.io.OutputStream;
|
||||
import java.math.BigDecimal;
|
||||
import java.math.BigInteger;
|
||||
import java.nio.ByteBuffer;
|
||||
import java.nio.charset.StandardCharsets;
|
||||
import java.sql.ResultSet;
|
||||
import java.sql.ResultSetMetaData;
|
||||
import java.sql.SQLException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Arrays;
|
||||
import java.util.List;
|
||||
|
||||
import static java.sql.Types.ARRAY;
|
||||
import static java.sql.Types.BIGINT;
|
||||
import static java.sql.Types.BINARY;
|
||||
import static java.sql.Types.BIT;
|
||||
import static java.sql.Types.BLOB;
|
||||
import static java.sql.Types.BOOLEAN;
|
||||
import static java.sql.Types.CHAR;
|
||||
import static java.sql.Types.CLOB;
|
||||
import static java.sql.Types.DATE;
|
||||
import static java.sql.Types.DECIMAL;
|
||||
import static java.sql.Types.DOUBLE;
|
||||
import static java.sql.Types.FLOAT;
|
||||
import static java.sql.Types.INTEGER;
|
||||
import static java.sql.Types.JAVA_OBJECT;
|
||||
import static java.sql.Types.LONGNVARCHAR;
|
||||
import static java.sql.Types.LONGVARBINARY;
|
||||
import static java.sql.Types.LONGVARCHAR;
|
||||
import static java.sql.Types.NCHAR;
|
||||
import static java.sql.Types.NUMERIC;
|
||||
import static java.sql.Types.NVARCHAR;
|
||||
import static java.sql.Types.REAL;
|
||||
import static java.sql.Types.ROWID;
|
||||
import static java.sql.Types.SMALLINT;
|
||||
import static java.sql.Types.STRUCT;
|
||||
import static java.sql.Types.TIME;
|
||||
import static java.sql.Types.TIMESTAMP;
|
||||
import static java.sql.Types.TINYINT;
|
||||
import static java.sql.Types.VARBINARY;
|
||||
import static java.sql.Types.VARCHAR;
|
||||
|
||||
/**
|
||||
* JDBC / HiveQL common functions.
|
||||
*/
|
||||
public class HiveJdbcCommon {
|
||||
|
||||
public static final String AVRO = "Avro";
|
||||
public static final String CSV = "CSV";
|
||||
|
||||
public static final String MIME_TYPE_AVRO_BINARY = "application/avro-binary";
|
||||
public static final String CSV_MIME_TYPE = "text/csv";
|
||||
|
||||
|
||||
public static final PropertyDescriptor NORMALIZE_NAMES_FOR_AVRO = new PropertyDescriptor.Builder()
|
||||
.name("hive-normalize-avro")
|
||||
.displayName("Normalize Table/Column Names")
|
||||
.description("Whether to change non-Avro-compatible characters in column names to Avro-compatible characters. For example, colons and periods "
|
||||
+ "will be changed to underscores in order to build a valid Avro record.")
|
||||
.allowableValues("true", "false")
|
||||
.defaultValue("false")
|
||||
.required(true)
|
||||
.build();
|
||||
|
||||
public static long convertToAvroStream(final ResultSet rs, final OutputStream outStream, final int maxRows, boolean convertNames) throws SQLException, IOException {
|
||||
return convertToAvroStream(rs, outStream, null, maxRows, convertNames, null);
|
||||
}
|
||||
|
||||
|
||||
public static long convertToAvroStream(final ResultSet rs, final OutputStream outStream, String recordName, final int maxRows, boolean convertNames, ResultSetRowCallback callback)
|
||||
throws SQLException, IOException {
|
||||
final Schema schema = createSchema(rs, recordName, convertNames);
|
||||
final GenericRecord rec = new GenericData.Record(schema);
|
||||
|
||||
final DatumWriter<GenericRecord> datumWriter = new GenericDatumWriter<>(schema);
|
||||
try (final DataFileWriter<GenericRecord> dataFileWriter = new DataFileWriter<>(datumWriter)) {
|
||||
dataFileWriter.create(schema, outStream);
|
||||
|
||||
final ResultSetMetaData meta = rs.getMetaData();
|
||||
final int nrOfColumns = meta.getColumnCount();
|
||||
long nrOfRows = 0;
|
||||
while (rs.next()) {
|
||||
if (callback != null) {
|
||||
callback.processRow(rs);
|
||||
}
|
||||
for (int i = 1; i <= nrOfColumns; i++) {
|
||||
final int javaSqlType = meta.getColumnType(i);
|
||||
Object value = rs.getObject(i);
|
||||
|
||||
if (value == null) {
|
||||
rec.put(i - 1, null);
|
||||
|
||||
} else if (javaSqlType == BINARY || javaSqlType == VARBINARY || javaSqlType == LONGVARBINARY || javaSqlType == BLOB || javaSqlType == CLOB) {
|
||||
// bytes requires little bit different handling
|
||||
ByteBuffer bb = null;
|
||||
if (value instanceof byte[]) {
|
||||
bb = ByteBuffer.wrap((byte[]) value);
|
||||
} else if (value instanceof ByteBuffer) {
|
||||
bb = (ByteBuffer) value;
|
||||
}
|
||||
if (bb != null) {
|
||||
rec.put(i - 1, bb);
|
||||
} else {
|
||||
throw new IOException("Could not process binary object of type " + value.getClass().getName());
|
||||
}
|
||||
|
||||
} else if (value instanceof Byte) {
|
||||
// tinyint(1) type is returned by JDBC driver as java.sql.Types.TINYINT
|
||||
// But value is returned by JDBC as java.lang.Byte
|
||||
// (at least H2 JDBC works this way)
|
||||
// direct put to avro record results:
|
||||
// org.apache.avro.AvroRuntimeException: Unknown datum type java.lang.Byte
|
||||
rec.put(i - 1, ((Byte) value).intValue());
|
||||
|
||||
} else if (value instanceof BigDecimal || value instanceof BigInteger) {
|
||||
// Avro can't handle BigDecimal and BigInteger as numbers - it will throw an AvroRuntimeException such as: "Unknown datum type: java.math.BigDecimal: 38"
|
||||
rec.put(i - 1, value.toString());
|
||||
|
||||
} else if (value instanceof Number) {
|
||||
// Need to call the right getXYZ() method (instead of the getObject() method above), since Doubles are sometimes returned
|
||||
// when the JDBC type is 6 (Float) for example.
|
||||
if (javaSqlType == FLOAT) {
|
||||
value = rs.getFloat(i);
|
||||
} else if (javaSqlType == DOUBLE) {
|
||||
value = rs.getDouble(i);
|
||||
} else if (javaSqlType == INTEGER || javaSqlType == TINYINT || javaSqlType == SMALLINT) {
|
||||
value = rs.getInt(i);
|
||||
}
|
||||
|
||||
rec.put(i - 1, value);
|
||||
|
||||
} else if (value instanceof Boolean) {
|
||||
rec.put(i - 1, value);
|
||||
} else {
|
||||
// The different types that we support are numbers (int, long, double, float),
|
||||
// as well as boolean values and Strings. Since Avro doesn't provide
|
||||
// timestamp types, we want to convert those to Strings. So we will cast anything other
|
||||
// than numbers or booleans to strings by using the toString() method.
|
||||
rec.put(i - 1, value.toString());
|
||||
}
|
||||
}
|
||||
dataFileWriter.append(rec);
|
||||
nrOfRows += 1;
|
||||
|
||||
if (maxRows > 0 && nrOfRows == maxRows)
|
||||
break;
|
||||
}
|
||||
|
||||
return nrOfRows;
|
||||
}
|
||||
}
|
||||
|
||||
public static Schema createSchema(final ResultSet rs, boolean convertNames) throws SQLException {
|
||||
return createSchema(rs, null, false);
|
||||
}
|
||||
|
||||
/**
|
||||
* Creates an Avro schema from a result set. If the table/record name is known a priori and provided, use that as a
|
||||
* fallback for the record name if it cannot be retrieved from the result set, and finally fall back to a default value.
|
||||
*
|
||||
* @param rs The result set to convert to Avro
|
||||
* @param recordName The a priori record name to use if it cannot be determined from the result set.
|
||||
* @param convertNames Whether to convert column/table names to be legal Avro names
|
||||
* @return A Schema object representing the result set converted to an Avro record
|
||||
* @throws SQLException if any error occurs during conversion
|
||||
*/
|
||||
public static Schema createSchema(final ResultSet rs, String recordName, boolean convertNames) throws SQLException {
|
||||
final ResultSetMetaData meta = rs.getMetaData();
|
||||
final int nrOfColumns = meta.getColumnCount();
|
||||
String tableName = StringUtils.isEmpty(recordName) ? "NiFi_SelectHiveQL_Record" : recordName;
|
||||
try {
|
||||
if (nrOfColumns > 0) {
|
||||
// Hive JDBC doesn't support getTableName, instead it returns table.column for column name. Grab the table name from the first column
|
||||
String firstColumnNameFromMeta = meta.getColumnName(1);
|
||||
int tableNameDelimiter = firstColumnNameFromMeta.lastIndexOf(".");
|
||||
if (tableNameDelimiter > -1) {
|
||||
String tableNameFromMeta = firstColumnNameFromMeta.substring(0, tableNameDelimiter);
|
||||
if (!StringUtils.isBlank(tableNameFromMeta)) {
|
||||
tableName = tableNameFromMeta;
|
||||
}
|
||||
}
|
||||
}
|
||||
} catch (SQLException se) {
|
||||
// Not all drivers support getTableName, so just use the previously-set default
|
||||
}
|
||||
|
||||
if (convertNames) {
|
||||
tableName = normalizeNameForAvro(tableName);
|
||||
}
|
||||
final FieldAssembler<Schema> builder = SchemaBuilder.record(tableName).namespace("any.data").fields();
|
||||
|
||||
/**
|
||||
* Some missing Avro types - Decimal, Date types. May need some additional work.
|
||||
*/
|
||||
for (int i = 1; i <= nrOfColumns; i++) {
|
||||
String columnNameFromMeta = meta.getColumnName(i);
|
||||
// Hive returns table.column for column name. Grab the column name as the string after the last period
|
||||
int columnNameDelimiter = columnNameFromMeta.lastIndexOf(".");
|
||||
String columnName = columnNameFromMeta.substring(columnNameDelimiter + 1);
|
||||
switch (meta.getColumnType(i)) {
|
||||
case CHAR:
|
||||
case LONGNVARCHAR:
|
||||
case LONGVARCHAR:
|
||||
case NCHAR:
|
||||
case NVARCHAR:
|
||||
case VARCHAR:
|
||||
case ARRAY:
|
||||
case STRUCT:
|
||||
case JAVA_OBJECT:
|
||||
builder.name(columnName).type().unionOf().nullBuilder().endNull().and().stringType().endUnion().noDefault();
|
||||
break;
|
||||
|
||||
case BIT:
|
||||
case BOOLEAN:
|
||||
builder.name(columnName).type().unionOf().nullBuilder().endNull().and().booleanType().endUnion().noDefault();
|
||||
break;
|
||||
|
||||
case INTEGER:
|
||||
// Default to signed type unless otherwise noted. Some JDBC drivers don't implement isSigned()
|
||||
boolean signedType = true;
|
||||
try {
|
||||
signedType = meta.isSigned(i);
|
||||
} catch (SQLException se) {
|
||||
// Use signed types as default
|
||||
}
|
||||
if (signedType) {
|
||||
builder.name(columnName).type().unionOf().nullBuilder().endNull().and().intType().endUnion().noDefault();
|
||||
} else {
|
||||
builder.name(columnName).type().unionOf().nullBuilder().endNull().and().longType().endUnion().noDefault();
|
||||
}
|
||||
break;
|
||||
|
||||
case SMALLINT:
|
||||
case TINYINT:
|
||||
builder.name(columnName).type().unionOf().nullBuilder().endNull().and().intType().endUnion().noDefault();
|
||||
break;
|
||||
|
||||
case BIGINT:
|
||||
builder.name(columnName).type().unionOf().nullBuilder().endNull().and().longType().endUnion().noDefault();
|
||||
break;
|
||||
|
||||
// java.sql.RowId is interface, is seems to be database
|
||||
// implementation specific, let's convert to String
|
||||
case ROWID:
|
||||
builder.name(columnName).type().unionOf().nullBuilder().endNull().and().stringType().endUnion().noDefault();
|
||||
break;
|
||||
|
||||
case FLOAT:
|
||||
case REAL:
|
||||
builder.name(columnName).type().unionOf().nullBuilder().endNull().and().floatType().endUnion().noDefault();
|
||||
break;
|
||||
|
||||
case DOUBLE:
|
||||
builder.name(columnName).type().unionOf().nullBuilder().endNull().and().doubleType().endUnion().noDefault();
|
||||
break;
|
||||
|
||||
// Did not find direct suitable type, need to be clarified!!!!
|
||||
case DECIMAL:
|
||||
case NUMERIC:
|
||||
builder.name(columnName).type().unionOf().nullBuilder().endNull().and().stringType().endUnion().noDefault();
|
||||
break;
|
||||
|
||||
// Did not find direct suitable type, need to be clarified!!!!
|
||||
case DATE:
|
||||
case TIME:
|
||||
case TIMESTAMP:
|
||||
builder.name(columnName).type().unionOf().nullBuilder().endNull().and().stringType().endUnion().noDefault();
|
||||
break;
|
||||
|
||||
case BINARY:
|
||||
case VARBINARY:
|
||||
case LONGVARBINARY:
|
||||
case BLOB:
|
||||
case CLOB:
|
||||
builder.name(columnName).type().unionOf().nullBuilder().endNull().and().bytesType().endUnion().noDefault();
|
||||
break;
|
||||
|
||||
|
||||
default:
|
||||
throw new IllegalArgumentException("createSchema: Unknown SQL type " + meta.getColumnType(i) + " cannot be converted to Avro type");
|
||||
}
|
||||
}
|
||||
|
||||
return builder.endRecord();
|
||||
}
|
||||
|
||||
public static long convertToCsvStream(final ResultSet rs, final OutputStream outStream, CsvOutputOptions outputOptions) throws SQLException, IOException {
|
||||
return convertToCsvStream(rs, outStream, null, null, outputOptions);
|
||||
}
|
||||
|
||||
public static long convertToCsvStream(final ResultSet rs, final OutputStream outStream, String recordName, ResultSetRowCallback callback, CsvOutputOptions outputOptions)
|
||||
throws SQLException, IOException {
|
||||
|
||||
final ResultSetMetaData meta = rs.getMetaData();
|
||||
final int nrOfColumns = meta.getColumnCount();
|
||||
List<String> columnNames = new ArrayList<>(nrOfColumns);
|
||||
|
||||
if (outputOptions.isHeader()) {
|
||||
if (outputOptions.getAltHeader() == null) {
|
||||
for (int i = 1; i <= nrOfColumns; i++) {
|
||||
String columnNameFromMeta = meta.getColumnName(i);
|
||||
// Hive returns table.column for column name. Grab the column name as the string after the last period
|
||||
int columnNameDelimiter = columnNameFromMeta.lastIndexOf(".");
|
||||
columnNames.add(columnNameFromMeta.substring(columnNameDelimiter + 1));
|
||||
}
|
||||
} else {
|
||||
String[] altHeaderNames = outputOptions.getAltHeader().split(",");
|
||||
columnNames = Arrays.asList(altHeaderNames);
|
||||
}
|
||||
}
|
||||
|
||||
// Write column names as header row
|
||||
outStream.write(StringUtils.join(columnNames, outputOptions.getDelimiter()).getBytes(StandardCharsets.UTF_8));
|
||||
if (outputOptions.isHeader()) {
|
||||
outStream.write("\n".getBytes(StandardCharsets.UTF_8));
|
||||
}
|
||||
|
||||
// Iterate over the rows
|
||||
int maxRows = outputOptions.getMaxRowsPerFlowFile();
|
||||
long nrOfRows = 0;
|
||||
while (rs.next()) {
|
||||
if (callback != null) {
|
||||
callback.processRow(rs);
|
||||
}
|
||||
List<String> rowValues = new ArrayList<>(nrOfColumns);
|
||||
for (int i = 1; i <= nrOfColumns; i++) {
|
||||
final int javaSqlType = meta.getColumnType(i);
|
||||
final Object value = rs.getObject(i);
|
||||
|
||||
switch (javaSqlType) {
|
||||
case CHAR:
|
||||
case LONGNVARCHAR:
|
||||
case LONGVARCHAR:
|
||||
case NCHAR:
|
||||
case NVARCHAR:
|
||||
case VARCHAR:
|
||||
String valueString = rs.getString(i);
|
||||
if (valueString != null) {
|
||||
// Removed extra quotes as those are a part of the escapeCsv when required.
|
||||
StringBuilder sb = new StringBuilder();
|
||||
if (outputOptions.isQuote()) {
|
||||
sb.append("\"");
|
||||
if (outputOptions.isEscape()) {
|
||||
sb.append(StringEscapeUtils.escapeCsv(valueString));
|
||||
} else {
|
||||
sb.append(valueString);
|
||||
}
|
||||
sb.append("\"");
|
||||
rowValues.add(sb.toString());
|
||||
} else {
|
||||
if (outputOptions.isEscape()) {
|
||||
rowValues.add(StringEscapeUtils.escapeCsv(valueString));
|
||||
} else {
|
||||
rowValues.add(valueString);
|
||||
}
|
||||
}
|
||||
} else {
|
||||
rowValues.add("");
|
||||
}
|
||||
break;
|
||||
case ARRAY:
|
||||
case STRUCT:
|
||||
case JAVA_OBJECT:
|
||||
String complexValueString = rs.getString(i);
|
||||
if (complexValueString != null) {
|
||||
rowValues.add(StringEscapeUtils.escapeCsv(complexValueString));
|
||||
} else {
|
||||
rowValues.add("");
|
||||
}
|
||||
break;
|
||||
default:
|
||||
if (value != null) {
|
||||
rowValues.add(value.toString());
|
||||
} else {
|
||||
rowValues.add("");
|
||||
}
|
||||
}
|
||||
}
|
||||
// Write row values
|
||||
outStream.write(StringUtils.join(rowValues, outputOptions.getDelimiter()).getBytes(StandardCharsets.UTF_8));
|
||||
outStream.write("\n".getBytes(StandardCharsets.UTF_8));
|
||||
nrOfRows++;
|
||||
|
||||
if (maxRows > 0 && nrOfRows == maxRows)
|
||||
break;
|
||||
}
|
||||
return nrOfRows;
|
||||
}
|
||||
|
||||
public static String normalizeNameForAvro(String inputName) {
|
||||
String normalizedName = inputName.replaceAll("[^A-Za-z0-9_]", "_");
|
||||
if (Character.isDigit(normalizedName.charAt(0))) {
|
||||
normalizedName = "_" + normalizedName;
|
||||
}
|
||||
return normalizedName;
|
||||
}
|
||||
|
||||
/**
|
||||
* An interface for callback methods which allows processing of a row during the convertToXYZStream() processing.
|
||||
* <b>IMPORTANT:</b> This method should only work on the row pointed at by the current ResultSet reference.
|
||||
* Advancing the cursor (e.g.) can cause rows to be skipped during Avro transformation.
|
||||
*/
|
||||
public interface ResultSetRowCallback {
|
||||
void processRow(ResultSet resultSet) throws IOException;
|
||||
}
|
||||
|
||||
public static Configuration getConfigurationFromFiles(final String configFiles) {
|
||||
final Configuration hiveConfig = new HiveConf();
|
||||
if (StringUtils.isNotBlank(configFiles)) {
|
||||
for (final String configFile : configFiles.split(",")) {
|
||||
hiveConfig.addResource(new Path(configFile.trim()));
|
||||
}
|
||||
}
|
||||
return hiveConfig;
|
||||
}
|
||||
}
|
|
@ -0,0 +1,117 @@
|
|||
/**
|
||||
* 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
|
||||
* <p>
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
* <p>
|
||||
* 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.nifi.util.hive;
|
||||
|
||||
import org.apache.hadoop.hive.conf.HiveConf;
|
||||
|
||||
import java.io.Serializable;
|
||||
import java.util.List;
|
||||
|
||||
|
||||
public class HiveOptions implements Serializable {
|
||||
|
||||
protected String databaseName;
|
||||
protected String tableName;
|
||||
protected String metaStoreURI;
|
||||
protected Integer idleTimeout = 60000;
|
||||
protected Integer callTimeout = 0;
|
||||
protected List<String> staticPartitionValues = null;
|
||||
protected Boolean autoCreatePartitions = true;
|
||||
protected String kerberosPrincipal;
|
||||
protected String kerberosKeytab;
|
||||
protected HiveConf hiveConf;
|
||||
protected boolean streamingOptimizations = true;
|
||||
|
||||
public HiveOptions(String metaStoreURI, String databaseName, String tableName) {
|
||||
this.metaStoreURI = metaStoreURI;
|
||||
this.databaseName = databaseName;
|
||||
this.tableName = tableName;
|
||||
}
|
||||
|
||||
public HiveOptions withCallTimeout(Integer callTimeout) {
|
||||
this.callTimeout = callTimeout;
|
||||
return this;
|
||||
}
|
||||
|
||||
public HiveOptions withStaticPartitionValues(List<String> staticPartitionValues) {
|
||||
this.staticPartitionValues = staticPartitionValues;
|
||||
return this;
|
||||
}
|
||||
|
||||
public HiveOptions withAutoCreatePartitions(Boolean autoCreatePartitions) {
|
||||
this.autoCreatePartitions = autoCreatePartitions;
|
||||
return this;
|
||||
}
|
||||
|
||||
public HiveOptions withKerberosKeytab(String kerberosKeytab) {
|
||||
this.kerberosKeytab = kerberosKeytab;
|
||||
return this;
|
||||
}
|
||||
|
||||
public HiveOptions withKerberosPrincipal(String kerberosPrincipal) {
|
||||
this.kerberosPrincipal = kerberosPrincipal;
|
||||
return this;
|
||||
}
|
||||
|
||||
public HiveOptions withHiveConf(HiveConf hiveConf) {
|
||||
this.hiveConf = hiveConf;
|
||||
return this;
|
||||
}
|
||||
|
||||
public HiveOptions withStreamingOptimizations(boolean streamingOptimizations) {
|
||||
this.streamingOptimizations = streamingOptimizations;
|
||||
return this;
|
||||
}
|
||||
|
||||
public String getMetaStoreURI() {
|
||||
return metaStoreURI;
|
||||
}
|
||||
|
||||
public String getDatabaseName() {
|
||||
return databaseName;
|
||||
}
|
||||
|
||||
public String getTableName() {
|
||||
return tableName;
|
||||
}
|
||||
|
||||
public String getQualifiedTableName() {
|
||||
return databaseName + "." + tableName;
|
||||
}
|
||||
|
||||
public List<String> getStaticPartitionValues() {
|
||||
return staticPartitionValues;
|
||||
}
|
||||
|
||||
public Integer getCallTimeOut() {
|
||||
return callTimeout;
|
||||
}
|
||||
|
||||
public Integer getIdleTimeout() {
|
||||
return idleTimeout;
|
||||
}
|
||||
|
||||
public HiveConf getHiveConf() {
|
||||
return hiveConf;
|
||||
}
|
||||
|
||||
public boolean getStreamingOptimizations() {
|
||||
return streamingOptimizations;
|
||||
}
|
||||
}
|
|
@ -0,0 +1,76 @@
|
|||
/**
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.nifi.util.hive;
|
||||
|
||||
|
||||
import org.apache.nifi.components.ValidationResult;
|
||||
import org.apache.nifi.components.Validator;
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
||||
import java.io.File;
|
||||
|
||||
public class HiveUtils {
|
||||
private static final Logger LOG = LoggerFactory.getLogger(HiveUtils.class);
|
||||
|
||||
public static final Validator GREATER_THAN_ONE_VALIDATOR = (subject, value, context) -> {
|
||||
if (context.isExpressionLanguageSupported(subject) && context.isExpressionLanguagePresent(value)) {
|
||||
return new ValidationResult.Builder().subject(subject).input(value).explanation("Expression Language Present").valid(true).build();
|
||||
}
|
||||
|
||||
String reason = null;
|
||||
try {
|
||||
final int intVal = Integer.parseInt(value);
|
||||
|
||||
if (intVal < 2) {
|
||||
reason = "value is less than 2";
|
||||
}
|
||||
} catch (final NumberFormatException e) {
|
||||
reason = "value is not a valid integer";
|
||||
}
|
||||
|
||||
return new ValidationResult.Builder().subject(subject).input(value).explanation(reason).valid(reason == null).build();
|
||||
};
|
||||
|
||||
/**
|
||||
* Validates that one or more files exist, as specified in a single property.
|
||||
*/
|
||||
public static Validator createMultipleFilesExistValidator() {
|
||||
return (subject, input, context) -> {
|
||||
if (context.isExpressionLanguageSupported(subject) && context.isExpressionLanguagePresent(input)) {
|
||||
return new ValidationResult.Builder().subject(subject).input(input).explanation("Expression Language Present").valid(true).build();
|
||||
}
|
||||
final String[] files = input.split("\\s*,\\s*");
|
||||
for (String filename : files) {
|
||||
try {
|
||||
final File file = new File(filename.trim());
|
||||
final boolean valid = file.exists() && file.isFile();
|
||||
if (!valid) {
|
||||
final String message = "File " + file + " does not exist or is not a file";
|
||||
return new ValidationResult.Builder().subject(subject).input(input).valid(false).explanation(message).build();
|
||||
}
|
||||
} catch (SecurityException e) {
|
||||
final String message = "Unable to access " + filename + " due to " + e.getMessage();
|
||||
return new ValidationResult.Builder().subject(subject).input(input).valid(false).explanation(message).build();
|
||||
}
|
||||
}
|
||||
return new ValidationResult.Builder().subject(subject).input(input).valid(true).build();
|
||||
};
|
||||
}
|
||||
}
|
|
@ -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.nifi.util.hive;
|
||||
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
|
||||
/**
|
||||
* A helper class for maintaining loaded configurations (to avoid reloading on use unless necessary)
|
||||
*/
|
||||
public class ValidationResources {
|
||||
|
||||
private final String configResources;
|
||||
private final Configuration configuration;
|
||||
|
||||
public ValidationResources(String configResources, Configuration configuration) {
|
||||
this.configResources = configResources;
|
||||
this.configuration = configuration;
|
||||
}
|
||||
|
||||
public String getConfigResources() {
|
||||
return configResources;
|
||||
}
|
||||
|
||||
public Configuration getConfiguration() {
|
||||
return configuration;
|
||||
}
|
||||
}
|
|
@ -0,0 +1,15 @@
|
|||
# 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.
|
||||
org.apache.nifi.dbcp.hive.Hive3ConnectionPool
|
|
@ -0,0 +1,18 @@
|
|||
# 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.
|
||||
org.apache.nifi.processors.hive.SelectHive3QL
|
||||
org.apache.nifi.processors.hive.PutHive3QL
|
||||
org.apache.nifi.processors.hive.PutHive3Streaming
|
||||
org.apache.nifi.processors.orc.PutORC
|
|
@ -0,0 +1,31 @@
|
|||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
* contributor license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright ownership.
|
||||
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||
* (the "License"); you may not use this file except in compliance with
|
||||
* the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
package org.apache.hive.streaming;
|
||||
|
||||
public class StubConnectionError extends ConnectionError {
|
||||
public StubConnectionError(String msg) {
|
||||
super(msg);
|
||||
}
|
||||
|
||||
public StubConnectionError(String msg, Exception innerEx) {
|
||||
super(msg, innerEx);
|
||||
}
|
||||
|
||||
public StubConnectionError(StreamingConnection endPoint, Exception innerEx) {
|
||||
super(endPoint, innerEx);
|
||||
}
|
||||
}
|
|
@ -0,0 +1,23 @@
|
|||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
* contributor license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright ownership.
|
||||
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||
* (the "License"); you may not use this file except in compliance with
|
||||
* the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
package org.apache.hive.streaming;
|
||||
|
||||
public class StubSerializationError extends SerializationError {
|
||||
public StubSerializationError(String msg, Exception e) {
|
||||
super(msg, e);
|
||||
}
|
||||
}
|
|
@ -0,0 +1,28 @@
|
|||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
* contributor license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright ownership.
|
||||
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||
* (the "License"); you may not use this file except in compliance with
|
||||
* the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
package org.apache.hive.streaming;
|
||||
|
||||
public class StubStreamingIOFailure extends StreamingIOFailure {
|
||||
|
||||
public StubStreamingIOFailure(String msg, Exception cause) {
|
||||
super(msg, cause);
|
||||
}
|
||||
|
||||
public StubStreamingIOFailure(String msg) {
|
||||
super(msg);
|
||||
}
|
||||
}
|
|
@ -0,0 +1,27 @@
|
|||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
* contributor license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright ownership.
|
||||
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||
* (the "License"); you may not use this file except in compliance with
|
||||
* the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
package org.apache.hive.streaming;
|
||||
|
||||
public class StubTransactionError extends TransactionError {
|
||||
public StubTransactionError(String msg, Exception e) {
|
||||
super(msg, e);
|
||||
}
|
||||
|
||||
public StubTransactionError(String msg) {
|
||||
super(msg);
|
||||
}
|
||||
}
|
|
@ -0,0 +1,138 @@
|
|||
/*
|
||||
* 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.nifi.dbcp.hive;
|
||||
|
||||
import org.apache.commons.dbcp.BasicDataSource;
|
||||
import org.apache.hadoop.security.UserGroupInformation;
|
||||
import org.apache.nifi.components.PropertyDescriptor;
|
||||
import org.apache.nifi.controller.AbstractControllerService;
|
||||
import org.apache.nifi.logging.ComponentLog;
|
||||
import org.apache.nifi.processor.exception.ProcessException;
|
||||
import org.apache.nifi.registry.VariableDescriptor;
|
||||
import org.apache.nifi.util.MockConfigurationContext;
|
||||
import org.apache.nifi.util.MockVariableRegistry;
|
||||
import org.junit.Before;
|
||||
import org.junit.Test;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.lang.reflect.Field;
|
||||
import java.lang.reflect.UndeclaredThrowableException;
|
||||
import java.security.PrivilegedExceptionAction;
|
||||
import java.sql.SQLException;
|
||||
import java.util.HashMap;
|
||||
import java.util.Map;
|
||||
|
||||
import static org.junit.Assert.assertEquals;
|
||||
import static org.mockito.Matchers.isA;
|
||||
import static org.mockito.Mockito.mock;
|
||||
import static org.mockito.Mockito.when;
|
||||
|
||||
public class Hive3ConnectionPoolTest {
|
||||
private UserGroupInformation userGroupInformation;
|
||||
private Hive3ConnectionPool hive3ConnectionPool;
|
||||
private BasicDataSource basicDataSource;
|
||||
private ComponentLog componentLog;
|
||||
|
||||
@Before
|
||||
public void setup() throws Exception {
|
||||
userGroupInformation = mock(UserGroupInformation.class);
|
||||
basicDataSource = mock(BasicDataSource.class);
|
||||
componentLog = mock(ComponentLog.class);
|
||||
|
||||
when(userGroupInformation.doAs(isA(PrivilegedExceptionAction.class))).thenAnswer(invocation -> {
|
||||
try {
|
||||
return ((PrivilegedExceptionAction) invocation.getArguments()[0]).run();
|
||||
} catch (IOException | Error | RuntimeException | InterruptedException e) {
|
||||
throw e;
|
||||
} catch (Throwable e) {
|
||||
throw new UndeclaredThrowableException(e);
|
||||
}
|
||||
});
|
||||
initPool();
|
||||
}
|
||||
|
||||
private void initPool() throws Exception {
|
||||
hive3ConnectionPool = new Hive3ConnectionPool();
|
||||
|
||||
Field ugiField = Hive3ConnectionPool.class.getDeclaredField("ugi");
|
||||
ugiField.setAccessible(true);
|
||||
ugiField.set(hive3ConnectionPool, userGroupInformation);
|
||||
|
||||
Field dataSourceField = Hive3ConnectionPool.class.getDeclaredField("dataSource");
|
||||
dataSourceField.setAccessible(true);
|
||||
dataSourceField.set(hive3ConnectionPool, basicDataSource);
|
||||
|
||||
Field componentLogField = AbstractControllerService.class.getDeclaredField("logger");
|
||||
componentLogField.setAccessible(true);
|
||||
componentLogField.set(hive3ConnectionPool, componentLog);
|
||||
}
|
||||
|
||||
@Test(expected = ProcessException.class)
|
||||
public void testGetConnectionSqlException() throws SQLException {
|
||||
SQLException sqlException = new SQLException("bad sql");
|
||||
when(basicDataSource.getConnection()).thenThrow(sqlException);
|
||||
try {
|
||||
hive3ConnectionPool.getConnection();
|
||||
} catch (ProcessException e) {
|
||||
assertEquals(sqlException, e.getCause());
|
||||
throw e;
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testExpressionLanguageSupport() throws Exception {
|
||||
final String URL = "jdbc:hive2://localhost:10000/default";
|
||||
final String USER = "user";
|
||||
final String PASS = "pass";
|
||||
final int MAX_CONN = 7;
|
||||
final String MAX_WAIT = "10 sec"; // 10000 milliseconds
|
||||
final String CONF = "/path/to/hive-site.xml";
|
||||
hive3ConnectionPool = new Hive3ConnectionPool();
|
||||
|
||||
Map<PropertyDescriptor, String> props = new HashMap<PropertyDescriptor, String>() {{
|
||||
put(Hive3ConnectionPool.DATABASE_URL, "${url}");
|
||||
put(Hive3ConnectionPool.DB_USER, "${username}");
|
||||
put(Hive3ConnectionPool.DB_PASSWORD, "${password}");
|
||||
put(Hive3ConnectionPool.MAX_TOTAL_CONNECTIONS, "${maxconn}");
|
||||
put(Hive3ConnectionPool.MAX_WAIT_TIME, "${maxwait}");
|
||||
put(Hive3ConnectionPool.HIVE_CONFIGURATION_RESOURCES, "${hiveconf}");
|
||||
}};
|
||||
|
||||
MockVariableRegistry registry = new MockVariableRegistry();
|
||||
registry.setVariable(new VariableDescriptor("url"), URL);
|
||||
registry.setVariable(new VariableDescriptor("username"), USER);
|
||||
registry.setVariable(new VariableDescriptor("password"), PASS);
|
||||
registry.setVariable(new VariableDescriptor("maxconn"), Integer.toString(MAX_CONN));
|
||||
registry.setVariable(new VariableDescriptor("maxwait"), MAX_WAIT);
|
||||
registry.setVariable(new VariableDescriptor("hiveconf"), CONF);
|
||||
|
||||
|
||||
MockConfigurationContext context = new MockConfigurationContext(props, null, registry);
|
||||
hive3ConnectionPool.onConfigured(context);
|
||||
|
||||
Field dataSourceField = Hive3ConnectionPool.class.getDeclaredField("dataSource");
|
||||
dataSourceField.setAccessible(true);
|
||||
basicDataSource = (BasicDataSource) dataSourceField.get(hive3ConnectionPool);
|
||||
assertEquals(URL, basicDataSource.getUrl());
|
||||
assertEquals(USER, basicDataSource.getUsername());
|
||||
assertEquals(PASS, basicDataSource.getPassword());
|
||||
assertEquals(MAX_CONN, basicDataSource.getMaxActive());
|
||||
assertEquals(10000L, basicDataSource.getMaxWait());
|
||||
assertEquals(URL, hive3ConnectionPool.getConnectionURL());
|
||||
}
|
||||
}
|
|
@ -0,0 +1,292 @@
|
|||
/*
|
||||
* 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.nifi.processors.hive;
|
||||
|
||||
import org.apache.nifi.processor.ProcessContext;
|
||||
import org.apache.nifi.processor.ProcessSessionFactory;
|
||||
import org.apache.nifi.processor.ProcessorInitializationContext;
|
||||
import org.apache.nifi.processor.exception.ProcessException;
|
||||
import org.apache.nifi.util.MockProcessContext;
|
||||
import org.apache.nifi.util.MockProcessorInitializationContext;
|
||||
import org.junit.Before;
|
||||
import org.junit.Test;
|
||||
|
||||
import java.util.Set;
|
||||
import java.util.concurrent.atomic.AtomicInteger;
|
||||
|
||||
import static org.junit.Assert.assertEquals;
|
||||
import static org.junit.Assert.assertTrue;
|
||||
|
||||
public class TestHive3Parser extends AbstractHive3QLProcessor {
|
||||
|
||||
@Before
|
||||
public void initialize() {
|
||||
final MockProcessContext processContext = new MockProcessContext(this);
|
||||
final ProcessorInitializationContext initializationContext = new MockProcessorInitializationContext(this, processContext);
|
||||
initialize(initializationContext);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void onTrigger(ProcessContext context, ProcessSessionFactory sessionFactory) throws ProcessException {
|
||||
|
||||
}
|
||||
|
||||
@Test
|
||||
public void parseSelect() {
|
||||
String query = "select a.empid, to_something(b.saraly) from " +
|
||||
"company.emp a inner join default.salary b where a.empid = b.empid";
|
||||
final Set<TableName> tableNames = findTableNames(query);
|
||||
System.out.printf("tableNames=%s\n", tableNames);
|
||||
assertEquals(2, tableNames.size());
|
||||
assertTrue(tableNames.contains(new TableName("company", "emp", true)));
|
||||
assertTrue(tableNames.contains(new TableName("default", "salary", true)));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void parseSelectPrepared() {
|
||||
String query = "select empid from company.emp a where a.firstName = ?";
|
||||
final Set<TableName> tableNames = findTableNames(query);
|
||||
System.out.printf("tableNames=%s\n", tableNames);
|
||||
assertEquals(1, tableNames.size());
|
||||
assertTrue(tableNames.contains(new TableName("company", "emp", true)));
|
||||
}
|
||||
|
||||
|
||||
@Test
|
||||
public void parseLongSelect() {
|
||||
String query = "select\n" +
|
||||
"\n" +
|
||||
" i_item_id,\n" +
|
||||
"\n" +
|
||||
" i_item_desc,\n" +
|
||||
"\n" +
|
||||
" s_state,\n" +
|
||||
"\n" +
|
||||
" count(ss_quantity) as store_sales_quantitycount,\n" +
|
||||
"\n" +
|
||||
" avg(ss_quantity) as store_sales_quantityave,\n" +
|
||||
"\n" +
|
||||
" stddev_samp(ss_quantity) as store_sales_quantitystdev,\n" +
|
||||
"\n" +
|
||||
" stddev_samp(ss_quantity) / avg(ss_quantity) as store_sales_quantitycov,\n" +
|
||||
"\n" +
|
||||
" count(sr_return_quantity) as store_returns_quantitycount,\n" +
|
||||
"\n" +
|
||||
" avg(sr_return_quantity) as store_returns_quantityave,\n" +
|
||||
"\n" +
|
||||
" stddev_samp(sr_return_quantity) as store_returns_quantitystdev,\n" +
|
||||
"\n" +
|
||||
" stddev_samp(sr_return_quantity) / avg(sr_return_quantity) as store_returns_quantitycov,\n" +
|
||||
"\n" +
|
||||
" count(cs_quantity) as catalog_sales_quantitycount,\n" +
|
||||
"\n" +
|
||||
" avg(cs_quantity) as catalog_sales_quantityave,\n" +
|
||||
"\n" +
|
||||
" stddev_samp(cs_quantity) / avg(cs_quantity) as catalog_sales_quantitystdev,\n" +
|
||||
"\n" +
|
||||
" stddev_samp(cs_quantity) / avg(cs_quantity) as catalog_sales_quantitycov\n" +
|
||||
"\n" +
|
||||
"from\n" +
|
||||
"\n" +
|
||||
" store_sales,\n" +
|
||||
"\n" +
|
||||
" store_returns,\n" +
|
||||
"\n" +
|
||||
" catalog_sales,\n" +
|
||||
"\n" +
|
||||
" date_dim d1,\n" +
|
||||
"\n" +
|
||||
" date_dim d2,\n" +
|
||||
"\n" +
|
||||
" date_dim d3,\n" +
|
||||
"\n" +
|
||||
" store,\n" +
|
||||
"\n" +
|
||||
" item\n" +
|
||||
"\n" +
|
||||
"where\n" +
|
||||
"\n" +
|
||||
" d1.d_quarter_name = '2000Q1'\n" +
|
||||
"\n" +
|
||||
" and d1.d_date_sk = ss_sold_date_sk\n" +
|
||||
"\n" +
|
||||
" and i_item_sk = ss_item_sk\n" +
|
||||
"\n" +
|
||||
" and s_store_sk = ss_store_sk\n" +
|
||||
"\n" +
|
||||
" and ss_customer_sk = sr_customer_sk\n" +
|
||||
"\n" +
|
||||
" and ss_item_sk = sr_item_sk\n" +
|
||||
"\n" +
|
||||
" and ss_ticket_number = sr_ticket_number\n" +
|
||||
"\n" +
|
||||
" and sr_returned_date_sk = d2.d_date_sk\n" +
|
||||
"\n" +
|
||||
" and d2.d_quarter_name in ('2000Q1' , '2000Q2', '2000Q3')\n" +
|
||||
"\n" +
|
||||
" and sr_customer_sk = cs_bill_customer_sk\n" +
|
||||
"\n" +
|
||||
" and sr_item_sk = cs_item_sk\n" +
|
||||
"\n" +
|
||||
" and cs_sold_date_sk = d3.d_date_sk\n" +
|
||||
"\n" +
|
||||
" and d3.d_quarter_name in ('2000Q1' , '2000Q2', '2000Q3')\n" +
|
||||
"\n" +
|
||||
"group by i_item_id , i_item_desc , s_state\n" +
|
||||
"\n" +
|
||||
"order by i_item_id , i_item_desc , s_state\n" +
|
||||
"\n" +
|
||||
"limit 100";
|
||||
|
||||
final Set<TableName> tableNames = findTableNames(query);
|
||||
System.out.printf("tableNames=%s\n", tableNames);
|
||||
assertEquals(6, tableNames.size());
|
||||
AtomicInteger cnt = new AtomicInteger(0);
|
||||
for (TableName tableName : tableNames) {
|
||||
if (tableName.equals(new TableName(null, "store_sales", true))) {
|
||||
cnt.incrementAndGet();
|
||||
} else if (tableName.equals(new TableName(null, "store_returns", true))) {
|
||||
cnt.incrementAndGet();
|
||||
} else if (tableName.equals(new TableName(null, "catalog_sales", true))) {
|
||||
cnt.incrementAndGet();
|
||||
} else if (tableName.equals(new TableName(null, "date_dim", true))) {
|
||||
cnt.incrementAndGet();
|
||||
} else if (tableName.equals(new TableName(null, "store", true))) {
|
||||
cnt.incrementAndGet();
|
||||
} else if (tableName.equals(new TableName(null, "item", true))) {
|
||||
cnt.incrementAndGet();
|
||||
}
|
||||
}
|
||||
assertEquals(6, cnt.get());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void parseSelectInsert() {
|
||||
String query = "insert into databaseA.tableA select key, max(value) from databaseA.tableA where category = 'x'";
|
||||
|
||||
// The same database.tableName can appear two times for input and output.
|
||||
final Set<TableName> tableNames = findTableNames(query);
|
||||
System.out.printf("tableNames=%s\n", tableNames);
|
||||
assertEquals(2, tableNames.size());
|
||||
AtomicInteger cnt = new AtomicInteger(0);
|
||||
tableNames.forEach(tableName -> {
|
||||
if (tableName.equals(new TableName("databaseA", "tableA", false))) {
|
||||
cnt.incrementAndGet();
|
||||
} else if (tableName.equals(new TableName("databaseA", "tableA", true))) {
|
||||
cnt.incrementAndGet();
|
||||
}
|
||||
});
|
||||
assertEquals(2, cnt.get());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void parseInsert() {
|
||||
String query = "insert into databaseB.tableB1 select something from tableA1 a1 inner join tableA2 a2 where a1.id = a2.id";
|
||||
|
||||
final Set<TableName> tableNames = findTableNames(query);
|
||||
System.out.printf("tableNames=%s\n", tableNames);
|
||||
assertEquals(3, tableNames.size());
|
||||
AtomicInteger cnt = new AtomicInteger(0);
|
||||
tableNames.forEach(tableName -> {
|
||||
if (tableName.equals(new TableName("databaseB", "tableB1", false))) {
|
||||
cnt.incrementAndGet();
|
||||
} else if (tableName.equals(new TableName(null, "tableA1", true))) {
|
||||
cnt.incrementAndGet();
|
||||
} else if (tableName.equals(new TableName(null, "tableA2", true))) {
|
||||
cnt.incrementAndGet();
|
||||
}
|
||||
});
|
||||
assertEquals(3, cnt.get());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void parseUpdate() {
|
||||
String query = "update table_a set y = 'updated' where x > 100";
|
||||
|
||||
final Set<TableName> tableNames = findTableNames(query);
|
||||
System.out.printf("tableNames=%s\n", tableNames);
|
||||
assertEquals(1, tableNames.size());
|
||||
assertTrue(tableNames.contains(new TableName(null, "table_a", false)));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void parseDelete() {
|
||||
String query = "delete from table_a where x > 100";
|
||||
|
||||
final Set<TableName> tableNames = findTableNames(query);
|
||||
System.out.printf("tableNames=%s\n", tableNames);
|
||||
assertEquals(1, tableNames.size());
|
||||
assertTrue(tableNames.contains(new TableName(null, "table_a", false)));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void parseDDL() {
|
||||
String query = "CREATE TABLE IF NOT EXISTS EMPLOYEES(\n" +
|
||||
"EmployeeID INT,FirstName STRING, Title STRING,\n" +
|
||||
"State STRING, Laptop STRING)\n" +
|
||||
"COMMENT 'Employee Names'\n" +
|
||||
"STORED AS ORC";
|
||||
|
||||
|
||||
final Set<TableName> tableNames = findTableNames(query);
|
||||
System.out.printf("tableNames=%s\n", tableNames);
|
||||
assertEquals(1, tableNames.size());
|
||||
assertTrue(tableNames.contains(new TableName(null, "EMPLOYEES", false)));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void parseSetProperty() {
|
||||
String query = " set 'hive.exec.dynamic.partition.mode'=nonstrict";
|
||||
final Set<TableName> tableNames = findTableNames(query);
|
||||
System.out.printf("tableNames=%s\n", tableNames);
|
||||
assertEquals(0, tableNames.size());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void parseSetRole() {
|
||||
String query = "set role all";
|
||||
final Set<TableName> tableNames = findTableNames(query);
|
||||
System.out.printf("tableNames=%s\n", tableNames);
|
||||
assertEquals(0, tableNames.size());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void parseShowRoles() {
|
||||
String query = "show roles";
|
||||
final Set<TableName> tableNames = findTableNames(query);
|
||||
System.out.printf("tableNames=%s\n", tableNames);
|
||||
assertEquals(0, tableNames.size());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void parseMsck() {
|
||||
String query = "msck repair table table_a";
|
||||
final Set<TableName> tableNames = findTableNames(query);
|
||||
System.out.printf("tableNames=%s\n", tableNames);
|
||||
assertEquals(1, tableNames.size());
|
||||
assertTrue(tableNames.contains(new TableName(null, "table_a", false)));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void parseAddJar() {
|
||||
String query = "ADD JAR hdfs:///tmp/my_jar.jar";
|
||||
final Set<TableName> tableNames = findTableNames(query);
|
||||
System.out.printf("tableNames=%s\n", tableNames);
|
||||
assertEquals(0, tableNames.size());
|
||||
}
|
||||
|
||||
}
|
|
@ -0,0 +1,792 @@
|
|||
package org.apache.nifi.processors.hive;/*
|
||||
* 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.
|
||||
*/
|
||||
|
||||
import org.apache.nifi.controller.AbstractControllerService;
|
||||
import org.apache.nifi.dbcp.DBCPService;
|
||||
import org.apache.nifi.dbcp.hive.Hive3DBCPService;
|
||||
import org.apache.nifi.processor.exception.ProcessException;
|
||||
import org.apache.nifi.processor.util.pattern.RollbackOnFailure;
|
||||
import org.apache.nifi.reporting.InitializationException;
|
||||
import org.apache.nifi.util.TestRunner;
|
||||
import org.apache.nifi.util.TestRunners;
|
||||
import org.junit.BeforeClass;
|
||||
import org.junit.Rule;
|
||||
import org.junit.Test;
|
||||
import org.junit.rules.TemporaryFolder;
|
||||
import org.mockito.Mockito;
|
||||
|
||||
import java.io.File;
|
||||
import java.io.IOException;
|
||||
import java.sql.Connection;
|
||||
import java.sql.DriverManager;
|
||||
import java.sql.ResultSet;
|
||||
import java.sql.SQLException;
|
||||
import java.sql.Statement;
|
||||
import java.sql.Types;
|
||||
import java.util.HashMap;
|
||||
import java.util.Map;
|
||||
|
||||
import static org.junit.Assert.assertEquals;
|
||||
import static org.junit.Assert.assertFalse;
|
||||
import static org.junit.Assert.assertTrue;
|
||||
import static org.junit.Assert.fail;
|
||||
|
||||
public class TestPutHive3QL {
|
||||
private static final String createPersons = "CREATE TABLE PERSONS (id integer primary key, name varchar(100), code integer)";
|
||||
private static final String createPersonsAutoId = "CREATE TABLE PERSONS (id INTEGER NOT NULL GENERATED ALWAYS AS IDENTITY (START WITH 1), name VARCHAR(100), code INTEGER check(code <= 100))";
|
||||
|
||||
@Rule
|
||||
public TemporaryFolder folder = new TemporaryFolder();
|
||||
|
||||
@BeforeClass
|
||||
public static void setup() {
|
||||
System.setProperty("derby.stream.error.file", "target/derby.log");
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testDirectStatements() throws InitializationException, ProcessException, SQLException, IOException {
|
||||
final TestRunner runner = TestRunners.newTestRunner(PutHive3QL.class);
|
||||
final File tempDir = folder.getRoot();
|
||||
final File dbDir = new File(tempDir, "db");
|
||||
final DBCPService service = new MockDBCPService(dbDir.getAbsolutePath());
|
||||
runner.addControllerService("dbcp", service);
|
||||
runner.enableControllerService(service);
|
||||
|
||||
try (final Connection conn = service.getConnection()) {
|
||||
try (final Statement stmt = conn.createStatement()) {
|
||||
stmt.executeUpdate(createPersons);
|
||||
}
|
||||
}
|
||||
|
||||
runner.setProperty(PutHive3QL.HIVE_DBCP_SERVICE, "dbcp");
|
||||
runner.enqueue("INSERT INTO PERSONS (ID, NAME, CODE) VALUES (1, 'Mark', 84)".getBytes());
|
||||
runner.run();
|
||||
|
||||
runner.assertAllFlowFilesTransferred(PutHive3QL.REL_SUCCESS, 1);
|
||||
|
||||
try (final Connection conn = service.getConnection()) {
|
||||
try (final Statement stmt = conn.createStatement()) {
|
||||
final ResultSet rs = stmt.executeQuery("SELECT * FROM PERSONS");
|
||||
assertTrue(rs.next());
|
||||
assertEquals(1, rs.getInt(1));
|
||||
assertEquals("Mark", rs.getString(2));
|
||||
assertEquals(84, rs.getInt(3));
|
||||
assertFalse(rs.next());
|
||||
}
|
||||
}
|
||||
|
||||
runner.enqueue("UPDATE PERSONS SET NAME='George' WHERE ID=1".getBytes());
|
||||
runner.run();
|
||||
|
||||
try (final Connection conn = service.getConnection()) {
|
||||
try (final Statement stmt = conn.createStatement()) {
|
||||
final ResultSet rs = stmt.executeQuery("SELECT * FROM PERSONS");
|
||||
assertTrue(rs.next());
|
||||
assertEquals(1, rs.getInt(1));
|
||||
assertEquals("George", rs.getString(2));
|
||||
assertEquals(84, rs.getInt(3));
|
||||
assertFalse(rs.next());
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testFailInMiddleWithBadStatement() throws InitializationException, ProcessException, SQLException, IOException {
|
||||
final TestRunner runner = TestRunners.newTestRunner(PutHive3QL.class);
|
||||
final File tempDir = folder.getRoot();
|
||||
final File dbDir = new File(tempDir, "db");
|
||||
final DBCPService service = new MockDBCPService(dbDir.getAbsolutePath());
|
||||
runner.addControllerService("dbcp", service);
|
||||
runner.enableControllerService(service);
|
||||
|
||||
try (final Connection conn = service.getConnection()) {
|
||||
try (final Statement stmt = conn.createStatement()) {
|
||||
stmt.executeUpdate(createPersonsAutoId);
|
||||
}
|
||||
}
|
||||
|
||||
runner.setProperty(PutHive3QL.HIVE_DBCP_SERVICE, "dbcp");
|
||||
runner.enqueue("INSERT INTO PERSONS (NAME, CODE) VALUES ('Mark', 84)".getBytes());
|
||||
runner.enqueue("INSERT INTO PERSONS".getBytes()); // intentionally wrong syntax
|
||||
runner.enqueue("INSERT INTO PERSONS (NAME, CODE) VALUES ('Tom', 3)".getBytes());
|
||||
runner.enqueue("INSERT INTO PERSONS (NAME, CODE) VALUES ('Harry', 44)".getBytes());
|
||||
runner.run();
|
||||
|
||||
runner.assertTransferCount(PutHive3QL.REL_FAILURE, 1);
|
||||
runner.assertTransferCount(PutHive3QL.REL_SUCCESS, 3);
|
||||
runner.getFlowFilesForRelationship(PutHive3QL.REL_SUCCESS)
|
||||
.forEach(f -> f.assertAttributeEquals(PutHive3QL.ATTR_OUTPUT_TABLES, "PERSONS"));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testFailInMiddleWithBadStatementRollbackOnFailure() throws InitializationException, ProcessException, SQLException, IOException {
|
||||
final TestRunner runner = TestRunners.newTestRunner(PutHive3QL.class);
|
||||
final File tempDir = folder.getRoot();
|
||||
final File dbDir = new File(tempDir, "db");
|
||||
final DBCPService service = new MockDBCPService(dbDir.getAbsolutePath());
|
||||
runner.addControllerService("dbcp", service);
|
||||
runner.enableControllerService(service);
|
||||
runner.setProperty(RollbackOnFailure.ROLLBACK_ON_FAILURE, "true");
|
||||
|
||||
try (final Connection conn = service.getConnection()) {
|
||||
try (final Statement stmt = conn.createStatement()) {
|
||||
stmt.executeUpdate(createPersonsAutoId);
|
||||
}
|
||||
}
|
||||
|
||||
runner.setProperty(PutHive3QL.HIVE_DBCP_SERVICE, "dbcp");
|
||||
runner.enqueue("INSERT INTO PERSONS (NAME, CODE) VALUES ('Mark', 84)".getBytes());
|
||||
runner.enqueue("INSERT INTO PERSONS".getBytes()); // intentionally wrong syntax
|
||||
runner.enqueue("INSERT INTO PERSONS (NAME, CODE) VALUES ('Tom', 3)".getBytes());
|
||||
runner.enqueue("INSERT INTO PERSONS (NAME, CODE) VALUES ('Harry', 44)".getBytes());
|
||||
runner.run();
|
||||
|
||||
// The 1st one should be routed to success, others should stay in queue.
|
||||
assertEquals(3, runner.getQueueSize().getObjectCount());
|
||||
runner.assertTransferCount(PutHive3QL.REL_FAILURE, 0);
|
||||
runner.assertTransferCount(PutHive3QL.REL_SUCCESS, 1);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testFailAtBeginning() throws InitializationException, ProcessException, SQLException, IOException {
|
||||
final TestRunner runner = TestRunners.newTestRunner(PutHive3QL.class);
|
||||
final File tempDir = folder.getRoot();
|
||||
final File dbDir = new File(tempDir, "db");
|
||||
final DBCPService service = new MockDBCPService(dbDir.getAbsolutePath());
|
||||
runner.addControllerService("dbcp", service);
|
||||
runner.enableControllerService(service);
|
||||
|
||||
try (final Connection conn = service.getConnection()) {
|
||||
try (final Statement stmt = conn.createStatement()) {
|
||||
stmt.executeUpdate(createPersonsAutoId);
|
||||
}
|
||||
}
|
||||
|
||||
runner.setProperty(PutHive3QL.HIVE_DBCP_SERVICE, "dbcp");
|
||||
runner.enqueue("INSERT INTO PERSONS".getBytes()); // intentionally wrong syntax
|
||||
runner.enqueue("INSERT INTO PERSONS (NAME, CODE) VALUES ('Tom', 3)".getBytes());
|
||||
runner.enqueue("INSERT INTO PERSONS (NAME, CODE) VALUES ('Harry', 44)".getBytes());
|
||||
runner.run();
|
||||
|
||||
runner.assertTransferCount(PutHive3QL.REL_FAILURE, 1);
|
||||
runner.assertTransferCount(PutHive3QL.REL_SUCCESS, 2);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testFailAtBeginningRollbackOnFailure() throws InitializationException, ProcessException, SQLException, IOException {
|
||||
final TestRunner runner = TestRunners.newTestRunner(PutHive3QL.class);
|
||||
final File tempDir = folder.getRoot();
|
||||
final File dbDir = new File(tempDir, "db");
|
||||
final DBCPService service = new MockDBCPService(dbDir.getAbsolutePath());
|
||||
runner.addControllerService("dbcp", service);
|
||||
runner.enableControllerService(service);
|
||||
runner.setProperty(RollbackOnFailure.ROLLBACK_ON_FAILURE, "true");
|
||||
|
||||
try (final Connection conn = service.getConnection()) {
|
||||
try (final Statement stmt = conn.createStatement()) {
|
||||
stmt.executeUpdate(createPersonsAutoId);
|
||||
}
|
||||
}
|
||||
|
||||
runner.setProperty(PutHive3QL.HIVE_DBCP_SERVICE, "dbcp");
|
||||
runner.enqueue("INSERT INTO PERSONS".getBytes()); // intentionally wrong syntax
|
||||
runner.enqueue("INSERT INTO PERSONS (NAME, CODE) VALUES ('Tom', 3)".getBytes());
|
||||
runner.enqueue("INSERT INTO PERSONS (NAME, CODE) VALUES ('Harry', 44)".getBytes());
|
||||
try {
|
||||
runner.run();
|
||||
fail("ProcessException should be thrown");
|
||||
} catch (AssertionError e) {
|
||||
assertTrue(e.getCause() instanceof ProcessException);
|
||||
}
|
||||
|
||||
assertEquals(3, runner.getQueueSize().getObjectCount());
|
||||
runner.assertTransferCount(PutHive3QL.REL_FAILURE, 0);
|
||||
runner.assertTransferCount(PutHive3QL.REL_SUCCESS, 0);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testFailInMiddleWithBadParameterType() throws InitializationException, ProcessException, SQLException, IOException {
|
||||
final TestRunner runner = TestRunners.newTestRunner(PutHive3QL.class);
|
||||
final File tempDir = folder.getRoot();
|
||||
final File dbDir = new File(tempDir, "db");
|
||||
final DBCPService service = new MockDBCPService(dbDir.getAbsolutePath());
|
||||
runner.addControllerService("dbcp", service);
|
||||
runner.enableControllerService(service);
|
||||
|
||||
try (final Connection conn = service.getConnection()) {
|
||||
try (final Statement stmt = conn.createStatement()) {
|
||||
stmt.executeUpdate(createPersonsAutoId);
|
||||
}
|
||||
}
|
||||
|
||||
runner.setProperty(PutHive3QL.HIVE_DBCP_SERVICE, "dbcp");
|
||||
|
||||
final Map<String, String> goodAttributes = new HashMap<>();
|
||||
goodAttributes.put("hiveql.args.1.type", String.valueOf(Types.INTEGER));
|
||||
goodAttributes.put("hiveql.args.1.value", "84");
|
||||
|
||||
final Map<String, String> badAttributes = new HashMap<>();
|
||||
badAttributes.put("hiveql.args.1.type", String.valueOf(Types.VARCHAR));
|
||||
badAttributes.put("hiveql.args.1.value", "hello");
|
||||
|
||||
final byte[] data = "INSERT INTO PERSONS (NAME, CODE) VALUES ('Mark', ?)".getBytes();
|
||||
runner.enqueue(data, goodAttributes);
|
||||
runner.enqueue(data, badAttributes);
|
||||
runner.enqueue(data, goodAttributes);
|
||||
runner.enqueue(data, goodAttributes);
|
||||
runner.run();
|
||||
|
||||
runner.assertTransferCount(PutHive3QL.REL_FAILURE, 1);
|
||||
runner.assertTransferCount(PutHive3QL.REL_SUCCESS, 3);
|
||||
}
|
||||
|
||||
|
||||
@Test
|
||||
public void testFailInMiddleWithBadParameterValue() throws InitializationException, ProcessException, SQLException, IOException {
|
||||
final TestRunner runner = TestRunners.newTestRunner(PutHive3QL.class);
|
||||
final File tempDir = folder.getRoot();
|
||||
final File dbDir = new File(tempDir, "db");
|
||||
final DBCPService service = new MockDBCPService(dbDir.getAbsolutePath());
|
||||
runner.addControllerService("dbcp", service);
|
||||
runner.enableControllerService(service);
|
||||
|
||||
try (final Connection conn = service.getConnection()) {
|
||||
try (final Statement stmt = conn.createStatement()) {
|
||||
stmt.executeUpdate(createPersonsAutoId);
|
||||
}
|
||||
}
|
||||
|
||||
runner.setProperty(PutHive3QL.HIVE_DBCP_SERVICE, "dbcp");
|
||||
|
||||
final Map<String, String> goodAttributes = new HashMap<>();
|
||||
goodAttributes.put("hiveql.args.1.type", String.valueOf(Types.INTEGER));
|
||||
goodAttributes.put("hiveql.args.1.value", "84");
|
||||
|
||||
final Map<String, String> badAttributes = new HashMap<>();
|
||||
badAttributes.put("hiveql.args.1.type", String.valueOf(Types.INTEGER));
|
||||
badAttributes.put("hiveql.args.1.value", "101"); // Constraint violation, up to 100
|
||||
|
||||
final byte[] data = "INSERT INTO PERSONS (NAME, CODE) VALUES ('Mark', ?)".getBytes();
|
||||
runner.enqueue(data, goodAttributes);
|
||||
runner.enqueue(data, badAttributes);
|
||||
runner.enqueue(data, goodAttributes);
|
||||
runner.enqueue(data, goodAttributes);
|
||||
runner.run();
|
||||
|
||||
runner.assertTransferCount(PutHive3QL.REL_SUCCESS, 3);
|
||||
runner.assertTransferCount(PutHive3QL.REL_FAILURE, 1);
|
||||
|
||||
try (final Connection conn = service.getConnection()) {
|
||||
try (final Statement stmt = conn.createStatement()) {
|
||||
final ResultSet rs = stmt.executeQuery("SELECT * FROM PERSONS");
|
||||
assertTrue(rs.next());
|
||||
assertEquals(1, rs.getInt(1));
|
||||
assertEquals("Mark", rs.getString(2));
|
||||
assertEquals(84, rs.getInt(3));
|
||||
assertTrue(rs.next());
|
||||
assertTrue(rs.next());
|
||||
assertFalse(rs.next());
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testFailInMiddleWithBadNumberFormat() throws InitializationException, ProcessException, SQLException, IOException {
|
||||
final TestRunner runner = TestRunners.newTestRunner(PutHive3QL.class);
|
||||
final File tempDir = folder.getRoot();
|
||||
final File dbDir = new File(tempDir, "db");
|
||||
final DBCPService service = new MockDBCPService(dbDir.getAbsolutePath());
|
||||
runner.addControllerService("dbcp", service);
|
||||
runner.enableControllerService(service);
|
||||
|
||||
try (final Connection conn = service.getConnection()) {
|
||||
try (final Statement stmt = conn.createStatement()) {
|
||||
stmt.executeUpdate(createPersonsAutoId);
|
||||
}
|
||||
}
|
||||
|
||||
runner.setProperty(PutHive3QL.HIVE_DBCP_SERVICE, "dbcp");
|
||||
|
||||
final Map<String, String> goodAttributes = new HashMap<>();
|
||||
goodAttributes.put("hiveql.args.1.type", String.valueOf(Types.INTEGER));
|
||||
goodAttributes.put("hiveql.args.1.value", "84");
|
||||
|
||||
final Map<String, String> badAttributes = new HashMap<>();
|
||||
badAttributes.put("hiveql.args.1.type", String.valueOf(Types.INTEGER));
|
||||
badAttributes.put("hiveql.args.1.value", "NOT_NUMBER");
|
||||
|
||||
final byte[] data = "INSERT INTO PERSONS (NAME, CODE) VALUES ('Mark', ?)".getBytes();
|
||||
runner.enqueue(data, goodAttributes);
|
||||
runner.enqueue(data, badAttributes);
|
||||
runner.enqueue(data, goodAttributes);
|
||||
runner.enqueue(data, goodAttributes);
|
||||
runner.run();
|
||||
|
||||
runner.assertTransferCount(PutHive3QL.REL_SUCCESS, 3);
|
||||
runner.assertTransferCount(PutHive3QL.REL_FAILURE, 1);
|
||||
|
||||
try (final Connection conn = service.getConnection()) {
|
||||
try (final Statement stmt = conn.createStatement()) {
|
||||
final ResultSet rs = stmt.executeQuery("SELECT * FROM PERSONS");
|
||||
assertTrue(rs.next());
|
||||
assertEquals(1, rs.getInt(1));
|
||||
assertEquals("Mark", rs.getString(2));
|
||||
assertEquals(84, rs.getInt(3));
|
||||
assertTrue(rs.next());
|
||||
assertTrue(rs.next());
|
||||
assertFalse(rs.next());
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
@Test
|
||||
public void testUsingSqlDataTypesWithNegativeValues() throws InitializationException, ProcessException, SQLException, IOException {
|
||||
final TestRunner runner = TestRunners.newTestRunner(PutHive3QL.class);
|
||||
final File tempDir = folder.getRoot();
|
||||
final File dbDir = new File(tempDir, "db");
|
||||
final DBCPService service = new MockDBCPService(dbDir.getAbsolutePath());
|
||||
runner.addControllerService("dbcp", service);
|
||||
runner.enableControllerService(service);
|
||||
|
||||
try (final Connection conn = service.getConnection()) {
|
||||
try (final Statement stmt = conn.createStatement()) {
|
||||
stmt.executeUpdate("CREATE TABLE PERSONS (id integer primary key, name varchar(100), code bigint)");
|
||||
}
|
||||
}
|
||||
|
||||
runner.setProperty(PutHive3QL.HIVE_DBCP_SERVICE, "dbcp");
|
||||
final Map<String, String> attributes = new HashMap<>();
|
||||
attributes.put("hiveql.args.1.type", "-5");
|
||||
attributes.put("hiveql.args.1.value", "84");
|
||||
runner.enqueue("INSERT INTO PERSONS (ID, NAME, CODE) VALUES (1, 'Mark', ?)".getBytes(), attributes);
|
||||
runner.run();
|
||||
|
||||
runner.assertAllFlowFilesTransferred(PutHive3QL.REL_SUCCESS, 1);
|
||||
runner.getFlowFilesForRelationship(PutHive3QL.REL_SUCCESS).get(0).assertAttributeEquals(PutHive3QL.ATTR_OUTPUT_TABLES, "PERSONS");
|
||||
|
||||
try (final Connection conn = service.getConnection()) {
|
||||
try (final Statement stmt = conn.createStatement()) {
|
||||
final ResultSet rs = stmt.executeQuery("SELECT * FROM PERSONS");
|
||||
assertTrue(rs.next());
|
||||
assertEquals(1, rs.getInt(1));
|
||||
assertEquals("Mark", rs.getString(2));
|
||||
assertEquals(84, rs.getInt(3));
|
||||
assertFalse(rs.next());
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testStatementsWithPreparedParameters() throws InitializationException, ProcessException, SQLException, IOException {
|
||||
final TestRunner runner = TestRunners.newTestRunner(PutHive3QL.class);
|
||||
final File tempDir = folder.getRoot();
|
||||
final File dbDir = new File(tempDir, "db");
|
||||
final DBCPService service = new MockDBCPService(dbDir.getAbsolutePath());
|
||||
runner.addControllerService("dbcp", service);
|
||||
runner.enableControllerService(service);
|
||||
|
||||
try (final Connection conn = service.getConnection()) {
|
||||
try (final Statement stmt = conn.createStatement()) {
|
||||
stmt.executeUpdate(createPersons);
|
||||
}
|
||||
}
|
||||
|
||||
runner.setProperty(PutHive3QL.HIVE_DBCP_SERVICE, "dbcp");
|
||||
final Map<String, String> attributes = new HashMap<>();
|
||||
attributes.put("hiveql.args.1.type", String.valueOf(Types.INTEGER));
|
||||
attributes.put("hiveql.args.1.value", "1");
|
||||
|
||||
attributes.put("hiveql.args.2.type", String.valueOf(Types.VARCHAR));
|
||||
attributes.put("hiveql.args.2.value", "Mark");
|
||||
|
||||
attributes.put("hiveql.args.3.type", String.valueOf(Types.INTEGER));
|
||||
attributes.put("hiveql.args.3.value", "84");
|
||||
|
||||
runner.enqueue("INSERT INTO PERSONS (ID, NAME, CODE) VALUES (?, ?, ?)".getBytes(), attributes);
|
||||
runner.run();
|
||||
|
||||
runner.assertAllFlowFilesTransferred(PutHive3QL.REL_SUCCESS, 1);
|
||||
|
||||
try (final Connection conn = service.getConnection()) {
|
||||
try (final Statement stmt = conn.createStatement()) {
|
||||
final ResultSet rs = stmt.executeQuery("SELECT * FROM PERSONS");
|
||||
assertTrue(rs.next());
|
||||
assertEquals(1, rs.getInt(1));
|
||||
assertEquals("Mark", rs.getString(2));
|
||||
assertEquals(84, rs.getInt(3));
|
||||
assertFalse(rs.next());
|
||||
}
|
||||
}
|
||||
|
||||
runner.clearTransferState();
|
||||
|
||||
attributes.clear();
|
||||
attributes.put("hiveql.args.1.type", String.valueOf(Types.VARCHAR));
|
||||
attributes.put("hiveql.args.1.value", "George");
|
||||
|
||||
attributes.put("hiveql.args.2.type", String.valueOf(Types.INTEGER));
|
||||
attributes.put("hiveql.args.2.value", "1");
|
||||
|
||||
runner.enqueue("UPDATE PERSONS SET NAME=? WHERE ID=?".getBytes(), attributes);
|
||||
runner.run();
|
||||
runner.assertAllFlowFilesTransferred(PutHive3QL.REL_SUCCESS, 1);
|
||||
|
||||
try (final Connection conn = service.getConnection()) {
|
||||
try (final Statement stmt = conn.createStatement()) {
|
||||
final ResultSet rs = stmt.executeQuery("SELECT * FROM PERSONS");
|
||||
assertTrue(rs.next());
|
||||
assertEquals(1, rs.getInt(1));
|
||||
assertEquals("George", rs.getString(2));
|
||||
assertEquals(84, rs.getInt(3));
|
||||
assertFalse(rs.next());
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
@Test
|
||||
public void testMultipleStatementsWithinFlowFile() throws InitializationException, ProcessException, SQLException, IOException {
|
||||
final TestRunner runner = TestRunners.newTestRunner(PutHive3QL.class);
|
||||
final File tempDir = folder.getRoot();
|
||||
final File dbDir = new File(tempDir, "db");
|
||||
final DBCPService service = new MockDBCPService(dbDir.getAbsolutePath());
|
||||
runner.addControllerService("dbcp", service);
|
||||
runner.enableControllerService(service);
|
||||
|
||||
try (final Connection conn = service.getConnection()) {
|
||||
try (final Statement stmt = conn.createStatement()) {
|
||||
stmt.executeUpdate(createPersons);
|
||||
}
|
||||
}
|
||||
|
||||
runner.setProperty(PutHive3QL.HIVE_DBCP_SERVICE, "dbcp");
|
||||
|
||||
final String sql = "INSERT INTO PERSONS (ID, NAME, CODE) VALUES (?, ?, ?); " +
|
||||
"UPDATE PERSONS SET NAME='George' WHERE ID=?; ";
|
||||
final Map<String, String> attributes = new HashMap<>();
|
||||
attributes.put("hiveql.args.1.type", String.valueOf(Types.INTEGER));
|
||||
attributes.put("hiveql.args.1.value", "1");
|
||||
|
||||
attributes.put("hiveql.args.2.type", String.valueOf(Types.VARCHAR));
|
||||
attributes.put("hiveql.args.2.value", "Mark");
|
||||
|
||||
attributes.put("hiveql.args.3.type", String.valueOf(Types.INTEGER));
|
||||
attributes.put("hiveql.args.3.value", "84");
|
||||
|
||||
attributes.put("hiveql.args.4.type", String.valueOf(Types.INTEGER));
|
||||
attributes.put("hiveql.args.4.value", "1");
|
||||
|
||||
runner.enqueue(sql.getBytes(), attributes);
|
||||
runner.run();
|
||||
|
||||
// should fail because of the semicolon
|
||||
runner.assertAllFlowFilesTransferred(PutHive3QL.REL_SUCCESS, 1);
|
||||
runner.getFlowFilesForRelationship(PutHive3QL.REL_SUCCESS)
|
||||
.forEach(f -> f.assertAttributeEquals(PutHive3QL.ATTR_OUTPUT_TABLES, "PERSONS"));
|
||||
|
||||
// Now we can check that the values were inserted by the multi-statement script.
|
||||
try (final Connection conn = service.getConnection()) {
|
||||
try (final Statement stmt = conn.createStatement()) {
|
||||
final ResultSet rs = stmt.executeQuery("SELECT * FROM PERSONS");
|
||||
assertTrue(rs.next());
|
||||
assertEquals("Record ID mismatch", 1, rs.getInt(1));
|
||||
assertEquals("Record NAME mismatch", "George", rs.getString(2));
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testMultipleStatementsWithinFlowFilePlusEmbeddedDelimiter() throws InitializationException, ProcessException, SQLException, IOException {
|
||||
final TestRunner runner = TestRunners.newTestRunner(PutHive3QL.class);
|
||||
final File tempDir = folder.getRoot();
|
||||
final File dbDir = new File(tempDir, "db");
|
||||
final DBCPService service = new MockDBCPService(dbDir.getAbsolutePath());
|
||||
runner.addControllerService("dbcp", service);
|
||||
runner.enableControllerService(service);
|
||||
|
||||
try (final Connection conn = service.getConnection()) {
|
||||
try (final Statement stmt = conn.createStatement()) {
|
||||
stmt.executeUpdate(createPersons);
|
||||
}
|
||||
}
|
||||
|
||||
runner.setProperty(PutHive3QL.HIVE_DBCP_SERVICE, "dbcp");
|
||||
|
||||
final String sql = "INSERT INTO PERSONS (ID, NAME, CODE) VALUES (?, ?, ?); " +
|
||||
"UPDATE PERSONS SET NAME='George\\;' WHERE ID=?; ";
|
||||
final Map<String, String> attributes = new HashMap<>();
|
||||
attributes.put("hiveql.args.1.type", String.valueOf(Types.INTEGER));
|
||||
attributes.put("hiveql.args.1.value", "1");
|
||||
|
||||
attributes.put("hiveql.args.2.type", String.valueOf(Types.VARCHAR));
|
||||
attributes.put("hiveql.args.2.value", "Mark");
|
||||
|
||||
attributes.put("hiveql.args.3.type", String.valueOf(Types.INTEGER));
|
||||
attributes.put("hiveql.args.3.value", "84");
|
||||
|
||||
attributes.put("hiveql.args.4.type", String.valueOf(Types.INTEGER));
|
||||
attributes.put("hiveql.args.4.value", "1");
|
||||
|
||||
runner.enqueue(sql.getBytes(), attributes);
|
||||
runner.run();
|
||||
|
||||
// should fail because of the semicolon
|
||||
runner.assertAllFlowFilesTransferred(PutHive3QL.REL_SUCCESS, 1);
|
||||
|
||||
// Now we can check that the values were inserted by the multi-statement script.
|
||||
try (final Connection conn = service.getConnection()) {
|
||||
try (final Statement stmt = conn.createStatement()) {
|
||||
final ResultSet rs = stmt.executeQuery("SELECT * FROM PERSONS");
|
||||
assertTrue(rs.next());
|
||||
assertEquals("Record ID mismatch", 1, rs.getInt(1));
|
||||
assertEquals("Record NAME mismatch", "George\\;", rs.getString(2));
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
@Test
|
||||
public void testWithNullParameter() throws InitializationException, ProcessException, SQLException, IOException {
|
||||
final TestRunner runner = TestRunners.newTestRunner(PutHive3QL.class);
|
||||
final File tempDir = folder.getRoot();
|
||||
final File dbDir = new File(tempDir, "db");
|
||||
final DBCPService service = new MockDBCPService(dbDir.getAbsolutePath());
|
||||
runner.addControllerService("dbcp", service);
|
||||
runner.enableControllerService(service);
|
||||
|
||||
try (final Connection conn = service.getConnection()) {
|
||||
try (final Statement stmt = conn.createStatement()) {
|
||||
stmt.executeUpdate(createPersons);
|
||||
}
|
||||
}
|
||||
|
||||
runner.setProperty(PutHive3QL.HIVE_DBCP_SERVICE, "dbcp");
|
||||
final Map<String, String> attributes = new HashMap<>();
|
||||
attributes.put("hiveql.args.1.type", String.valueOf(Types.INTEGER));
|
||||
attributes.put("hiveql.args.1.value", "1");
|
||||
|
||||
attributes.put("hiveql.args.2.type", String.valueOf(Types.VARCHAR));
|
||||
attributes.put("hiveql.args.2.value", "Mark");
|
||||
|
||||
attributes.put("hiveql.args.3.type", String.valueOf(Types.INTEGER));
|
||||
|
||||
runner.enqueue("INSERT INTO PERSONS (ID, NAME, CODE) VALUES (?, ?, ?)".getBytes(), attributes);
|
||||
runner.run();
|
||||
|
||||
runner.assertAllFlowFilesTransferred(PutHive3QL.REL_SUCCESS, 1);
|
||||
|
||||
try (final Connection conn = service.getConnection()) {
|
||||
try (final Statement stmt = conn.createStatement()) {
|
||||
final ResultSet rs = stmt.executeQuery("SELECT * FROM PERSONS");
|
||||
assertTrue(rs.next());
|
||||
assertEquals(1, rs.getInt(1));
|
||||
assertEquals("Mark", rs.getString(2));
|
||||
assertEquals(0, rs.getInt(3));
|
||||
assertFalse(rs.next());
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testInvalidStatement() throws InitializationException, ProcessException, SQLException, IOException {
|
||||
final TestRunner runner = TestRunners.newTestRunner(PutHive3QL.class);
|
||||
final File tempDir = folder.getRoot();
|
||||
final File dbDir = new File(tempDir, "db");
|
||||
final DBCPService service = new MockDBCPService(dbDir.getAbsolutePath());
|
||||
runner.addControllerService("dbcp", service);
|
||||
runner.enableControllerService(service);
|
||||
|
||||
try (final Connection conn = service.getConnection()) {
|
||||
try (final Statement stmt = conn.createStatement()) {
|
||||
stmt.executeUpdate(createPersons);
|
||||
}
|
||||
}
|
||||
|
||||
runner.setProperty(PutHive3QL.HIVE_DBCP_SERVICE, "dbcp");
|
||||
|
||||
final String sql = "INSERT INTO PERSONS (ID, NAME, CODE) VALUES (?, ?, ?); " +
|
||||
"UPDATE SOME_RANDOM_TABLE NAME='George' WHERE ID=?; ";
|
||||
final Map<String, String> attributes = new HashMap<>();
|
||||
attributes.put("hiveql.args.1.type", String.valueOf(Types.INTEGER));
|
||||
attributes.put("hiveql.args.1.value", "1");
|
||||
|
||||
attributes.put("hiveql.args.2.type", String.valueOf(Types.VARCHAR));
|
||||
attributes.put("hiveql.args.2.value", "Mark");
|
||||
|
||||
attributes.put("hiveql.args.3.type", String.valueOf(Types.INTEGER));
|
||||
attributes.put("hiveql.args.3.value", "84");
|
||||
|
||||
attributes.put("hiveql.args.4.type", String.valueOf(Types.INTEGER));
|
||||
attributes.put("hiveql.args.4.value", "1");
|
||||
|
||||
runner.enqueue(sql.getBytes(), attributes);
|
||||
runner.run();
|
||||
|
||||
// should fail because of the table is invalid
|
||||
runner.assertAllFlowFilesTransferred(PutHive3QL.REL_FAILURE, 1);
|
||||
|
||||
try (final Connection conn = service.getConnection()) {
|
||||
try (final Statement stmt = conn.createStatement()) {
|
||||
final ResultSet rs = stmt.executeQuery("SELECT * FROM PERSONS");
|
||||
assertTrue(rs.next());
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
@Test
|
||||
public void testRetryableFailure() throws InitializationException, ProcessException, SQLException, IOException {
|
||||
final TestRunner runner = TestRunners.newTestRunner(PutHive3QL.class);
|
||||
final DBCPService service = new SQLExceptionService(null);
|
||||
runner.addControllerService("dbcp", service);
|
||||
runner.enableControllerService(service);
|
||||
|
||||
runner.setProperty(PutHive3QL.HIVE_DBCP_SERVICE, "dbcp");
|
||||
|
||||
final String sql = "INSERT INTO PERSONS (ID, NAME, CODE) VALUES (?, ?, ?); " +
|
||||
"UPDATE PERSONS SET NAME='George' WHERE ID=?; ";
|
||||
|
||||
final Map<String, String> attributes = new HashMap<>();
|
||||
attributes.put("hiveql.args.1.type", String.valueOf(Types.INTEGER));
|
||||
attributes.put("hiveql.args.1.value", "1");
|
||||
|
||||
attributes.put("hiveql.args.2.type", String.valueOf(Types.VARCHAR));
|
||||
attributes.put("hiveql.args.2.value", "Mark");
|
||||
|
||||
attributes.put("hiveql.args.3.type", String.valueOf(Types.INTEGER));
|
||||
attributes.put("hiveql.args.3.value", "84");
|
||||
|
||||
attributes.put("hiveql.args.4.type", String.valueOf(Types.INTEGER));
|
||||
attributes.put("hiveql.args.4.value", "1");
|
||||
|
||||
runner.enqueue(sql.getBytes(), attributes);
|
||||
runner.run();
|
||||
|
||||
// should fail because there isn't a valid connection and tables don't exist.
|
||||
runner.assertAllFlowFilesTransferred(PutHive3QL.REL_RETRY, 1);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testRetryableFailureRollbackOnFailure() throws InitializationException, ProcessException, SQLException, IOException {
|
||||
final TestRunner runner = TestRunners.newTestRunner(PutHive3QL.class);
|
||||
final DBCPService service = new SQLExceptionService(null);
|
||||
runner.addControllerService("dbcp", service);
|
||||
runner.enableControllerService(service);
|
||||
|
||||
runner.setProperty(PutHive3QL.HIVE_DBCP_SERVICE, "dbcp");
|
||||
runner.setProperty(RollbackOnFailure.ROLLBACK_ON_FAILURE, "true");
|
||||
|
||||
final String sql = "INSERT INTO PERSONS (ID, NAME, CODE) VALUES (?, ?, ?); " +
|
||||
"UPDATE PERSONS SET NAME='George' WHERE ID=?; ";
|
||||
|
||||
final Map<String, String> attributes = new HashMap<>();
|
||||
attributes.put("hiveql.args.1.type", String.valueOf(Types.INTEGER));
|
||||
attributes.put("hiveql.args.1.value", "1");
|
||||
|
||||
attributes.put("hiveql.args.2.type", String.valueOf(Types.VARCHAR));
|
||||
attributes.put("hiveql.args.2.value", "Mark");
|
||||
|
||||
attributes.put("hiveql.args.3.type", String.valueOf(Types.INTEGER));
|
||||
attributes.put("hiveql.args.3.value", "84");
|
||||
|
||||
attributes.put("hiveql.args.4.type", String.valueOf(Types.INTEGER));
|
||||
attributes.put("hiveql.args.4.value", "1");
|
||||
|
||||
runner.enqueue(sql.getBytes(), attributes);
|
||||
try {
|
||||
runner.run();
|
||||
fail("Should throw ProcessException");
|
||||
} catch (AssertionError e) {
|
||||
assertTrue(e.getCause() instanceof ProcessException);
|
||||
}
|
||||
|
||||
assertEquals(1, runner.getQueueSize().getObjectCount());
|
||||
runner.assertAllFlowFilesTransferred(PutHive3QL.REL_RETRY, 0);
|
||||
}
|
||||
|
||||
/**
|
||||
* Simple implementation only for testing purposes
|
||||
*/
|
||||
private static class MockDBCPService extends AbstractControllerService implements Hive3DBCPService {
|
||||
private final String dbLocation;
|
||||
|
||||
MockDBCPService(final String dbLocation) {
|
||||
this.dbLocation = dbLocation;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String getIdentifier() {
|
||||
return "dbcp";
|
||||
}
|
||||
|
||||
@Override
|
||||
public Connection getConnection() throws ProcessException {
|
||||
try {
|
||||
Class.forName("org.apache.derby.jdbc.EmbeddedDriver");
|
||||
return DriverManager.getConnection("jdbc:derby:" + dbLocation + ";create=true");
|
||||
} catch (final Exception e) {
|
||||
e.printStackTrace();
|
||||
throw new ProcessException("getConnection failed: " + e);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public String getConnectionURL() {
|
||||
return "jdbc:derby:" + dbLocation + ";create=true";
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Simple implementation only for testing purposes
|
||||
*/
|
||||
private static class SQLExceptionService extends AbstractControllerService implements Hive3DBCPService {
|
||||
private final Hive3DBCPService service;
|
||||
private int allowedBeforeFailure = 0;
|
||||
private int successful = 0;
|
||||
|
||||
SQLExceptionService(final Hive3DBCPService service) {
|
||||
this.service = service;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String getIdentifier() {
|
||||
return "dbcp";
|
||||
}
|
||||
|
||||
@Override
|
||||
public Connection getConnection() throws ProcessException {
|
||||
try {
|
||||
if (++successful > allowedBeforeFailure) {
|
||||
final Connection conn = Mockito.mock(Connection.class);
|
||||
Mockito.when(conn.prepareStatement(Mockito.any(String.class))).thenThrow(new SQLException("Unit Test Generated SQLException"));
|
||||
return conn;
|
||||
} else {
|
||||
return service.getConnection();
|
||||
}
|
||||
} catch (final Exception e) {
|
||||
e.printStackTrace();
|
||||
throw new ProcessException("getConnection failed: " + e);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public String getConnectionURL() {
|
||||
return service != null ? service.getConnectionURL() : null;
|
||||
}
|
||||
}
|
||||
}
|
|
@ -0,0 +1,878 @@
|
|||
/*
|
||||
* 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.nifi.processors.hive;
|
||||
|
||||
import org.apache.avro.Schema;
|
||||
import org.apache.avro.file.DataFileStream;
|
||||
import org.apache.avro.file.DataFileWriter;
|
||||
import org.apache.avro.generic.GenericData;
|
||||
import org.apache.avro.generic.GenericDatumReader;
|
||||
import org.apache.avro.generic.GenericDatumWriter;
|
||||
import org.apache.avro.generic.GenericRecord;
|
||||
import org.apache.avro.io.DatumWriter;
|
||||
import org.apache.commons.io.FileUtils;
|
||||
import org.apache.commons.io.IOUtils;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.hive.conf.HiveConf;
|
||||
import org.apache.hadoop.hive.metastore.api.FieldSchema;
|
||||
import org.apache.hadoop.hive.metastore.api.StorageDescriptor;
|
||||
import org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat;
|
||||
import org.apache.hadoop.hive.ql.metadata.Table;
|
||||
import org.apache.hadoop.hive.serde.serdeConstants;
|
||||
import org.apache.hadoop.security.UserGroupInformation;
|
||||
import org.apache.hive.streaming.ConnectionStats;
|
||||
import org.apache.hive.streaming.HiveRecordWriter;
|
||||
import org.apache.hive.streaming.PartitionInfo;
|
||||
import org.apache.hive.streaming.RecordWriter;
|
||||
import org.apache.hive.streaming.StreamingConnection;
|
||||
import org.apache.hive.streaming.StreamingException;
|
||||
import org.apache.hive.streaming.StubConnectionError;
|
||||
import org.apache.hive.streaming.StubSerializationError;
|
||||
import org.apache.hive.streaming.StubStreamingIOFailure;
|
||||
import org.apache.hive.streaming.StubTransactionError;
|
||||
import org.apache.nifi.avro.AvroTypeUtil;
|
||||
import org.apache.nifi.components.PropertyDescriptor;
|
||||
import org.apache.nifi.components.ValidationContext;
|
||||
import org.apache.nifi.components.ValidationResult;
|
||||
import org.apache.nifi.controller.ControllerService;
|
||||
import org.apache.nifi.controller.ControllerServiceInitializationContext;
|
||||
import org.apache.nifi.hadoop.SecurityUtil;
|
||||
import org.apache.nifi.kerberos.KerberosCredentialsService;
|
||||
import org.apache.nifi.processor.exception.ProcessException;
|
||||
import org.apache.nifi.reporting.InitializationException;
|
||||
import org.apache.nifi.serialization.RecordReader;
|
||||
import org.apache.nifi.serialization.record.MockRecordParser;
|
||||
import org.apache.nifi.serialization.record.RecordField;
|
||||
import org.apache.nifi.serialization.record.RecordSchema;
|
||||
import org.apache.nifi.util.MockFlowFile;
|
||||
import org.apache.nifi.util.TestRunner;
|
||||
import org.apache.nifi.util.TestRunners;
|
||||
import org.apache.nifi.util.hive.HiveConfigurator;
|
||||
import org.apache.nifi.util.hive.HiveOptions;
|
||||
import org.junit.Before;
|
||||
import org.junit.Test;
|
||||
|
||||
import java.io.ByteArrayInputStream;
|
||||
import java.io.ByteArrayOutputStream;
|
||||
import java.io.File;
|
||||
import java.io.FileInputStream;
|
||||
import java.io.IOException;
|
||||
import java.io.InputStream;
|
||||
import java.nio.charset.StandardCharsets;
|
||||
import java.util.Arrays;
|
||||
import java.util.Collection;
|
||||
import java.util.Collections;
|
||||
import java.util.HashMap;
|
||||
import java.util.LinkedList;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.function.BiFunction;
|
||||
|
||||
import static org.apache.nifi.processors.hive.AbstractHive3QLProcessor.ATTR_OUTPUT_TABLES;
|
||||
import static org.apache.nifi.processors.hive.PutHive3Streaming.HIVE_STREAMING_RECORD_COUNT_ATTR;
|
||||
import static org.apache.nifi.processors.hive.PutHive3Streaming.KERBEROS_CREDENTIALS_SERVICE;
|
||||
import static org.junit.Assert.assertEquals;
|
||||
import static org.junit.Assert.assertFalse;
|
||||
import static org.junit.Assert.assertNotNull;
|
||||
import static org.junit.Assert.assertNull;
|
||||
import static org.junit.Assert.assertTrue;
|
||||
import static org.junit.Assert.fail;
|
||||
import static org.mockito.Matchers.anyString;
|
||||
import static org.mockito.Matchers.eq;
|
||||
import static org.mockito.Mockito.mock;
|
||||
import static org.mockito.Mockito.when;
|
||||
|
||||
/**
|
||||
* Unit tests for PutHive3Streaming processor.
|
||||
*/
|
||||
public class TestPutHive3Streaming {
|
||||
|
||||
private static final String TEST_CONF_PATH = "src/test/resources/core-site.xml";
|
||||
private static final String TARGET_HIVE = "target/hive";
|
||||
|
||||
private TestRunner runner;
|
||||
private MockPutHive3Streaming processor;
|
||||
|
||||
private HiveConfigurator hiveConfigurator;
|
||||
private HiveConf hiveConf;
|
||||
private UserGroupInformation ugi;
|
||||
private Schema schema;
|
||||
|
||||
@Before
|
||||
public void setUp() throws Exception {
|
||||
|
||||
final String avroSchema = IOUtils.toString(new FileInputStream("src/test/resources/user.avsc"), StandardCharsets.UTF_8);
|
||||
schema = new Schema.Parser().parse(avroSchema);
|
||||
|
||||
Configuration testConf = new Configuration();
|
||||
testConf.addResource(new Path(TEST_CONF_PATH));
|
||||
|
||||
// needed for calls to UserGroupInformation.setConfiguration() to work when passing in
|
||||
// config with Kerberos authentication enabled
|
||||
System.setProperty("java.security.krb5.realm", "nifi.com");
|
||||
System.setProperty("java.security.krb5.kdc", "nifi.kdc");
|
||||
|
||||
ugi = null;
|
||||
processor = new MockPutHive3Streaming();
|
||||
hiveConfigurator = mock(HiveConfigurator.class);
|
||||
hiveConf = new HiveConf();
|
||||
when(hiveConfigurator.getConfigurationFromFiles(anyString())).thenReturn(hiveConf);
|
||||
processor.hiveConfigurator = hiveConfigurator;
|
||||
|
||||
// Delete any temp files from previous tests
|
||||
try {
|
||||
FileUtils.deleteDirectory(new File(TARGET_HIVE));
|
||||
} catch (IOException ioe) {
|
||||
// Do nothing, directory may not have existed
|
||||
}
|
||||
}
|
||||
|
||||
private void configure(final PutHive3Streaming processor, final int numUsers) throws InitializationException {
|
||||
configure(processor, numUsers, -1);
|
||||
}
|
||||
|
||||
private void configure(final PutHive3Streaming processor, final int numUsers, int failAfter) throws InitializationException {
|
||||
configure(processor, numUsers, failAfter, null);
|
||||
}
|
||||
|
||||
private void configure(final PutHive3Streaming processor, final int numUsers, final int failAfter,
|
||||
final BiFunction<Integer, MockRecordParser, Void> recordGenerator) throws InitializationException {
|
||||
runner = TestRunners.newTestRunner(processor);
|
||||
runner.setProperty(PutHive3Streaming.HIVE_CONFIGURATION_RESOURCES, TEST_CONF_PATH);
|
||||
MockRecordParser readerFactory = new MockRecordParser();
|
||||
final RecordSchema recordSchema = AvroTypeUtil.createSchema(schema);
|
||||
for (final RecordField recordField : recordSchema.getFields()) {
|
||||
readerFactory.addSchemaField(recordField.getFieldName(), recordField.getDataType().getFieldType(), recordField.isNullable());
|
||||
}
|
||||
|
||||
if (recordGenerator == null) {
|
||||
for (int i = 0; i < numUsers; i++) {
|
||||
readerFactory.addRecord("name" + i, i, "blue" + i, i * 10.0);
|
||||
}
|
||||
} else {
|
||||
recordGenerator.apply(numUsers, readerFactory);
|
||||
}
|
||||
|
||||
readerFactory.failAfter(failAfter);
|
||||
|
||||
runner.addControllerService("mock-reader-factory", readerFactory);
|
||||
runner.enableControllerService(readerFactory);
|
||||
|
||||
runner.setProperty(PutHive3Streaming.RECORD_READER, "mock-reader-factory");
|
||||
}
|
||||
|
||||
private void configureComplex(final MockPutHive3Streaming processor, final int numUsers, final int failAfter,
|
||||
final BiFunction<Integer, MockRecordParser, Void> recordGenerator) throws IOException, InitializationException {
|
||||
final String avroSchema = IOUtils.toString(new FileInputStream("src/test/resources/array_of_records.avsc"), StandardCharsets.UTF_8);
|
||||
schema = new Schema.Parser().parse(avroSchema);
|
||||
processor.setFields(Arrays.asList(new FieldSchema("records",
|
||||
serdeConstants.LIST_TYPE_NAME + "<"
|
||||
+ serdeConstants.MAP_TYPE_NAME + "<"
|
||||
+ serdeConstants.STRING_TYPE_NAME + ","
|
||||
+ serdeConstants.STRING_TYPE_NAME + ">>", "")));
|
||||
runner = TestRunners.newTestRunner(processor);
|
||||
runner.setProperty(PutHive3Streaming.HIVE_CONFIGURATION_RESOURCES, TEST_CONF_PATH);
|
||||
MockRecordParser readerFactory = new MockRecordParser();
|
||||
final RecordSchema recordSchema = AvroTypeUtil.createSchema(schema);
|
||||
for (final RecordField recordField : recordSchema.getFields()) {
|
||||
readerFactory.addSchemaField(recordField.getFieldName(), recordField.getDataType().getFieldType(), recordField.isNullable());
|
||||
}
|
||||
|
||||
if (recordGenerator == null) {
|
||||
Object[] mapArray = new Object[numUsers];
|
||||
for (int i = 0; i < numUsers; i++) {
|
||||
final int x = i;
|
||||
Map<String, Object> map = new HashMap<String, Object>() {{
|
||||
put("name", "name" + x);
|
||||
put("age", x * 5);
|
||||
}};
|
||||
mapArray[i] = map;
|
||||
}
|
||||
readerFactory.addRecord((Object)mapArray);
|
||||
} else {
|
||||
recordGenerator.apply(numUsers, readerFactory);
|
||||
}
|
||||
|
||||
readerFactory.failAfter(failAfter);
|
||||
|
||||
runner.addControllerService("mock-reader-factory", readerFactory);
|
||||
runner.enableControllerService(readerFactory);
|
||||
|
||||
runner.setProperty(PutHive3Streaming.RECORD_READER, "mock-reader-factory");
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testSetup() throws Exception {
|
||||
configure(processor, 0);
|
||||
runner.assertNotValid();
|
||||
runner.setProperty(PutHive3Streaming.METASTORE_URI, "thrift://localhost:9083");
|
||||
runner.setProperty(PutHive3Streaming.DB_NAME, "default");
|
||||
runner.assertNotValid();
|
||||
runner.setProperty(PutHive3Streaming.TABLE_NAME, "users");
|
||||
runner.assertValid();
|
||||
runner.run();
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testUgiGetsCleared() throws Exception {
|
||||
configure(processor, 0);
|
||||
runner.setProperty(PutHive3Streaming.METASTORE_URI, "thrift://localhost:9083");
|
||||
runner.setProperty(PutHive3Streaming.DB_NAME, "default");
|
||||
runner.setProperty(PutHive3Streaming.TABLE_NAME, "users");
|
||||
processor.ugi = mock(UserGroupInformation.class);
|
||||
runner.run();
|
||||
assertNull(processor.ugi);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testUgiGetsSetIfSecure() throws Exception {
|
||||
configure(processor, 1);
|
||||
hiveConf.set(SecurityUtil.HADOOP_SECURITY_AUTHENTICATION, SecurityUtil.KERBEROS);
|
||||
KerberosCredentialsService kcs = new MockKerberosCredentialsService();
|
||||
runner.addControllerService("kcs", kcs);
|
||||
runner.setProperty(KERBEROS_CREDENTIALS_SERVICE, "kcs");
|
||||
runner.enableControllerService(kcs);
|
||||
ugi = mock(UserGroupInformation.class);
|
||||
when(hiveConfigurator.authenticate(eq(hiveConf), anyString(), anyString())).thenReturn(ugi);
|
||||
runner.setProperty(PutHive3Streaming.METASTORE_URI, "thrift://localhost:9083");
|
||||
runner.setProperty(PutHive3Streaming.DB_NAME, "default");
|
||||
runner.setProperty(PutHive3Streaming.TABLE_NAME, "users");
|
||||
runner.enqueue(new byte[0]);
|
||||
runner.run();
|
||||
}
|
||||
|
||||
@Test(expected = AssertionError.class)
|
||||
public void testSetupWithKerberosAuthFailed() throws Exception {
|
||||
configure(processor, 0);
|
||||
runner.setProperty(PutHive3Streaming.METASTORE_URI, "thrift://localhost:9083");
|
||||
runner.setProperty(PutHive3Streaming.DB_NAME, "default");
|
||||
runner.setProperty(PutHive3Streaming.TABLE_NAME, "users");
|
||||
runner.setProperty(PutHive3Streaming.HIVE_CONFIGURATION_RESOURCES, "src/test/resources/core-site-security.xml, src/test/resources/hive-site-security.xml");
|
||||
|
||||
hiveConf.set(SecurityUtil.HADOOP_SECURITY_AUTHENTICATION, SecurityUtil.KERBEROS);
|
||||
KerberosCredentialsService kcs = new MockKerberosCredentialsService(null, null);
|
||||
runner.addControllerService("kcs", kcs);
|
||||
runner.setProperty(KERBEROS_CREDENTIALS_SERVICE, "kcs");
|
||||
runner.enableControllerService(kcs);
|
||||
runner.assertNotValid();
|
||||
runner.run();
|
||||
}
|
||||
|
||||
@Test
|
||||
public void onTrigger() throws Exception {
|
||||
configure(processor, 1);
|
||||
runner.setProperty(PutHive3Streaming.METASTORE_URI, "thrift://localhost:9083");
|
||||
runner.setProperty(PutHive3Streaming.DB_NAME, "default");
|
||||
runner.setProperty(PutHive3Streaming.TABLE_NAME, "users");
|
||||
runner.enqueue(new byte[0]);
|
||||
runner.run();
|
||||
|
||||
runner.assertTransferCount(PutHive3Streaming.REL_SUCCESS, 1);
|
||||
final MockFlowFile flowFile = runner.getFlowFilesForRelationship(PutHive3Streaming.REL_SUCCESS).get(0);
|
||||
assertEquals("1", flowFile.getAttribute(HIVE_STREAMING_RECORD_COUNT_ATTR));
|
||||
assertEquals("default.users", flowFile.getAttribute(ATTR_OUTPUT_TABLES));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void onTriggerComplex() throws Exception {
|
||||
configureComplex(processor, 10, -1, null);
|
||||
runner.setProperty(PutHive3Streaming.METASTORE_URI, "thrift://localhost:9083");
|
||||
runner.setProperty(PutHive3Streaming.DB_NAME, "default");
|
||||
runner.setProperty(PutHive3Streaming.TABLE_NAME, "users");
|
||||
runner.enqueue(new byte[0]);
|
||||
runner.run();
|
||||
|
||||
runner.assertTransferCount(PutHive3Streaming.REL_SUCCESS, 1);
|
||||
final MockFlowFile flowFile = runner.getFlowFilesForRelationship(PutHive3Streaming.REL_SUCCESS).get(0);
|
||||
// Schema is an array of size 10, so only one record is output
|
||||
assertEquals("1", flowFile.getAttribute(HIVE_STREAMING_RECORD_COUNT_ATTR));
|
||||
assertEquals("default.users", flowFile.getAttribute(ATTR_OUTPUT_TABLES));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void onTriggerBadInput() throws Exception {
|
||||
configure(processor, 1, 0);
|
||||
runner.setProperty(PutHive3Streaming.METASTORE_URI, "thrift://localhost:9083");
|
||||
runner.setProperty(PutHive3Streaming.DB_NAME, "default");
|
||||
runner.setProperty(PutHive3Streaming.TABLE_NAME, "users");
|
||||
runner.enqueue("I am not an Avro record".getBytes());
|
||||
runner.run();
|
||||
|
||||
runner.assertTransferCount(PutHive3Streaming.REL_FAILURE, 1);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void onTriggerBadInputRollbackOnFailure() throws Exception {
|
||||
configure(processor, 1, 0);
|
||||
runner.setProperty(PutHive3Streaming.METASTORE_URI, "thrift://localhost:9083");
|
||||
runner.setProperty(PutHive3Streaming.DB_NAME, "default");
|
||||
runner.setProperty(PutHive3Streaming.TABLE_NAME, "users");
|
||||
|
||||
runner.setProperty(PutHive3Streaming.ROLLBACK_ON_FAILURE, "true");
|
||||
runner.enqueue("I am not an Avro record".getBytes());
|
||||
try {
|
||||
runner.run();
|
||||
fail("ProcessException should be thrown");
|
||||
} catch (AssertionError e) {
|
||||
assertTrue(e.getCause() instanceof ProcessException);
|
||||
}
|
||||
|
||||
runner.assertTransferCount(PutHive3Streaming.REL_FAILURE, 0);
|
||||
// Assert incoming FlowFile stays in input queue.
|
||||
assertEquals(1, runner.getQueueSize().getObjectCount());
|
||||
}
|
||||
|
||||
|
||||
@Test
|
||||
public void onTriggerMultipleRecordsSingleTransaction() throws Exception {
|
||||
configure(processor, 3);
|
||||
runner.setProperty(PutHive3Streaming.METASTORE_URI, "thrift://localhost:9083");
|
||||
runner.setProperty(PutHive3Streaming.DB_NAME, "default");
|
||||
runner.setProperty(PutHive3Streaming.TABLE_NAME, "users");
|
||||
Map<String, Object> user1 = new HashMap<String, Object>() {
|
||||
{
|
||||
put("name", "Joe");
|
||||
put("favorite_number", 146);
|
||||
}
|
||||
};
|
||||
Map<String, Object> user2 = new HashMap<String, Object>() {
|
||||
{
|
||||
put("name", "Mary");
|
||||
put("favorite_number", 42);
|
||||
}
|
||||
};
|
||||
Map<String, Object> user3 = new HashMap<String, Object>() {
|
||||
{
|
||||
put("name", "Matt");
|
||||
put("favorite_number", 3);
|
||||
}
|
||||
};
|
||||
final List<Map<String, Object>> users = Arrays.asList(user1, user2, user3);
|
||||
runner.enqueue(createAvroRecord(users));
|
||||
runner.run();
|
||||
|
||||
runner.assertTransferCount(PutHive3Streaming.REL_SUCCESS, 1);
|
||||
MockFlowFile resultFlowFile = runner.getFlowFilesForRelationship(PutHive3Streaming.REL_SUCCESS).get(0);
|
||||
assertOutputAvroRecords(users, resultFlowFile);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void onTriggerMultipleRecordsFailInMiddle() throws Exception {
|
||||
configure(processor, 4);
|
||||
runner.setProperty(PutHive3Streaming.METASTORE_URI, "thrift://localhost:9083");
|
||||
runner.setProperty(PutHive3Streaming.DB_NAME, "default");
|
||||
runner.setProperty(PutHive3Streaming.TABLE_NAME, "users");
|
||||
processor.setGenerateWriteFailure(true);
|
||||
runner.enqueue(new byte[0]);
|
||||
runner.run();
|
||||
|
||||
runner.assertTransferCount(PutHive3Streaming.REL_SUCCESS, 0);
|
||||
runner.assertTransferCount(PutHive3Streaming.REL_FAILURE, 1);
|
||||
runner.assertTransferCount(PutHive3Streaming.REL_RETRY, 0);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void onTriggerMultipleRecordsFailInMiddleRollbackOnFailure() throws Exception {
|
||||
configure(processor, 3);
|
||||
runner.setProperty(PutHive3Streaming.METASTORE_URI, "thrift://localhost:9083");
|
||||
runner.setProperty(PutHive3Streaming.DB_NAME, "default");
|
||||
runner.setProperty(PutHive3Streaming.TABLE_NAME, "users");
|
||||
runner.setProperty(PutHive3Streaming.ROLLBACK_ON_FAILURE, "true");
|
||||
processor.setGenerateWriteFailure(true);
|
||||
runner.enqueue(new byte[0]);
|
||||
try {
|
||||
runner.run();
|
||||
fail("ProcessException should be thrown, because any Hive Transaction is committed yet.");
|
||||
} catch (AssertionError e) {
|
||||
assertTrue(e.getCause() instanceof ProcessException);
|
||||
}
|
||||
|
||||
runner.assertTransferCount(PutHive3Streaming.REL_SUCCESS, 0);
|
||||
runner.assertTransferCount(PutHive3Streaming.REL_RETRY, 0);
|
||||
runner.assertTransferCount(PutHive3Streaming.REL_FAILURE, 0);
|
||||
// Assert incoming FlowFile stays in input queue.
|
||||
assertEquals(1, runner.getQueueSize().getObjectCount());
|
||||
}
|
||||
|
||||
private void assertOutputAvroRecords(List<Map<String, Object>> expectedRecords, MockFlowFile resultFlowFile) throws IOException {
|
||||
assertEquals(String.valueOf(expectedRecords.size()), resultFlowFile.getAttribute(PutHive3Streaming.HIVE_STREAMING_RECORD_COUNT_ATTR));
|
||||
|
||||
final DataFileStream<GenericRecord> reader = new DataFileStream<>(
|
||||
new ByteArrayInputStream(resultFlowFile.toByteArray()),
|
||||
new GenericDatumReader<>());
|
||||
|
||||
Schema schema = reader.getSchema();
|
||||
|
||||
// Verify that the schema is preserved
|
||||
assertEquals(schema, new Schema.Parser().parse(new File("src/test/resources/user.avsc")));
|
||||
|
||||
GenericRecord record = null;
|
||||
for (Map<String, Object> expectedRecord : expectedRecords) {
|
||||
assertTrue(reader.hasNext());
|
||||
record = reader.next(record);
|
||||
final String name = record.get("name").toString();
|
||||
final Integer favorite_number = (Integer) record.get("favorite_number");
|
||||
assertNotNull(name);
|
||||
assertNotNull(favorite_number);
|
||||
assertNull(record.get("favorite_color"));
|
||||
assertNull(record.get("scale"));
|
||||
|
||||
assertEquals(expectedRecord.get("name"), name);
|
||||
assertEquals(expectedRecord.get("favorite_number"), favorite_number);
|
||||
}
|
||||
assertFalse(reader.hasNext());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void onTriggerWithConnectFailure() throws Exception {
|
||||
configure(processor, 1);
|
||||
processor.setGenerateConnectFailure(true);
|
||||
runner.setProperty(PutHive3Streaming.METASTORE_URI, "thrift://localhost:9083");
|
||||
runner.setProperty(PutHive3Streaming.DB_NAME, "default");
|
||||
runner.setProperty(PutHive3Streaming.TABLE_NAME, "users");
|
||||
runner.enqueue(new byte[0]);
|
||||
try {
|
||||
runner.run();
|
||||
fail("ProcessException should be thrown");
|
||||
} catch (AssertionError e) {
|
||||
assertTrue(e.getCause() instanceof ProcessException);
|
||||
}
|
||||
runner.assertTransferCount(PutHive3Streaming.REL_RETRY, 0);
|
||||
runner.assertTransferCount(PutHive3Streaming.REL_FAILURE, 0);
|
||||
runner.assertTransferCount(PutHive3Streaming.REL_SUCCESS, 0);
|
||||
// Assert incoming FlowFile stays in input queue.
|
||||
assertEquals(1, runner.getQueueSize().getObjectCount());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void onTriggerWithConnectFailureRollbackOnFailure() throws Exception {
|
||||
configure(processor, 1);
|
||||
processor.setGenerateConnectFailure(true);
|
||||
runner.setProperty(PutHive3Streaming.METASTORE_URI, "thrift://localhost:9083");
|
||||
runner.setProperty(PutHive3Streaming.DB_NAME, "default");
|
||||
runner.setProperty(PutHive3Streaming.TABLE_NAME, "users");
|
||||
runner.setProperty(PutHive3Streaming.ROLLBACK_ON_FAILURE, "true");
|
||||
runner.enqueue(new byte[0]);
|
||||
try {
|
||||
runner.run();
|
||||
fail("ProcessException should be thrown");
|
||||
} catch (AssertionError e) {
|
||||
assertTrue(e.getCause() instanceof ProcessException);
|
||||
}
|
||||
|
||||
runner.assertTransferCount(PutHive3Streaming.REL_RETRY, 0);
|
||||
runner.assertTransferCount(PutHive3Streaming.REL_FAILURE, 0);
|
||||
runner.assertTransferCount(PutHive3Streaming.REL_SUCCESS, 0);
|
||||
// Assert incoming FlowFile stays in input queue.
|
||||
assertEquals(1, runner.getQueueSize().getObjectCount());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void onTriggerWithWriteFailure() throws Exception {
|
||||
configure(processor, 2);
|
||||
processor.setGenerateWriteFailure(true);
|
||||
runner.setProperty(PutHive3Streaming.METASTORE_URI, "thrift://localhost:9083");
|
||||
runner.setProperty(PutHive3Streaming.DB_NAME, "default");
|
||||
runner.setProperty(PutHive3Streaming.TABLE_NAME, "users");
|
||||
runner.enqueue(new byte[0]);
|
||||
runner.run();
|
||||
|
||||
runner.assertTransferCount(PutHive3Streaming.REL_FAILURE, 1);
|
||||
final MockFlowFile flowFile = runner.getFlowFilesForRelationship(PutHive3Streaming.REL_FAILURE).get(0);
|
||||
assertEquals("0", flowFile.getAttribute(HIVE_STREAMING_RECORD_COUNT_ATTR));
|
||||
assertEquals("default.users", flowFile.getAttribute(ATTR_OUTPUT_TABLES));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void onTriggerWithWriteFailureRollbackOnFailure() throws Exception {
|
||||
configure(processor, 2);
|
||||
processor.setGenerateWriteFailure(true);
|
||||
runner.setProperty(PutHive3Streaming.METASTORE_URI, "thrift://localhost:9083");
|
||||
runner.setProperty(PutHive3Streaming.DB_NAME, "default");
|
||||
runner.setProperty(PutHive3Streaming.TABLE_NAME, "users");
|
||||
runner.setProperty(PutHive3Streaming.ROLLBACK_ON_FAILURE, "true");
|
||||
Map<String, Object> user1 = new HashMap<String, Object>() {
|
||||
{
|
||||
put("name", "Joe");
|
||||
put("favorite_number", 146);
|
||||
}
|
||||
};
|
||||
Map<String, Object> user2 = new HashMap<String, Object>() {
|
||||
{
|
||||
put("name", "Mary");
|
||||
put("favorite_number", 42);
|
||||
}
|
||||
};
|
||||
runner.enqueue(createAvroRecord(Arrays.asList(user1, user2)));
|
||||
try {
|
||||
runner.run();
|
||||
fail("ProcessException should be thrown");
|
||||
} catch (AssertionError e) {
|
||||
assertTrue(e.getCause() instanceof ProcessException);
|
||||
}
|
||||
|
||||
runner.assertTransferCount(PutHive3Streaming.REL_FAILURE, 0);
|
||||
// Assert incoming FlowFile stays in input queue.
|
||||
assertEquals(1, runner.getQueueSize().getObjectCount());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void onTriggerWithSerializationError() throws Exception {
|
||||
configure(processor, 1);
|
||||
processor.setGenerateSerializationError(true);
|
||||
runner.setProperty(PutHive3Streaming.METASTORE_URI, "thrift://localhost:9083");
|
||||
runner.setProperty(PutHive3Streaming.DB_NAME, "default");
|
||||
runner.setProperty(PutHive3Streaming.TABLE_NAME, "users");
|
||||
Map<String, Object> user1 = new HashMap<String, Object>() {
|
||||
{
|
||||
put("name", "Joe");
|
||||
put("favorite_number", 146);
|
||||
}
|
||||
};
|
||||
runner.enqueue(createAvroRecord(Collections.singletonList(user1)));
|
||||
runner.run();
|
||||
|
||||
runner.assertTransferCount(PutHive3Streaming.REL_SUCCESS, 0);
|
||||
runner.assertTransferCount(PutHive3Streaming.REL_FAILURE, 1);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void onTriggerWithSerializationErrorRollbackOnFailure() throws Exception {
|
||||
configure(processor, 1);
|
||||
processor.setGenerateSerializationError(true);
|
||||
runner.setProperty(PutHive3Streaming.METASTORE_URI, "thrift://localhost:9083");
|
||||
runner.setProperty(PutHive3Streaming.DB_NAME, "default");
|
||||
runner.setProperty(PutHive3Streaming.TABLE_NAME, "users");
|
||||
runner.setProperty(PutHive3Streaming.ROLLBACK_ON_FAILURE, "true");
|
||||
Map<String, Object> user1 = new HashMap<String, Object>() {
|
||||
{
|
||||
put("name", "Joe");
|
||||
put("favorite_number", 146);
|
||||
}
|
||||
};
|
||||
runner.enqueue(createAvroRecord(Collections.singletonList(user1)));
|
||||
try {
|
||||
runner.run();
|
||||
fail("ProcessException should be thrown");
|
||||
} catch (AssertionError e) {
|
||||
assertTrue(e.getCause() instanceof ProcessException);
|
||||
}
|
||||
|
||||
runner.assertTransferCount(PutHive3Streaming.REL_SUCCESS, 0);
|
||||
runner.assertTransferCount(PutHive3Streaming.REL_FAILURE, 0);
|
||||
// Assert incoming FlowFile stays in input queue.
|
||||
assertEquals(1, runner.getQueueSize().getObjectCount());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void onTriggerWithCommitFailure() throws Exception {
|
||||
configure(processor, 1);
|
||||
processor.setGenerateCommitFailure(true);
|
||||
runner.setProperty(PutHive3Streaming.METASTORE_URI, "thrift://localhost:9083");
|
||||
runner.setProperty(PutHive3Streaming.DB_NAME, "default");
|
||||
runner.setProperty(PutHive3Streaming.TABLE_NAME, "users");
|
||||
runner.setProperty(PutHive3Streaming.ROLLBACK_ON_FAILURE, "false");
|
||||
runner.enqueue(new byte[0]);
|
||||
runner.run();
|
||||
|
||||
runner.assertTransferCount(PutHive3Streaming.REL_FAILURE, 0);
|
||||
runner.assertTransferCount(PutHive3Streaming.REL_SUCCESS, 0);
|
||||
runner.assertTransferCount(PutHive3Streaming.REL_RETRY, 1);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void onTriggerWithCommitFailureRollbackOnFailure() throws Exception {
|
||||
configure(processor, 1);
|
||||
processor.setGenerateCommitFailure(true);
|
||||
runner.setProperty(PutHive3Streaming.METASTORE_URI, "thrift://localhost:9083");
|
||||
runner.setProperty(PutHive3Streaming.DB_NAME, "default");
|
||||
runner.setProperty(PutHive3Streaming.TABLE_NAME, "users");
|
||||
runner.setProperty(PutHive3Streaming.ROLLBACK_ON_FAILURE, "true");
|
||||
runner.enqueue(new byte[0]);
|
||||
try {
|
||||
runner.run();
|
||||
fail("ProcessException should be thrown");
|
||||
} catch (AssertionError e) {
|
||||
assertTrue(e.getCause() instanceof ProcessException);
|
||||
}
|
||||
|
||||
runner.assertTransferCount(PutHive3Streaming.REL_FAILURE, 0);
|
||||
runner.assertTransferCount(PutHive3Streaming.REL_SUCCESS, 0);
|
||||
runner.assertTransferCount(PutHive3Streaming.REL_RETRY, 0);
|
||||
// Assert incoming FlowFile stays in input queue.
|
||||
assertEquals(1, runner.getQueueSize().getObjectCount());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void cleanup() {
|
||||
processor.cleanup();
|
||||
}
|
||||
|
||||
private byte[] createAvroRecord(List<Map<String, Object>> records) throws IOException {
|
||||
final Schema schema = new Schema.Parser().parse(new File("src/test/resources/user.avsc"));
|
||||
|
||||
List<GenericRecord> users = new LinkedList<>();
|
||||
for (Map<String, Object> record : records) {
|
||||
final GenericRecord user = new GenericData.Record(schema);
|
||||
user.put("name", record.get("name"));
|
||||
user.put("favorite_number", record.get("favorite_number"));
|
||||
user.put("favorite_color", record.get("favorite_color"));
|
||||
users.add(user);
|
||||
}
|
||||
final DatumWriter<GenericRecord> datumWriter = new GenericDatumWriter<>(schema);
|
||||
ByteArrayOutputStream out = new ByteArrayOutputStream();
|
||||
try (DataFileWriter<GenericRecord> dataFileWriter = new DataFileWriter<>(datumWriter)) {
|
||||
dataFileWriter.create(schema, out);
|
||||
for (final GenericRecord user : users) {
|
||||
dataFileWriter.append(user);
|
||||
}
|
||||
}
|
||||
return out.toByteArray();
|
||||
|
||||
}
|
||||
|
||||
private class MockPutHive3Streaming extends PutHive3Streaming {
|
||||
|
||||
private boolean generateConnectFailure = false;
|
||||
private boolean generateWriteFailure = false;
|
||||
private boolean generateSerializationError = false;
|
||||
private boolean generateCommitFailure = false;
|
||||
private List<FieldSchema> schema = Arrays.asList(
|
||||
new FieldSchema("name", serdeConstants.STRING_TYPE_NAME, ""),
|
||||
new FieldSchema("favorite_number", serdeConstants.INT_TYPE_NAME, ""),
|
||||
new FieldSchema("favorite_color", serdeConstants.STRING_TYPE_NAME, ""),
|
||||
new FieldSchema("scale", serdeConstants.DOUBLE_TYPE_NAME, "")
|
||||
);
|
||||
|
||||
@Override
|
||||
StreamingConnection makeStreamingConnection(HiveOptions options, RecordReader reader) throws StreamingException {
|
||||
|
||||
if (generateConnectFailure) {
|
||||
throw new StubConnectionError("Unit Test - Connection Error");
|
||||
}
|
||||
|
||||
HiveRecordWriter hiveRecordWriter = new HiveRecordWriter(reader, getLogger());
|
||||
MockHiveStreamingConnection hiveConnection = new MockHiveStreamingConnection(options, reader, hiveRecordWriter, schema);
|
||||
hiveConnection.setGenerateWriteFailure(generateWriteFailure);
|
||||
hiveConnection.setGenerateSerializationError(generateSerializationError);
|
||||
hiveConnection.setGenerateCommitFailure(generateCommitFailure);
|
||||
return hiveConnection;
|
||||
}
|
||||
|
||||
void setGenerateConnectFailure(boolean generateConnectFailure) {
|
||||
this.generateConnectFailure = generateConnectFailure;
|
||||
}
|
||||
|
||||
void setGenerateWriteFailure(boolean generateWriteFailure) {
|
||||
this.generateWriteFailure = generateWriteFailure;
|
||||
}
|
||||
|
||||
void setGenerateSerializationError(boolean generateSerializationError) {
|
||||
this.generateSerializationError = generateSerializationError;
|
||||
}
|
||||
|
||||
void setGenerateCommitFailure(boolean generateCommitFailure) {
|
||||
this.generateCommitFailure = generateCommitFailure;
|
||||
}
|
||||
|
||||
void setFields(List<FieldSchema> schema) {
|
||||
this.schema = schema;
|
||||
}
|
||||
}
|
||||
|
||||
private class MockHiveStreamingConnection implements StreamingConnection {
|
||||
|
||||
private boolean generateWriteFailure = false;
|
||||
private boolean generateSerializationError = false;
|
||||
private boolean generateCommitFailure = false;
|
||||
private int writeAttemptCount = 0;
|
||||
private ConnectionStats connectionStats;
|
||||
private HiveOptions options;
|
||||
private RecordWriter writer;
|
||||
private HiveConf hiveConf;
|
||||
private Table table;
|
||||
private String metastoreURI;
|
||||
|
||||
MockHiveStreamingConnection(HiveOptions options, RecordReader reader, RecordWriter recordWriter, List<FieldSchema> schema) {
|
||||
this.options = options;
|
||||
metastoreURI = options.getMetaStoreURI();
|
||||
this.writer = recordWriter;
|
||||
this.hiveConf = this.options.getHiveConf();
|
||||
connectionStats = new ConnectionStats();
|
||||
this.table = new Table(Table.getEmptyTable(options.getDatabaseName(), options.getTableName()));
|
||||
this.table.setFields(schema);
|
||||
StorageDescriptor sd = this.table.getSd();
|
||||
sd.setOutputFormat(OrcOutputFormat.class.getName());
|
||||
sd.setLocation(TARGET_HIVE);
|
||||
}
|
||||
|
||||
@Override
|
||||
public HiveConf getHiveConf() {
|
||||
return hiveConf;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void beginTransaction() throws StreamingException {
|
||||
writer.init(this, 0, 100);
|
||||
}
|
||||
|
||||
@Override
|
||||
public synchronized void write(byte[] record) throws StreamingException {
|
||||
throw new UnsupportedOperationException(this.getClass().getName() + " does not support writing of records via bytes, only via an InputStream");
|
||||
}
|
||||
|
||||
@Override
|
||||
public void write(InputStream inputStream) throws StreamingException {
|
||||
try {
|
||||
if (generateWriteFailure) {
|
||||
throw new StubStreamingIOFailure("Unit Test - Streaming IO Failure");
|
||||
}
|
||||
if (generateSerializationError) {
|
||||
throw new StubSerializationError("Unit Test - Serialization error", new Exception());
|
||||
}
|
||||
this.writer.write(writeAttemptCount, inputStream);
|
||||
} finally {
|
||||
writeAttemptCount++;
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void commitTransaction() throws StreamingException {
|
||||
if (generateCommitFailure) {
|
||||
throw new StubTransactionError("Unit Test - Commit Failure");
|
||||
}
|
||||
connectionStats.incrementCommittedTransactions();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void abortTransaction() throws StreamingException {
|
||||
connectionStats.incrementAbortedTransactions();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void close() {
|
||||
// closing the connection shouldn't throw an exception
|
||||
}
|
||||
|
||||
@Override
|
||||
public ConnectionStats getConnectionStats() {
|
||||
return connectionStats;
|
||||
}
|
||||
|
||||
public void setGenerateWriteFailure(boolean generateWriteFailure) {
|
||||
this.generateWriteFailure = generateWriteFailure;
|
||||
}
|
||||
|
||||
public void setGenerateSerializationError(boolean generateSerializationError) {
|
||||
this.generateSerializationError = generateSerializationError;
|
||||
}
|
||||
|
||||
public void setGenerateCommitFailure(boolean generateCommitFailure) {
|
||||
this.generateCommitFailure = generateCommitFailure;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String getMetastoreUri() {
|
||||
return metastoreURI;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Table getTable() {
|
||||
return table;
|
||||
}
|
||||
|
||||
@Override
|
||||
public List<String> getStaticPartitionValues() {
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean isPartitionedTable() {
|
||||
return false;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean isDynamicPartitioning() {
|
||||
return false;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String getAgentInfo() {
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public PartitionInfo createPartitionIfNotExists(List<String> list) throws StreamingException {
|
||||
return null;
|
||||
}
|
||||
}
|
||||
|
||||
private static class MockKerberosCredentialsService implements KerberosCredentialsService, ControllerService {
|
||||
|
||||
private String keytab = "src/test/resources/fake.keytab";
|
||||
private String principal = "test@REALM.COM";
|
||||
|
||||
public MockKerberosCredentialsService() {
|
||||
}
|
||||
|
||||
public MockKerberosCredentialsService(String keytab, String principal) {
|
||||
this.keytab = keytab;
|
||||
this.principal = principal;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String getKeytab() {
|
||||
return keytab;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String getPrincipal() {
|
||||
return principal;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void initialize(ControllerServiceInitializationContext context) throws InitializationException {
|
||||
|
||||
}
|
||||
|
||||
@Override
|
||||
public Collection<ValidationResult> validate(ValidationContext context) {
|
||||
return Collections.EMPTY_LIST;
|
||||
}
|
||||
|
||||
@Override
|
||||
public PropertyDescriptor getPropertyDescriptor(String name) {
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void onPropertyModified(PropertyDescriptor descriptor, String oldValue, String newValue) {
|
||||
|
||||
}
|
||||
|
||||
@Override
|
||||
public List<PropertyDescriptor> getPropertyDescriptors() {
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String getIdentifier() {
|
||||
return "kcs";
|
||||
}
|
||||
}
|
||||
}
|
|
@ -0,0 +1,539 @@
|
|||
/*
|
||||
* 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.nifi.processors.hive;
|
||||
|
||||
import org.apache.avro.file.DataFileStream;
|
||||
import org.apache.avro.generic.GenericDatumReader;
|
||||
import org.apache.avro.generic.GenericRecord;
|
||||
import org.apache.avro.io.DatumReader;
|
||||
import org.apache.nifi.controller.AbstractControllerService;
|
||||
import org.apache.nifi.dbcp.DBCPService;
|
||||
import org.apache.nifi.dbcp.hive.Hive3DBCPService;
|
||||
import org.apache.nifi.flowfile.attributes.CoreAttributes;
|
||||
import org.apache.nifi.processor.exception.ProcessException;
|
||||
import org.apache.nifi.reporting.InitializationException;
|
||||
import org.apache.nifi.util.MockFlowFile;
|
||||
import org.apache.nifi.util.TestRunner;
|
||||
import org.apache.nifi.util.TestRunners;
|
||||
import org.apache.nifi.util.hive.HiveJdbcCommon;
|
||||
import org.junit.Before;
|
||||
import org.junit.BeforeClass;
|
||||
import org.junit.Test;
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
||||
import java.io.BufferedReader;
|
||||
import java.io.ByteArrayInputStream;
|
||||
import java.io.File;
|
||||
import java.io.IOException;
|
||||
import java.io.InputStream;
|
||||
import java.io.InputStreamReader;
|
||||
import java.sql.Connection;
|
||||
import java.sql.DriverManager;
|
||||
import java.sql.SQLException;
|
||||
import java.sql.Statement;
|
||||
import java.sql.Types;
|
||||
import java.util.HashMap;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Random;
|
||||
|
||||
import static org.apache.nifi.processors.hive.SelectHive3QL.HIVEQL_OUTPUT_FORMAT;
|
||||
import static org.apache.nifi.util.hive.HiveJdbcCommon.AVRO;
|
||||
import static org.apache.nifi.util.hive.HiveJdbcCommon.CSV;
|
||||
import static org.apache.nifi.util.hive.HiveJdbcCommon.CSV_MIME_TYPE;
|
||||
import static org.apache.nifi.util.hive.HiveJdbcCommon.MIME_TYPE_AVRO_BINARY;
|
||||
import static org.junit.Assert.assertEquals;
|
||||
import static org.junit.Assert.assertTrue;
|
||||
|
||||
public class TestSelectHive3QL {
|
||||
|
||||
private static final Logger LOGGER;
|
||||
private final static String MAX_ROWS_KEY = "maxRows";
|
||||
|
||||
static {
|
||||
System.setProperty("org.slf4j.simpleLogger.defaultLogLevel", "info");
|
||||
System.setProperty("org.slf4j.simpleLogger.showDateTime", "true");
|
||||
System.setProperty("org.slf4j.simpleLogger.log.nifi.io.nio", "debug");
|
||||
System.setProperty("org.slf4j.simpleLogger.log.nifi.processors.hive.SelectHive3QL", "debug");
|
||||
System.setProperty("org.slf4j.simpleLogger.log.nifi.processors.hive.TestSelectHive3QL", "debug");
|
||||
LOGGER = LoggerFactory.getLogger(TestSelectHive3QL.class);
|
||||
}
|
||||
|
||||
private final static String DB_LOCATION = "target/db";
|
||||
|
||||
private final static String QUERY_WITH_EL = "select "
|
||||
+ " PER.ID as PersonId, PER.NAME as PersonName, PER.CODE as PersonCode"
|
||||
+ " from persons PER"
|
||||
+ " where PER.ID > ${person.id}";
|
||||
|
||||
private final static String QUERY_WITHOUT_EL = "select "
|
||||
+ " PER.ID as PersonId, PER.NAME as PersonName, PER.CODE as PersonCode"
|
||||
+ " from persons PER"
|
||||
+ " where PER.ID > 10";
|
||||
|
||||
|
||||
@BeforeClass
|
||||
public static void setupClass() {
|
||||
System.setProperty("derby.stream.error.file", "target/derby.log");
|
||||
}
|
||||
|
||||
private TestRunner runner;
|
||||
|
||||
@Before
|
||||
public void setup() throws InitializationException {
|
||||
final DBCPService dbcp = new DBCPServiceSimpleImpl();
|
||||
final Map<String, String> dbcpProperties = new HashMap<>();
|
||||
|
||||
runner = TestRunners.newTestRunner(SelectHive3QL.class);
|
||||
runner.addControllerService("dbcp", dbcp, dbcpProperties);
|
||||
runner.enableControllerService(dbcp);
|
||||
runner.setProperty(SelectHive3QL.HIVE_DBCP_SERVICE, "dbcp");
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testIncomingConnectionWithNoFlowFile() throws InitializationException {
|
||||
runner.setIncomingConnection(true);
|
||||
runner.setProperty(SelectHive3QL.HIVEQL_SELECT_QUERY, "SELECT * FROM persons");
|
||||
runner.run();
|
||||
runner.assertTransferCount(SelectHive3QL.REL_SUCCESS, 0);
|
||||
runner.assertTransferCount(SelectHive3QL.REL_FAILURE, 0);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testNoIncomingConnection() throws ClassNotFoundException, SQLException, InitializationException, IOException {
|
||||
runner.setIncomingConnection(false);
|
||||
invokeOnTrigger(QUERY_WITHOUT_EL, false, "Avro");
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testNoTimeLimit() throws InitializationException, ClassNotFoundException, SQLException, IOException {
|
||||
invokeOnTrigger(QUERY_WITH_EL, true, "Avro");
|
||||
}
|
||||
|
||||
|
||||
@Test
|
||||
public void testWithNullIntColumn() throws SQLException {
|
||||
// remove previous test database, if any
|
||||
final File dbLocation = new File(DB_LOCATION);
|
||||
dbLocation.delete();
|
||||
|
||||
// load test data to database
|
||||
final Connection con = ((Hive3DBCPService) runner.getControllerService("dbcp")).getConnection();
|
||||
Statement stmt = con.createStatement();
|
||||
|
||||
try {
|
||||
stmt.execute("drop table TEST_NULL_INT");
|
||||
} catch (final SQLException sqle) {
|
||||
// Nothing to do, probably means the table didn't exist
|
||||
}
|
||||
|
||||
stmt.execute("create table TEST_NULL_INT (id integer not null, val1 integer, val2 integer, constraint my_pk primary key (id))");
|
||||
|
||||
stmt.execute("insert into TEST_NULL_INT (id, val1, val2) VALUES (0, NULL, 1)");
|
||||
stmt.execute("insert into TEST_NULL_INT (id, val1, val2) VALUES (1, 1, 1)");
|
||||
|
||||
runner.setIncomingConnection(false);
|
||||
runner.setProperty(SelectHive3QL.HIVEQL_SELECT_QUERY, "SELECT * FROM TEST_NULL_INT");
|
||||
runner.run();
|
||||
|
||||
runner.assertAllFlowFilesTransferred(SelectHive3QL.REL_SUCCESS, 1);
|
||||
runner.getFlowFilesForRelationship(SelectHive3QL.REL_SUCCESS).get(0).assertAttributeEquals(SelectHive3QL.RESULT_ROW_COUNT, "2");
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testWithSqlException() throws SQLException {
|
||||
// remove previous test database, if any
|
||||
final File dbLocation = new File(DB_LOCATION);
|
||||
dbLocation.delete();
|
||||
|
||||
// load test data to database
|
||||
final Connection con = ((Hive3DBCPService) runner.getControllerService("dbcp")).getConnection();
|
||||
Statement stmt = con.createStatement();
|
||||
|
||||
try {
|
||||
stmt.execute("drop table TEST_NO_ROWS");
|
||||
} catch (final SQLException sqle) {
|
||||
// Nothing to do, probably means the table didn't exist
|
||||
}
|
||||
|
||||
stmt.execute("create table TEST_NO_ROWS (id integer)");
|
||||
|
||||
runner.setIncomingConnection(false);
|
||||
// Try a valid SQL statement that will generate an error (val1 does not exist, e.g.)
|
||||
runner.setProperty(SelectHive3QL.HIVEQL_SELECT_QUERY, "SELECT val1 FROM TEST_NO_ROWS");
|
||||
runner.run();
|
||||
|
||||
runner.assertAllFlowFilesTransferred(SelectHive3QL.REL_FAILURE, 1);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testWithBadSQL() throws SQLException {
|
||||
final String BAD_SQL = "create table TEST_NO_ROWS (id integer)";
|
||||
|
||||
// Test with incoming flow file (it should be routed to failure intact, i.e. same content and no parent)
|
||||
runner.setIncomingConnection(true);
|
||||
// Try a valid SQL statement that will generate an error (val1 does not exist, e.g.)
|
||||
runner.enqueue(BAD_SQL);
|
||||
runner.run();
|
||||
runner.assertAllFlowFilesTransferred(SelectHive3QL.REL_FAILURE, 1);
|
||||
MockFlowFile flowFile = runner.getFlowFilesForRelationship(SelectHive3QL.REL_FAILURE).get(0);
|
||||
flowFile.assertContentEquals(BAD_SQL);
|
||||
flowFile.assertAttributeEquals("parentIds", null);
|
||||
runner.clearTransferState();
|
||||
|
||||
// Test with no incoming flow file (an empty flow file is transferred)
|
||||
runner.setIncomingConnection(false);
|
||||
// Try a valid SQL statement that will generate an error (val1 does not exist, e.g.)
|
||||
runner.setProperty(SelectHive3QL.HIVEQL_SELECT_QUERY, BAD_SQL);
|
||||
runner.run();
|
||||
runner.assertAllFlowFilesTransferred(SelectHive3QL.REL_FAILURE, 1);
|
||||
flowFile = runner.getFlowFilesForRelationship(SelectHive3QL.REL_FAILURE).get(0);
|
||||
flowFile.assertContentEquals("");
|
||||
}
|
||||
|
||||
@Test
|
||||
public void invokeOnTriggerWithCsv()
|
||||
throws InitializationException, ClassNotFoundException, SQLException, IOException {
|
||||
invokeOnTrigger(QUERY_WITHOUT_EL, false, CSV);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void invokeOnTriggerWithAvro()
|
||||
throws InitializationException, ClassNotFoundException, SQLException, IOException {
|
||||
invokeOnTrigger(QUERY_WITHOUT_EL, false, AVRO);
|
||||
}
|
||||
|
||||
public void invokeOnTrigger(final String query, final boolean incomingFlowFile, String outputFormat)
|
||||
throws InitializationException, ClassNotFoundException, SQLException, IOException {
|
||||
|
||||
// remove previous test database, if any
|
||||
final File dbLocation = new File(DB_LOCATION);
|
||||
dbLocation.delete();
|
||||
|
||||
// load test data to database
|
||||
final Connection con = ((Hive3DBCPService) runner.getControllerService("dbcp")).getConnection();
|
||||
final Statement stmt = con.createStatement();
|
||||
try {
|
||||
stmt.execute("drop table persons");
|
||||
} catch (final SQLException sqle) {
|
||||
// Nothing to do here, the table didn't exist
|
||||
}
|
||||
|
||||
stmt.execute("create table persons (id integer, name varchar(100), code integer)");
|
||||
Random rng = new Random(53496);
|
||||
final int nrOfRows = 100;
|
||||
stmt.executeUpdate("insert into persons values (1, 'Joe Smith', " + rng.nextInt(469947) + ")");
|
||||
for (int i = 2; i < nrOfRows; i++) {
|
||||
stmt.executeUpdate("insert into persons values (" + i + ", 'Someone Else', " + rng.nextInt(469947) + ")");
|
||||
}
|
||||
stmt.executeUpdate("insert into persons values (" + nrOfRows + ", 'Last Person', NULL)");
|
||||
|
||||
LOGGER.info("test data loaded");
|
||||
|
||||
runner.setProperty(SelectHive3QL.HIVEQL_SELECT_QUERY, query);
|
||||
runner.setProperty(HIVEQL_OUTPUT_FORMAT, outputFormat);
|
||||
|
||||
if (incomingFlowFile) {
|
||||
// incoming FlowFile content is not used, but attributes are used
|
||||
final Map<String, String> attributes = new HashMap<>();
|
||||
attributes.put("person.id", "10");
|
||||
runner.enqueue("Hello".getBytes(), attributes);
|
||||
}
|
||||
|
||||
runner.setIncomingConnection(incomingFlowFile);
|
||||
runner.run();
|
||||
runner.assertAllFlowFilesTransferred(SelectHive3QL.REL_SUCCESS, 1);
|
||||
|
||||
final List<MockFlowFile> flowfiles = runner.getFlowFilesForRelationship(SelectHive3QL.REL_SUCCESS);
|
||||
MockFlowFile flowFile = flowfiles.get(0);
|
||||
final InputStream in = new ByteArrayInputStream(flowFile.toByteArray());
|
||||
long recordsFromStream = 0;
|
||||
if (AVRO.equals(outputFormat)) {
|
||||
assertEquals(MIME_TYPE_AVRO_BINARY, flowFile.getAttribute(CoreAttributes.MIME_TYPE.key()));
|
||||
final DatumReader<GenericRecord> datumReader = new GenericDatumReader<>();
|
||||
try (DataFileStream<GenericRecord> dataFileReader = new DataFileStream<>(in, datumReader)) {
|
||||
GenericRecord record = null;
|
||||
while (dataFileReader.hasNext()) {
|
||||
// Reuse record object by passing it to next(). This saves us from
|
||||
// allocating and garbage collecting many objects for files with
|
||||
// many items.
|
||||
record = dataFileReader.next(record);
|
||||
recordsFromStream++;
|
||||
}
|
||||
}
|
||||
} else {
|
||||
assertEquals(CSV_MIME_TYPE, flowFile.getAttribute(CoreAttributes.MIME_TYPE.key()));
|
||||
BufferedReader br = new BufferedReader(new InputStreamReader(in));
|
||||
|
||||
String headerRow = br.readLine();
|
||||
// Derby capitalizes column names
|
||||
assertEquals("PERSONID,PERSONNAME,PERSONCODE", headerRow);
|
||||
|
||||
// Validate rows
|
||||
String line;
|
||||
while ((line = br.readLine()) != null) {
|
||||
recordsFromStream++;
|
||||
String[] values = line.split(",");
|
||||
if (recordsFromStream < (nrOfRows - 10)) {
|
||||
assertEquals(3, values.length);
|
||||
assertTrue(values[1].startsWith("\""));
|
||||
assertTrue(values[1].endsWith("\""));
|
||||
} else {
|
||||
assertEquals(2, values.length); // Middle value is null
|
||||
}
|
||||
}
|
||||
}
|
||||
assertEquals(nrOfRows - 10, recordsFromStream);
|
||||
assertEquals(recordsFromStream, Integer.parseInt(flowFile.getAttribute(SelectHive3QL.RESULT_ROW_COUNT)));
|
||||
flowFile.assertAttributeEquals(AbstractHive3QLProcessor.ATTR_INPUT_TABLES, "persons");
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testMaxRowsPerFlowFileAvro() throws ClassNotFoundException, SQLException, InitializationException, IOException {
|
||||
|
||||
// load test data to database
|
||||
final Connection con = ((DBCPService) runner.getControllerService("dbcp")).getConnection();
|
||||
Statement stmt = con.createStatement();
|
||||
InputStream in;
|
||||
MockFlowFile mff;
|
||||
|
||||
try {
|
||||
stmt.execute("drop table TEST_QUERY_DB_TABLE");
|
||||
} catch (final SQLException sqle) {
|
||||
// Ignore this error, probably a "table does not exist" since Derby doesn't yet support DROP IF EXISTS [DERBY-4842]
|
||||
}
|
||||
|
||||
stmt.execute("create table TEST_QUERY_DB_TABLE (id integer not null, name varchar(100), scale float, created_on timestamp, bignum bigint default 0)");
|
||||
int rowCount = 0;
|
||||
//create larger row set
|
||||
for (int batch = 0; batch < 100; batch++) {
|
||||
stmt.execute("insert into TEST_QUERY_DB_TABLE (id, name, scale, created_on) VALUES (" + rowCount + ", 'Joe Smith', 1.0, '1962-09-23 03:23:34.234')");
|
||||
rowCount++;
|
||||
}
|
||||
|
||||
runner.setIncomingConnection(false);
|
||||
runner.setProperty(SelectHive3QL.HIVEQL_SELECT_QUERY, "SELECT * FROM TEST_QUERY_DB_TABLE");
|
||||
runner.setProperty(SelectHive3QL.MAX_ROWS_PER_FLOW_FILE, "${" + MAX_ROWS_KEY + "}");
|
||||
runner.setProperty(SelectHive3QL.HIVEQL_OUTPUT_FORMAT, HiveJdbcCommon.AVRO);
|
||||
runner.setVariable(MAX_ROWS_KEY, "9");
|
||||
|
||||
runner.run();
|
||||
runner.assertAllFlowFilesTransferred(SelectHive3QL.REL_SUCCESS, 12);
|
||||
|
||||
//ensure all but the last file have 9 records each
|
||||
for (int ff = 0; ff < 11; ff++) {
|
||||
mff = runner.getFlowFilesForRelationship(SelectHive3QL.REL_SUCCESS).get(ff);
|
||||
in = new ByteArrayInputStream(mff.toByteArray());
|
||||
assertEquals(9, getNumberOfRecordsFromStream(in));
|
||||
|
||||
mff.assertAttributeExists("fragment.identifier");
|
||||
assertEquals(Integer.toString(ff), mff.getAttribute("fragment.index"));
|
||||
assertEquals("12", mff.getAttribute("fragment.count"));
|
||||
}
|
||||
|
||||
//last file should have 1 record
|
||||
mff = runner.getFlowFilesForRelationship(SelectHive3QL.REL_SUCCESS).get(11);
|
||||
in = new ByteArrayInputStream(mff.toByteArray());
|
||||
assertEquals(1, getNumberOfRecordsFromStream(in));
|
||||
mff.assertAttributeExists("fragment.identifier");
|
||||
assertEquals(Integer.toString(11), mff.getAttribute("fragment.index"));
|
||||
assertEquals("12", mff.getAttribute("fragment.count"));
|
||||
runner.clearTransferState();
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testParametrizedQuery() throws ClassNotFoundException, SQLException, InitializationException, IOException {
|
||||
// load test data to database
|
||||
final Connection con = ((DBCPService) runner.getControllerService("dbcp")).getConnection();
|
||||
Statement stmt = con.createStatement();
|
||||
|
||||
try {
|
||||
stmt.execute("drop table TEST_QUERY_DB_TABLE");
|
||||
} catch (final SQLException sqle) {
|
||||
// Ignore this error, probably a "table does not exist" since Derby doesn't yet support DROP IF EXISTS [DERBY-4842]
|
||||
}
|
||||
|
||||
stmt.execute("create table TEST_QUERY_DB_TABLE (id integer not null, name varchar(100), scale float, created_on timestamp, bignum bigint default 0)");
|
||||
int rowCount = 0;
|
||||
//create larger row set
|
||||
for (int batch = 0; batch < 100; batch++) {
|
||||
stmt.execute("insert into TEST_QUERY_DB_TABLE (id, name, scale, created_on) VALUES (" + rowCount + ", 'Joe Smith', 1.0, '1962-09-23 03:23:34.234')");
|
||||
rowCount++;
|
||||
}
|
||||
|
||||
runner.setIncomingConnection(true);
|
||||
runner.setProperty(SelectHive3QL.MAX_ROWS_PER_FLOW_FILE, "${" + MAX_ROWS_KEY + "}");
|
||||
runner.setProperty(SelectHive3QL.HIVEQL_OUTPUT_FORMAT, HiveJdbcCommon.AVRO);
|
||||
runner.setVariable(MAX_ROWS_KEY, "9");
|
||||
|
||||
Map<String, String> attributes = new HashMap<String, String>();
|
||||
attributes.put("hiveql.args.1.value", "1");
|
||||
attributes.put("hiveql.args.1.type", String.valueOf(Types.INTEGER));
|
||||
runner.enqueue("SELECT * FROM TEST_QUERY_DB_TABLE WHERE id = ?", attributes );
|
||||
|
||||
runner.run();
|
||||
runner.assertAllFlowFilesTransferred(SelectHive3QL.REL_SUCCESS, 1);
|
||||
runner.clearTransferState();
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testMaxRowsPerFlowFileCSV() throws ClassNotFoundException, SQLException, InitializationException, IOException {
|
||||
|
||||
// load test data to database
|
||||
final Connection con = ((DBCPService) runner.getControllerService("dbcp")).getConnection();
|
||||
Statement stmt = con.createStatement();
|
||||
InputStream in;
|
||||
MockFlowFile mff;
|
||||
|
||||
try {
|
||||
stmt.execute("drop table TEST_QUERY_DB_TABLE");
|
||||
} catch (final SQLException sqle) {
|
||||
// Ignore this error, probably a "table does not exist" since Derby doesn't yet support DROP IF EXISTS [DERBY-4842]
|
||||
}
|
||||
|
||||
stmt.execute("create table TEST_QUERY_DB_TABLE (id integer not null, name varchar(100), scale float, created_on timestamp, bignum bigint default 0)");
|
||||
int rowCount = 0;
|
||||
//create larger row set
|
||||
for (int batch = 0; batch < 100; batch++) {
|
||||
stmt.execute("insert into TEST_QUERY_DB_TABLE (id, name, scale, created_on) VALUES (" + rowCount + ", 'Joe Smith', 1.0, '1962-09-23 03:23:34.234')");
|
||||
rowCount++;
|
||||
}
|
||||
|
||||
runner.setIncomingConnection(true);
|
||||
runner.setProperty(SelectHive3QL.MAX_ROWS_PER_FLOW_FILE, "${" + MAX_ROWS_KEY + "}");
|
||||
runner.setProperty(SelectHive3QL.HIVEQL_OUTPUT_FORMAT, HiveJdbcCommon.CSV);
|
||||
|
||||
runner.enqueue("SELECT * FROM TEST_QUERY_DB_TABLE", new HashMap<String, String>() {{
|
||||
put(MAX_ROWS_KEY, "9");
|
||||
}});
|
||||
|
||||
runner.run();
|
||||
runner.assertAllFlowFilesTransferred(SelectHive3QL.REL_SUCCESS, 12);
|
||||
|
||||
//ensure all but the last file have 9 records (10 lines = 9 records + header) each
|
||||
for (int ff = 0; ff < 11; ff++) {
|
||||
mff = runner.getFlowFilesForRelationship(SelectHive3QL.REL_SUCCESS).get(ff);
|
||||
in = new ByteArrayInputStream(mff.toByteArray());
|
||||
BufferedReader br = new BufferedReader(new InputStreamReader(in));
|
||||
assertEquals(10, br.lines().count());
|
||||
|
||||
mff.assertAttributeExists("fragment.identifier");
|
||||
assertEquals(Integer.toString(ff), mff.getAttribute("fragment.index"));
|
||||
assertEquals("12", mff.getAttribute("fragment.count"));
|
||||
}
|
||||
|
||||
//last file should have 1 record (2 lines = 1 record + header)
|
||||
mff = runner.getFlowFilesForRelationship(SelectHive3QL.REL_SUCCESS).get(11);
|
||||
in = new ByteArrayInputStream(mff.toByteArray());
|
||||
BufferedReader br = new BufferedReader(new InputStreamReader(in));
|
||||
assertEquals(2, br.lines().count());
|
||||
mff.assertAttributeExists("fragment.identifier");
|
||||
assertEquals(Integer.toString(11), mff.getAttribute("fragment.index"));
|
||||
assertEquals("12", mff.getAttribute("fragment.count"));
|
||||
runner.clearTransferState();
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testMaxRowsPerFlowFileWithMaxFragments() throws ClassNotFoundException, SQLException, InitializationException, IOException {
|
||||
|
||||
// load test data to database
|
||||
final Connection con = ((DBCPService) runner.getControllerService("dbcp")).getConnection();
|
||||
Statement stmt = con.createStatement();
|
||||
InputStream in;
|
||||
MockFlowFile mff;
|
||||
|
||||
try {
|
||||
stmt.execute("drop table TEST_QUERY_DB_TABLE");
|
||||
} catch (final SQLException sqle) {
|
||||
// Ignore this error, probably a "table does not exist" since Derby doesn't yet support DROP IF EXISTS [DERBY-4842]
|
||||
}
|
||||
|
||||
stmt.execute("create table TEST_QUERY_DB_TABLE (id integer not null, name varchar(100), scale float, created_on timestamp, bignum bigint default 0)");
|
||||
int rowCount = 0;
|
||||
//create larger row set
|
||||
for (int batch = 0; batch < 100; batch++) {
|
||||
stmt.execute("insert into TEST_QUERY_DB_TABLE (id, name, scale, created_on) VALUES (" + rowCount + ", 'Joe Smith', 1.0, '1962-09-23 03:23:34.234')");
|
||||
rowCount++;
|
||||
}
|
||||
|
||||
runner.setIncomingConnection(false);
|
||||
runner.setProperty(SelectHive3QL.HIVEQL_SELECT_QUERY, "SELECT * FROM TEST_QUERY_DB_TABLE");
|
||||
runner.setProperty(SelectHive3QL.MAX_ROWS_PER_FLOW_FILE, "9");
|
||||
Integer maxFragments = 3;
|
||||
runner.setProperty(SelectHive3QL.MAX_FRAGMENTS, maxFragments.toString());
|
||||
|
||||
runner.run();
|
||||
runner.assertAllFlowFilesTransferred(SelectHive3QL.REL_SUCCESS, maxFragments);
|
||||
|
||||
for (int i = 0; i < maxFragments; i++) {
|
||||
mff = runner.getFlowFilesForRelationship(SelectHive3QL.REL_SUCCESS).get(i);
|
||||
in = new ByteArrayInputStream(mff.toByteArray());
|
||||
assertEquals(9, getNumberOfRecordsFromStream(in));
|
||||
|
||||
mff.assertAttributeExists("fragment.identifier");
|
||||
assertEquals(Integer.toString(i), mff.getAttribute("fragment.index"));
|
||||
assertEquals(maxFragments.toString(), mff.getAttribute("fragment.count"));
|
||||
}
|
||||
|
||||
runner.clearTransferState();
|
||||
}
|
||||
|
||||
private long getNumberOfRecordsFromStream(InputStream in) throws IOException {
|
||||
final DatumReader<GenericRecord> datumReader = new GenericDatumReader<>();
|
||||
try (DataFileStream<GenericRecord> dataFileReader = new DataFileStream<>(in, datumReader)) {
|
||||
GenericRecord record = null;
|
||||
long recordsFromStream = 0;
|
||||
while (dataFileReader.hasNext()) {
|
||||
// Reuse record object by passing it to next(). This saves us from
|
||||
// allocating and garbage collecting many objects for files with
|
||||
// many items.
|
||||
record = dataFileReader.next(record);
|
||||
recordsFromStream += 1;
|
||||
}
|
||||
|
||||
return recordsFromStream;
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Simple implementation only for SelectHive3QL processor testing.
|
||||
*/
|
||||
private class DBCPServiceSimpleImpl extends AbstractControllerService implements Hive3DBCPService {
|
||||
|
||||
@Override
|
||||
public String getIdentifier() {
|
||||
return "dbcp";
|
||||
}
|
||||
|
||||
@Override
|
||||
public Connection getConnection() throws ProcessException {
|
||||
try {
|
||||
Class.forName("org.apache.derby.jdbc.EmbeddedDriver");
|
||||
return DriverManager.getConnection("jdbc:derby:" + DB_LOCATION + ";create=true");
|
||||
} catch (final Exception e) {
|
||||
throw new ProcessException("getConnection failed: " + e);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public String getConnectionURL() {
|
||||
return "jdbc:derby:" + DB_LOCATION + ";create=true";
|
||||
}
|
||||
}
|
||||
|
||||
}
|
|
@ -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.nifi.processors.orc;
|
||||
|
||||
import org.apache.avro.Schema;
|
||||
import org.apache.commons.io.IOUtils;
|
||||
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.orc.OrcFile;
|
||||
import org.apache.hadoop.hive.ql.io.orc.OrcStruct;
|
||||
import org.apache.hadoop.hive.ql.io.orc.Reader;
|
||||
import org.apache.hadoop.hive.ql.io.orc.RecordReader;
|
||||
import org.apache.hadoop.hive.serde2.io.DateWritable;
|
||||
import org.apache.hadoop.hive.serde2.io.DoubleWritable;
|
||||
import org.apache.hadoop.hive.serde2.io.TimestampWritable;
|
||||
import org.apache.hadoop.hive.serde2.objectinspector.StructObjectInspector;
|
||||
import org.apache.hadoop.hive.serde2.typeinfo.TypeInfo;
|
||||
import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoUtils;
|
||||
import org.apache.hadoop.io.IntWritable;
|
||||
import org.apache.log4j.BasicConfigurator;
|
||||
import org.apache.nifi.avro.AvroTypeUtil;
|
||||
import org.apache.nifi.flowfile.FlowFile;
|
||||
import org.apache.nifi.flowfile.attributes.CoreAttributes;
|
||||
import org.apache.nifi.logging.ComponentLog;
|
||||
import org.apache.nifi.processor.ProcessContext;
|
||||
import org.apache.nifi.processors.hadoop.exception.FailureException;
|
||||
import org.apache.nifi.processors.hadoop.record.HDFSRecordWriter;
|
||||
import org.apache.nifi.provenance.ProvenanceEventRecord;
|
||||
import org.apache.nifi.provenance.ProvenanceEventType;
|
||||
import org.apache.nifi.reporting.InitializationException;
|
||||
import org.apache.nifi.schema.access.SchemaNotFoundException;
|
||||
import org.apache.nifi.serialization.MalformedRecordException;
|
||||
import org.apache.nifi.serialization.RecordReaderFactory;
|
||||
import org.apache.nifi.serialization.record.MockRecordParser;
|
||||
import org.apache.nifi.serialization.record.RecordField;
|
||||
import org.apache.nifi.serialization.record.RecordSchema;
|
||||
import org.apache.nifi.serialization.record.RecordSet;
|
||||
import org.apache.nifi.util.MockFlowFile;
|
||||
import org.apache.nifi.util.TestRunner;
|
||||
import org.apache.nifi.util.TestRunners;
|
||||
import org.junit.Assert;
|
||||
import org.junit.Before;
|
||||
import org.junit.BeforeClass;
|
||||
import org.junit.Test;
|
||||
import org.mockito.Mockito;
|
||||
|
||||
import java.io.File;
|
||||
import java.io.FileInputStream;
|
||||
import java.io.IOException;
|
||||
import java.io.InputStream;
|
||||
import java.nio.charset.StandardCharsets;
|
||||
import java.sql.Date;
|
||||
import java.sql.Timestamp;
|
||||
import java.time.LocalDate;
|
||||
import java.time.LocalDateTime;
|
||||
import java.time.LocalTime;
|
||||
import java.time.temporal.ChronoField;
|
||||
import java.util.Calendar;
|
||||
import java.util.HashMap;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.function.BiFunction;
|
||||
|
||||
import static org.junit.Assert.assertEquals;
|
||||
import static org.mockito.Matchers.any;
|
||||
import static org.mockito.Mockito.when;
|
||||
|
||||
public class PutORCTest {
|
||||
|
||||
private static final String DIRECTORY = "target";
|
||||
private static final String TEST_CONF_PATH = "src/test/resources/core-site.xml";
|
||||
|
||||
private Schema schema;
|
||||
private Configuration testConf;
|
||||
private PutORC proc;
|
||||
private TestRunner testRunner;
|
||||
|
||||
@BeforeClass
|
||||
public static void setupLogging() {
|
||||
BasicConfigurator.configure();
|
||||
}
|
||||
|
||||
@Before
|
||||
public void setup() throws IOException {
|
||||
final String avroSchema = IOUtils.toString(new FileInputStream("src/test/resources/user.avsc"), StandardCharsets.UTF_8);
|
||||
schema = new Schema.Parser().parse(avroSchema);
|
||||
|
||||
testConf = new Configuration();
|
||||
testConf.addResource(new Path(TEST_CONF_PATH));
|
||||
|
||||
proc = new PutORC();
|
||||
}
|
||||
|
||||
private void configure(final PutORC putORC, final int numUsers) throws InitializationException {
|
||||
configure(putORC, numUsers, null);
|
||||
}
|
||||
|
||||
private void configure(final PutORC putORC, final int numUsers, final BiFunction<Integer, MockRecordParser, Void> recordGenerator) throws InitializationException {
|
||||
testRunner = TestRunners.newTestRunner(putORC);
|
||||
testRunner.setProperty(PutORC.HADOOP_CONFIGURATION_RESOURCES, TEST_CONF_PATH);
|
||||
testRunner.setProperty(PutORC.DIRECTORY, DIRECTORY);
|
||||
|
||||
MockRecordParser readerFactory = new MockRecordParser();
|
||||
|
||||
final RecordSchema recordSchema = AvroTypeUtil.createSchema(schema);
|
||||
for (final RecordField recordField : recordSchema.getFields()) {
|
||||
readerFactory.addSchemaField(recordField.getFieldName(), recordField.getDataType().getFieldType(), recordField.isNullable());
|
||||
}
|
||||
|
||||
if (recordGenerator == null) {
|
||||
for (int i = 0; i < numUsers; i++) {
|
||||
readerFactory.addRecord("name" + i, i, "blue" + i, i * 10.0);
|
||||
}
|
||||
} else {
|
||||
recordGenerator.apply(numUsers, readerFactory);
|
||||
}
|
||||
|
||||
testRunner.addControllerService("mock-reader-factory", readerFactory);
|
||||
testRunner.enableControllerService(readerFactory);
|
||||
|
||||
testRunner.setProperty(PutORC.RECORD_READER, "mock-reader-factory");
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testWriteORCWithDefaults() throws IOException, InitializationException {
|
||||
configure(proc, 100);
|
||||
|
||||
final String filename = "testORCWithDefaults-" + System.currentTimeMillis();
|
||||
|
||||
final Map<String, String> flowFileAttributes = new HashMap<>();
|
||||
flowFileAttributes.put(CoreAttributes.FILENAME.key(), filename);
|
||||
|
||||
testRunner.setProperty(PutORC.HIVE_TABLE_NAME, "myTable");
|
||||
|
||||
testRunner.enqueue("trigger", flowFileAttributes);
|
||||
testRunner.run();
|
||||
testRunner.assertAllFlowFilesTransferred(PutORC.REL_SUCCESS, 1);
|
||||
|
||||
final Path orcFile = new Path(DIRECTORY + "/" + filename);
|
||||
|
||||
// verify the successful flow file has the expected attributes
|
||||
final MockFlowFile mockFlowFile = testRunner.getFlowFilesForRelationship(PutORC.REL_SUCCESS).get(0);
|
||||
mockFlowFile.assertAttributeEquals(PutORC.ABSOLUTE_HDFS_PATH_ATTRIBUTE, orcFile.getParent().toString());
|
||||
mockFlowFile.assertAttributeEquals(CoreAttributes.FILENAME.key(), filename);
|
||||
mockFlowFile.assertAttributeEquals(PutORC.RECORD_COUNT_ATTR, "100");
|
||||
mockFlowFile.assertAttributeEquals(PutORC.HIVE_DDL_ATTRIBUTE,
|
||||
"CREATE EXTERNAL TABLE IF NOT EXISTS myTable (name STRING, favorite_number INT, favorite_color STRING, scale DOUBLE) STORED AS ORC");
|
||||
|
||||
// verify we generated a provenance event
|
||||
final List<ProvenanceEventRecord> provEvents = testRunner.getProvenanceEvents();
|
||||
assertEquals(1, provEvents.size());
|
||||
|
||||
// verify it was a SEND event with the correct URI
|
||||
final ProvenanceEventRecord provEvent = provEvents.get(0);
|
||||
assertEquals(ProvenanceEventType.SEND, provEvent.getEventType());
|
||||
// If it runs with a real HDFS, the protocol will be "hdfs://", but with a local filesystem, just assert the filename.
|
||||
Assert.assertTrue(provEvent.getTransitUri().endsWith(DIRECTORY + "/" + filename));
|
||||
|
||||
// verify the content of the ORC file by reading it back in
|
||||
verifyORCUsers(orcFile, 100);
|
||||
|
||||
// verify we don't have the temp dot file after success
|
||||
final File tempOrcFile = new File(DIRECTORY + "/." + filename);
|
||||
Assert.assertFalse(tempOrcFile.exists());
|
||||
|
||||
// verify we DO have the CRC file after success
|
||||
final File crcAvroORCFile = new File(DIRECTORY + "/." + filename + ".crc");
|
||||
Assert.assertTrue(crcAvroORCFile.exists());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testWriteORCWithAvroLogicalTypes() throws IOException, InitializationException {
|
||||
final String avroSchema = IOUtils.toString(new FileInputStream("src/test/resources/user_logical_types.avsc"), StandardCharsets.UTF_8);
|
||||
schema = new Schema.Parser().parse(avroSchema);
|
||||
Calendar now = Calendar.getInstance();
|
||||
LocalTime nowTime = LocalTime.now();
|
||||
LocalDateTime nowDateTime = LocalDateTime.now();
|
||||
LocalDate epoch = LocalDate.ofEpochDay(0);
|
||||
LocalDate nowDate = LocalDate.now();
|
||||
|
||||
final int timeMillis = nowTime.get(ChronoField.MILLI_OF_DAY);
|
||||
final Timestamp timestampMillis = Timestamp.valueOf(nowDateTime);
|
||||
final Date dt = Date.valueOf(nowDate);
|
||||
final double dec = 1234.56;
|
||||
|
||||
configure(proc, 10, (numUsers, readerFactory) -> {
|
||||
for (int i = 0; i < numUsers; i++) {
|
||||
readerFactory.addRecord(
|
||||
i,
|
||||
timeMillis,
|
||||
timestampMillis,
|
||||
dt,
|
||||
dec);
|
||||
}
|
||||
return null;
|
||||
});
|
||||
|
||||
final String filename = "testORCWithDefaults-" + System.currentTimeMillis();
|
||||
|
||||
final Map<String, String> flowFileAttributes = new HashMap<>();
|
||||
flowFileAttributes.put(CoreAttributes.FILENAME.key(), filename);
|
||||
|
||||
testRunner.setProperty(PutORC.HIVE_TABLE_NAME, "myTable");
|
||||
|
||||
testRunner.enqueue("trigger", flowFileAttributes);
|
||||
testRunner.run();
|
||||
testRunner.assertAllFlowFilesTransferred(PutORC.REL_SUCCESS, 1);
|
||||
|
||||
final Path orcFile = new Path(DIRECTORY + "/" + filename);
|
||||
|
||||
// verify the successful flow file has the expected attributes
|
||||
final MockFlowFile mockFlowFile = testRunner.getFlowFilesForRelationship(PutORC.REL_SUCCESS).get(0);
|
||||
mockFlowFile.assertAttributeEquals(PutORC.ABSOLUTE_HDFS_PATH_ATTRIBUTE, orcFile.getParent().toString());
|
||||
mockFlowFile.assertAttributeEquals(CoreAttributes.FILENAME.key(), filename);
|
||||
mockFlowFile.assertAttributeEquals(PutORC.RECORD_COUNT_ATTR, "10");
|
||||
// DDL will be created with field names normalized (lowercased, e.g.) for Hive by default
|
||||
mockFlowFile.assertAttributeEquals(PutORC.HIVE_DDL_ATTRIBUTE,
|
||||
"CREATE EXTERNAL TABLE IF NOT EXISTS myTable (id INT, timemillis INT, timestampmillis TIMESTAMP, dt DATE, dec DOUBLE) STORED AS ORC");
|
||||
|
||||
// verify we generated a provenance event
|
||||
final List<ProvenanceEventRecord> provEvents = testRunner.getProvenanceEvents();
|
||||
assertEquals(1, provEvents.size());
|
||||
|
||||
// verify it was a SEND event with the correct URI
|
||||
final ProvenanceEventRecord provEvent = provEvents.get(0);
|
||||
assertEquals(ProvenanceEventType.SEND, provEvent.getEventType());
|
||||
// If it runs with a real HDFS, the protocol will be "hdfs://", but with a local filesystem, just assert the filename.
|
||||
Assert.assertTrue(provEvent.getTransitUri().endsWith(DIRECTORY + "/" + filename));
|
||||
|
||||
// verify the content of the ORC file by reading it back in
|
||||
verifyORCUsers(orcFile, 10, (x, currUser) -> {
|
||||
assertEquals((int) currUser, ((IntWritable) x.get(0)).get());
|
||||
assertEquals(timeMillis, ((IntWritable) x.get(1)).get());
|
||||
assertEquals(timestampMillis, ((TimestampWritable) x.get(2)).getTimestamp());
|
||||
assertEquals(dt, ((DateWritable) x.get(3)).get());
|
||||
assertEquals(dec, ((DoubleWritable) x.get(4)).get(), Double.MIN_VALUE);
|
||||
return null;
|
||||
}
|
||||
);
|
||||
|
||||
// verify we don't have the temp dot file after success
|
||||
final File tempOrcFile = new File(DIRECTORY + "/." + filename);
|
||||
Assert.assertFalse(tempOrcFile.exists());
|
||||
|
||||
// verify we DO have the CRC file after success
|
||||
final File crcAvroORCFile = new File(DIRECTORY + "/." + filename + ".crc");
|
||||
Assert.assertTrue(crcAvroORCFile.exists());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testValidSchemaWithELShouldBeSuccessful() throws InitializationException {
|
||||
configure(proc, 10);
|
||||
|
||||
final String filename = "testValidSchemaWithELShouldBeSuccessful-" + System.currentTimeMillis();
|
||||
|
||||
// don't provide my.schema as an attribute
|
||||
final Map<String, String> flowFileAttributes = new HashMap<>();
|
||||
flowFileAttributes.put(CoreAttributes.FILENAME.key(), filename);
|
||||
flowFileAttributes.put("my.schema", schema.toString());
|
||||
|
||||
testRunner.enqueue("trigger", flowFileAttributes);
|
||||
testRunner.run();
|
||||
testRunner.assertAllFlowFilesTransferred(PutORC.REL_SUCCESS, 1);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testMalformedRecordExceptionFromReaderShouldRouteToFailure() throws InitializationException, IOException, MalformedRecordException, SchemaNotFoundException {
|
||||
configure(proc, 10);
|
||||
|
||||
final org.apache.nifi.serialization.RecordReader recordReader = Mockito.mock(org.apache.nifi.serialization.RecordReader.class);
|
||||
when(recordReader.nextRecord()).thenThrow(new MalformedRecordException("ERROR"));
|
||||
|
||||
final RecordReaderFactory readerFactory = Mockito.mock(RecordReaderFactory.class);
|
||||
when(readerFactory.getIdentifier()).thenReturn("mock-reader-factory");
|
||||
when(readerFactory.createRecordReader(any(FlowFile.class), any(InputStream.class), any(ComponentLog.class))).thenReturn(recordReader);
|
||||
|
||||
testRunner.addControllerService("mock-reader-factory", readerFactory);
|
||||
testRunner.enableControllerService(readerFactory);
|
||||
testRunner.setProperty(PutORC.RECORD_READER, "mock-reader-factory");
|
||||
|
||||
final String filename = "testMalformedRecordExceptionShouldRouteToFailure-" + System.currentTimeMillis();
|
||||
|
||||
final Map<String, String> flowFileAttributes = new HashMap<>();
|
||||
flowFileAttributes.put(CoreAttributes.FILENAME.key(), filename);
|
||||
|
||||
testRunner.enqueue("trigger", flowFileAttributes);
|
||||
testRunner.run();
|
||||
testRunner.assertAllFlowFilesTransferred(PutORC.REL_FAILURE, 1);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testIOExceptionCreatingWriterShouldRouteToRetry() throws InitializationException {
|
||||
final PutORC proc = new PutORC() {
|
||||
@Override
|
||||
public HDFSRecordWriter createHDFSRecordWriter(ProcessContext context, FlowFile flowFile, Configuration conf, Path path, RecordSchema schema)
|
||||
throws IOException {
|
||||
throw new IOException("IOException");
|
||||
}
|
||||
};
|
||||
configure(proc, 0);
|
||||
|
||||
final String filename = "testMalformedRecordExceptionShouldRouteToFailure-" + System.currentTimeMillis();
|
||||
|
||||
final Map<String, String> flowFileAttributes = new HashMap<>();
|
||||
flowFileAttributes.put(CoreAttributes.FILENAME.key(), filename);
|
||||
|
||||
testRunner.enqueue("trigger", flowFileAttributes);
|
||||
testRunner.run();
|
||||
testRunner.assertAllFlowFilesTransferred(PutORC.REL_RETRY, 1);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testIOExceptionFromReaderShouldRouteToRetry() throws InitializationException, IOException, MalformedRecordException, SchemaNotFoundException {
|
||||
configure(proc, 10);
|
||||
|
||||
final RecordSet recordSet = Mockito.mock(RecordSet.class);
|
||||
when(recordSet.next()).thenThrow(new IOException("ERROR"));
|
||||
|
||||
final org.apache.nifi.serialization.RecordReader recordReader = Mockito.mock(org.apache.nifi.serialization.RecordReader.class);
|
||||
when(recordReader.createRecordSet()).thenReturn(recordSet);
|
||||
when(recordReader.getSchema()).thenReturn(AvroTypeUtil.createSchema(schema));
|
||||
|
||||
final RecordReaderFactory readerFactory = Mockito.mock(RecordReaderFactory.class);
|
||||
when(readerFactory.getIdentifier()).thenReturn("mock-reader-factory");
|
||||
when(readerFactory.createRecordReader(any(FlowFile.class), any(InputStream.class), any(ComponentLog.class))).thenReturn(recordReader);
|
||||
|
||||
testRunner.addControllerService("mock-reader-factory", readerFactory);
|
||||
testRunner.enableControllerService(readerFactory);
|
||||
testRunner.setProperty(PutORC.RECORD_READER, "mock-reader-factory");
|
||||
|
||||
final String filename = "testMalformedRecordExceptionShouldRouteToFailure-" + System.currentTimeMillis();
|
||||
|
||||
final Map<String, String> flowFileAttributes = new HashMap<>();
|
||||
flowFileAttributes.put(CoreAttributes.FILENAME.key(), filename);
|
||||
|
||||
testRunner.enqueue("trigger", flowFileAttributes);
|
||||
testRunner.run();
|
||||
testRunner.assertAllFlowFilesTransferred(PutORC.REL_RETRY, 1);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testIOExceptionRenamingShouldRouteToRetry() throws InitializationException {
|
||||
final PutORC proc = new PutORC() {
|
||||
@Override
|
||||
protected void rename(FileSystem fileSystem, Path srcFile, Path destFile)
|
||||
throws IOException, InterruptedException, FailureException {
|
||||
throw new IOException("IOException renaming");
|
||||
}
|
||||
};
|
||||
|
||||
configure(proc, 10);
|
||||
|
||||
final String filename = "testIOExceptionRenamingShouldRouteToRetry-" + System.currentTimeMillis();
|
||||
|
||||
final Map<String, String> flowFileAttributes = new HashMap<>();
|
||||
flowFileAttributes.put(CoreAttributes.FILENAME.key(), filename);
|
||||
|
||||
testRunner.enqueue("trigger", flowFileAttributes);
|
||||
testRunner.run();
|
||||
testRunner.assertAllFlowFilesTransferred(PutORC.REL_RETRY, 1);
|
||||
|
||||
// verify we don't have the temp dot file after success
|
||||
final File tempAvroORCFile = new File(DIRECTORY + "/." + filename);
|
||||
Assert.assertFalse(tempAvroORCFile.exists());
|
||||
}
|
||||
|
||||
private void verifyORCUsers(final Path orcUsers, final int numExpectedUsers) throws IOException {
|
||||
verifyORCUsers(orcUsers, numExpectedUsers, null);
|
||||
}
|
||||
|
||||
private void verifyORCUsers(final Path orcUsers, final int numExpectedUsers, BiFunction<List<Object>, Integer, Void> assertFunction) throws IOException {
|
||||
Reader reader = OrcFile.createReader(orcUsers, OrcFile.readerOptions(testConf));
|
||||
RecordReader recordReader = reader.rows();
|
||||
|
||||
TypeInfo typeInfo =
|
||||
TypeInfoUtils.getTypeInfoFromTypeString("struct<name:string,favorite_number:int,favorite_color:string,scale:double>");
|
||||
StructObjectInspector inspector = (StructObjectInspector)
|
||||
OrcStruct.createObjectInspector(typeInfo);
|
||||
|
||||
int currUser = 0;
|
||||
Object nextRecord = null;
|
||||
while ((nextRecord = recordReader.next(nextRecord)) != null) {
|
||||
Assert.assertNotNull(nextRecord);
|
||||
Assert.assertTrue("Not an OrcStruct", nextRecord instanceof OrcStruct);
|
||||
List<Object> x = inspector.getStructFieldsDataAsList(nextRecord);
|
||||
|
||||
if (assertFunction == null) {
|
||||
assertEquals("name" + currUser, x.get(0).toString());
|
||||
assertEquals(currUser, ((IntWritable) x.get(1)).get());
|
||||
assertEquals("blue" + currUser, x.get(2).toString());
|
||||
assertEquals(10.0 * currUser, ((DoubleWritable) x.get(3)).get(), Double.MIN_VALUE);
|
||||
} else {
|
||||
assertFunction.apply(x, currUser);
|
||||
}
|
||||
currUser++;
|
||||
}
|
||||
|
||||
assertEquals(numExpectedUsers, currUser);
|
||||
}
|
||||
|
||||
}
|
|
@ -0,0 +1,437 @@
|
|||
/*
|
||||
* 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.nifi.util.orc;
|
||||
|
||||
|
||||
import org.apache.avro.Schema;
|
||||
import org.apache.avro.SchemaBuilder;
|
||||
import org.apache.avro.generic.GenericData;
|
||||
import org.apache.avro.util.Utf8;
|
||||
import org.apache.hadoop.hive.ql.io.orc.NiFiOrcUtils;
|
||||
import org.apache.hadoop.hive.serde2.objectinspector.UnionObject;
|
||||
import org.apache.hadoop.hive.serde2.typeinfo.TypeInfo;
|
||||
import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoFactory;
|
||||
import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoUtils;
|
||||
import org.apache.hadoop.io.DoubleWritable;
|
||||
import org.apache.hadoop.io.FloatWritable;
|
||||
import org.apache.hadoop.io.IntWritable;
|
||||
import org.apache.hadoop.io.LongWritable;
|
||||
import org.apache.hadoop.io.Text;
|
||||
import org.junit.Test;
|
||||
|
||||
import java.nio.ByteBuffer;
|
||||
import java.util.Arrays;
|
||||
import java.util.HashMap;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
|
||||
import static org.junit.Assert.assertEquals;
|
||||
import static org.junit.Assert.assertTrue;
|
||||
|
||||
/**
|
||||
* Unit tests for the NiFiOrcUtils helper class
|
||||
*/
|
||||
public class TestNiFiOrcUtils {
|
||||
|
||||
@Test
|
||||
public void test_getOrcField_primitive() {
|
||||
// Expected ORC types
|
||||
TypeInfo[] expectedTypes = {
|
||||
TypeInfoFactory.getPrimitiveTypeInfo("int"),
|
||||
TypeInfoFactory.getPrimitiveTypeInfo("bigint"),
|
||||
TypeInfoFactory.getPrimitiveTypeInfo("boolean"),
|
||||
TypeInfoFactory.getPrimitiveTypeInfo("float"),
|
||||
TypeInfoFactory.getPrimitiveTypeInfo("double"),
|
||||
TypeInfoFactory.getPrimitiveTypeInfo("binary"),
|
||||
TypeInfoFactory.getPrimitiveTypeInfo("string")
|
||||
};
|
||||
|
||||
// Build a fake Avro record with all types
|
||||
Schema testSchema = buildPrimitiveAvroSchema();
|
||||
List<Schema.Field> fields = testSchema.getFields();
|
||||
for (int i = 0; i < fields.size(); i++) {
|
||||
assertEquals(expectedTypes[i], NiFiOrcUtils.getOrcField(fields.get(i).schema(), false));
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
@Test
|
||||
public void test_getOrcField_union_optional_type() {
|
||||
final SchemaBuilder.FieldAssembler<Schema> builder = SchemaBuilder.record("testRecord").namespace("any.data").fields();
|
||||
builder.name("union").type().unionOf().nullBuilder().endNull().and().booleanType().endUnion().noDefault();
|
||||
Schema testSchema = builder.endRecord();
|
||||
TypeInfo orcType = NiFiOrcUtils.getOrcField(testSchema.getField("union").schema(), false);
|
||||
assertEquals(TypeInfoCreator.createBoolean(), orcType);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void test_getOrcField_union() {
|
||||
final SchemaBuilder.FieldAssembler<Schema> builder = SchemaBuilder.record("testRecord").namespace("any.data").fields();
|
||||
builder.name("union").type().unionOf().intType().and().booleanType().endUnion().noDefault();
|
||||
Schema testSchema = builder.endRecord();
|
||||
TypeInfo orcType = NiFiOrcUtils.getOrcField(testSchema.getField("union").schema(), false);
|
||||
assertEquals(
|
||||
TypeInfoFactory.getUnionTypeInfo(Arrays.asList(
|
||||
TypeInfoCreator.createInt(),
|
||||
TypeInfoCreator.createBoolean())),
|
||||
orcType);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void test_getOrcField_map() {
|
||||
final SchemaBuilder.FieldAssembler<Schema> builder = SchemaBuilder.record("testRecord").namespace("any.data").fields();
|
||||
builder.name("map").type().map().values().doubleType().noDefault();
|
||||
Schema testSchema = builder.endRecord();
|
||||
TypeInfo orcType = NiFiOrcUtils.getOrcField(testSchema.getField("map").schema(), true);
|
||||
assertEquals(
|
||||
TypeInfoFactory.getMapTypeInfo(
|
||||
TypeInfoCreator.createString(),
|
||||
TypeInfoCreator.createDouble()),
|
||||
orcType);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void test_getOrcField_nested_map() {
|
||||
final SchemaBuilder.FieldAssembler<Schema> builder = SchemaBuilder.record("testRecord").namespace("any.data").fields();
|
||||
builder.name("map").type().map().values().map().values().doubleType().noDefault();
|
||||
Schema testSchema = builder.endRecord();
|
||||
TypeInfo orcType = NiFiOrcUtils.getOrcField(testSchema.getField("map").schema(), false);
|
||||
assertEquals(
|
||||
TypeInfoFactory.getMapTypeInfo(TypeInfoCreator.createString(),
|
||||
TypeInfoFactory.getMapTypeInfo(TypeInfoCreator.createString(), TypeInfoCreator.createDouble())),
|
||||
orcType);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void test_getOrcField_array() {
|
||||
final SchemaBuilder.FieldAssembler<Schema> builder = SchemaBuilder.record("testRecord").namespace("any.data").fields();
|
||||
builder.name("array").type().array().items().longType().noDefault();
|
||||
Schema testSchema = builder.endRecord();
|
||||
TypeInfo orcType = NiFiOrcUtils.getOrcField(testSchema.getField("array").schema(), false);
|
||||
assertEquals(
|
||||
TypeInfoFactory.getListTypeInfo(TypeInfoCreator.createLong()),
|
||||
orcType);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void test_getOrcField_complex_array() {
|
||||
final SchemaBuilder.FieldAssembler<Schema> builder = SchemaBuilder.record("testRecord").namespace("any.data").fields();
|
||||
builder.name("Array").type().array().items().map().values().floatType().noDefault();
|
||||
Schema testSchema = builder.endRecord();
|
||||
TypeInfo orcType = NiFiOrcUtils.getOrcField(testSchema.getField("Array").schema(), true);
|
||||
assertEquals(
|
||||
TypeInfoFactory.getListTypeInfo(TypeInfoFactory.getMapTypeInfo(TypeInfoCreator.createString(), TypeInfoCreator.createFloat())),
|
||||
orcType);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void test_getOrcField_record() {
|
||||
final SchemaBuilder.FieldAssembler<Schema> builder = SchemaBuilder.record("testRecord").namespace("any.data").fields();
|
||||
builder.name("Int").type().intType().noDefault();
|
||||
builder.name("Long").type().longType().longDefault(1L);
|
||||
builder.name("Array").type().array().items().stringType().noDefault();
|
||||
Schema testSchema = builder.endRecord();
|
||||
// Normalize field names for Hive, assert that their names are now lowercase
|
||||
TypeInfo orcType = NiFiOrcUtils.getOrcField(testSchema, true);
|
||||
assertEquals(
|
||||
TypeInfoFactory.getStructTypeInfo(
|
||||
Arrays.asList("int", "long", "array"),
|
||||
Arrays.asList(
|
||||
TypeInfoCreator.createInt(),
|
||||
TypeInfoCreator.createLong(),
|
||||
TypeInfoFactory.getListTypeInfo(TypeInfoCreator.createString()))),
|
||||
orcType);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void test_getOrcField_enum() {
|
||||
final SchemaBuilder.FieldAssembler<Schema> builder = SchemaBuilder.record("testRecord").namespace("any.data").fields();
|
||||
builder.name("enumField").type().enumeration("enum").symbols("a", "b", "c").enumDefault("a");
|
||||
Schema testSchema = builder.endRecord();
|
||||
TypeInfo orcType = NiFiOrcUtils.getOrcField(testSchema.getField("enumField").schema(), true);
|
||||
assertEquals(TypeInfoCreator.createString(), orcType);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void test_getPrimitiveOrcTypeFromPrimitiveAvroType() {
|
||||
// Expected ORC types
|
||||
TypeInfo[] expectedTypes = {
|
||||
TypeInfoCreator.createInt(),
|
||||
TypeInfoCreator.createLong(),
|
||||
TypeInfoCreator.createBoolean(),
|
||||
TypeInfoCreator.createFloat(),
|
||||
TypeInfoCreator.createDouble(),
|
||||
TypeInfoCreator.createBinary(),
|
||||
TypeInfoCreator.createString(),
|
||||
};
|
||||
|
||||
Schema testSchema = buildPrimitiveAvroSchema();
|
||||
List<Schema.Field> fields = testSchema.getFields();
|
||||
for (int i = 0; i < fields.size(); i++) {
|
||||
assertEquals(expectedTypes[i], NiFiOrcUtils.getPrimitiveOrcTypeFromPrimitiveAvroType(fields.get(i).schema().getType()));
|
||||
}
|
||||
}
|
||||
|
||||
@Test(expected = IllegalArgumentException.class)
|
||||
public void test_getPrimitiveOrcTypeFromPrimitiveAvroType_badType() {
|
||||
Schema.Type nonPrimitiveType = Schema.Type.ARRAY;
|
||||
NiFiOrcUtils.getPrimitiveOrcTypeFromPrimitiveAvroType(nonPrimitiveType);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void test_getWritable() throws Exception {
|
||||
assertTrue(NiFiOrcUtils.convertToORCObject(null, 1, true) instanceof IntWritable);
|
||||
assertTrue(NiFiOrcUtils.convertToORCObject(null, 1L, true) instanceof LongWritable);
|
||||
assertTrue(NiFiOrcUtils.convertToORCObject(null, 1.0f, true) instanceof FloatWritable);
|
||||
assertTrue(NiFiOrcUtils.convertToORCObject(null, 1.0, true) instanceof DoubleWritable);
|
||||
assertTrue(NiFiOrcUtils.convertToORCObject(null, new int[]{1, 2, 3}, true) instanceof List);
|
||||
assertTrue(NiFiOrcUtils.convertToORCObject(null, Arrays.asList(1, 2, 3), true) instanceof List);
|
||||
Map<String, Float> map = new HashMap<>();
|
||||
map.put("Hello", 1.0f);
|
||||
map.put("World", 2.0f);
|
||||
|
||||
Object convMap = NiFiOrcUtils.convertToORCObject(TypeInfoUtils.getTypeInfoFromTypeString("map<string,float>"), map, true);
|
||||
assertTrue(convMap instanceof Map);
|
||||
((Map) convMap).forEach((key, value) -> {
|
||||
assertTrue(key instanceof Text);
|
||||
assertTrue(value instanceof FloatWritable);
|
||||
});
|
||||
}
|
||||
|
||||
@Test
|
||||
public void test_getHiveTypeFromAvroType_primitive() {
|
||||
// Expected ORC types
|
||||
String[] expectedTypes = {
|
||||
"INT",
|
||||
"BIGINT",
|
||||
"BOOLEAN",
|
||||
"FLOAT",
|
||||
"DOUBLE",
|
||||
"BINARY",
|
||||
"STRING",
|
||||
};
|
||||
|
||||
Schema testSchema = buildPrimitiveAvroSchema();
|
||||
List<Schema.Field> fields = testSchema.getFields();
|
||||
for (int i = 0; i < fields.size(); i++) {
|
||||
assertEquals(expectedTypes[i], NiFiOrcUtils.getHiveTypeFromAvroType(fields.get(i).schema(), false));
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void test_getHiveTypeFromAvroType_complex() {
|
||||
// Expected ORC types
|
||||
String[] expectedTypes = {
|
||||
"INT",
|
||||
"MAP<STRING, DOUBLE>",
|
||||
"STRING",
|
||||
"UNIONTYPE<BIGINT, FLOAT>",
|
||||
"ARRAY<INT>"
|
||||
};
|
||||
|
||||
Schema testSchema = buildComplexAvroSchema();
|
||||
List<Schema.Field> fields = testSchema.getFields();
|
||||
for (int i = 0; i < fields.size(); i++) {
|
||||
assertEquals(expectedTypes[i], NiFiOrcUtils.getHiveTypeFromAvroType(fields.get(i).schema(), false));
|
||||
}
|
||||
|
||||
assertEquals("STRUCT<myInt:INT, myMap:MAP<STRING, DOUBLE>, myEnum:STRING, myLongOrFloat:UNIONTYPE<BIGINT, FLOAT>, myIntList:ARRAY<INT>>",
|
||||
NiFiOrcUtils.getHiveTypeFromAvroType(testSchema, false));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void test_generateHiveDDL_primitive() {
|
||||
Schema avroSchema = buildPrimitiveAvroSchema();
|
||||
String ddl = NiFiOrcUtils.generateHiveDDL(avroSchema, "myHiveTable", false);
|
||||
assertEquals("CREATE EXTERNAL TABLE IF NOT EXISTS myHiveTable (int INT, long BIGINT, boolean BOOLEAN, float FLOAT, double DOUBLE, bytes BINARY, string STRING)"
|
||||
+ " STORED AS ORC", ddl);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void test_generateHiveDDL_complex() {
|
||||
Schema avroSchema = buildComplexAvroSchema();
|
||||
String ddl = NiFiOrcUtils.generateHiveDDL(avroSchema, "myHiveTable", false);
|
||||
assertEquals("CREATE EXTERNAL TABLE IF NOT EXISTS myHiveTable "
|
||||
+ "(myInt INT, myMap MAP<STRING, DOUBLE>, myEnum STRING, myLongOrFloat UNIONTYPE<BIGINT, FLOAT>, myIntList ARRAY<INT>)"
|
||||
+ " STORED AS ORC", ddl);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void test_generateHiveDDL_complex_normalize() {
|
||||
Schema avroSchema = buildComplexAvroSchema();
|
||||
String ddl = NiFiOrcUtils.generateHiveDDL(avroSchema, "myHiveTable", true);
|
||||
assertEquals("CREATE EXTERNAL TABLE IF NOT EXISTS myHiveTable "
|
||||
+ "(myint INT, mymap MAP<STRING, DOUBLE>, myenum STRING, mylongorfloat UNIONTYPE<BIGINT, FLOAT>, myintlist ARRAY<INT>)"
|
||||
+ " STORED AS ORC", ddl);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void test_convertToORCObject() {
|
||||
Schema schema = SchemaBuilder.enumeration("myEnum").symbols("x", "y", "z");
|
||||
List<Object> objects = Arrays.asList(new Utf8("Hello"), new GenericData.EnumSymbol(schema, "x"));
|
||||
objects.forEach((avroObject) -> {
|
||||
Object o = NiFiOrcUtils.convertToORCObject(TypeInfoUtils.getTypeInfoFromTypeString("uniontype<bigint,string>"), avroObject, true);
|
||||
assertTrue(o instanceof UnionObject);
|
||||
UnionObject uo = (UnionObject) o;
|
||||
assertTrue(uo.getObject() instanceof Text);
|
||||
});
|
||||
}
|
||||
|
||||
@Test(expected = IllegalArgumentException.class)
|
||||
public void test_convertToORCObjectBadUnion() {
|
||||
NiFiOrcUtils.convertToORCObject(TypeInfoUtils.getTypeInfoFromTypeString("uniontype<bigint,long>"), "Hello", true);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void test_getHiveTypeFromAvroType_complex_normalize() {
|
||||
// Expected ORC types
|
||||
String[] expectedTypes = {
|
||||
"INT",
|
||||
"MAP<STRING, DOUBLE>",
|
||||
"STRING",
|
||||
"UNIONTYPE<BIGINT, FLOAT>",
|
||||
"ARRAY<INT>"
|
||||
};
|
||||
|
||||
Schema testSchema = buildComplexAvroSchema();
|
||||
List<Schema.Field> fields = testSchema.getFields();
|
||||
for (int i = 0; i < fields.size(); i++) {
|
||||
assertEquals(expectedTypes[i], NiFiOrcUtils.getHiveTypeFromAvroType(fields.get(i).schema(), true));
|
||||
}
|
||||
|
||||
assertEquals("STRUCT<myint:INT, mymap:MAP<STRING, DOUBLE>, myenum:STRING, mylongorfloat:UNIONTYPE<BIGINT, FLOAT>, myintlist:ARRAY<INT>>",
|
||||
NiFiOrcUtils.getHiveTypeFromAvroType(testSchema, true));
|
||||
}
|
||||
|
||||
//////////////////
|
||||
// Helper methods
|
||||
//////////////////
|
||||
|
||||
public static Schema buildPrimitiveAvroSchema() {
|
||||
// Build a fake Avro record with all primitive types
|
||||
final SchemaBuilder.FieldAssembler<Schema> builder = SchemaBuilder.record("test.record").namespace("any.data").fields();
|
||||
builder.name("int").type().intType().noDefault();
|
||||
builder.name("long").type().longType().longDefault(1L);
|
||||
builder.name("boolean").type().booleanType().booleanDefault(true);
|
||||
builder.name("float").type().floatType().floatDefault(0.0f);
|
||||
builder.name("double").type().doubleType().doubleDefault(0.0);
|
||||
builder.name("bytes").type().bytesType().noDefault();
|
||||
builder.name("string").type().stringType().stringDefault("default");
|
||||
return builder.endRecord();
|
||||
}
|
||||
|
||||
public static GenericData.Record buildPrimitiveAvroRecord(int i, long l, boolean b, float f, double d, ByteBuffer bytes, String string) {
|
||||
Schema schema = buildPrimitiveAvroSchema();
|
||||
GenericData.Record row = new GenericData.Record(schema);
|
||||
row.put("int", i);
|
||||
row.put("long", l);
|
||||
row.put("boolean", b);
|
||||
row.put("float", f);
|
||||
row.put("double", d);
|
||||
row.put("bytes", bytes);
|
||||
row.put("string", string);
|
||||
return row;
|
||||
}
|
||||
|
||||
public static TypeInfo buildPrimitiveOrcSchema() {
|
||||
return TypeInfoFactory.getStructTypeInfo(Arrays.asList("int", "long", "boolean", "float", "double", "bytes", "string"),
|
||||
Arrays.asList(
|
||||
TypeInfoCreator.createInt(),
|
||||
TypeInfoCreator.createLong(),
|
||||
TypeInfoCreator.createBoolean(),
|
||||
TypeInfoCreator.createFloat(),
|
||||
TypeInfoCreator.createDouble(),
|
||||
TypeInfoCreator.createBinary(),
|
||||
TypeInfoCreator.createString()));
|
||||
}
|
||||
|
||||
public static Schema buildComplexAvroSchema() {
|
||||
// Build a fake Avro record with nested types
|
||||
final SchemaBuilder.FieldAssembler<Schema> builder = SchemaBuilder.record("complex.record").namespace("any.data").fields();
|
||||
builder.name("myInt").type().unionOf().nullType().and().intType().endUnion().nullDefault();
|
||||
builder.name("myMap").type().map().values().doubleType().noDefault();
|
||||
builder.name("myEnum").type().enumeration("myEnum").symbols("ABC", "DEF", "XYZ").enumDefault("ABC");
|
||||
builder.name("myLongOrFloat").type().unionOf().longType().and().floatType().endUnion().noDefault();
|
||||
builder.name("myIntList").type().array().items().intType().noDefault();
|
||||
return builder.endRecord();
|
||||
}
|
||||
|
||||
public static GenericData.Record buildComplexAvroRecord(Integer i, Map<String, Double> m, String e, Object unionVal, List<Integer> intArray) {
|
||||
Schema schema = buildComplexAvroSchema();
|
||||
GenericData.Record row = new GenericData.Record(schema);
|
||||
row.put("myInt", i);
|
||||
row.put("myMap", m);
|
||||
row.put("myEnum", e);
|
||||
row.put("myLongOrFloat", unionVal);
|
||||
row.put("myIntList", intArray);
|
||||
return row;
|
||||
}
|
||||
|
||||
public static TypeInfo buildComplexOrcSchema() {
|
||||
return TypeInfoUtils.getTypeInfoFromTypeString("struct<myInt:int,myMap:map<string,double>,myEnum:string,myLongOrFloat:uniontype<int>,myIntList:array<int>>");
|
||||
}
|
||||
|
||||
public static Schema buildNestedComplexAvroSchema() {
|
||||
// Build a fake Avro record with nested complex types
|
||||
final SchemaBuilder.FieldAssembler<Schema> builder = SchemaBuilder.record("nested.complex.record").namespace("any.data").fields();
|
||||
builder.name("myMapOfArray").type().map().values().array().items().doubleType().noDefault();
|
||||
builder.name("myArrayOfMap").type().array().items().map().values().stringType().noDefault();
|
||||
return builder.endRecord();
|
||||
}
|
||||
|
||||
public static GenericData.Record buildNestedComplexAvroRecord(Map<String, List<Double>> m, List<Map<String, String>> a) {
|
||||
Schema schema = buildNestedComplexAvroSchema();
|
||||
GenericData.Record row = new GenericData.Record(schema);
|
||||
row.put("myMapOfArray", m);
|
||||
row.put("myArrayOfMap", a);
|
||||
return row;
|
||||
}
|
||||
|
||||
public static TypeInfo buildNestedComplexOrcSchema() {
|
||||
return TypeInfoUtils.getTypeInfoFromTypeString("struct<myMapOfArray:map<string,array<double>>,myArrayOfMap:array<map<string,string>>>");
|
||||
}
|
||||
|
||||
private static class TypeInfoCreator {
|
||||
static TypeInfo createInt() {
|
||||
return TypeInfoFactory.getPrimitiveTypeInfo("int");
|
||||
}
|
||||
|
||||
static TypeInfo createLong() {
|
||||
return TypeInfoFactory.getPrimitiveTypeInfo("bigint");
|
||||
}
|
||||
|
||||
static TypeInfo createBoolean() {
|
||||
return TypeInfoFactory.getPrimitiveTypeInfo("boolean");
|
||||
}
|
||||
|
||||
static TypeInfo createFloat() {
|
||||
return TypeInfoFactory.getPrimitiveTypeInfo("float");
|
||||
}
|
||||
|
||||
static TypeInfo createDouble() {
|
||||
return TypeInfoFactory.getPrimitiveTypeInfo("double");
|
||||
}
|
||||
|
||||
static TypeInfo createBinary() {
|
||||
return TypeInfoFactory.getPrimitiveTypeInfo("binary");
|
||||
}
|
||||
|
||||
static TypeInfo createString() {
|
||||
return TypeInfoFactory.getPrimitiveTypeInfo("string");
|
||||
}
|
||||
}
|
||||
}
|
|
@ -0,0 +1,38 @@
|
|||
/*
|
||||
* 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.
|
||||
*/
|
||||
{
|
||||
"namespace" : "org.apache.nifi",
|
||||
"name" : "outer_record",
|
||||
"type" : "record",
|
||||
"fields" : [ {
|
||||
"name" : "records",
|
||||
"type" : {
|
||||
"type" : "array",
|
||||
"items" : {
|
||||
"type" : "record",
|
||||
"name" : "inner_record",
|
||||
"fields" : [ {
|
||||
"name" : "name",
|
||||
"type" : "string"
|
||||
}, {
|
||||
"name" : "age",
|
||||
"type" : "string"
|
||||
} ]
|
||||
}
|
||||
}
|
||||
} ]
|
||||
}
|
|
@ -0,0 +1,30 @@
|
|||
<?xml version="1.0"?>
|
||||
<?xml-stylesheet type="text/xsl" href="configuration.xsl"?>
|
||||
<!--
|
||||
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.
|
||||
-->
|
||||
<configuration>
|
||||
<property>
|
||||
<name>fs.default.name</name>
|
||||
<value>hdfs://hive</value>
|
||||
</property>
|
||||
<property>
|
||||
<name>hadoop.security.authentication</name>
|
||||
<value>kerberos</value>
|
||||
</property>
|
||||
<property>
|
||||
<name>hadoop.security.authorization</name>
|
||||
<value>true</value>
|
||||
</property>
|
||||
</configuration>
|
|
@ -0,0 +1,22 @@
|
|||
<?xml version="1.0"?>
|
||||
<?xml-stylesheet type="text/xsl" href="configuration.xsl"?>
|
||||
<!--
|
||||
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.
|
||||
-->
|
||||
<configuration>
|
||||
<property>
|
||||
<name>fs.default.name</name>
|
||||
<value>file:///</value>
|
||||
</property>
|
||||
</configuration>
|
|
@ -0,0 +1,26 @@
|
|||
<?xml version="1.0"?>
|
||||
<?xml-stylesheet type="text/xsl" href="configuration.xsl"?>
|
||||
<!--
|
||||
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.
|
||||
-->
|
||||
<configuration>
|
||||
<property>
|
||||
<name>fs.default.name</name>
|
||||
<value>hdfs://hive</value>
|
||||
</property>
|
||||
<property>
|
||||
<name>hive.server2.authentication</name>
|
||||
<value>KERBEROS</value>
|
||||
</property>
|
||||
</configuration>
|
|
@ -0,0 +1,22 @@
|
|||
<?xml version="1.0"?>
|
||||
<?xml-stylesheet type="text/xsl" href="configuration.xsl"?>
|
||||
<!--
|
||||
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.
|
||||
-->
|
||||
<configuration>
|
||||
<property>
|
||||
<name>fs.default.name</name>
|
||||
<value>file:///</value>
|
||||
</property>
|
||||
</configuration>
|
|
@ -0,0 +1,26 @@
|
|||
/*
|
||||
* 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.
|
||||
*/
|
||||
{"namespace": "example.avro",
|
||||
"type": "record",
|
||||
"name": "User",
|
||||
"fields": [
|
||||
{"name": "name", "type": "string"},
|
||||
{"name": "favorite_number", "type": ["int", "null"]},
|
||||
{"name": "favorite_color", "type": ["string", "null"]},
|
||||
{"name": "scale", "type": ["double", "null"]}
|
||||
]
|
||||
}
|
|
@ -0,0 +1,27 @@
|
|||
/*
|
||||
* 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.
|
||||
*/
|
||||
{"namespace": "example.avro",
|
||||
"type": "record",
|
||||
"name": "User",
|
||||
"fields": [
|
||||
{"name": "id", "type": ["null","int"]},
|
||||
{"name": "timeMillis","type": {"type": "int","logicalType": "time-millis"}},
|
||||
{"name": "timestampMillis","type": {"type": "long","logicalType": "timestamp-millis"}},
|
||||
{"name": "dt","type": {"type": "int","logicalType": "date"}},
|
||||
{"name": "dec","type": {"type": "bytes","logicalType": "decimal", "precision": 4, "scale": 2}}
|
||||
]
|
||||
}
|
|
@ -26,63 +26,22 @@
|
|||
<version>1.7.0-SNAPSHOT</version>
|
||||
<packaging>pom</packaging>
|
||||
|
||||
<properties>
|
||||
<!-- Need to override hadoop.version here, for Hive and hadoop-client transitive dependencies -->
|
||||
<hive.hadoop.version>2.6.2</hive.hadoop.version>
|
||||
<hadoop.version>${hive.hadoop.version}</hadoop.version>
|
||||
</properties>
|
||||
|
||||
<modules>
|
||||
<module>nifi-hive-services-api</module>
|
||||
<module>nifi-hive-services-api-nar</module>
|
||||
<module>nifi-hive-processors</module>
|
||||
<module>nifi-hive-nar</module>
|
||||
<module>nifi-hive3-processors</module>
|
||||
<module>nifi-hive3-nar</module>
|
||||
</modules>
|
||||
|
||||
<dependencyManagement>
|
||||
<dependencies>
|
||||
<dependency>
|
||||
<groupId>org.apache.hadoop</groupId>
|
||||
<artifactId>hadoop-common</artifactId>
|
||||
<version>${hadoop.version}</version>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.apache.hadoop</groupId>
|
||||
<artifactId>hadoop-hdfs</artifactId>
|
||||
<version>${hadoop.version}</version>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.apache.hadoop</groupId>
|
||||
<artifactId>hadoop-yarn-api</artifactId>
|
||||
<version>${hadoop.version}</version>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.apache.hadoop</groupId>
|
||||
<artifactId>hadoop-mapreduce-client-core</artifactId>
|
||||
<version>${hadoop.version}</version>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.apache.hadoop</groupId>
|
||||
<artifactId>hadoop-annotations</artifactId>
|
||||
<version>${hadoop.version}</version>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.apache.hadoop</groupId>
|
||||
<artifactId>hadoop-client</artifactId>
|
||||
<version>${hadoop.version}</version>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.apache.hadoop</groupId>
|
||||
<artifactId>hadoop-auth</artifactId>
|
||||
<version>${hadoop.version}</version>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.apache.avro</groupId>
|
||||
<artifactId>avro</artifactId>
|
||||
<version>1.7.7</version>
|
||||
</dependency>
|
||||
</dependencies>
|
||||
</dependencyManagement>
|
||||
<properties>
|
||||
<hive.version>1.2.1</hive.version>
|
||||
<hive.hadoop.version>2.6.2</hive.hadoop.version>
|
||||
<hive3.version>3.0.0</hive3.version>
|
||||
<hive3.hadoop.version>3.0.0</hive3.hadoop.version>
|
||||
<orc.version>1.4.3</orc.version>
|
||||
</properties>
|
||||
|
||||
<build>
|
||||
<plugins>
|
||||
|
|
Loading…
Reference in New Issue