hiveFields = recordFields.stream().map(
- recordField -> ("`" + (hiveFieldNames ? recordField.getFieldName().toLowerCase() : recordField.getFieldName()) + "`:"
- + getHiveTypeFromFieldType(recordField.getDataType(), hiveFieldNames))).collect(Collectors.toList());
- return "STRUCT<" + StringUtils.join(hiveFields, ", ") + ">";
- }
- return null;
- }
-
- throw new IllegalArgumentException("Error converting Avro type " + dataType.name() + " to Hive type");
- }
-
-
- public static OrcFlowFileWriter createWriter(OutputStream flowFileOutputStream,
- Path path,
- Configuration conf,
- TypeInfo orcSchema,
- long stripeSize,
- CompressionKind compress,
- int bufferSize) throws IOException {
-
- int rowIndexStride = HiveConf.getIntVar(conf, HIVE_ORC_DEFAULT_ROW_INDEX_STRIDE);
-
- boolean addBlockPadding = HiveConf.getBoolVar(conf, HIVE_ORC_DEFAULT_BLOCK_PADDING);
-
- String versionName = HiveConf.getVar(conf, HIVE_ORC_WRITE_FORMAT);
- OrcFile.Version versionValue = (versionName == null)
- ? OrcFile.Version.CURRENT
- : OrcFile.Version.byName(versionName);
-
- OrcFile.EncodingStrategy encodingStrategy;
- String enString = conf.get(HiveConf.ConfVars.HIVE_ORC_ENCODING_STRATEGY.varname);
- if (enString == null) {
- encodingStrategy = OrcFile.EncodingStrategy.SPEED;
- } else {
- encodingStrategy = OrcFile.EncodingStrategy.valueOf(enString);
- }
-
- OrcFile.CompressionStrategy compressionStrategy;
- String compString = conf.get(HiveConf.ConfVars.HIVE_ORC_COMPRESSION_STRATEGY.varname);
- if (compString == null) {
- compressionStrategy = OrcFile.CompressionStrategy.SPEED;
- } else {
- compressionStrategy = OrcFile.CompressionStrategy.valueOf(compString);
- }
-
- float paddingTolerance;
- paddingTolerance = conf.getFloat(HiveConf.ConfVars.HIVE_ORC_BLOCK_PADDING_TOLERANCE.varname,
- HiveConf.ConfVars.HIVE_ORC_BLOCK_PADDING_TOLERANCE.defaultFloatVal);
-
- long blockSizeValue = HiveConf.getLongVar(conf, HIVE_ORC_DEFAULT_BLOCK_SIZE);
-
- double bloomFilterFpp = BloomFilterIO.DEFAULT_FPP;
-
- ObjectInspector inspector = OrcStruct.createObjectInspector(orcSchema);
-
- return new OrcFlowFileWriter(flowFileOutputStream,
- path,
- conf,
- inspector,
- stripeSize,
- compress,
- bufferSize,
- rowIndexStride,
- getMemoryManager(conf),
- addBlockPadding,
- versionValue,
- null, // no callback
- encodingStrategy,
- compressionStrategy,
- paddingTolerance,
- blockSizeValue,
- null, // no Bloom Filter column names
- bloomFilterFpp);
- }
-
- private static MemoryManager memoryManager = null;
-
- private static synchronized MemoryManager getMemoryManager(Configuration conf) {
- if (memoryManager == null) {
- memoryManager = new MemoryManager(conf);
- }
- return memoryManager;
- }
-}
diff --git a/nifi-nar-bundles/nifi-hive-bundle/nifi-hive-processors/src/main/java/org/apache/hadoop/hive/ql/io/orc/OrcFlowFileWriter.java b/nifi-nar-bundles/nifi-hive-bundle/nifi-hive-processors/src/main/java/org/apache/hadoop/hive/ql/io/orc/OrcFlowFileWriter.java
deleted file mode 100644
index c868b91a52..0000000000
--- a/nifi-nar-bundles/nifi-hive-bundle/nifi-hive-processors/src/main/java/org/apache/hadoop/hive/ql/io/orc/OrcFlowFileWriter.java
+++ /dev/null
@@ -1,2649 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements. See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.hive.ql.io.orc;
-
-import com.google.common.annotations.VisibleForTesting;
-import com.google.common.base.Joiner;
-import com.google.common.collect.Lists;
-import com.google.common.primitives.Longs;
-import com.google.protobuf.ByteString;
-import com.google.protobuf.CodedOutputStream;
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.hive.common.JavaUtils;
-import org.apache.hadoop.hive.common.type.HiveDecimal;
-import org.apache.hadoop.hive.conf.HiveConf;
-import org.apache.hadoop.hive.ql.exec.vector.ColumnVector;
-import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch;
-import org.apache.hadoop.hive.ql.io.IOConstants;
-import org.apache.hadoop.hive.ql.io.filters.BloomFilterIO;
-import org.apache.hadoop.hive.ql.io.orc.CompressionCodec.Modifier;
-import org.apache.hadoop.hive.ql.io.orc.OrcFile.CompressionStrategy;
-import org.apache.hadoop.hive.ql.io.orc.OrcFile.EncodingStrategy;
-import org.apache.hadoop.hive.ql.io.orc.OrcProto.RowIndexEntry;
-import org.apache.hadoop.hive.ql.io.orc.OrcProto.StripeStatistics;
-import org.apache.hadoop.hive.ql.io.orc.OrcProto.Type;
-import org.apache.hadoop.hive.ql.io.orc.OrcProto.UserMetadataItem;
-import org.apache.hadoop.hive.ql.util.JavaDataModel;
-import org.apache.hadoop.hive.serde2.io.DateWritable;
-import org.apache.hadoop.hive.serde2.objectinspector.ListObjectInspector;
-import org.apache.hadoop.hive.serde2.objectinspector.MapObjectInspector;
-import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector;
-import org.apache.hadoop.hive.serde2.objectinspector.PrimitiveObjectInspector;
-import org.apache.hadoop.hive.serde2.objectinspector.StructField;
-import org.apache.hadoop.hive.serde2.objectinspector.StructObjectInspector;
-import org.apache.hadoop.hive.serde2.objectinspector.UnionObjectInspector;
-import org.apache.hadoop.hive.serde2.objectinspector.primitive.BinaryObjectInspector;
-import org.apache.hadoop.hive.serde2.objectinspector.primitive.BooleanObjectInspector;
-import org.apache.hadoop.hive.serde2.objectinspector.primitive.ByteObjectInspector;
-import org.apache.hadoop.hive.serde2.objectinspector.primitive.DateObjectInspector;
-import org.apache.hadoop.hive.serde2.objectinspector.primitive.DoubleObjectInspector;
-import org.apache.hadoop.hive.serde2.objectinspector.primitive.FloatObjectInspector;
-import org.apache.hadoop.hive.serde2.objectinspector.primitive.HiveCharObjectInspector;
-import org.apache.hadoop.hive.serde2.objectinspector.primitive.HiveDecimalObjectInspector;
-import org.apache.hadoop.hive.serde2.objectinspector.primitive.HiveVarcharObjectInspector;
-import org.apache.hadoop.hive.serde2.objectinspector.primitive.IntObjectInspector;
-import org.apache.hadoop.hive.serde2.objectinspector.primitive.LongObjectInspector;
-import org.apache.hadoop.hive.serde2.objectinspector.primitive.ShortObjectInspector;
-import org.apache.hadoop.hive.serde2.objectinspector.primitive.StringObjectInspector;
-import org.apache.hadoop.hive.serde2.objectinspector.primitive.TimestampObjectInspector;
-import org.apache.hadoop.hive.serde2.typeinfo.CharTypeInfo;
-import org.apache.hadoop.hive.serde2.typeinfo.DecimalTypeInfo;
-import org.apache.hadoop.hive.serde2.typeinfo.VarcharTypeInfo;
-import org.apache.hadoop.io.BytesWritable;
-import org.apache.hadoop.io.Text;
-import org.apache.nifi.stream.io.ByteCountingOutputStream;
-
-import java.io.IOException;
-import java.io.OutputStream;
-import java.lang.management.ManagementFactory;
-import java.nio.ByteBuffer;
-import java.sql.Timestamp;
-import java.util.ArrayList;
-import java.util.EnumSet;
-import java.util.List;
-import java.util.Map;
-import java.util.Objects;
-import java.util.TimeZone;
-import java.util.TreeMap;
-import java.util.function.Function;
-import java.util.stream.Collectors;
-
-/**
- * An ORC file writer. The file is divided into stripes, which is the natural
- * unit of work when reading. Each stripe is buffered in memory until the
- * memory reaches the stripe size and then it is written out broken down by
- * columns. Each column is written by a TreeWriter that is specific to that
- * type of column. TreeWriters may have children TreeWriters that handle the
- * sub-types. Each of the TreeWriters writes the column's data as a set of
- * streams.
- *
- * This class is synchronized so that multi-threaded access is ok. In
- * particular, because the MemoryManager is shared between writers, this class
- * assumes that checkMemory may be called from a separate thread.
- */
-public class OrcFlowFileWriter implements Writer, MemoryManager.Callback {
-
- private static final Log LOG = LogFactory.getLog(WriterImpl.class);
-
- private static final int HDFS_BUFFER_SIZE = 256 * 1024;
- private static final int MIN_ROW_INDEX_STRIDE = 1000;
-
- // threshold above which buffer size will be automatically resized
- private static final int COLUMN_COUNT_THRESHOLD = 1000;
-
- private final Path path;
- private final long defaultStripeSize;
- private long adjustedStripeSize;
- private final int rowIndexStride;
- private final CompressionKind compress;
- private final CompressionCodec codec;
- private final boolean addBlockPadding;
- private final int bufferSize;
- private final long blockSize;
- private final float paddingTolerance;
- // the streams that make up the current stripe
- private final Map streams = new TreeMap<>();
-
- private final OutputStream flowFileOutputStream;
-
- private ByteCountingOutputStream rawWriter = null;
- // the compressed metadata information outStream
- private OutStream writer = null;
- // a protobuf outStream around streamFactory
- private CodedOutputStream protobufWriter = null;
- private long headerLength;
- private int columnCount;
- private long rowCount = 0;
- private long rowsInStripe = 0;
- private long rawDataSize = 0;
- private int rowsInIndex = 0;
- private int stripesAtLastFlush = -1;
- private final List stripes = new ArrayList<>();
- private final Map userMetadata = new TreeMap<>();
- private final StreamFactory streamFactory = new StreamFactory();
- private final TreeWriter treeWriter;
- private final boolean buildIndex;
- private final MemoryManager memoryManager;
- private final OrcFile.Version version;
- private final Configuration conf;
- private final OrcFile.WriterCallback callback;
- private final OrcFile.WriterContext callbackContext;
- private final OrcFile.EncodingStrategy encodingStrategy;
- private final OrcFile.CompressionStrategy compressionStrategy;
- private final boolean[] bloomFilterColumns;
- private final double bloomFilterFpp;
-
-
- public OrcFlowFileWriter(OutputStream flowFileOutputStream,
- Path path,
- Configuration conf,
- ObjectInspector inspector,
- long stripeSize,
- CompressionKind compress,
- int bufferSize,
- int rowIndexStride,
- MemoryManager memoryManager,
- boolean addBlockPadding,
- OrcFile.Version version,
- OrcFile.WriterCallback callback,
- EncodingStrategy encodingStrategy,
- CompressionStrategy compressionStrategy,
- float paddingTolerance,
- long blockSizeValue,
- String bloomFilterColumnNames,
- double bloomFilterFpp) throws IOException {
- this.flowFileOutputStream = flowFileOutputStream;
- this.path = path;
- this.conf = conf;
- this.callback = callback;
- callbackContext = (callback != null) ? () -> OrcFlowFileWriter.this : null;
- this.adjustedStripeSize = stripeSize;
- this.defaultStripeSize = stripeSize;
- this.version = version;
- this.encodingStrategy = encodingStrategy;
- this.compressionStrategy = compressionStrategy;
- this.addBlockPadding = addBlockPadding;
- this.blockSize = blockSizeValue;
- this.paddingTolerance = paddingTolerance;
- this.compress = compress;
- this.rowIndexStride = rowIndexStride;
- this.memoryManager = memoryManager;
- buildIndex = rowIndexStride > 0;
- codec = createCodec(compress);
- String allColumns = conf.get(IOConstants.COLUMNS);
- if (allColumns == null) {
- allColumns = getColumnNamesFromInspector(inspector);
- }
- this.bufferSize = getEstimatedBufferSize(allColumns, bufferSize);
- if (version == OrcFile.Version.V_0_11) {
- /* do not write bloom filters for ORC v11 */
- this.bloomFilterColumns =
- OrcUtils.includeColumns(null, allColumns, inspector);
- } else {
- this.bloomFilterColumns =
- OrcUtils.includeColumns(bloomFilterColumnNames, allColumns, inspector);
- }
- this.bloomFilterFpp = bloomFilterFpp;
- treeWriter = createTreeWriter(inspector, streamFactory, false);
- if (buildIndex && rowIndexStride < MIN_ROW_INDEX_STRIDE) {
- throw new IllegalArgumentException("Row stride must be at least " +
- MIN_ROW_INDEX_STRIDE);
- }
-
- // ensure that we are able to handle callbacks before we register ourselves
- memoryManager.addWriter(path, stripeSize, this);
- }
-
- private String getColumnNamesFromInspector(ObjectInspector inspector) {
- List fieldNames = Lists.newArrayList();
- Joiner joiner = Joiner.on(",");
- if (inspector instanceof StructObjectInspector) {
- StructObjectInspector soi = (StructObjectInspector) inspector;
- List extends StructField> fields = soi.getAllStructFieldRefs();
- fieldNames.addAll(fields.stream().map((Function) StructField::getFieldName).collect(Collectors.toList()));
- }
- return joiner.join(fieldNames);
- }
-
- @VisibleForTesting
- int getEstimatedBufferSize(int bs) {
- return getEstimatedBufferSize(conf.get(IOConstants.COLUMNS), bs);
- }
-
- int getEstimatedBufferSize(String colNames, int bs) {
- long availableMem = getMemoryAvailableForORC();
- if (colNames != null) {
- final int numCols = colNames.split(",").length;
- if (numCols > COLUMN_COUNT_THRESHOLD) {
- // In BufferedStream, there are 3 outstream buffers (compressed,
- // uncompressed and overflow) and list of previously compressed buffers.
- // Since overflow buffer is rarely used, lets consider only 2 allocation.
- // Also, initially, the list of compression buffers will be empty.
- final int outStreamBuffers = codec == null ? 1 : 2;
-
- // max possible streams per column is 5. For string columns, there is
- // ROW_INDEX, PRESENT, DATA, LENGTH, DICTIONARY_DATA streams.
- final int maxStreams = 5;
-
- // Lets assume 10% memory for holding dictionary in memory and other
- // object allocations
- final long miscAllocation = (long) (0.1f * availableMem);
-
- // compute the available memory
- final long remainingMem = availableMem - miscAllocation;
-
- int estBufferSize = (int) (remainingMem / (maxStreams * outStreamBuffers * numCols));
- estBufferSize = getClosestBufferSize(estBufferSize, bs);
- if (estBufferSize > bs) {
- estBufferSize = bs;
- }
-
- LOG.info("WIDE TABLE - Number of columns: " + numCols + " Chosen compression buffer size: " + estBufferSize);
- return estBufferSize;
- }
- }
- return bs;
- }
-
- private int getClosestBufferSize(int estBufferSize, int bs) {
- final int kb4 = 4 * 1024;
- final int kb8 = 8 * 1024;
- final int kb16 = 16 * 1024;
- final int kb32 = 32 * 1024;
- final int kb64 = 64 * 1024;
- final int kb128 = 128 * 1024;
- final int kb256 = 256 * 1024;
- if (estBufferSize <= kb4) {
- return kb4;
- } else if (estBufferSize > kb4 && estBufferSize <= kb8) {
- return kb8;
- } else if (estBufferSize > kb8 && estBufferSize <= kb16) {
- return kb16;
- } else if (estBufferSize > kb16 && estBufferSize <= kb32) {
- return kb32;
- } else if (estBufferSize > kb32 && estBufferSize <= kb64) {
- return kb64;
- } else if (estBufferSize > kb64 && estBufferSize <= kb128) {
- return kb128;
- } else {
- return kb256;
- }
- }
-
- // the assumption is only one ORC writer open at a time, which holds true for
- // most of the cases. HIVE-6455 forces single writer case.
- private long getMemoryAvailableForORC() {
- HiveConf.ConfVars poolVar = HiveConf.ConfVars.HIVE_ORC_FILE_MEMORY_POOL;
- double maxLoad = conf.getFloat(poolVar.varname, poolVar.defaultFloatVal);
- long totalMemoryPool = Math.round(ManagementFactory.getMemoryMXBean().getHeapMemoryUsage().getMax() * maxLoad);
- return totalMemoryPool;
- }
-
- public static CompressionCodec createCodec(CompressionKind kind) {
- switch (kind) {
- case NONE:
- return null;
- case ZLIB:
- return new ZlibCodec();
- case SNAPPY:
- return new SnappyCodec();
- case LZO:
- try {
- Class extends CompressionCodec> lzo =
- (Class extends CompressionCodec>)
- JavaUtils.loadClass("org.apache.hadoop.hive.ql.io.orc.LzoCodec");
- return lzo.newInstance();
- } catch (ClassNotFoundException e) {
- throw new IllegalArgumentException("LZO is not available.", e);
- } catch (InstantiationException e) {
- throw new IllegalArgumentException("Problem initializing LZO", e);
- } catch (IllegalAccessException e) {
- throw new IllegalArgumentException("Insufficient access to LZO", e);
- }
- default:
- throw new IllegalArgumentException("Unknown compression codec: " +
- kind);
- }
- }
-
- @Override
- public synchronized boolean checkMemory(double newScale) throws IOException {
- long limit = Math.round(adjustedStripeSize * newScale);
- long size = estimateStripeSize();
- if (LOG.isDebugEnabled()) {
- LOG.debug("ORC writer " + path + " size = " + size + " limit = " +
- limit);
- }
- if (size > limit) {
- flushStripe();
- return true;
- }
- return false;
- }
-
- /**
- * This class is used to hold the contents of streams as they are buffered.
- * The TreeWriters write to the outStream and the codec compresses the
- * data as buffers fill up and stores them in the output list. When the
- * stripe is being written, the whole stream is written to the file.
- */
- private class BufferedStream implements OutStream.OutputReceiver {
- private final OutStream outStream;
- private final List output = new ArrayList();
-
- BufferedStream(String name, int bufferSize,
- CompressionCodec codec) throws IOException {
- outStream = new OutStream(name, bufferSize, codec, this);
- }
-
- /**
- * Receive a buffer from the compression codec.
- *
- * @param buffer the buffer to save
- * @throws IOException if an error occurs while receiving a buffer
- */
- @Override
- public void output(ByteBuffer buffer) {
- output.add(buffer);
- }
-
- /**
- * Get the number of bytes in buffers that are allocated to this stream.
- *
- * @return number of bytes in buffers
- */
- public long getBufferSize() {
- long result = 0;
- for (ByteBuffer buf : output) {
- result += buf.capacity();
- }
- return outStream.getBufferSize() + result;
- }
-
- /**
- * Flush the stream to the codec.
- *
- * @throws IOException if an error occurs while flushing the stream
- */
- public void flush() throws IOException {
- outStream.flush();
- }
-
- /**
- * Clear all of the buffers.
- *
- * @throws IOException if an error occurs while clearing the buffers
- */
- public void clear() throws IOException {
- outStream.clear();
- output.clear();
- }
-
- /**
- * Check the state of suppress flag in output stream
- *
- * @return value of suppress flag
- */
- public boolean isSuppressed() {
- return outStream.isSuppressed();
- }
-
- /**
- * Get the number of bytes that will be written to the output. Assumes
- * the stream has already been flushed.
- *
- * @return the number of bytes
- */
- public long getOutputSize() {
- long result = 0;
- for (ByteBuffer buffer : output) {
- result += buffer.remaining();
- }
- return result;
- }
-
- /**
- * Write the saved compressed buffers to the OutputStream.
- *
- * @param out the stream to write to
- * @throws IOException if an error occurs during write
- */
- void spillTo(OutputStream out) throws IOException {
- for (ByteBuffer buffer : output) {
- out.write(buffer.array(), buffer.arrayOffset() + buffer.position(),
- buffer.remaining());
- }
- }
-
- @Override
- public String toString() {
- return outStream.toString();
- }
- }
-
- /**
- * An output receiver that writes the ByteBuffers to the output stream
- * as they are received.
- */
- private class DirectStream implements OutStream.OutputReceiver {
- private final OutputStream output;
-
- DirectStream(OutputStream output) {
- this.output = output;
- }
-
- @Override
- public void output(ByteBuffer buffer) throws IOException {
- output.write(buffer.array(), buffer.arrayOffset() + buffer.position(),
- buffer.remaining());
- }
- }
-
- private static class RowIndexPositionRecorder implements PositionRecorder {
- private final OrcProto.RowIndexEntry.Builder builder;
-
- RowIndexPositionRecorder(OrcProto.RowIndexEntry.Builder builder) {
- this.builder = builder;
- }
-
- @Override
- public void addPosition(long position) {
- builder.addPositions(position);
- }
- }
-
- /**
- * Interface from the Writer to the TreeWriters. This limits the visibility
- * that the TreeWriters have into the Writer.
- */
- private class StreamFactory {
- /**
- * Create a stream to store part of a column.
- *
- * @param column the column id for the stream
- * @param kind the kind of stream
- * @return The output outStream that the section needs to be written to.
- * @throws IOException if an error occurs while creating the stream
- */
- public OutStream createStream(int column,
- OrcProto.Stream.Kind kind
- ) throws IOException {
- final StreamName name = new StreamName(column, kind);
- final EnumSet modifiers;
-
- switch (kind) {
- case BLOOM_FILTER:
- case DATA:
- case DICTIONARY_DATA:
- if (getCompressionStrategy() == CompressionStrategy.SPEED) {
- modifiers = EnumSet.of(Modifier.FAST, Modifier.TEXT);
- } else {
- modifiers = EnumSet.of(Modifier.DEFAULT, Modifier.TEXT);
- }
- break;
- case LENGTH:
- case DICTIONARY_COUNT:
- case PRESENT:
- case ROW_INDEX:
- case SECONDARY:
- // easily compressed using the fastest modes
- modifiers = EnumSet.of(Modifier.FASTEST, Modifier.BINARY);
- break;
- default:
- LOG.warn("Missing ORC compression modifiers for " + kind);
- modifiers = null;
- break;
- }
-
- BufferedStream result = streams.get(name);
- if (result == null) {
- result = new BufferedStream(name.toString(), bufferSize,
- codec == null ? codec : codec.modify(modifiers));
- streams.put(name, result);
- }
- return result.outStream;
- }
-
- /**
- * Get the next column id.
- *
- * @return a number from 0 to the number of columns - 1
- */
- public int getNextColumnId() {
- return columnCount++;
- }
-
- /**
- * Get the current column id. After creating all tree writers this count should tell how many
- * columns (including columns within nested complex objects) are created in total.
- *
- * @return current column id
- */
- public int getCurrentColumnId() {
- return columnCount;
- }
-
- /**
- * Get the stride rate of the row index.
- */
- public int getRowIndexStride() {
- return rowIndexStride;
- }
-
- /**
- * Should be building the row index.
- *
- * @return true if we are building the index
- */
- public boolean buildIndex() {
- return buildIndex;
- }
-
- /**
- * Is the ORC file compressed?
- *
- * @return are the streams compressed
- */
- public boolean isCompressed() {
- return codec != null;
- }
-
- /**
- * Get the encoding strategy to use.
- *
- * @return encoding strategy
- */
- public EncodingStrategy getEncodingStrategy() {
- return encodingStrategy;
- }
-
- /**
- * Get the compression strategy to use.
- *
- * @return compression strategy
- */
- public CompressionStrategy getCompressionStrategy() {
- return compressionStrategy;
- }
-
- /**
- * Get the bloom filter columns
- *
- * @return bloom filter columns
- */
- public boolean[] getBloomFilterColumns() {
- return bloomFilterColumns;
- }
-
- /**
- * Get bloom filter false positive percentage.
- *
- * @return fpp
- */
- public double getBloomFilterFPP() {
- return bloomFilterFpp;
- }
-
- /**
- * Get the writer's configuration.
- *
- * @return configuration
- */
- public Configuration getConfiguration() {
- return conf;
- }
-
- /**
- * Get the version of the file to write.
- */
- public OrcFile.Version getVersion() {
- return version;
- }
- }
-
- /**
- * The parent class of all of the writers for each column. Each column
- * is written by an instance of this class. The compound types (struct,
- * list, map, and union) have children tree writers that write the children
- * types.
- */
- private abstract static class TreeWriter {
- protected final int id;
- protected final ObjectInspector inspector;
- private final BitFieldWriter isPresent;
- private final boolean isCompressed;
- protected final ColumnStatisticsImpl indexStatistics;
- protected final ColumnStatisticsImpl stripeColStatistics;
- private final ColumnStatisticsImpl fileStatistics;
- protected TreeWriter[] childrenWriters;
- protected final RowIndexPositionRecorder rowIndexPosition;
- private final OrcProto.RowIndex.Builder rowIndex;
- private final OrcProto.RowIndexEntry.Builder rowIndexEntry;
- private final PositionedOutputStream rowIndexStream;
- private final PositionedOutputStream bloomFilterStream;
- protected final BloomFilterIO bloomFilter;
- protected final boolean createBloomFilter;
- private final OrcProto.BloomFilterIndex.Builder bloomFilterIndex;
- private final OrcProto.BloomFilter.Builder bloomFilterEntry;
- private boolean foundNulls;
- private OutStream isPresentOutStream;
- private final List stripeStatsBuilders;
-
- /**
- * Create a tree writer.
- *
- * @param columnId the column id of the column to write
- * @param inspector the object inspector to use
- * @param streamFactory limited access to the Writer's data.
- * @param nullable can the value be null?
- * @throws IOException if an error occurs during creation
- */
- TreeWriter(int columnId, ObjectInspector inspector,
- StreamFactory streamFactory,
- boolean nullable) throws IOException {
- this.isCompressed = streamFactory.isCompressed();
- this.id = columnId;
- this.inspector = inspector;
- if (nullable) {
- isPresentOutStream = streamFactory.createStream(id,
- OrcProto.Stream.Kind.PRESENT);
- isPresent = new BitFieldWriter(isPresentOutStream, 1);
- } else {
- isPresent = null;
- }
- this.foundNulls = false;
- createBloomFilter = streamFactory.getBloomFilterColumns()[columnId];
- indexStatistics = ColumnStatisticsImpl.create(inspector);
- stripeColStatistics = ColumnStatisticsImpl.create(inspector);
- fileStatistics = ColumnStatisticsImpl.create(inspector);
- childrenWriters = new TreeWriter[0];
- rowIndex = OrcProto.RowIndex.newBuilder();
- rowIndexEntry = OrcProto.RowIndexEntry.newBuilder();
- rowIndexPosition = new RowIndexPositionRecorder(rowIndexEntry);
- stripeStatsBuilders = Lists.newArrayList();
- if (streamFactory.buildIndex()) {
- rowIndexStream = streamFactory.createStream(id, OrcProto.Stream.Kind.ROW_INDEX);
- } else {
- rowIndexStream = null;
- }
- if (createBloomFilter) {
- bloomFilterEntry = OrcProto.BloomFilter.newBuilder();
- bloomFilterIndex = OrcProto.BloomFilterIndex.newBuilder();
- bloomFilterStream = streamFactory.createStream(id, OrcProto.Stream.Kind.BLOOM_FILTER);
- bloomFilter = new BloomFilterIO(streamFactory.getRowIndexStride(),
- streamFactory.getBloomFilterFPP());
- } else {
- bloomFilterEntry = null;
- bloomFilterIndex = null;
- bloomFilterStream = null;
- bloomFilter = null;
- }
- }
-
- protected OrcProto.RowIndex.Builder getRowIndex() {
- return rowIndex;
- }
-
- protected ColumnStatisticsImpl getStripeStatistics() {
- return stripeColStatistics;
- }
-
- protected ColumnStatisticsImpl getFileStatistics() {
- return fileStatistics;
- }
-
- protected OrcProto.RowIndexEntry.Builder getRowIndexEntry() {
- return rowIndexEntry;
- }
-
- IntegerWriter createIntegerWriter(PositionedOutputStream output,
- boolean signed, boolean isDirectV2,
- StreamFactory writer) {
- if (isDirectV2) {
- boolean alignedBitpacking = false;
- if (writer.getEncodingStrategy().equals(EncodingStrategy.SPEED)) {
- alignedBitpacking = true;
- }
- return new RunLengthIntegerWriterV2(output, signed, alignedBitpacking);
- } else {
- return new RunLengthIntegerWriter(output, signed);
- }
- }
-
- boolean isNewWriteFormat(StreamFactory writer) {
- return writer.getVersion() != OrcFile.Version.V_0_11;
- }
-
- /**
- * Add a new value to the column.
- *
- * @param obj The value to write to the column
- * @throws IOException if an error occurs during add
- */
- void write(Object obj) throws IOException {
- if (obj != null) {
- indexStatistics.increment();
- } else {
- indexStatistics.setNull();
- }
- if (isPresent != null) {
- isPresent.write(obj == null ? 0 : 1);
- if (obj == null) {
- foundNulls = true;
- }
- }
- }
-
- private void removeIsPresentPositions() {
- for (int i = 0; i < rowIndex.getEntryCount(); ++i) {
- RowIndexEntry.Builder entry = rowIndex.getEntryBuilder(i);
- List positions = entry.getPositionsList();
- // bit streams use 3 positions if uncompressed, 4 if compressed
- positions = positions.subList(isCompressed ? 4 : 3, positions.size());
- entry.clearPositions();
- entry.addAllPositions(positions);
- }
- }
-
- /**
- * Write the stripe out to the file.
- *
- * @param builder the stripe footer that contains the information about the
- * layout of the stripe. The TreeWriter is required to update
- * the footer with its information.
- * @param requiredIndexEntries the number of index entries that are
- * required. this is to check to make sure the
- * row index is well formed.
- * @throws IOException if an error occurs during write
- */
- void writeStripe(OrcProto.StripeFooter.Builder builder,
- int requiredIndexEntries) throws IOException {
- if (isPresent != null) {
- isPresent.flush();
-
- // if no nulls are found in a stream, then suppress the stream
- if (!foundNulls) {
- isPresentOutStream.suppress();
- // since isPresent bitstream is suppressed, update the index to
- // remove the positions of the isPresent stream
- if (rowIndexStream != null) {
- removeIsPresentPositions();
- }
- }
- }
-
- // merge stripe-level column statistics to file statistics and write it to
- // stripe statistics
- OrcProto.StripeStatistics.Builder stripeStatsBuilder = OrcProto.StripeStatistics.newBuilder();
- writeStripeStatistics(stripeStatsBuilder, this);
- stripeStatsBuilders.add(stripeStatsBuilder);
-
- // reset the flag for next stripe
- foundNulls = false;
-
- builder.addColumns(getEncoding());
- builder.setWriterTimezone(TimeZone.getDefault().getID());
- if (rowIndexStream != null) {
- if (rowIndex.getEntryCount() != requiredIndexEntries) {
- throw new IllegalArgumentException("Column has wrong number of " +
- "index entries found: " + rowIndex.getEntryCount() + " expected: " +
- requiredIndexEntries);
- }
- rowIndex.build().writeTo(rowIndexStream);
- rowIndexStream.flush();
- }
- rowIndex.clear();
- rowIndexEntry.clear();
-
- // write the bloom filter to out stream
- if (bloomFilterStream != null) {
- bloomFilterIndex.build().writeTo(bloomFilterStream);
- bloomFilterStream.flush();
- bloomFilterIndex.clear();
- bloomFilterEntry.clear();
- }
- }
-
- private void writeStripeStatistics(OrcProto.StripeStatistics.Builder builder,
- TreeWriter treeWriter) {
- treeWriter.fileStatistics.merge(treeWriter.stripeColStatistics);
- builder.addColStats(treeWriter.stripeColStatistics.serialize().build());
- treeWriter.stripeColStatistics.reset();
- for (TreeWriter child : treeWriter.getChildrenWriters()) {
- writeStripeStatistics(builder, child);
- }
- }
-
- TreeWriter[] getChildrenWriters() {
- return childrenWriters;
- }
-
- /**
- * Get the encoding for this column.
- *
- * @return the information about the encoding of this column
- */
- OrcProto.ColumnEncoding getEncoding() {
- return OrcProto.ColumnEncoding.newBuilder().setKind(
- OrcProto.ColumnEncoding.Kind.DIRECT).build();
- }
-
- /**
- * Create a row index entry with the previous location and the current
- * index statistics. Also merges the index statistics into the file
- * statistics before they are cleared. Finally, it records the start of the
- * next index and ensures all of the children columns also create an entry.
- *
- * @throws IOException if an error occurs during create
- */
- void createRowIndexEntry() throws IOException {
- stripeColStatistics.merge(indexStatistics);
- rowIndexEntry.setStatistics(indexStatistics.serialize());
- indexStatistics.reset();
- rowIndex.addEntry(rowIndexEntry);
- rowIndexEntry.clear();
- addBloomFilterEntry();
- recordPosition(rowIndexPosition);
- for (TreeWriter child : childrenWriters) {
- child.createRowIndexEntry();
- }
- }
-
- void addBloomFilterEntry() {
- if (createBloomFilter) {
- bloomFilterEntry.setNumHashFunctions(bloomFilter.getNumHashFunctions());
- bloomFilterEntry.addAllBitset(Longs.asList(bloomFilter.getBitSet()));
- bloomFilterIndex.addBloomFilter(bloomFilterEntry.build());
- bloomFilter.reset();
- bloomFilterEntry.clear();
- }
- }
-
- /**
- * Record the current position in each of this column's streams.
- *
- * @param recorder where should the locations be recorded
- * @throws IOException if an error occurs during position recording
- */
- void recordPosition(PositionRecorder recorder) throws IOException {
- if (isPresent != null) {
- isPresent.getPosition(recorder);
- }
- }
-
- /**
- * Estimate how much memory the writer is consuming excluding the streams.
- *
- * @return the number of bytes.
- */
- long estimateMemory() {
- long result = 0;
- for (TreeWriter child : childrenWriters) {
- result += child.estimateMemory();
- }
- return result;
- }
-
- /**
- * Handle the top level object write.
- *
- * This default method is used for all types except structs, which are the
- * typical case. VectorizedRowBatch assumes the top level object is a
- * struct, so we use the first column for all other types.
- * @param batch the batch to write from
- * @param offset the row to start on
- * @param length the number of rows to write
- * @throws IOException if an error occurs during write
- */
- void writeRootBatch(VectorizedRowBatch batch, int offset,
- int length) throws IOException {
- writeBatch(batch.cols[0], offset, length);
- }
-
- /**
- * Write the values from the given vector from offset for length elements.
- * @param vector the vector to write from
- * @param offset the first value from the vector to write
- * @param length the number of values from the vector to write
- * @throws IOException if an error occurs during write
- */
- void writeBatch(ColumnVector vector, int offset,
- int length) throws IOException {
- if (vector.noNulls) {
- if (isPresent != null) {
- for (int i = 0; i < length; ++i) {
- isPresent.write(1);
- indexStatistics.increment();
- }
- }
- } else {
- if (vector.isRepeating) {
- boolean isNull = vector.isNull[0];
- if (isPresent != null) {
- for (int i = 0; i < length; ++i) {
- isPresent.write(isNull ? 0 : 1);
- }
- }
- if (isNull) {
- foundNulls = true;
- indexStatistics.setNull();
- } else {
- indexStatistics.increment();
- }
- } else {
- // count the number of non-null values
- int nonNullCount = 0;
- for(int i = 0; i < length; ++i) {
- boolean isNull = vector.isNull[i + offset];
- if (!isNull) {
- nonNullCount += 1;
- }
- if (isPresent != null) {
- isPresent.write(isNull ? 0 : 1);
- }
- }
- indexStatistics.increment();
- if (nonNullCount != length) {
- foundNulls = true;
- indexStatistics.setNull();
- }
- }
- }
- }
- }
-
- private static class BooleanTreeWriter extends TreeWriter {
- private final BitFieldWriter writer;
-
- BooleanTreeWriter(int columnId,
- ObjectInspector inspector,
- StreamFactory writer,
- boolean nullable) throws IOException {
- super(columnId, inspector, writer, nullable);
- PositionedOutputStream out = writer.createStream(id,
- OrcProto.Stream.Kind.DATA);
- this.writer = new BitFieldWriter(out, 1);
- recordPosition(rowIndexPosition);
- }
-
- @Override
- void write(Object obj) throws IOException {
- super.write(obj);
- if (obj != null) {
- boolean val = ((BooleanObjectInspector) inspector).get(obj);
- indexStatistics.updateBoolean(val);
- writer.write(val ? 1 : 0);
- }
- }
-
- @Override
- void writeStripe(OrcProto.StripeFooter.Builder builder,
- int requiredIndexEntries) throws IOException {
- super.writeStripe(builder, requiredIndexEntries);
- writer.flush();
- recordPosition(rowIndexPosition);
- }
-
- @Override
- void recordPosition(PositionRecorder recorder) throws IOException {
- super.recordPosition(recorder);
- writer.getPosition(recorder);
- }
- }
-
- private static class ByteTreeWriter extends TreeWriter {
- private final RunLengthByteWriter writer;
-
- ByteTreeWriter(int columnId,
- ObjectInspector inspector,
- StreamFactory writer,
- boolean nullable) throws IOException {
- super(columnId, inspector, writer, nullable);
- this.writer = new RunLengthByteWriter(writer.createStream(id,
- OrcProto.Stream.Kind.DATA));
- recordPosition(rowIndexPosition);
- }
-
- @Override
- void write(Object obj) throws IOException {
- super.write(obj);
- if (obj != null) {
- byte val = ((ByteObjectInspector) inspector).get(obj);
- indexStatistics.updateInteger(val);
- if (createBloomFilter) {
- bloomFilter.addLong(val);
- }
- writer.write(val);
- }
- }
-
- @Override
- void writeStripe(OrcProto.StripeFooter.Builder builder,
- int requiredIndexEntries) throws IOException {
- super.writeStripe(builder, requiredIndexEntries);
- writer.flush();
- recordPosition(rowIndexPosition);
- }
-
- @Override
- void recordPosition(PositionRecorder recorder) throws IOException {
- super.recordPosition(recorder);
- writer.getPosition(recorder);
- }
- }
-
- private static class IntegerTreeWriter extends TreeWriter {
- private final IntegerWriter writer;
- private final ShortObjectInspector shortInspector;
- private final IntObjectInspector intInspector;
- private final LongObjectInspector longInspector;
- private boolean isDirectV2 = true;
-
- IntegerTreeWriter(int columnId,
- ObjectInspector inspector,
- StreamFactory writer,
- boolean nullable) throws IOException {
- super(columnId, inspector, writer, nullable);
- OutStream out = writer.createStream(id,
- OrcProto.Stream.Kind.DATA);
- this.isDirectV2 = isNewWriteFormat(writer);
- this.writer = createIntegerWriter(out, true, isDirectV2, writer);
- if (inspector instanceof IntObjectInspector) {
- intInspector = (IntObjectInspector) inspector;
- shortInspector = null;
- longInspector = null;
- } else {
- intInspector = null;
- if (inspector instanceof LongObjectInspector) {
- longInspector = (LongObjectInspector) inspector;
- shortInspector = null;
- } else {
- shortInspector = (ShortObjectInspector) inspector;
- longInspector = null;
- }
- }
- recordPosition(rowIndexPosition);
- }
-
- @Override
- OrcProto.ColumnEncoding getEncoding() {
- if (isDirectV2) {
- return OrcProto.ColumnEncoding.newBuilder()
- .setKind(OrcProto.ColumnEncoding.Kind.DIRECT_V2).build();
- }
- return OrcProto.ColumnEncoding.newBuilder()
- .setKind(OrcProto.ColumnEncoding.Kind.DIRECT).build();
- }
-
- @Override
- void write(Object obj) throws IOException {
- super.write(obj);
- if (obj != null) {
- long val;
- if (intInspector != null) {
- val = intInspector.get(obj);
- } else if (longInspector != null) {
- val = longInspector.get(obj);
- } else {
- val = shortInspector.get(obj);
- }
- indexStatistics.updateInteger(val);
- if (createBloomFilter) {
- // integers are converted to longs in column statistics and during SARG evaluation
- bloomFilter.addLong(val);
- }
- writer.write(val);
- }
- }
-
- @Override
- void writeStripe(OrcProto.StripeFooter.Builder builder,
- int requiredIndexEntries) throws IOException {
- super.writeStripe(builder, requiredIndexEntries);
- writer.flush();
- recordPosition(rowIndexPosition);
- }
-
- @Override
- void recordPosition(PositionRecorder recorder) throws IOException {
- super.recordPosition(recorder);
- writer.getPosition(recorder);
- }
- }
-
- private static class FloatTreeWriter extends TreeWriter {
- private final PositionedOutputStream stream;
- private final SerializationUtils utils;
-
- FloatTreeWriter(int columnId,
- ObjectInspector inspector,
- StreamFactory writer,
- boolean nullable) throws IOException {
- super(columnId, inspector, writer, nullable);
- this.stream = writer.createStream(id,
- OrcProto.Stream.Kind.DATA);
- this.utils = new SerializationUtils();
- recordPosition(rowIndexPosition);
- }
-
- @Override
- void write(Object obj) throws IOException {
- super.write(obj);
- if (obj != null) {
- float val = ((FloatObjectInspector) inspector).get(obj);
- indexStatistics.updateDouble(val);
- if (createBloomFilter) {
- // floats are converted to doubles in column statistics and during SARG evaluation
- bloomFilter.addDouble(val);
- }
- utils.writeFloat(stream, val);
- }
- }
-
- @Override
- void writeStripe(OrcProto.StripeFooter.Builder builder,
- int requiredIndexEntries) throws IOException {
- super.writeStripe(builder, requiredIndexEntries);
- stream.flush();
- recordPosition(rowIndexPosition);
- }
-
- @Override
- void recordPosition(PositionRecorder recorder) throws IOException {
- super.recordPosition(recorder);
- stream.getPosition(recorder);
- }
- }
-
- private static class DoubleTreeWriter extends TreeWriter {
- private final PositionedOutputStream stream;
- private final SerializationUtils utils;
-
- DoubleTreeWriter(int columnId,
- ObjectInspector inspector,
- StreamFactory writer,
- boolean nullable) throws IOException {
- super(columnId, inspector, writer, nullable);
- this.stream = writer.createStream(id,
- OrcProto.Stream.Kind.DATA);
- this.utils = new SerializationUtils();
- recordPosition(rowIndexPosition);
- }
-
- @Override
- void write(Object obj) throws IOException {
- super.write(obj);
- if (obj != null) {
- double val = ((DoubleObjectInspector) inspector).get(obj);
- indexStatistics.updateDouble(val);
- if (createBloomFilter) {
- bloomFilter.addDouble(val);
- }
- utils.writeDouble(stream, val);
- }
- }
-
- @Override
- void writeStripe(OrcProto.StripeFooter.Builder builder,
- int requiredIndexEntries) throws IOException {
- super.writeStripe(builder, requiredIndexEntries);
- stream.flush();
- recordPosition(rowIndexPosition);
- }
-
- @Override
- void recordPosition(PositionRecorder recorder) throws IOException {
- super.recordPosition(recorder);
- stream.getPosition(recorder);
- }
- }
-
- private static class StringTreeWriter extends TreeWriter {
- private static final int INITIAL_DICTIONARY_SIZE = 4096;
- private final OutStream stringOutput;
- private final IntegerWriter lengthOutput;
- private final IntegerWriter rowOutput;
- private final StringRedBlackTree dictionary =
- new StringRedBlackTree(INITIAL_DICTIONARY_SIZE);
- private final DynamicIntArray rows = new DynamicIntArray();
- private final PositionedOutputStream directStreamOutput;
- private final IntegerWriter directLengthOutput;
- private final List savedRowIndex =
- new ArrayList();
- private final boolean buildIndex;
- private final List rowIndexValueCount = new ArrayList();
- // If the number of keys in a dictionary is greater than this fraction of
- //the total number of non-null rows, turn off dictionary encoding
- private final float dictionaryKeySizeThreshold;
- private boolean useDictionaryEncoding = true;
- private boolean isDirectV2 = true;
- private boolean doneDictionaryCheck;
- private final boolean strideDictionaryCheck;
-
- StringTreeWriter(int columnId,
- ObjectInspector inspector,
- StreamFactory writer,
- boolean nullable) throws IOException {
- super(columnId, inspector, writer, nullable);
- this.isDirectV2 = isNewWriteFormat(writer);
- stringOutput = writer.createStream(id,
- OrcProto.Stream.Kind.DICTIONARY_DATA);
- lengthOutput = createIntegerWriter(writer.createStream(id,
- OrcProto.Stream.Kind.LENGTH), false, isDirectV2, writer);
- rowOutput = createIntegerWriter(writer.createStream(id,
- OrcProto.Stream.Kind.DATA), false, isDirectV2, writer);
- recordPosition(rowIndexPosition);
- rowIndexValueCount.add(0L);
- buildIndex = writer.buildIndex();
- directStreamOutput = writer.createStream(id, OrcProto.Stream.Kind.DATA);
- directLengthOutput = createIntegerWriter(writer.createStream(id,
- OrcProto.Stream.Kind.LENGTH), false, isDirectV2, writer);
- dictionaryKeySizeThreshold = writer.getConfiguration().getFloat(
- HiveConf.ConfVars.HIVE_ORC_DICTIONARY_KEY_SIZE_THRESHOLD.varname,
- HiveConf.ConfVars.HIVE_ORC_DICTIONARY_KEY_SIZE_THRESHOLD.defaultFloatVal);
- strideDictionaryCheck = writer.getConfiguration().getBoolean(
- HiveConf.ConfVars.HIVE_ORC_ROW_INDEX_STRIDE_DICTIONARY_CHECK.varname,
- HiveConf.ConfVars.HIVE_ORC_ROW_INDEX_STRIDE_DICTIONARY_CHECK.defaultBoolVal);
- doneDictionaryCheck = false;
- }
-
- /**
- * Method to retrieve text values from the value object, which can be overridden
- * by subclasses.
- *
- * @param obj value
- * @return Text text value from obj
- */
- Text getTextValue(Object obj) {
- return ((StringObjectInspector) inspector).getPrimitiveWritableObject(obj);
- }
-
- @Override
- void write(Object obj) throws IOException {
- super.write(obj);
- if (obj != null) {
- Text val = getTextValue(obj);
- if (useDictionaryEncoding || !strideDictionaryCheck) {
- rows.add(dictionary.add(val));
- } else {
- // write data and length
- directStreamOutput.write(val.getBytes(), 0, val.getLength());
- directLengthOutput.write(val.getLength());
- }
- indexStatistics.updateString(val);
- if (createBloomFilter) {
- bloomFilter.addBytes(val.getBytes(), val.getLength());
- }
- }
- }
-
- private boolean checkDictionaryEncoding() {
- if (!doneDictionaryCheck) {
- // Set the flag indicating whether or not to use dictionary encoding
- // based on whether or not the fraction of distinct keys over number of
- // non-null rows is less than the configured threshold
- float ratio = rows.size() > 0 ? (float) (dictionary.size()) / rows.size() : 0.0f;
- useDictionaryEncoding = !isDirectV2 || ratio <= dictionaryKeySizeThreshold;
- doneDictionaryCheck = true;
- }
- return useDictionaryEncoding;
- }
-
- @Override
- void writeStripe(OrcProto.StripeFooter.Builder builder,
- int requiredIndexEntries) throws IOException {
- // if rows in stripe is less than dictionaryCheckAfterRows, dictionary
- // checking would not have happened. So do it again here.
- checkDictionaryEncoding();
-
- if (useDictionaryEncoding) {
- flushDictionary();
- } else {
- // flushout any left over entries from dictionary
- if (rows.size() > 0) {
- flushDictionary();
- }
-
- // suppress the stream for every stripe if dictionary is disabled
- stringOutput.suppress();
- }
-
- // we need to build the rowindex before calling super, since it
- // writes it out.
- super.writeStripe(builder, requiredIndexEntries);
- stringOutput.flush();
- lengthOutput.flush();
- rowOutput.flush();
- directStreamOutput.flush();
- directLengthOutput.flush();
- // reset all of the fields to be ready for the next stripe.
- dictionary.clear();
- savedRowIndex.clear();
- rowIndexValueCount.clear();
- recordPosition(rowIndexPosition);
- rowIndexValueCount.add(0L);
-
- if (!useDictionaryEncoding) {
- // record the start positions of first index stride of next stripe i.e
- // beginning of the direct streams when dictionary is disabled
- recordDirectStreamPosition();
- }
- }
-
- private void flushDictionary() throws IOException {
- final int[] dumpOrder = new int[dictionary.size()];
-
- if (useDictionaryEncoding) {
- // Write the dictionary by traversing the red-black tree writing out
- // the bytes and lengths; and creating the map from the original order
- // to the final sorted order.
-
- dictionary.visit(new StringRedBlackTree.Visitor() {
- private int currentId = 0;
-
- @Override
- public void visit(StringRedBlackTree.VisitorContext context
- ) throws IOException {
- context.writeBytes(stringOutput);
- lengthOutput.write(context.getLength());
- dumpOrder[context.getOriginalPosition()] = currentId++;
- }
- });
- } else {
- // for direct encoding, we don't want the dictionary data stream
- stringOutput.suppress();
- }
- int length = rows.size();
- int rowIndexEntry = 0;
- OrcProto.RowIndex.Builder rowIndex = getRowIndex();
- Text text = new Text();
- // write the values translated into the dump order.
- for (int i = 0; i <= length; ++i) {
- // now that we are writing out the row values, we can finalize the
- // row index
- if (buildIndex) {
- while (i == rowIndexValueCount.get(rowIndexEntry) && rowIndexEntry < savedRowIndex.size()) {
- OrcProto.RowIndexEntry.Builder base =
- savedRowIndex.get(rowIndexEntry++).toBuilder();
- if (useDictionaryEncoding) {
- rowOutput.getPosition(new RowIndexPositionRecorder(base));
- } else {
- PositionRecorder posn = new RowIndexPositionRecorder(base);
- directStreamOutput.getPosition(posn);
- directLengthOutput.getPosition(posn);
- }
- rowIndex.addEntry(base.build());
- }
- }
- if (i != length) {
- if (useDictionaryEncoding) {
- rowOutput.write(dumpOrder[rows.get(i)]);
- } else {
- dictionary.getText(text, rows.get(i));
- directStreamOutput.write(text.getBytes(), 0, text.getLength());
- directLengthOutput.write(text.getLength());
- }
- }
- }
- rows.clear();
- }
-
- @Override
- OrcProto.ColumnEncoding getEncoding() {
- // Returns the encoding used for the last call to writeStripe
- if (useDictionaryEncoding) {
- if (isDirectV2) {
- return OrcProto.ColumnEncoding.newBuilder().setKind(
- OrcProto.ColumnEncoding.Kind.DICTIONARY_V2).setDictionarySize(dictionary.size()).build();
- }
- return OrcProto.ColumnEncoding.newBuilder().setKind(
- OrcProto.ColumnEncoding.Kind.DICTIONARY).setDictionarySize(dictionary.size()).build();
- } else {
- if (isDirectV2) {
- return OrcProto.ColumnEncoding.newBuilder().setKind(
- OrcProto.ColumnEncoding.Kind.DIRECT_V2).build();
- }
- return OrcProto.ColumnEncoding.newBuilder().setKind(
- OrcProto.ColumnEncoding.Kind.DIRECT).build();
- }
- }
-
- /**
- * This method doesn't call the super method, because unlike most of the
- * other TreeWriters, this one can't record the position in the streams
- * until the stripe is being flushed. Therefore it saves all of the entries
- * and augments them with the final information as the stripe is written.
- *
- * @throws IOException if an error occurs on create
- */
- @Override
- void createRowIndexEntry() throws IOException {
- getStripeStatistics().merge(indexStatistics);
- OrcProto.RowIndexEntry.Builder rowIndexEntry = getRowIndexEntry();
- rowIndexEntry.setStatistics(indexStatistics.serialize());
- indexStatistics.reset();
- OrcProto.RowIndexEntry base = rowIndexEntry.build();
- savedRowIndex.add(base);
- rowIndexEntry.clear();
- addBloomFilterEntry();
- recordPosition(rowIndexPosition);
- rowIndexValueCount.add(Long.valueOf(rows.size()));
- if (strideDictionaryCheck) {
- checkDictionaryEncoding();
- }
- if (!useDictionaryEncoding) {
- if (rows.size() > 0) {
- flushDictionary();
- // just record the start positions of next index stride
- recordDirectStreamPosition();
- } else {
- // record the start positions of next index stride
- recordDirectStreamPosition();
- getRowIndex().addEntry(base);
- }
- }
- }
-
- private void recordDirectStreamPosition() throws IOException {
- directStreamOutput.getPosition(rowIndexPosition);
- directLengthOutput.getPosition(rowIndexPosition);
- }
-
- @Override
- long estimateMemory() {
- return rows.getSizeInBytes() + dictionary.getSizeInBytes();
- }
- }
-
- /**
- * Under the covers, char is written to ORC the same way as string.
- */
- private static class CharTreeWriter extends StringTreeWriter {
-
- CharTreeWriter(int columnId,
- ObjectInspector inspector,
- StreamFactory writer,
- boolean nullable) throws IOException {
- super(columnId, inspector, writer, nullable);
- }
-
- /**
- * Override base class implementation to support char values.
- */
- @Override
- Text getTextValue(Object obj) {
- return (((HiveCharObjectInspector) inspector)
- .getPrimitiveWritableObject(obj)).getTextValue();
- }
- }
-
- /**
- * Under the covers, varchar is written to ORC the same way as string.
- */
- private static class VarcharTreeWriter extends StringTreeWriter {
-
- VarcharTreeWriter(int columnId,
- ObjectInspector inspector,
- StreamFactory writer,
- boolean nullable) throws IOException {
- super(columnId, inspector, writer, nullable);
- }
-
- /**
- * Override base class implementation to support varchar values.
- */
- @Override
- Text getTextValue(Object obj) {
- return (((HiveVarcharObjectInspector) inspector)
- .getPrimitiveWritableObject(obj)).getTextValue();
- }
- }
-
- private static class BinaryTreeWriter extends TreeWriter {
- private final PositionedOutputStream stream;
- private final IntegerWriter length;
- private boolean isDirectV2 = true;
-
- BinaryTreeWriter(int columnId,
- ObjectInspector inspector,
- StreamFactory writer,
- boolean nullable) throws IOException {
- super(columnId, inspector, writer, nullable);
- this.stream = writer.createStream(id,
- OrcProto.Stream.Kind.DATA);
- this.isDirectV2 = isNewWriteFormat(writer);
- this.length = createIntegerWriter(writer.createStream(id,
- OrcProto.Stream.Kind.LENGTH), false, isDirectV2, writer);
- recordPosition(rowIndexPosition);
- }
-
- @Override
- OrcProto.ColumnEncoding getEncoding() {
- if (isDirectV2) {
- return OrcProto.ColumnEncoding.newBuilder()
- .setKind(OrcProto.ColumnEncoding.Kind.DIRECT_V2).build();
- }
- return OrcProto.ColumnEncoding.newBuilder()
- .setKind(OrcProto.ColumnEncoding.Kind.DIRECT).build();
- }
-
- @Override
- void write(Object obj) throws IOException {
- super.write(obj);
- if (obj != null) {
- BytesWritable val =
- ((BinaryObjectInspector) inspector).getPrimitiveWritableObject(obj);
- stream.write(val.getBytes(), 0, val.getLength());
- length.write(val.getLength());
- indexStatistics.updateBinary(val);
- if (createBloomFilter) {
- bloomFilter.addBytes(val.getBytes(), val.getLength());
- }
- }
- }
-
- @Override
- void writeStripe(OrcProto.StripeFooter.Builder builder,
- int requiredIndexEntries) throws IOException {
- super.writeStripe(builder, requiredIndexEntries);
- stream.flush();
- length.flush();
- recordPosition(rowIndexPosition);
- }
-
- @Override
- void recordPosition(PositionRecorder recorder) throws IOException {
- super.recordPosition(recorder);
- stream.getPosition(recorder);
- length.getPosition(recorder);
- }
- }
-
- static final int MILLIS_PER_SECOND = 1000;
- static final String BASE_TIMESTAMP_STRING = "2015-01-01 00:00:00";
-
- private static class TimestampTreeWriter extends TreeWriter {
- private final IntegerWriter seconds;
- private final IntegerWriter nanos;
- private final boolean isDirectV2;
- private final long base_timestamp;
-
- TimestampTreeWriter(int columnId,
- ObjectInspector inspector,
- StreamFactory writer,
- boolean nullable) throws IOException {
- super(columnId, inspector, writer, nullable);
- this.isDirectV2 = isNewWriteFormat(writer);
- this.seconds = createIntegerWriter(writer.createStream(id,
- OrcProto.Stream.Kind.DATA), true, isDirectV2, writer);
- this.nanos = createIntegerWriter(writer.createStream(id,
- OrcProto.Stream.Kind.SECONDARY), false, isDirectV2, writer);
- recordPosition(rowIndexPosition);
- // for unit tests to set different time zones
- this.base_timestamp = Timestamp.valueOf(BASE_TIMESTAMP_STRING).getTime() / MILLIS_PER_SECOND;
- }
-
- @Override
- OrcProto.ColumnEncoding getEncoding() {
- if (isDirectV2) {
- return OrcProto.ColumnEncoding.newBuilder()
- .setKind(OrcProto.ColumnEncoding.Kind.DIRECT_V2).build();
- }
- return OrcProto.ColumnEncoding.newBuilder()
- .setKind(OrcProto.ColumnEncoding.Kind.DIRECT).build();
- }
-
- @Override
- void write(Object obj) throws IOException {
- super.write(obj);
- if (obj != null) {
- Timestamp val =
- ((TimestampObjectInspector) inspector).getPrimitiveJavaObject(obj);
- indexStatistics.updateTimestamp(val);
- seconds.write((val.getTime() / MILLIS_PER_SECOND) - base_timestamp);
- nanos.write(formatNanos(val.getNanos()));
- if (createBloomFilter) {
- bloomFilter.addLong(val.getTime());
- }
- }
- }
-
- @Override
- void writeStripe(OrcProto.StripeFooter.Builder builder,
- int requiredIndexEntries) throws IOException {
- super.writeStripe(builder, requiredIndexEntries);
- seconds.flush();
- nanos.flush();
- recordPosition(rowIndexPosition);
- }
-
- private static long formatNanos(int nanos) {
- if (nanos == 0) {
- return 0;
- } else if (nanos % 100 != 0) {
- return ((long) nanos) << 3;
- } else {
- nanos /= 100;
- int trailingZeros = 1;
- while (nanos % 10 == 0 && trailingZeros < 7) {
- nanos /= 10;
- trailingZeros += 1;
- }
- return ((long) nanos) << 3 | trailingZeros;
- }
- }
-
- @Override
- void recordPosition(PositionRecorder recorder) throws IOException {
- super.recordPosition(recorder);
- seconds.getPosition(recorder);
- nanos.getPosition(recorder);
- }
- }
-
- private static class DateTreeWriter extends TreeWriter {
- private final IntegerWriter writer;
- private final boolean isDirectV2;
-
- DateTreeWriter(int columnId,
- ObjectInspector inspector,
- StreamFactory writer,
- boolean nullable) throws IOException {
- super(columnId, inspector, writer, nullable);
- OutStream out = writer.createStream(id,
- OrcProto.Stream.Kind.DATA);
- this.isDirectV2 = isNewWriteFormat(writer);
- this.writer = createIntegerWriter(out, true, isDirectV2, writer);
- recordPosition(rowIndexPosition);
- }
-
- @Override
- void write(Object obj) throws IOException {
- super.write(obj);
- if (obj != null) {
- // Using the Writable here as it's used directly for writing as well as for stats.
- DateWritable val = ((DateObjectInspector) inspector).getPrimitiveWritableObject(obj);
- indexStatistics.updateDate(val);
- writer.write(val.getDays());
- if (createBloomFilter) {
- bloomFilter.addLong(val.getDays());
- }
- }
- }
-
- @Override
- void writeStripe(OrcProto.StripeFooter.Builder builder,
- int requiredIndexEntries) throws IOException {
- super.writeStripe(builder, requiredIndexEntries);
- writer.flush();
- recordPosition(rowIndexPosition);
- }
-
- @Override
- void recordPosition(PositionRecorder recorder) throws IOException {
- super.recordPosition(recorder);
- writer.getPosition(recorder);
- }
-
- @Override
- OrcProto.ColumnEncoding getEncoding() {
- if (isDirectV2) {
- return OrcProto.ColumnEncoding.newBuilder()
- .setKind(OrcProto.ColumnEncoding.Kind.DIRECT_V2).build();
- }
- return OrcProto.ColumnEncoding.newBuilder()
- .setKind(OrcProto.ColumnEncoding.Kind.DIRECT).build();
- }
- }
-
- private static class DecimalTreeWriter extends TreeWriter {
- private final PositionedOutputStream valueStream;
- private final IntegerWriter scaleStream;
- private final boolean isDirectV2;
-
- DecimalTreeWriter(int columnId,
- ObjectInspector inspector,
- StreamFactory writer,
- boolean nullable) throws IOException {
- super(columnId, inspector, writer, nullable);
- this.isDirectV2 = isNewWriteFormat(writer);
- valueStream = writer.createStream(id, OrcProto.Stream.Kind.DATA);
- this.scaleStream = createIntegerWriter(writer.createStream(id,
- OrcProto.Stream.Kind.SECONDARY), true, isDirectV2, writer);
- recordPosition(rowIndexPosition);
- }
-
- @Override
- OrcProto.ColumnEncoding getEncoding() {
- if (isDirectV2) {
- return OrcProto.ColumnEncoding.newBuilder()
- .setKind(OrcProto.ColumnEncoding.Kind.DIRECT_V2).build();
- }
- return OrcProto.ColumnEncoding.newBuilder()
- .setKind(OrcProto.ColumnEncoding.Kind.DIRECT).build();
- }
-
- @Override
- void write(Object obj) throws IOException {
- super.write(obj);
- if (obj != null) {
- HiveDecimal decimal = ((HiveDecimalObjectInspector) inspector).getPrimitiveJavaObject(obj);
- if (decimal == null) {
- return;
- }
- SerializationUtils.writeBigInteger(valueStream,
- decimal.unscaledValue());
- scaleStream.write(decimal.scale());
- indexStatistics.updateDecimal(decimal);
- if (createBloomFilter) {
- bloomFilter.addString(decimal.toString());
- }
- }
- }
-
- @Override
- void writeStripe(OrcProto.StripeFooter.Builder builder,
- int requiredIndexEntries) throws IOException {
- super.writeStripe(builder, requiredIndexEntries);
- valueStream.flush();
- scaleStream.flush();
- recordPosition(rowIndexPosition);
- }
-
- @Override
- void recordPosition(PositionRecorder recorder) throws IOException {
- super.recordPosition(recorder);
- valueStream.getPosition(recorder);
- scaleStream.getPosition(recorder);
- }
- }
-
- private static class StructTreeWriter extends TreeWriter {
- private final List extends StructField> fields;
-
- StructTreeWriter(int columnId,
- ObjectInspector inspector,
- StreamFactory writer,
- boolean nullable) throws IOException {
- super(columnId, inspector, writer, nullable);
- StructObjectInspector structObjectInspector =
- (StructObjectInspector) inspector;
- fields = structObjectInspector.getAllStructFieldRefs();
- childrenWriters = new TreeWriter[fields.size()];
- for (int i = 0; i < childrenWriters.length; ++i) {
- childrenWriters[i] = createTreeWriter(
- fields.get(i).getFieldObjectInspector(), writer, true);
- }
- recordPosition(rowIndexPosition);
- }
-
- @Override
- void write(Object obj) throws IOException {
- super.write(obj);
- if (obj != null) {
- StructObjectInspector insp = (StructObjectInspector) inspector;
- for (int i = 0; i < fields.size(); ++i) {
- StructField field = fields.get(i);
- TreeWriter writer = childrenWriters[i];
- writer.write(insp.getStructFieldData(obj, field));
- }
- }
- }
-
- @Override
- void writeStripe(OrcProto.StripeFooter.Builder builder,
- int requiredIndexEntries) throws IOException {
- super.writeStripe(builder, requiredIndexEntries);
- for (TreeWriter child : childrenWriters) {
- child.writeStripe(builder, requiredIndexEntries);
- }
- recordPosition(rowIndexPosition);
- }
- }
-
- private static class ListTreeWriter extends TreeWriter {
- private final IntegerWriter lengths;
- private final boolean isDirectV2;
-
- ListTreeWriter(int columnId,
- ObjectInspector inspector,
- StreamFactory writer,
- boolean nullable) throws IOException {
- super(columnId, inspector, writer, nullable);
- this.isDirectV2 = isNewWriteFormat(writer);
- ListObjectInspector listObjectInspector = (ListObjectInspector) inspector;
- childrenWriters = new TreeWriter[1];
- childrenWriters[0] =
- createTreeWriter(listObjectInspector.getListElementObjectInspector(),
- writer, true);
- lengths = createIntegerWriter(writer.createStream(columnId,
- OrcProto.Stream.Kind.LENGTH), false, isDirectV2, writer);
- recordPosition(rowIndexPosition);
- }
-
- @Override
- OrcProto.ColumnEncoding getEncoding() {
- if (isDirectV2) {
- return OrcProto.ColumnEncoding.newBuilder()
- .setKind(OrcProto.ColumnEncoding.Kind.DIRECT_V2).build();
- }
- return OrcProto.ColumnEncoding.newBuilder()
- .setKind(OrcProto.ColumnEncoding.Kind.DIRECT).build();
- }
-
- @Override
- void write(Object obj) throws IOException {
- super.write(obj);
- if (obj != null) {
- ListObjectInspector insp = (ListObjectInspector) inspector;
- int len = insp.getListLength(obj);
- lengths.write(len);
- if (createBloomFilter) {
- bloomFilter.addLong(len);
- }
- for (int i = 0; i < len; ++i) {
- childrenWriters[0].write(insp.getListElement(obj, i));
- }
- }
- }
-
- @Override
- void writeStripe(OrcProto.StripeFooter.Builder builder,
- int requiredIndexEntries) throws IOException {
- super.writeStripe(builder, requiredIndexEntries);
- lengths.flush();
- for (TreeWriter child : childrenWriters) {
- child.writeStripe(builder, requiredIndexEntries);
- }
- recordPosition(rowIndexPosition);
- }
-
- @Override
- void recordPosition(PositionRecorder recorder) throws IOException {
- super.recordPosition(recorder);
- lengths.getPosition(recorder);
- }
- }
-
- private static class MapTreeWriter extends TreeWriter {
- private final IntegerWriter lengths;
- private final boolean isDirectV2;
-
- MapTreeWriter(int columnId,
- ObjectInspector inspector,
- StreamFactory writer,
- boolean nullable) throws IOException {
- super(columnId, inspector, writer, nullable);
- this.isDirectV2 = isNewWriteFormat(writer);
- MapObjectInspector insp = (MapObjectInspector) inspector;
- childrenWriters = new TreeWriter[2];
- childrenWriters[0] =
- createTreeWriter(insp.getMapKeyObjectInspector(), writer, true);
- childrenWriters[1] =
- createTreeWriter(insp.getMapValueObjectInspector(), writer, true);
- lengths = createIntegerWriter(writer.createStream(columnId,
- OrcProto.Stream.Kind.LENGTH), false, isDirectV2, writer);
- recordPosition(rowIndexPosition);
- }
-
- @Override
- OrcProto.ColumnEncoding getEncoding() {
- if (isDirectV2) {
- return OrcProto.ColumnEncoding.newBuilder()
- .setKind(OrcProto.ColumnEncoding.Kind.DIRECT_V2).build();
- }
- return OrcProto.ColumnEncoding.newBuilder()
- .setKind(OrcProto.ColumnEncoding.Kind.DIRECT).build();
- }
-
- @Override
- void write(Object obj) throws IOException {
- super.write(obj);
- if (obj != null) {
- MapObjectInspector insp = (MapObjectInspector) inspector;
- // this sucks, but it will have to do until we can get a better
- // accessor in the MapObjectInspector.
- Map, ?> valueMap = insp.getMap(obj);
- lengths.write(valueMap.size());
- if (createBloomFilter) {
- bloomFilter.addLong(valueMap.size());
- }
- for (Map.Entry, ?> entry : valueMap.entrySet()) {
- childrenWriters[0].write(entry.getKey());
- childrenWriters[1].write(entry.getValue());
- }
- }
- }
-
- @Override
- void writeStripe(OrcProto.StripeFooter.Builder builder,
- int requiredIndexEntries) throws IOException {
- super.writeStripe(builder, requiredIndexEntries);
- lengths.flush();
- for (TreeWriter child : childrenWriters) {
- child.writeStripe(builder, requiredIndexEntries);
- }
- recordPosition(rowIndexPosition);
- }
-
- @Override
- void recordPosition(PositionRecorder recorder) throws IOException {
- super.recordPosition(recorder);
- lengths.getPosition(recorder);
- }
- }
-
- private static class UnionTreeWriter extends TreeWriter {
- private final RunLengthByteWriter tags;
-
- UnionTreeWriter(int columnId,
- ObjectInspector inspector,
- StreamFactory writer,
- boolean nullable) throws IOException {
- super(columnId, inspector, writer, nullable);
- UnionObjectInspector insp = (UnionObjectInspector) inspector;
- List choices = insp.getObjectInspectors();
- childrenWriters = new TreeWriter[choices.size()];
- for (int i = 0; i < childrenWriters.length; ++i) {
- childrenWriters[i] = createTreeWriter(choices.get(i), writer, true);
- }
- tags =
- new RunLengthByteWriter(writer.createStream(columnId,
- OrcProto.Stream.Kind.DATA));
- recordPosition(rowIndexPosition);
- }
-
- @Override
- void write(Object obj) throws IOException {
- super.write(obj);
- if (obj != null) {
- UnionObjectInspector insp = (UnionObjectInspector) inspector;
- byte tag = insp.getTag(obj);
- tags.write(tag);
- if (createBloomFilter) {
- bloomFilter.addLong(tag);
- }
- childrenWriters[tag].write(insp.getField(obj));
- }
- }
-
- @Override
- void writeStripe(OrcProto.StripeFooter.Builder builder,
- int requiredIndexEntries) throws IOException {
- super.writeStripe(builder, requiredIndexEntries);
- tags.flush();
- for (TreeWriter child : childrenWriters) {
- child.writeStripe(builder, requiredIndexEntries);
- }
- recordPosition(rowIndexPosition);
- }
-
- @Override
- void recordPosition(PositionRecorder recorder) throws IOException {
- super.recordPosition(recorder);
- tags.getPosition(recorder);
- }
- }
-
- private static TreeWriter createTreeWriter(ObjectInspector inspector,
- StreamFactory streamFactory,
- boolean nullable) throws IOException {
- switch (inspector.getCategory()) {
- case PRIMITIVE:
- switch (((PrimitiveObjectInspector) inspector).getPrimitiveCategory()) {
- case BOOLEAN:
- return new BooleanTreeWriter(streamFactory.getNextColumnId(),
- inspector, streamFactory, nullable);
- case BYTE:
- return new ByteTreeWriter(streamFactory.getNextColumnId(),
- inspector, streamFactory, nullable);
- case SHORT:
- case INT:
- case LONG:
- return new IntegerTreeWriter(streamFactory.getNextColumnId(),
- inspector, streamFactory, nullable);
- case FLOAT:
- return new FloatTreeWriter(streamFactory.getNextColumnId(),
- inspector, streamFactory, nullable);
- case DOUBLE:
- return new DoubleTreeWriter(streamFactory.getNextColumnId(),
- inspector, streamFactory, nullable);
- case STRING:
- return new StringTreeWriter(streamFactory.getNextColumnId(),
- inspector, streamFactory, nullable);
- case CHAR:
- return new CharTreeWriter(streamFactory.getNextColumnId(),
- inspector, streamFactory, nullable);
- case VARCHAR:
- return new VarcharTreeWriter(streamFactory.getNextColumnId(),
- inspector, streamFactory, nullable);
- case BINARY:
- return new BinaryTreeWriter(streamFactory.getNextColumnId(),
- inspector, streamFactory, nullable);
- case TIMESTAMP:
- return new TimestampTreeWriter(streamFactory.getNextColumnId(),
- inspector, streamFactory, nullable);
- case DATE:
- return new DateTreeWriter(streamFactory.getNextColumnId(),
- inspector, streamFactory, nullable);
- case DECIMAL:
- return new DecimalTreeWriter(streamFactory.getNextColumnId(),
- inspector, streamFactory, nullable);
- default:
- throw new IllegalArgumentException("Bad primitive category " +
- ((PrimitiveObjectInspector) inspector).getPrimitiveCategory());
- }
- case STRUCT:
- return new StructTreeWriter(streamFactory.getNextColumnId(), inspector,
- streamFactory, nullable);
- case MAP:
- return new MapTreeWriter(streamFactory.getNextColumnId(), inspector,
- streamFactory, nullable);
- case LIST:
- return new ListTreeWriter(streamFactory.getNextColumnId(), inspector,
- streamFactory, nullable);
- case UNION:
- return new UnionTreeWriter(streamFactory.getNextColumnId(), inspector,
- streamFactory, nullable);
- default:
- throw new IllegalArgumentException("Bad category: " +
- inspector.getCategory());
- }
- }
-
- private static void writeTypes(OrcProto.Footer.Builder builder,
- TreeWriter treeWriter) {
- OrcProto.Type.Builder type = OrcProto.Type.newBuilder();
- switch (treeWriter.inspector.getCategory()) {
- case PRIMITIVE:
- switch (((PrimitiveObjectInspector) treeWriter.inspector).getPrimitiveCategory()) {
- case BOOLEAN:
- type.setKind(OrcProto.Type.Kind.BOOLEAN);
- break;
- case BYTE:
- type.setKind(OrcProto.Type.Kind.BYTE);
- break;
- case SHORT:
- type.setKind(OrcProto.Type.Kind.SHORT);
- break;
- case INT:
- type.setKind(OrcProto.Type.Kind.INT);
- break;
- case LONG:
- type.setKind(OrcProto.Type.Kind.LONG);
- break;
- case FLOAT:
- type.setKind(OrcProto.Type.Kind.FLOAT);
- break;
- case DOUBLE:
- type.setKind(OrcProto.Type.Kind.DOUBLE);
- break;
- case STRING:
- type.setKind(OrcProto.Type.Kind.STRING);
- break;
- case CHAR:
- // The char length needs to be written to file and should be available
- // from the object inspector
- CharTypeInfo charTypeInfo = (CharTypeInfo) ((PrimitiveObjectInspector) treeWriter.inspector).getTypeInfo();
- type.setKind(Type.Kind.CHAR);
- type.setMaximumLength(charTypeInfo.getLength());
- break;
- case VARCHAR:
- // The varchar length needs to be written to file and should be available
- // from the object inspector
- VarcharTypeInfo typeInfo = (VarcharTypeInfo) ((PrimitiveObjectInspector) treeWriter.inspector).getTypeInfo();
- type.setKind(Type.Kind.VARCHAR);
- type.setMaximumLength(typeInfo.getLength());
- break;
- case BINARY:
- type.setKind(OrcProto.Type.Kind.BINARY);
- break;
- case TIMESTAMP:
- type.setKind(OrcProto.Type.Kind.TIMESTAMP);
- break;
- case DATE:
- type.setKind(OrcProto.Type.Kind.DATE);
- break;
- case DECIMAL:
- DecimalTypeInfo decTypeInfo = (DecimalTypeInfo) ((PrimitiveObjectInspector) treeWriter.inspector).getTypeInfo();
- type.setKind(OrcProto.Type.Kind.DECIMAL);
- type.setPrecision(decTypeInfo.precision());
- type.setScale(decTypeInfo.scale());
- break;
- default:
- throw new IllegalArgumentException("Unknown primitive category: "
- + ((PrimitiveObjectInspector) treeWriter.inspector).getPrimitiveCategory());
- }
- break;
- case LIST:
- type.setKind(OrcProto.Type.Kind.LIST);
- type.addSubtypes(treeWriter.childrenWriters[0].id);
- break;
- case MAP:
- type.setKind(OrcProto.Type.Kind.MAP);
- type.addSubtypes(treeWriter.childrenWriters[0].id);
- type.addSubtypes(treeWriter.childrenWriters[1].id);
- break;
- case STRUCT:
- type.setKind(OrcProto.Type.Kind.STRUCT);
- for (TreeWriter child : treeWriter.childrenWriters) {
- type.addSubtypes(child.id);
- }
- for (StructField field : ((StructTreeWriter) treeWriter).fields) {
- type.addFieldNames(field.getFieldName());
- }
- break;
- case UNION:
- type.setKind(OrcProto.Type.Kind.UNION);
- for (TreeWriter child : treeWriter.childrenWriters) {
- type.addSubtypes(child.id);
- }
- break;
- default:
- throw new IllegalArgumentException("Unknown category: " + treeWriter.inspector.getCategory());
- }
- builder.addTypes(type);
- for (TreeWriter child : treeWriter.childrenWriters) {
- writeTypes(builder, child);
- }
- }
-
- @VisibleForTesting
- public OutputStream getStream() throws IOException {
- if (rawWriter == null) {
- rawWriter = new ByteCountingOutputStream(flowFileOutputStream);
- rawWriter.write(OrcFile.MAGIC.getBytes());
- headerLength = rawWriter.getBytesWritten();
- writer = new OutStream("metadata", bufferSize, codec,
- new DirectStream(rawWriter));
- protobufWriter = CodedOutputStream.newInstance(writer);
- }
- return rawWriter;
- }
-
- private void createRowIndexEntry() throws IOException {
- treeWriter.createRowIndexEntry();
- rowsInIndex = 0;
- }
-
- private void flushStripe() throws IOException {
- getStream();
- if (buildIndex && rowsInIndex != 0) {
- createRowIndexEntry();
- }
- if (rowsInStripe != 0) {
- if (callback != null) {
- callback.preStripeWrite(callbackContext);
- }
- // finalize the data for the stripe
- int requiredIndexEntries = rowIndexStride == 0 ? 0 :
- (int) ((rowsInStripe + rowIndexStride - 1) / rowIndexStride);
- OrcProto.StripeFooter.Builder builder =
- OrcProto.StripeFooter.newBuilder();
- treeWriter.writeStripe(builder, requiredIndexEntries);
- long indexSize = 0;
- long dataSize = 0;
- for (Map.Entry pair : streams.entrySet()) {
- BufferedStream stream = pair.getValue();
- if (!stream.isSuppressed()) {
- stream.flush();
- StreamName name = pair.getKey();
- long streamSize = pair.getValue().getOutputSize();
- builder.addStreams(OrcProto.Stream.newBuilder()
- .setColumn(name.getColumn())
- .setKind(name.getKind())
- .setLength(streamSize));
- if (StreamName.Area.INDEX == name.getArea()) {
- indexSize += streamSize;
- } else {
- dataSize += streamSize;
- }
- }
- }
- OrcProto.StripeFooter footer = builder.build();
-
- // Do we need to pad the file so the stripe doesn't straddle a block
- // boundary?
- long start = rawWriter.getBytesWritten();
- final long currentStripeSize = indexSize + dataSize + footer.getSerializedSize();
- final long available = blockSize - (start % blockSize);
- final long overflow = currentStripeSize - adjustedStripeSize;
- final float availRatio = (float) available / (float) defaultStripeSize;
-
- if (availRatio > 0.0f && availRatio < 1.0f
- && availRatio > paddingTolerance) {
- // adjust default stripe size to fit into remaining space, also adjust
- // the next stripe for correction based on the current stripe size
- // and user specified padding tolerance. Since stripe size can overflow
- // the default stripe size we should apply this correction to avoid
- // writing portion of last stripe to next hdfs block.
- float correction = overflow > 0 ? (float) overflow
- / (float) adjustedStripeSize : 0.0f;
-
- // correction should not be greater than user specified padding
- // tolerance
- correction = correction > paddingTolerance ? paddingTolerance
- : correction;
-
- // adjust next stripe size based on current stripe estimate correction
- adjustedStripeSize = (long) ((1.0f - correction) * (availRatio * defaultStripeSize));
- } else if (availRatio >= 1.0) {
- adjustedStripeSize = defaultStripeSize;
- }
-
- if (availRatio < paddingTolerance && addBlockPadding) {
- long padding = blockSize - (start % blockSize);
- byte[] pad = new byte[(int) Math.min(HDFS_BUFFER_SIZE, padding)];
- LOG.info(String.format("Padding ORC by %d bytes (<= %.2f * %d)",
- padding, availRatio, defaultStripeSize));
- start += padding;
- while (padding > 0) {
- int writeLen = (int) Math.min(padding, pad.length);
- rawWriter.write(pad, 0, writeLen);
- padding -= writeLen;
- }
- adjustedStripeSize = defaultStripeSize;
- } else if (currentStripeSize < blockSize
- && (start % blockSize) + currentStripeSize > blockSize) {
- // even if you don't pad, reset the default stripe size when crossing a
- // block boundary
- adjustedStripeSize = defaultStripeSize;
- }
-
- // write out the data streams
- for (Map.Entry pair : streams.entrySet()) {
- BufferedStream stream = pair.getValue();
- if (!stream.isSuppressed()) {
- stream.spillTo(rawWriter);
- }
- stream.clear();
- }
- footer.writeTo(protobufWriter);
- protobufWriter.flush();
- writer.flush();
- long footerLength = rawWriter.getBytesWritten() - start - dataSize - indexSize;
- OrcProto.StripeInformation dirEntry =
- OrcProto.StripeInformation.newBuilder()
- .setOffset(start)
- .setNumberOfRows(rowsInStripe)
- .setIndexLength(indexSize)
- .setDataLength(dataSize)
- .setFooterLength(footerLength).build();
- stripes.add(dirEntry);
- rowCount += rowsInStripe;
- rowsInStripe = 0;
- }
- }
-
- private long computeRawDataSize() {
- long result = 0;
- for (TreeWriter child : treeWriter.getChildrenWriters()) {
- result += getRawDataSizeFromInspectors(child, child.inspector);
- }
- return result;
- }
-
- private long getRawDataSizeFromInspectors(TreeWriter child, ObjectInspector oi) {
- long total = 0;
- switch (oi.getCategory()) {
- case PRIMITIVE:
- total += getRawDataSizeFromPrimitives(child, oi);
- break;
- case LIST:
- case MAP:
- case UNION:
- case STRUCT:
- for (TreeWriter tw : child.childrenWriters) {
- total += getRawDataSizeFromInspectors(tw, tw.inspector);
- }
- break;
- default:
- LOG.debug("Unknown object inspector category.");
- break;
- }
- return total;
- }
-
- private long getRawDataSizeFromPrimitives(TreeWriter child, ObjectInspector oi) {
- long result = 0;
- long numVals = child.fileStatistics.getNumberOfValues();
- switch (((PrimitiveObjectInspector) oi).getPrimitiveCategory()) {
- case BOOLEAN:
- case BYTE:
- case SHORT:
- case INT:
- case FLOAT:
- return numVals * JavaDataModel.get().primitive1();
- case LONG:
- case DOUBLE:
- return numVals * JavaDataModel.get().primitive2();
- case STRING:
- case VARCHAR:
- case CHAR:
- // ORC strings are converted to java Strings. so use JavaDataModel to
- // compute the overall size of strings
- child = (StringTreeWriter) child;
- StringColumnStatistics scs = (StringColumnStatistics) child.fileStatistics;
- numVals = numVals == 0 ? 1 : numVals;
- int avgStringLen = (int) (scs.getSum() / numVals);
- return numVals * JavaDataModel.get().lengthForStringOfLength(avgStringLen);
- case DECIMAL:
- return numVals * JavaDataModel.get().lengthOfDecimal();
- case DATE:
- return numVals * JavaDataModel.get().lengthOfDate();
- case BINARY:
- // get total length of binary blob
- BinaryColumnStatistics bcs = (BinaryColumnStatistics) child.fileStatistics;
- return bcs.getSum();
- case TIMESTAMP:
- return numVals * JavaDataModel.get().lengthOfTimestamp();
- default:
- LOG.debug("Unknown primitive category.");
- break;
- }
-
- return result;
- }
-
- private OrcProto.CompressionKind writeCompressionKind(CompressionKind kind) {
- switch (kind) {
- case NONE:
- return OrcProto.CompressionKind.NONE;
- case ZLIB:
- return OrcProto.CompressionKind.ZLIB;
- case SNAPPY:
- return OrcProto.CompressionKind.SNAPPY;
- case LZO:
- return OrcProto.CompressionKind.LZO;
- default:
- throw new IllegalArgumentException("Unknown compression " + kind);
- }
- }
-
- private void writeFileStatistics(OrcProto.Footer.Builder builder,
- TreeWriter writer) throws IOException {
- builder.addStatistics(writer.fileStatistics.serialize());
- for (TreeWriter child : writer.getChildrenWriters()) {
- writeFileStatistics(builder, child);
- }
- }
-
- private int writeMetadata() throws IOException {
- getStream();
- OrcProto.Metadata.Builder builder = OrcProto.Metadata.newBuilder();
- for (OrcProto.StripeStatistics.Builder ssb : treeWriter.stripeStatsBuilders) {
- builder.addStripeStats(ssb.build());
- }
-
- long startPosn = rawWriter.getBytesWritten();
- OrcProto.Metadata metadata = builder.build();
- metadata.writeTo(protobufWriter);
- protobufWriter.flush();
- writer.flush();
- return (int) (rawWriter.getBytesWritten() - startPosn);
- }
-
- private int writeFooter(long bodyLength) throws IOException {
- getStream();
- OrcProto.Footer.Builder builder = OrcProto.Footer.newBuilder();
- builder.setContentLength(bodyLength);
- builder.setHeaderLength(headerLength);
- builder.setNumberOfRows(rowCount);
- builder.setRowIndexStride(rowIndexStride);
- // populate raw data size
- rawDataSize = computeRawDataSize();
- // serialize the types
- writeTypes(builder, treeWriter);
- // add the stripe information
- for (OrcProto.StripeInformation stripe : stripes) {
- builder.addStripes(stripe);
- }
- // add the column statistics
- writeFileStatistics(builder, treeWriter);
- // add all of the user metadata
- for (Map.Entry entry : userMetadata.entrySet()) {
- builder.addMetadata(OrcProto.UserMetadataItem.newBuilder()
- .setName(entry.getKey()).setValue(entry.getValue()));
- }
- long startPosn = rawWriter.getBytesWritten();
- OrcProto.Footer footer = builder.build();
- footer.writeTo(protobufWriter);
- protobufWriter.flush();
- writer.flush();
- return (int) (rawWriter.getBytesWritten() - startPosn);
- }
-
- private int writePostScript(int footerLength, int metadataLength) throws IOException {
- OrcProto.PostScript.Builder builder =
- OrcProto.PostScript.newBuilder()
- .setCompression(writeCompressionKind(compress))
- .setFooterLength(footerLength)
- .setMetadataLength(metadataLength)
- .setMagic(OrcFile.MAGIC)
- .addVersion(version.getMajor())
- .addVersion(version.getMinor())
- .setWriterVersion(OrcFile.WriterVersion.HIVE_8732.getId());
- if (compress != CompressionKind.NONE) {
- builder.setCompressionBlockSize(bufferSize);
- }
- OrcProto.PostScript ps = builder.build();
- // need to write this uncompressed
- long startPosn = rawWriter.getBytesWritten();
- ps.writeTo(rawWriter);
- long length = rawWriter.getBytesWritten() - startPosn;
- if (length > 255) {
- throw new IllegalArgumentException("PostScript too large at " + length);
- }
- return (int) length;
- }
-
- private long estimateStripeSize() {
- long result = 0;
- for (BufferedStream stream : streams.values()) {
- result += stream.getBufferSize();
- }
- result += treeWriter.estimateMemory();
- return result;
- }
-
- @Override
- public synchronized void addUserMetadata(String name, ByteBuffer value) {
- userMetadata.put(name, ByteString.copyFrom(value));
- }
-
- @Override
- public void addRow(Object row) throws IOException {
- synchronized (this) {
- treeWriter.write(row);
- rowsInStripe += 1;
- if (buildIndex) {
- rowsInIndex += 1;
-
- if (rowsInIndex >= rowIndexStride) {
- createRowIndexEntry();
- }
- }
- }
- memoryManager.addedRow();
- }
-
- public void addRowBatch(VectorizedRowBatch batch) throws IOException {
- if (buildIndex) {
- // Batch the writes up to the rowIndexStride so that we can get the
- // right size indexes.
- int posn = 0;
- while (posn < batch.size) {
- int chunkSize = Math.min(batch.size - posn,
- rowIndexStride - rowsInIndex);
- treeWriter.writeRootBatch(batch, posn, chunkSize);
- posn += chunkSize;
- rowsInIndex += chunkSize;
- rowsInStripe += chunkSize;
- if (rowsInIndex >= rowIndexStride) {
- createRowIndexEntry();
- }
- }
- } else {
- rowsInStripe += batch.size;
- treeWriter.writeRootBatch(batch, 0, batch.size);
- }
- memoryManager.addedRow();
- }
-
-
- @Override
- public void close() throws IOException {
- if (callback != null) {
- callback.preFooterWrite(callbackContext);
- }
- // remove us from the memory manager so that we don't get any callbacks
- memoryManager.removeWriter(path);
- // actually close the file
- flushStripe();
- int metadataLength = writeMetadata();
- int footerLength = writeFooter(rawWriter.getBytesWritten() - metadataLength);
- rawWriter.write(writePostScript(footerLength, metadataLength));
- rawWriter.close();
-
- }
-
- /**
- * Raw data size will be compute when writing the file footer. Hence raw data
- * size value will be available only after closing the writer.
- */
- @Override
- public long getRawDataSize() {
- return rawDataSize;
- }
-
- /**
- * Row count gets updated when flushing the stripes. To get accurate row
- * count call this method after writer is closed.
- */
- @Override
- public long getNumberOfRows() {
- return rowCount;
- }
-
- @Override
- public long writeIntermediateFooter() throws IOException {
- // flush any buffered rows
- flushStripe();
- // write a footer
- if (stripesAtLastFlush != stripes.size()) {
- if (callback != null) {
- callback.preFooterWrite(callbackContext);
- }
- int metaLength = writeMetadata();
- int footLength = writeFooter(rawWriter.getBytesWritten() - metaLength);
- rawWriter.write(writePostScript(footLength, metaLength));
- stripesAtLastFlush = stripes.size();
- rawWriter.flush();
- }
- return rawWriter.getBytesWritten();
- }
-
- @Override
- public void appendStripe(byte[] stripe, int offset, int length,
- StripeInformation stripeInfo,
- OrcProto.StripeStatistics stripeStatistics) throws IOException {
- Objects.requireNonNull(stripe, "Stripe must not be null");
- if (length > stripe.length) {
- throw new IllegalArgumentException("Specified length must not be greater specified array length");
- }
- Objects.requireNonNull(stripeInfo, "Stripe information must not be null");
- Objects.requireNonNull(stripeStatistics,
- "Stripe statistics must not be null");
-
- getStream();
- long start = rawWriter.getBytesWritten();
- long stripeLen = length;
- long availBlockSpace = blockSize - (start % blockSize);
-
- // see if stripe can fit in the current hdfs block, else pad the remaining
- // space in the block
- if (stripeLen < blockSize && stripeLen > availBlockSpace && addBlockPadding) {
- byte[] pad = new byte[(int) Math.min(HDFS_BUFFER_SIZE, availBlockSpace)];
- LOG.info(String.format("Padding ORC by %d bytes while merging..",
- availBlockSpace));
- start += availBlockSpace;
- while (availBlockSpace > 0) {
- int writeLen = (int) Math.min(availBlockSpace, pad.length);
- rawWriter.write(pad, 0, writeLen);
- availBlockSpace -= writeLen;
- }
- }
-
- rawWriter.write(stripe);
- rowsInStripe = stripeStatistics.getColStats(0).getNumberOfValues();
- rowCount += rowsInStripe;
-
- // since we have already written the stripe, just update stripe statistics
- treeWriter.stripeStatsBuilders.add(stripeStatistics.toBuilder());
-
- // update file level statistics
- updateFileStatistics(stripeStatistics);
-
- // update stripe information
- OrcProto.StripeInformation dirEntry = OrcProto.StripeInformation
- .newBuilder()
- .setOffset(start)
- .setNumberOfRows(rowsInStripe)
- .setIndexLength(stripeInfo.getIndexLength())
- .setDataLength(stripeInfo.getDataLength())
- .setFooterLength(stripeInfo.getFooterLength())
- .build();
- stripes.add(dirEntry);
-
- // reset it after writing the stripe
- rowsInStripe = 0;
- }
-
- private void updateFileStatistics(OrcProto.StripeStatistics stripeStatistics) {
- List cs = stripeStatistics.getColStatsList();
- List allWriters = getAllColumnTreeWriters(treeWriter);
- for (int i = 0; i < allWriters.size(); i++) {
- allWriters.get(i).fileStatistics.merge(ColumnStatisticsImpl.deserialize(cs.get(i)));
- }
- }
-
- private List getAllColumnTreeWriters(TreeWriter rootTreeWriter) {
- List result = Lists.newArrayList();
- getAllColumnTreeWritersImpl(rootTreeWriter, result);
- return result;
- }
-
- private void getAllColumnTreeWritersImpl(TreeWriter tw,
- List result) {
- result.add(tw);
- for (TreeWriter child : tw.childrenWriters) {
- getAllColumnTreeWritersImpl(child, result);
- }
- }
-
- @Override
- public void appendUserMetadata(List userMetadata) {
- if (userMetadata != null) {
- for (UserMetadataItem item : userMetadata) {
- this.userMetadata.put(item.getName(), item.getValue());
- }
- }
- }
-}
diff --git a/nifi-nar-bundles/nifi-hive-bundle/nifi-hive-processors/src/main/java/org/apache/nifi/dbcp/hive/HiveConnectionPool.java b/nifi-nar-bundles/nifi-hive-bundle/nifi-hive-processors/src/main/java/org/apache/nifi/dbcp/hive/HiveConnectionPool.java
deleted file mode 100644
index d9b2999378..0000000000
--- a/nifi-nar-bundles/nifi-hive-bundle/nifi-hive-processors/src/main/java/org/apache/nifi/dbcp/hive/HiveConnectionPool.java
+++ /dev/null
@@ -1,459 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements. See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.nifi.dbcp.hive;
-
-import org.apache.commons.dbcp2.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.DeprecationNotice;
-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.PropertyValue;
-import org.apache.nifi.components.ValidationContext;
-import org.apache.nifi.components.ValidationResult;
-import org.apache.nifi.components.resource.ResourceCardinality;
-import org.apache.nifi.components.resource.ResourceType;
-import org.apache.nifi.controller.AbstractControllerService;
-import org.apache.nifi.controller.ConfigurationContext;
-import org.apache.nifi.controller.ControllerServiceInitializationContext;
-import org.apache.nifi.dbcp.DBCPValidator;
-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.security.krb.KerberosKeytabUser;
-import org.apache.nifi.security.krb.KerberosLoginException;
-import org.apache.nifi.security.krb.KerberosPasswordUser;
-import org.apache.nifi.security.krb.KerberosUser;
-import org.apache.nifi.util.hive.AuthenticationFailedException;
-import org.apache.nifi.util.hive.HiveConfigurator;
-import org.apache.nifi.util.hive.ValidationResources;
-
-import java.io.File;
-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;
-
-/**
- * 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. Connections can be asked from pool and returned after usage.")
-@DeprecationNotice(classNames = "org.apache.nifi.dbcp.hive.Hive3ConnectionPool")
-public class HiveConnectionPool extends AbstractControllerService implements HiveDBCPService {
- private static final String ALLOW_EXPLICIT_KEYTAB = "NIFI_ALLOW_EXPLICIT_KEYTAB";
-
- private static final String DEFAULT_MAX_CONN_LIFETIME = "-1";
-
- public 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();
-
- public 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)
- .identifiesExternalResource(ResourceCardinality.MULTIPLE, ResourceType.FILE)
- .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
- .build();
-
- public 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();
-
- public 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();
-
- public 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();
-
- public 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();
-
- public static final PropertyDescriptor MAX_CONN_LIFETIME = new PropertyDescriptor.Builder()
- .displayName("Max Connection Lifetime")
- .name("hive-max-conn-lifetime")
- .description("The maximum lifetime in milliseconds of a connection. After this time is exceeded the " +
- "connection pool will invalidate the connection. A value of zero or -1 " +
- "means the connection has an infinite lifetime.")
- .defaultValue(DEFAULT_MAX_CONN_LIFETIME)
- .required(true)
- .addValidator(DBCPValidator.CUSTOM_TIME_PERIOD_VALIDATOR)
- .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
- .build();
-
- public 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();
-
- 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 properties;
-
- private String connectionUrl = "unknown";
-
- // Holder of cached Configuration information so validation does not reload the same config over and over
- private final AtomicReference validationResourceHolder = new AtomicReference<>();
-
- private volatile BasicDataSource dataSource;
-
- private volatile HiveConfigurator hiveConfigurator = new HiveConfigurator();
- private volatile UserGroupInformation ugi;
- private final AtomicReference kerberosUserReference = new AtomicReference<>();
- private volatile File kerberosConfigFile = null;
- private volatile KerberosProperties kerberosProperties;
-
- @Override
- protected void init(final ControllerServiceInitializationContext context) {
- List 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(MAX_CONN_LIFETIME);
- props.add(VALIDATION_QUERY);
- props.add(KERBEROS_CREDENTIALS_SERVICE);
-
- kerberosConfigFile = context.getKerberosConfigurationFile();
- kerberosProperties = new KerberosProperties(kerberosConfigFile);
- props.add(kerberosProperties.getKerberosPrincipal());
- props.add(kerberosProperties.getKerberosKeytab());
- props.add(kerberosProperties.getKerberosPassword());
- properties = props;
- }
-
- @Override
- protected List getSupportedPropertyDescriptors() {
- return properties;
- }
-
- @Override
- protected Collection customValidate(ValidationContext validationContext) {
- boolean confFileProvided = validationContext.getProperty(HIVE_CONFIGURATION_RESOURCES).isSet();
-
- final List problems = new ArrayList<>();
-
- if (confFileProvided) {
- final String explicitPrincipal = validationContext.getProperty(kerberosProperties.getKerberosPrincipal()).evaluateAttributeExpressions().getValue();
- final String explicitKeytab = validationContext.getProperty(kerberosProperties.getKerberosKeytab()).evaluateAttributeExpressions().getValue();
- final String explicitPassword = validationContext.getProperty(kerberosProperties.getKerberosPassword()).getValue();
- final KerberosCredentialsService credentialsService = validationContext.getProperty(KERBEROS_CREDENTIALS_SERVICE).asControllerService(KerberosCredentialsService.class);
-
- final String resolvedPrincipal;
- final String resolvedKeytab;
- if (credentialsService != null) {
- resolvedPrincipal = credentialsService.getPrincipal();
- resolvedKeytab = credentialsService.getKeytab();
- } else {
- resolvedPrincipal = explicitPrincipal;
- resolvedKeytab = explicitKeytab;
- }
-
-
- final String configFiles = validationContext.getProperty(HIVE_CONFIGURATION_RESOURCES).evaluateAttributeExpressions().getValue();
- problems.addAll(hiveConfigurator.validate(configFiles, resolvedPrincipal, resolvedKeytab, explicitPassword, validationResourceHolder, getLogger()));
-
- if (credentialsService != null && (explicitPrincipal != null || explicitKeytab != null || explicitPassword != null)) {
- problems.add(new ValidationResult.Builder()
- .subject("Kerberos Credentials")
- .valid(false)
- .explanation("Cannot specify a Kerberos Credentials Service while also specifying a Kerberos Principal, Kerberos Keytab, or Kerberos Password")
- .build());
- }
-
- if (!isAllowExplicitKeytab() && 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 Kerberos 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}.
- *
- * 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.
- *
- * 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 use a separate thread to
- * relogin, and instead call {@link UserGroupInformation#checkTGTAndReloginFromKeytab()} from
- * {@link HiveConnectionPool#getConnection()}. The relogin request is performed in a synchronized block to prevent
- * threads from requesting concurrent relogins. For more information, please read the documentation for
- * {@link SecurityUtil#loginKerberos(Configuration, String, String)}.
- *
- * 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 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 String explicitPassword = context.getProperty(kerberosProperties.getKerberosPassword()).getValue();
- final KerberosCredentialsService credentialsService = context.getProperty(KERBEROS_CREDENTIALS_SERVICE).asControllerService(KerberosCredentialsService.class);
-
- final String resolvedPrincipal;
- final String resolvedKeytab;
- if (credentialsService != null) {
- resolvedPrincipal = credentialsService.getPrincipal();
- resolvedKeytab = credentialsService.getKeytab();
- } else {
- resolvedPrincipal = explicitPrincipal;
- resolvedKeytab = explicitKeytab;
- }
-
- if (resolvedKeytab != null) {
- kerberosUserReference.set(new KerberosKeytabUser(resolvedPrincipal, resolvedKeytab));
- log.info("Hive Security Enabled, logging in as principal {} with keytab {}", new Object[] {resolvedPrincipal, resolvedKeytab});
- } else if (explicitPassword != null) {
- kerberosUserReference.set(new KerberosPasswordUser(resolvedPrincipal, explicitPassword));
- log.info("Hive Security Enabled, logging in as principal {} with password", new Object[] {resolvedPrincipal});
- } else {
- throw new InitializationException("Unable to authenticate with Kerberos, no keytab or password was provided");
- }
-
- try {
- ugi = hiveConfigurator.authenticate(hiveConfig, kerberosUserReference.get());
- } catch (AuthenticationFailedException ae) {
- log.error(ae.getMessage(), ae);
- throw new InitializationException(ae);
- }
-
- getLogger().info("Successfully logged in as principal " + resolvedPrincipal);
- }
-
- 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();
- final long maxConnectionLifetimeMillis = extractMillisWithInfinite(context.getProperty(MAX_CONN_LIFETIME).evaluateAttributeExpressions());
-
- dataSource = new BasicDataSource();
- dataSource.setDriverClassName(drv);
-
- connectionUrl = context.getProperty(DATABASE_URL).evaluateAttributeExpressions().getValue();
-
- dataSource.setMaxWaitMillis(maxWaitMillis);
- dataSource.setMaxTotal(maxTotal);
- dataSource.setMaxConnLifetimeMillis(maxConnectionLifetimeMillis);
-
- 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) {
- /*
- * Explicitly check the TGT and relogin if necessary with the KerberosUser instance. No synchronization
- * is necessary in the client code, since AbstractKerberosUser's checkTGTAndRelogin method is synchronized.
- */
- getLogger().trace("getting UGI instance");
- if (kerberosUserReference.get() != null) {
- // if there's a KerberosUser associated with this UGI, check the TGT and relogin if it is close to expiring
- KerberosUser kerberosUser = kerberosUserReference.get();
- getLogger().debug("kerberosUser is " + kerberosUser);
- try {
- getLogger().debug("checking TGT on kerberosUser [{}]", new Object[]{kerberosUser});
- kerberosUser.checkTGTAndRelogin();
- } catch (final KerberosLoginException e) {
- throw new ProcessException("Unable to relogin with kerberos credentials for " + kerberosUser.getPrincipal(), e);
- }
- } else {
- getLogger().debug("kerberosUser was null, will not refresh TGT with KerberosUser");
- // no synchronization is needed for UserGroupInformation.checkTGTAndReloginFromKeytab; UGI handles the synchronization internally
- ugi.checkTGTAndReloginFromKeytab();
- }
- try {
- return ugi.doAs((PrivilegedExceptionAction) () -> 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 "HiveConnectionPool[id=" + getIdentifier() + "]";
- }
-
- @Override
- public String getConnectionURL() {
- return connectionUrl;
- }
-
- /*
- * Overridable by subclasses in the same package, mainly intended for testing purposes to allow verification without having to set environment variables.
- */
- boolean isAllowExplicitKeytab() {
- return Boolean.parseBoolean(System.getenv(ALLOW_EXPLICIT_KEYTAB));
- }
-
- private long extractMillisWithInfinite(PropertyValue prop) {
- if (prop.getValue() == null || DEFAULT_MAX_CONN_LIFETIME.equals(prop.getValue())) {
- return -1;
- } else {
- return prop.asTimePeriod(TimeUnit.MILLISECONDS);
- }
- }
-
-}
diff --git a/nifi-nar-bundles/nifi-hive-bundle/nifi-hive-processors/src/main/java/org/apache/nifi/processors/hive/AbstractHiveQLProcessor.java b/nifi-nar-bundles/nifi-hive-bundle/nifi-hive-processors/src/main/java/org/apache/nifi/processors/hive/AbstractHiveQLProcessor.java
deleted file mode 100644
index f1ef5fd7e7..0000000000
--- a/nifi-nar-bundles/nifi-hive-bundle/nifi-hive-processors/src/main/java/org/apache/nifi/processors/hive/AbstractHiveQLProcessor.java
+++ /dev/null
@@ -1,344 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements. See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.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.HiveDBCPService;
-import org.apache.nifi.flowfile.FlowFile;
-import org.apache.nifi.processor.AbstractSessionFactoryProcessor;
-import org.apache.nifi.processor.ProcessSession;
-import org.apache.nifi.processor.io.InputStreamCallback;
-import org.apache.nifi.processor.util.StandardValidators;
-import org.apache.nifi.stream.io.StreamUtils;
-
-import java.io.IOException;
-import java.io.InputStream;
-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 AbstractHiveQLProcessor 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("Hive Database Connection Pooling Service")
- .description("The Hive Controller Service that is used to obtain connection(s) to the Hive database")
- .required(true)
- .identifiesControllerService(HiveDBCPService.class)
- .build();
-
- public static final PropertyDescriptor CHARSET = new PropertyDescriptor.Builder()
- .name("hive-charset")
- .displayName("Character Set")
- .description("Specifies the character set of the record data.")
- .required(true)
- .defaultValue("UTF-8")
- .addValidator(StandardValidators.CHARACTER_SET_VALIDATOR)
- .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, new InputStreamCallback() {
- @Override
- public void process(final InputStream in) throws IOException {
- 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 attributes) throws SQLException {
-
- Map parmMap = new TreeMap();
-
- for (final Map.Entry 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 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 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 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 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 toQueryTableAttributes(Set tableNames) {
- final Map 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;
- }
-}
diff --git a/nifi-nar-bundles/nifi-hive-bundle/nifi-hive-processors/src/main/java/org/apache/nifi/processors/hive/ConvertAvroToORC.java b/nifi-nar-bundles/nifi-hive-bundle/nifi-hive-processors/src/main/java/org/apache/nifi/processors/hive/ConvertAvroToORC.java
deleted file mode 100644
index d918365a37..0000000000
--- a/nifi-nar-bundles/nifi-hive-bundle/nifi-hive-processors/src/main/java/org/apache/nifi/processors/hive/ConvertAvroToORC.java
+++ /dev/null
@@ -1,293 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements. See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.nifi.processors.hive;
-
-import org.apache.avro.Schema;
-import org.apache.avro.file.DataFileStream;
-import org.apache.avro.generic.GenericDatumReader;
-import org.apache.avro.generic.GenericRecord;
-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.OrcFlowFileWriter;
-import org.apache.hadoop.hive.serde2.typeinfo.TypeInfo;
-import org.apache.nifi.annotation.behavior.InputRequirement;
-import org.apache.nifi.annotation.behavior.SideEffectFree;
-import org.apache.nifi.annotation.behavior.SupportsBatching;
-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.components.resource.ResourceCardinality;
-import org.apache.nifi.components.resource.ResourceType;
-import org.apache.nifi.expression.ExpressionLanguageScope;
-import org.apache.nifi.flowfile.FlowFile;
-import org.apache.nifi.flowfile.attributes.CoreAttributes;
-import org.apache.nifi.processor.AbstractProcessor;
-import org.apache.nifi.processor.DataUnit;
-import org.apache.nifi.processor.ProcessContext;
-import org.apache.nifi.processor.ProcessSession;
-import org.apache.nifi.processor.Relationship;
-import org.apache.nifi.processor.exception.ProcessException;
-import org.apache.nifi.processor.util.StandardValidators;
-import org.apache.nifi.util.hive.HiveJdbcCommon;
-
-import java.io.BufferedInputStream;
-import java.io.BufferedOutputStream;
-import java.io.IOException;
-import java.io.InputStream;
-import java.io.OutputStream;
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Set;
-import java.util.concurrent.atomic.AtomicInteger;
-import java.util.concurrent.atomic.AtomicReference;
-
-/**
- * The ConvertAvroToORC processor takes an Avro-formatted flow file as input and converts it into ORC format.
- */
-@SideEffectFree
-@SupportsBatching
-@Tags({"avro", "orc", "hive", "convert"})
-@InputRequirement(InputRequirement.Requirement.INPUT_REQUIRED)
-@CapabilityDescription("Converts an Avro record into ORC file format. This processor provides a direct mapping of an Avro record to an ORC record, such "
- + "that the resulting ORC file will have the same hierarchical structure as the Avro document. If an incoming FlowFile contains a stream of "
- + "multiple Avro records, the resultant FlowFile will contain a ORC file containing all of the Avro records. If an incoming FlowFile does "
- + "not contain any records, an empty ORC file is the output. NOTE: Many Avro datatypes (collections, primitives, and unions of primitives, e.g.) can "
- + "be converted to ORC, but unions of collections and other complex datatypes may not be able to be converted to ORC.")
-@WritesAttributes({
- @WritesAttribute(attribute = "mime.type", description = "Sets the mime type to application/octet-stream"),
- @WritesAttribute(attribute = "filename", description = "Sets the filename to the existing filename with the extension replaced by / added to by .orc"),
- @WritesAttribute(attribute = "record.count", description = "Sets the number of records in the ORC file."),
- @WritesAttribute(attribute = "hive.ddl", description = "Creates a partial Hive DDL statement for creating a table in Hive from this ORC file. "
- + "This can be used in ReplaceText for setting the content to the DDL. To make it valid DDL, add \"LOCATION ''\", where "
- + "the path is the directory that contains this ORC file on HDFS. For example, ConvertAvroToORC can send flow files to a PutHDFS processor to send the file to "
- + "HDFS, then to a 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.")
-})
-public class ConvertAvroToORC extends AbstractProcessor {
-
- // Attributes
- public static final String ORC_MIME_TYPE = "application/octet-stream";
- public static final String HIVE_DDL_ATTRIBUTE = "hive.ddl";
- public static final String RECORD_COUNT_ATTRIBUTE = "record.count";
-
-
- // Properties
- public static final PropertyDescriptor ORC_CONFIGURATION_RESOURCES = new PropertyDescriptor.Builder()
- .name("orc-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)
- .identifiesExternalResource(ResourceCardinality.MULTIPLE, ResourceType.FILE)
- .build();
-
- public static final PropertyDescriptor STRIPE_SIZE = new PropertyDescriptor.Builder()
- .name("orc-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("orc-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();
-
- public static final PropertyDescriptor COMPRESSION_TYPE = new PropertyDescriptor.Builder()
- .name("orc-compression-type")
- .displayName("Compression Type")
- .required(true)
- .allowableValues("NONE", "ZLIB", "SNAPPY", "LZO")
- .defaultValue("NONE")
- .build();
-
- public static final PropertyDescriptor HIVE_TABLE_NAME = new PropertyDescriptor.Builder()
- .name("orc-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 PutHiveQL 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();
-
- // Relationships
- static final Relationship REL_SUCCESS = new Relationship.Builder()
- .name("success")
- .description("A FlowFile is routed to this relationship after it has been converted to ORC format.")
- .build();
- static final Relationship REL_FAILURE = new Relationship.Builder()
- .name("failure")
- .description("A FlowFile is routed to this relationship if it cannot be parsed as Avro or cannot be converted to ORC for any reason")
- .build();
-
- private final static List propertyDescriptors;
- private final static Set relationships;
-
- private volatile Configuration orcConfig;
-
- /*
- * Will ensure that the list of property descriptors is built only once.
- * Will also create a Set of relationships
- */
- static {
- List _propertyDescriptors = new ArrayList<>();
- _propertyDescriptors.add(ORC_CONFIGURATION_RESOURCES);
- _propertyDescriptors.add(STRIPE_SIZE);
- _propertyDescriptors.add(BUFFER_SIZE);
- _propertyDescriptors.add(COMPRESSION_TYPE);
- _propertyDescriptors.add(HIVE_TABLE_NAME);
- propertyDescriptors = Collections.unmodifiableList(_propertyDescriptors);
-
- Set _relationships = new HashSet<>();
- _relationships.add(REL_SUCCESS);
- _relationships.add(REL_FAILURE);
- relationships = Collections.unmodifiableSet(_relationships);
- }
-
- @Override
- protected List getSupportedPropertyDescriptors() {
- return propertyDescriptors;
- }
-
- @Override
- public Set getRelationships() {
- return relationships;
- }
-
- @OnScheduled
- public void setup(ProcessContext context) {
- boolean confFileProvided = context.getProperty(ORC_CONFIGURATION_RESOURCES).isSet();
- if (confFileProvided) {
- final String configFiles = context.getProperty(ORC_CONFIGURATION_RESOURCES).getValue();
- orcConfig = HiveJdbcCommon.getConfigurationFromFiles(configFiles);
- }
- }
-
- @Override
- public void onTrigger(final ProcessContext context, final ProcessSession session) throws ProcessException {
- FlowFile flowFile = session.get();
- if (flowFile == null) {
- return;
- }
-
- try {
- long startTime = System.currentTimeMillis();
- 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 AtomicReference hiveAvroSchema = new AtomicReference<>(null);
- final AtomicInteger totalRecordCount = new AtomicInteger(0);
- final String fileName = flowFile.getAttribute(CoreAttributes.FILENAME.key());
- flowFile = session.write(flowFile, (rawIn, rawOut) -> {
- try (final InputStream in = new BufferedInputStream(rawIn);
- final OutputStream out = new BufferedOutputStream(rawOut);
- final DataFileStream reader = new DataFileStream<>(in, new GenericDatumReader<>())) {
-
- // Create ORC schema from Avro schema
- Schema avroSchema = reader.getSchema();
-
- TypeInfo orcSchema = NiFiOrcUtils.getOrcField(avroSchema);
-
- if (orcConfig == null) {
- orcConfig = new Configuration();
- }
-
- OrcFlowFileWriter orcWriter = NiFiOrcUtils.createWriter(
- out,
- new Path(fileName),
- orcConfig,
- orcSchema,
- stripeSize,
- compressionType,
- bufferSize);
- try {
-
- int recordCount = 0;
- while (reader.hasNext()) {
- GenericRecord currRecord = reader.next();
- List fields = currRecord.getSchema().getFields();
- if (fields != null) {
- Object[] row = new Object[fields.size()];
- for (int i = 0; i < fields.size(); i++) {
- Schema.Field field = fields.get(i);
- Schema fieldSchema = field.schema();
- Object o = currRecord.get(field.name());
- try {
- row[i] = NiFiOrcUtils.convertToORCObject(NiFiOrcUtils.getOrcField(fieldSchema), o);
- } catch (ArrayIndexOutOfBoundsException aioobe) {
- getLogger().error("Index out of bounds at record {} for column {}, type {}, and object {}",
- new Object[]{recordCount, i, fieldSchema.getType().getName(), o.toString()},
- aioobe);
- throw new IOException(aioobe);
- }
- }
- orcWriter.addRow(NiFiOrcUtils.createOrcStruct(orcSchema, row));
- recordCount++;
- }
- }
- hiveAvroSchema.set(avroSchema);
- totalRecordCount.set(recordCount);
- } finally {
- // finished writing this record, close the writer (which will flush to the flow file)
- orcWriter.close();
- }
- }
- });
-
- final String hiveTableName = context.getProperty(HIVE_TABLE_NAME).isSet()
- ? context.getProperty(HIVE_TABLE_NAME).evaluateAttributeExpressions(flowFile).getValue()
- : NiFiOrcUtils.normalizeHiveTableName(hiveAvroSchema.get().getFullName());
- String hiveDDL = NiFiOrcUtils.generateHiveDDL(hiveAvroSchema.get(), hiveTableName);
- // Add attributes and transfer to success
- flowFile = session.putAttribute(flowFile, RECORD_COUNT_ATTRIBUTE, Integer.toString(totalRecordCount.get()));
- flowFile = session.putAttribute(flowFile, HIVE_DDL_ATTRIBUTE, hiveDDL);
- StringBuilder newFilename = new StringBuilder();
- int extensionIndex = fileName.lastIndexOf(".");
- if (extensionIndex != -1) {
- newFilename.append(fileName.substring(0, extensionIndex));
- } else {
- newFilename.append(fileName);
- }
- newFilename.append(".orc");
- flowFile = session.putAttribute(flowFile, CoreAttributes.MIME_TYPE.key(), ORC_MIME_TYPE);
- flowFile = session.putAttribute(flowFile, CoreAttributes.FILENAME.key(), newFilename.toString());
- session.transfer(flowFile, REL_SUCCESS);
- session.getProvenanceReporter().modifyContent(flowFile, "Converted " + totalRecordCount.get() + " records", System.currentTimeMillis() - startTime);
-
- } catch (ProcessException | IllegalArgumentException e) {
- getLogger().error("Failed to convert {} from Avro to ORC due to {}; transferring to failure", new Object[]{flowFile, e});
- session.transfer(flowFile, REL_FAILURE);
- }
- }
-}
diff --git a/nifi-nar-bundles/nifi-hive-bundle/nifi-hive-processors/src/main/java/org/apache/nifi/processors/hive/PutHiveQL.java b/nifi-nar-bundles/nifi-hive-bundle/nifi-hive-processors/src/main/java/org/apache/nifi/processors/hive/PutHiveQL.java
deleted file mode 100644
index ffa0a3edc2..0000000000
--- a/nifi-nar-bundles/nifi-hive-bundle/nifi-hive-processors/src/main/java/org/apache/nifi/processors/hive/PutHiveQL.java
+++ /dev/null
@@ -1,300 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements. See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.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.DeprecationNotice;
-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.HiveDBCPService;
-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(SelectHiveQL.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.")
-})
-@DeprecationNotice(classNames = "org.apache.nifi.processors.hive.PutHive3QL")
-public class PutHiveQL extends AbstractHiveQLProcessor {
-
- 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 propertyDescriptors;
- private final static Set relationships;
-
- /*
- * Will ensure that the list of property descriptors is built only once.
- * Will also create a Set of relationships
- */
- static {
- List _propertyDescriptors = new ArrayList<>();
- _propertyDescriptors.add(HIVE_DBCP_SERVICE);
- _propertyDescriptors.add(BATCH_SIZE);
- _propertyDescriptors.add(CHARSET);
- _propertyDescriptors.add(STATEMENT_DELIMITER);
- _propertyDescriptors.add(RollbackOnFailure.ROLLBACK_ON_FAILURE);
- propertyDescriptors = Collections.unmodifiableList(_propertyDescriptors);
-
- Set _relationships = new HashSet<>();
- _relationships.add(REL_SUCCESS);
- _relationships.add(REL_FAILURE);
- _relationships.add(REL_RETRY);
- relationships = Collections.unmodifiableSet(_relationships);
- }
-
- private Put process;
- private ExceptionHandler exceptionHandler;
-
- @OnScheduled
- public void constructProcess() {
- exceptionHandler = new ExceptionHandler<>();
- exceptionHandler.mapException(e -> {
- if (e instanceof SQLNonTransientException) {
- return ErrorTypes.InvalidInput;
- } else if (e instanceof SQLException) {
- // Use the SQLException's vendor code for guidance -- see Hive's ErrorMsg class for details on error codes
- int errorCode = ((SQLException) e).getErrorCode();
- getLogger().debug("Error occurred during Hive operation, Hive returned error code {}", new Object[]{errorCode});
- if (errorCode >= 10000 && errorCode < 20000) {
- return ErrorTypes.InvalidInput;
- } else if (errorCode >= 20000 && errorCode < 30000) {
- return ErrorTypes.InvalidInput;
- } else if (errorCode >= 30000 && errorCode < 40000) {
- return ErrorTypes.TemporalInputFailure;
- } else if (errorCode >= 40000 && errorCode < 50000) {
- // These are unknown errors (to include some parse errors), but rather than generating an UnknownFailure which causes
- // a ProcessException, we'll route to failure via an InvalidInput error type.
- return ErrorTypes.InvalidInput;
- } else {
- // Default unknown errors to TemporalFailure (as they were implemented originally), so they can be routed to failure
- // or rolled back depending on the user's setting of Rollback On Failure.
- 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 getSupportedPropertyDescriptors() {
- return propertyDescriptors;
- }
-
- @Override
- public Set 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 initConnection = (context, session, fc, ffs) -> {
- final HiveDBCPService dbcpService = context.getProperty(HIVE_DBCP_SERVICE).asControllerService(HiveDBCPService.class);
- final Connection connection = dbcpService.getConnection(ffs == null || ffs.isEmpty() ? Collections.emptyMap() : ffs.get(0).getAttributes());
- fc.connectionUrl = dbcpService.getConnectionURL();
- return connection;
- };
-
- private FetchFlowFiles fetchFlowFiles = (context, session, functionContext, result) -> {
- final int batchSize = context.getProperty(BATCH_SIZE).asInteger();
- return session.get(batchSize);
- };
-
- private Put.PutFlowFile putFlowFile = (context, session, fc, conn, flowFile, result) -> {
- final String script = getHiveQL(session, flowFile, fc.charset);
- String regex = "(? 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)) {
- try (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);
- }
-
- // 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 onFlowFileError(final ProcessContext context, final ProcessSession session, final RoutingResult result) {
- OnError 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;
- case Self:
- getLogger().error("Failed to update Hive for {} due to {};", 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));
- }
-}
\ No newline at end of file
diff --git a/nifi-nar-bundles/nifi-hive-bundle/nifi-hive-processors/src/main/java/org/apache/nifi/processors/hive/PutHiveStreaming.java b/nifi-nar-bundles/nifi-hive-bundle/nifi-hive-processors/src/main/java/org/apache/nifi/processors/hive/PutHiveStreaming.java
deleted file mode 100644
index 7511e44e2e..0000000000
--- a/nifi-nar-bundles/nifi-hive-bundle/nifi-hive-processors/src/main/java/org/apache/nifi/processors/hive/PutHiveStreaming.java
+++ /dev/null
@@ -1,1216 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements. See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.nifi.processors.hive;
-
-import com.google.common.util.concurrent.ThreadFactoryBuilder;
-import org.apache.avro.AvroRuntimeException;
-import org.apache.avro.file.CodecFactory;
-import org.apache.avro.file.DataFileConstants;
-import org.apache.avro.file.DataFileStream;
-import org.apache.avro.file.DataFileWriter;
-import org.apache.avro.file.SeekableByteArrayInput;
-import org.apache.avro.generic.GenericDatumReader;
-import org.apache.avro.generic.GenericDatumWriter;
-import org.apache.avro.generic.GenericRecord;
-import org.apache.hadoop.hive.conf.HiveConf;
-import org.apache.hadoop.security.UserGroupInformation;
-import org.apache.hive.hcatalog.streaming.ConnectionError;
-import org.apache.hive.hcatalog.streaming.HiveEndPoint;
-import org.apache.hive.hcatalog.streaming.SerializationError;
-import org.apache.hive.hcatalog.streaming.StreamingException;
-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.DeprecationNotice;
-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.components.Validator;
-import org.apache.nifi.components.resource.ResourceCardinality;
-import org.apache.nifi.components.resource.ResourceType;
-import org.apache.nifi.expression.ExpressionLanguageScope;
-import org.apache.nifi.flowfile.FlowFile;
-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.AbstractSessionFactoryProcessor;
-import org.apache.nifi.processor.ProcessContext;
-import org.apache.nifi.processor.ProcessSession;
-import org.apache.nifi.processor.ProcessSessionFactory;
-import org.apache.nifi.processor.ProcessorInitializationContext;
-import org.apache.nifi.processor.Relationship;
-import org.apache.nifi.processor.exception.ProcessException;
-import org.apache.nifi.processor.io.InputStreamCallback;
-import org.apache.nifi.processor.util.StandardValidators;
-import org.apache.nifi.processor.util.pattern.DiscontinuedException;
-import org.apache.nifi.processor.util.pattern.ErrorTypes;
-import org.apache.nifi.processor.util.pattern.ExceptionHandler;
-import org.apache.nifi.processor.util.pattern.RollbackOnFailure;
-import org.apache.nifi.processor.util.pattern.RoutingResult;
-import org.apache.nifi.security.krb.KerberosKeytabUser;
-import org.apache.nifi.security.krb.KerberosLoginException;
-import org.apache.nifi.security.krb.KerberosPasswordUser;
-import org.apache.nifi.security.krb.KerberosUser;
-import org.apache.nifi.util.hive.AuthenticationFailedException;
-import org.apache.nifi.util.hive.HiveConfigurator;
-import org.apache.nifi.util.hive.HiveOptions;
-import org.apache.nifi.util.hive.HiveUtils;
-import org.apache.nifi.util.hive.HiveWriter;
-import org.apache.nifi.util.hive.ValidationResources;
-import org.xerial.snappy.Snappy;
-
-import java.io.ByteArrayOutputStream;
-import java.io.File;
-import java.io.IOException;
-import java.io.InputStream;
-import java.nio.charset.StandardCharsets;
-import java.util.ArrayList;
-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.Set;
-import java.util.Timer;
-import java.util.TimerTask;
-import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.ConcurrentLinkedQueue;
-import java.util.concurrent.ExecutorService;
-import java.util.concurrent.Executors;
-import java.util.concurrent.Semaphore;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.atomic.AtomicBoolean;
-import java.util.concurrent.atomic.AtomicInteger;
-import java.util.concurrent.atomic.AtomicReference;
-import java.util.function.BiFunction;
-import java.util.regex.Pattern;
-
-@Tags({"hive", "streaming", "put", "database", "store"})
-@CapabilityDescription("This processor uses Hive Streaming to send flow file data to an Apache Hive table. The incoming flow file is expected to be in "
- + "Avro format and the table must exist in Hive. Please see the Hive documentation for requirements on the Hive table (format, partitions, etc.). "
- + "The partition values are extracted from the Avro record based on the names of the partition columns as specified in the processor. NOTE: If "
- + "multiple concurrent tasks are configured for this processor, only one table can be written to at any time by a single thread. Additional tasks "
- + "intending to write to the same table will wait for the current task to finish writing to the table.")
-@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 written successfully and unsuccessfully, respectively."),
- @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
-@DeprecationNotice(classNames = "org.apache.nifi.processors.hive.PutHive3Streaming")
-public class PutHiveStreaming extends AbstractSessionFactoryProcessor {
- private static final String ALLOW_EXPLICIT_KEYTAB = "NIFI_ALLOW_EXPLICIT_KEYTAB";
-
- // 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";
-
- // Validators
- private 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();
- };
-
- // Metadata keys that are not transferred to split files when output strategy is datafile
- // Avro will write this key/values pairs on its own
- private static final Set RESERVED_METADATA;
-
- static {
- // This is used to prevent a race condition in Snappy 1.0.5 where two classloaders could
- // try to define the native loader class at the same time, causing an error. Make a no-op
- // call here to ensure Snappy's static initializers are called. Note that this block is
- // called once by the extensions loader before any actual processor instances are created,
- // so the race condition will not occur, and for each other instance, this is a no-op
- try {
- Snappy.compress("");
- } catch (IOException ioe) {
- // Do nothing here, should never happen as it is intended to be a no-op
- }
-
- Set reservedMetadata = new HashSet<>();
- reservedMetadata.add("avro.schema");
- reservedMetadata.add("avro.codec");
- RESERVED_METADATA = Collections.unmodifiableSet(reservedMetadata);
- }
-
- // Properties
- public static final PropertyDescriptor METASTORE_URI = new PropertyDescriptor.Builder()
- .name("hive-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();
-
- public 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. 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)
- .identifiesExternalResource(ResourceCardinality.MULTIPLE, ResourceType.FILE)
- .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
- .build();
-
- public static final PropertyDescriptor DB_NAME = new PropertyDescriptor.Builder()
- .name("hive-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();
-
- public static final PropertyDescriptor TABLE_NAME = new PropertyDescriptor.Builder()
- .name("hive-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();
-
- public static final PropertyDescriptor PARTITION_COLUMNS = new PropertyDescriptor.Builder()
- .name("hive-stream-partition-cols")
- .displayName("Partition Columns")
- .description("A comma-delimited list of column names on which the table has been partitioned. The order of values in this list must "
- + "correspond exactly to the order of partition columns specified during the table creation.")
- .required(false)
- .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
- .addValidator(StandardValidators.createRegexMatchingValidator(Pattern.compile("[^,]+(,[^,]+)*"))) // comma-separated list with non-empty entries
- .build();
-
- public static final PropertyDescriptor AUTOCREATE_PARTITIONS = new PropertyDescriptor.Builder()
- .name("hive-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();
-
- public static final PropertyDescriptor MAX_OPEN_CONNECTIONS = new PropertyDescriptor.Builder()
- .name("hive-stream-max-open-connections")
- .displayName("Max Open Connections")
- .description("The maximum number of open 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)
- .sensitive(false)
- .build();
-
- public static final PropertyDescriptor HEARTBEAT_INTERVAL = new PropertyDescriptor.Builder()
- .name("hive-stream-heartbeat-interval")
- .displayName("Heartbeat Interval")
- .description("Indicates that a heartbeat should be sent when the specified number of seconds has elapsed. "
- + "A value of 0 indicates that no heartbeat should be sent. "
- + "Note that although this property supports Expression Language, it will not be evaluated against incoming FlowFile attributes.")
- .defaultValue("60")
- .required(true)
- .addValidator(StandardValidators.NON_NEGATIVE_INTEGER_VALIDATOR)
- .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
- .build();
-
- public static final PropertyDescriptor TXNS_PER_BATCH = new PropertyDescriptor.Builder()
- .name("hive-stream-transactions-per-batch")
- .displayName("Transactions per Batch")
- .description("A hint to Hive Streaming indicating how many transactions the processor task will need. This value must be greater than 1.")
- .required(true)
- .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
- .addValidator(GREATER_THAN_ONE_VALIDATOR)
- .defaultValue("100")
- .build();
-
- public static final PropertyDescriptor RECORDS_PER_TXN = new PropertyDescriptor.Builder()
- .name("hive-stream-records-per-transaction")
- .displayName("Records per Transaction")
- .description("Number of records to process before committing the transaction. This value must be greater than 1.")
- .required(true)
- .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
- .addValidator(GREATER_THAN_ONE_VALIDATOR)
- .defaultValue("10000")
- .build();
-
- public static final PropertyDescriptor CALL_TIMEOUT = new PropertyDescriptor.Builder()
- .name("hive-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();
-
- public 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 propertyDescriptors;
- private Set relationships;
-
- protected KerberosProperties kerberosProperties;
- private volatile File kerberosConfigFile = null;
-
- protected volatile HiveConfigurator hiveConfigurator = new HiveConfigurator();
- protected volatile UserGroupInformation ugi;
- final protected AtomicReference kerberosUserReference = new AtomicReference<>();
- protected volatile HiveConf hiveConfig;
-
- protected final AtomicBoolean sendHeartBeat = new AtomicBoolean(false);
-
- protected volatile int callTimeout;
- protected ExecutorService callTimeoutPool;
- protected transient Timer heartBeatTimer;
-
- protected volatile ConcurrentLinkedQueue