diff --git a/parquet-arrow/pom.xml b/parquet-arrow/pom.xml index 09867eed8e..d34c4fc587 100644 --- a/parquet-arrow/pom.xml +++ b/parquet-arrow/pom.xml @@ -21,7 +21,7 @@ org.apache.parquet parquet ../pom.xml - 1.12.0-kylin-r2 + 1.12.0-kylin-r3 4.0.0 diff --git a/parquet-avro/pom.xml b/parquet-avro/pom.xml index c18abb8453..752081892d 100644 --- a/parquet-avro/pom.xml +++ b/parquet-avro/pom.xml @@ -21,7 +21,7 @@ org.apache.parquet parquet ../pom.xml - 1.12.0-kylin-r2 + 1.12.0-kylin-r3 4.0.0 diff --git a/parquet-benchmarks/pom.xml b/parquet-benchmarks/pom.xml index f06e5d8229..de03614611 100644 --- a/parquet-benchmarks/pom.xml +++ b/parquet-benchmarks/pom.xml @@ -21,7 +21,7 @@ org.apache.parquet parquet ../pom.xml - 1.12.0-kylin-r2 + 1.12.0-kylin-r3 4.0.0 diff --git a/parquet-cascading/pom.xml b/parquet-cascading/pom.xml index 17c3ac9cea..47536c27bb 100644 --- a/parquet-cascading/pom.xml +++ b/parquet-cascading/pom.xml @@ -21,7 +21,7 @@ org.apache.parquet parquet ../pom.xml - 1.12.0-kylin-r2 + 1.12.0-kylin-r3 4.0.0 diff --git a/parquet-cascading3/pom.xml b/parquet-cascading3/pom.xml index bddb478268..b76d9499aa 100644 --- a/parquet-cascading3/pom.xml +++ b/parquet-cascading3/pom.xml @@ -21,7 +21,7 @@ org.apache.parquet parquet ../pom.xml - 1.12.0-kylin-r2 + 1.12.0-kylin-r3 4.0.0 diff --git a/parquet-cli/pom.xml b/parquet-cli/pom.xml index adf676c2a5..8d95abafe3 100644 --- a/parquet-cli/pom.xml +++ b/parquet-cli/pom.xml @@ -21,7 +21,7 @@ org.apache.parquet parquet ../pom.xml - 1.12.0-kylin-r2 + 1.12.0-kylin-r3 4.0.0 diff --git a/parquet-column/pom.xml b/parquet-column/pom.xml index f1326902cc..bf3c3fcc69 100644 --- a/parquet-column/pom.xml +++ b/parquet-column/pom.xml @@ -21,7 +21,7 @@ org.apache.parquet parquet ../pom.xml - 1.12.0-kylin-r2 + 1.12.0-kylin-r3 4.0.0 diff --git a/parquet-column/src/main/java/org/apache/parquet/column/ParquetProperties.java b/parquet-column/src/main/java/org/apache/parquet/column/ParquetProperties.java index 41e482cfdd..843ed39efa 100644 --- a/parquet-column/src/main/java/org/apache/parquet/column/ParquetProperties.java +++ b/parquet-column/src/main/java/org/apache/parquet/column/ParquetProperties.java @@ -49,6 +49,8 @@ public class ParquetProperties { public static final int DEFAULT_MAXIMUM_RECORD_COUNT_FOR_CHECK = 10000; public static final int DEFAULT_COLUMN_INDEX_TRUNCATE_LENGTH = 64; public static final int DEFAULT_PAGE_ROW_COUNT_LIMIT = 20_000; + public static final boolean DEFAULT_PAGE_ROW_COUNT_CHECK_ENABLE = false; + public static final long DEFAULT_SINGLE_ROW_SIZE_LIMIT = 512 * 1024; public static final ValuesWriterFactory DEFAULT_VALUES_WRITER_FACTORY = new DefaultValuesWriterFactory(); @@ -87,10 +89,13 @@ public static WriterVersion fromString(String name) { private final ValuesWriterFactory valuesWriterFactory; private final int columnIndexTruncateLength; private final int pageRowCountLimit; + private final boolean pageSizeCheckEnable; + private final long singleRowSizeLimit; private ParquetProperties(WriterVersion writerVersion, int pageSize, int dictPageSize, boolean enableDict, int minRowCountForPageSizeCheck, int maxRowCountForPageSizeCheck, boolean estimateNextSizeCheck, ByteBufferAllocator allocator, - ValuesWriterFactory writerFactory, int columnIndexMinMaxTruncateLength, int pageRowCountLimit) { + ValuesWriterFactory writerFactory, int columnIndexMinMaxTruncateLength, int pageRowCountLimit, + boolean pageSizeCheckEnable, long singleRowSizeLimit) { this.pageSizeThreshold = pageSize; this.initialSlabSize = CapacityByteArrayOutputStream .initialSlabSizeHeuristic(MIN_SLAB_SIZE, pageSizeThreshold, 10); @@ -105,6 +110,8 @@ private ParquetProperties(WriterVersion writerVersion, int pageSize, int dictPag this.valuesWriterFactory = writerFactory; this.columnIndexTruncateLength = columnIndexMinMaxTruncateLength; this.pageRowCountLimit = pageRowCountLimit; + this.pageSizeCheckEnable = pageSizeCheckEnable; + this.singleRowSizeLimit = singleRowSizeLimit; } public ValuesWriter newRepetitionLevelWriter(ColumnDescriptor path) { @@ -145,6 +152,14 @@ public int getPageSizeThreshold() { return pageSizeThreshold; } + public static boolean isDefaultPageRowCountCheckEnable() { + return DEFAULT_PAGE_ROW_COUNT_CHECK_ENABLE; + } + + public static long getDefaultSingleRowSizeLimit() { + return DEFAULT_SINGLE_ROW_SIZE_LIMIT; + } + public int getInitialSlabSize() { return initialSlabSize; } @@ -181,6 +196,10 @@ public int getMinRowCountForPageSizeCheck() { return minRowCountForPageSizeCheck; } + public boolean isPageSizeCheckEnable() { + return pageSizeCheckEnable; + } + public int getMaxRowCountForPageSizeCheck() { return maxRowCountForPageSizeCheck; } @@ -201,6 +220,10 @@ public int getPageRowCountLimit() { return pageRowCountLimit; } + public long getSingleRowSizeLimit() { + return singleRowSizeLimit; + } + public static Builder builder() { return new Builder(); } @@ -221,6 +244,8 @@ public static class Builder { private ValuesWriterFactory valuesWriterFactory = DEFAULT_VALUES_WRITER_FACTORY; private int columnIndexTruncateLength = DEFAULT_COLUMN_INDEX_TRUNCATE_LENGTH; private int pageRowCountLimit = DEFAULT_PAGE_ROW_COUNT_LIMIT; + private boolean pageSizeCheckEnable = DEFAULT_PAGE_ROW_COUNT_CHECK_ENABLE; + private long singleRowSizeLimit = DEFAULT_SINGLE_ROW_SIZE_LIMIT; private Builder() { } @@ -236,6 +261,7 @@ private Builder(ParquetProperties toCopy) { this.valuesWriterFactory = toCopy.valuesWriterFactory; this.allocator = toCopy.allocator; this.pageRowCountLimit = toCopy.pageRowCountLimit; + this.singleRowSizeLimit = toCopy.singleRowSizeLimit; } /** @@ -330,11 +356,22 @@ public Builder withPageRowCountLimit(int rowCount) { return this; } + public Builder withPageRowCheckEnable(boolean enable) { + pageSizeCheckEnable = enable; + return this; + } + + public Builder withSingleRowSizeLimit(long size) { + singleRowSizeLimit = size; + return this; + } + public ParquetProperties build() { ParquetProperties properties = new ParquetProperties(writerVersion, pageSize, dictPageSize, enableDict, minRowCountForPageSizeCheck, maxRowCountForPageSizeCheck, - estimateNextSizeCheck, allocator, valuesWriterFactory, columnIndexTruncateLength, pageRowCountLimit); + estimateNextSizeCheck, allocator, valuesWriterFactory, columnIndexTruncateLength, + pageRowCountLimit, pageSizeCheckEnable, singleRowSizeLimit); // we pass a constructed but uninitialized factory to ParquetProperties above as currently // creation of ValuesWriters is invoked from within ParquetProperties. In the future // we'd like to decouple that and won't need to pass an object to properties and then pass the diff --git a/parquet-common/pom.xml b/parquet-common/pom.xml index af5b5263bd..6a9b08f0f5 100644 --- a/parquet-common/pom.xml +++ b/parquet-common/pom.xml @@ -21,7 +21,7 @@ org.apache.parquet parquet ../pom.xml - 1.12.0-kylin-r2 + 1.12.0-kylin-r3 4.0.0 diff --git a/parquet-encoding/pom.xml b/parquet-encoding/pom.xml index 7b50bca485..1e4ae29db8 100644 --- a/parquet-encoding/pom.xml +++ b/parquet-encoding/pom.xml @@ -21,7 +21,7 @@ org.apache.parquet parquet ../pom.xml - 1.12.0-kylin-r2 + 1.12.0-kylin-r3 4.0.0 diff --git a/parquet-format-structures/pom.xml b/parquet-format-structures/pom.xml index 4f951dbcf4..562cd68dc9 100644 --- a/parquet-format-structures/pom.xml +++ b/parquet-format-structures/pom.xml @@ -24,7 +24,7 @@ org.apache.parquet parquet ../pom.xml - 1.12.0-kylin-r2 + 1.12.0-kylin-r3 parquet-format-structures diff --git a/parquet-generator/pom.xml b/parquet-generator/pom.xml index bf010e0e97..6eafbc0f99 100644 --- a/parquet-generator/pom.xml +++ b/parquet-generator/pom.xml @@ -21,7 +21,7 @@ org.apache.parquet parquet ../pom.xml - 1.12.0-kylin-r2 + 1.12.0-kylin-r3 4.0.0 diff --git a/parquet-hadoop-bundle/pom.xml b/parquet-hadoop-bundle/pom.xml index 3b231a8f89..8315e5f781 100644 --- a/parquet-hadoop-bundle/pom.xml +++ b/parquet-hadoop-bundle/pom.xml @@ -21,7 +21,7 @@ org.apache.parquet parquet ../pom.xml - 1.12.0-kylin-r2 + 1.12.0-kylin-r3 4.0.0 diff --git a/parquet-hadoop/pom.xml b/parquet-hadoop/pom.xml index eec80f1a8a..2cf44268c4 100644 --- a/parquet-hadoop/pom.xml +++ b/parquet-hadoop/pom.xml @@ -21,7 +21,7 @@ org.apache.parquet parquet ../pom.xml - 1.12.0-kylin-r2 + 1.12.0-kylin-r3 4.0.0 diff --git a/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/InternalParquetRecordWriter.java b/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/InternalParquetRecordWriter.java index d8af379d13..b53d81f68f 100644 --- a/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/InternalParquetRecordWriter.java +++ b/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/InternalParquetRecordWriter.java @@ -18,14 +18,6 @@ */ package org.apache.parquet.hadoop; -import static java.lang.Math.max; -import static java.lang.Math.min; -import static org.apache.parquet.Preconditions.checkNotNull; - -import java.io.IOException; -import java.util.HashMap; -import java.util.Map; - import org.apache.parquet.column.ColumnWriteStore; import org.apache.parquet.column.ParquetProperties; import org.apache.parquet.hadoop.CodecFactory.BytesCompressor; @@ -39,6 +31,14 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import java.io.IOException; +import java.util.HashMap; +import java.util.Map; + +import static java.lang.Math.max; +import static java.lang.Math.min; +import static org.apache.parquet.Preconditions.checkNotNull; + class InternalParquetRecordWriter { private static final Logger LOG = LoggerFactory.getLogger(InternalParquetRecordWriter.class); @@ -66,23 +66,25 @@ class InternalParquetRecordWriter { private ColumnChunkPageWriteStore pageStore; private RecordConsumer recordConsumer; + private boolean pageSizeCheckEnable; + /** * @param parquetFileWriter the file to write to - * @param writeSupport the class to convert incoming records - * @param schema the schema of the records - * @param extraMetaData extra meta data to write in the footer of the file - * @param rowGroupSize the size of a block in the file (this will be approximate) - * @param compressor the codec used to compress + * @param writeSupport the class to convert incoming records + * @param schema the schema of the records + * @param extraMetaData extra meta data to write in the footer of the file + * @param rowGroupSize the size of a block in the file (this will be approximate) + * @param compressor the codec used to compress */ public InternalParquetRecordWriter( - ParquetFileWriter parquetFileWriter, - WriteSupport writeSupport, - MessageType schema, - Map extraMetaData, - long rowGroupSize, - BytesCompressor compressor, - boolean validating, - ParquetProperties props) { + ParquetFileWriter parquetFileWriter, + WriteSupport writeSupport, + MessageType schema, + Map extraMetaData, + long rowGroupSize, + BytesCompressor compressor, + boolean validating, + ParquetProperties props) { this.parquetFileWriter = parquetFileWriter; this.writeSupport = checkNotNull(writeSupport, "writeSupport"); this.schema = schema; @@ -92,6 +94,7 @@ public InternalParquetRecordWriter( this.nextRowGroupSize = rowGroupSizeThreshold; this.compressor = compressor; this.validating = validating; + this.pageSizeCheckEnable = props.isPageSizeCheckEnable(); this.props = props; initStore(); } @@ -102,7 +105,7 @@ public ParquetMetadata getFooter() { private void initStore() { pageStore = new ColumnChunkPageWriteStore(compressor, schema, props.getAllocator(), - props.getColumnIndexTruncateLength()); + props.getColumnIndexTruncateLength()); columnStore = props.newColumnWriteStore(schema, pageStore); MessageColumnIO columnIO = new ColumnIOFactory(validating).getColumnIO(schema); this.recordConsumer = columnIO.getRecordWriter(columnStore); @@ -125,9 +128,14 @@ public void close() throws IOException, InterruptedException { } public void write(T value) throws IOException, InterruptedException { - writeSupport.write(value); - ++ recordCount; - checkBlockSizeReached(); + Long singleRowLimit = props.getSingleRowSizeLimit(); + Long recordSize = writeSupport.writeAndFetchRecordSize(value); + ++recordCount; + if (singleRowLimit <= recordSize) { + checkBlockSizeReachedManual(); + } else { + checkBlockSizeReached(); + } } /** @@ -144,25 +152,38 @@ private void checkBlockSizeReached() throws IOException { // flush the row group if it is within ~2 records of the limit // it is much better to be slightly under size than to be over at all if (memSize > (nextRowGroupSize - 2 * recordSize)) { - LOG.debug("mem size {} > {}: flushing {} records to disk.", memSize, nextRowGroupSize, recordCount); + LOG.info("mem size {} > {}: flushing {} records to disk.", memSize, nextRowGroupSize, recordCount); flushRowGroupToStore(); initStore(); recordCountForNextMemCheck = min(max(MINIMUM_RECORD_COUNT_FOR_CHECK, recordCount / 2), MAXIMUM_RECORD_COUNT_FOR_CHECK); this.lastRowGroupEndPos = parquetFileWriter.getPos(); } else { recordCountForNextMemCheck = min( - max(MINIMUM_RECORD_COUNT_FOR_CHECK, (recordCount + (long)(nextRowGroupSize / ((float)recordSize))) / 2), // will check halfway - recordCount + MAXIMUM_RECORD_COUNT_FOR_CHECK // will not look more than max records ahead - ); - LOG.debug("Checked mem at {} will check again at: {}", recordCount, recordCountForNextMemCheck); + max(MINIMUM_RECORD_COUNT_FOR_CHECK, (recordCount + (long)(nextRowGroupSize / ((float)recordSize))) / 2), // will check halfway + recordCount + MAXIMUM_RECORD_COUNT_FOR_CHECK // will not look more than max records ahead + ); + LOG.info("Checked mem at {} will check again at: {}", recordCount, recordCountForNextMemCheck); } } } + private void checkBlockSizeReachedManual() throws IOException { + long memSize = columnStore.getBufferedSize(); + long recordSize = memSize / recordCount; + if (memSize > (nextRowGroupSize - 2 * recordSize)) { + LOG.info("mem size {} > {}: flushing {} records to disk.", memSize, nextRowGroupSize, recordCount); + flushRowGroupToStore(); + initStore(); + this.lastRowGroupEndPos = parquetFileWriter.getPos(); + } else { + LOG.info("Checked mem at {}", recordCount); + } + } + private void flushRowGroupToStore() - throws IOException { + throws IOException { recordConsumer.flush(); - LOG.debug("Flushing mem columnStore to file. allocated memory: {}", columnStore.getAllocatedSize()); + LOG.info("Flushing mem columnStore to file. allocated memory: {}", columnStore.getAllocatedSize()); if (columnStore.getAllocatedSize() > (3 * rowGroupSizeThreshold)) { LOG.warn("Too much memory used: {}", columnStore.memUsageString()); } @@ -174,8 +195,8 @@ private void flushRowGroupToStore() recordCount = 0; parquetFileWriter.endBlock(); this.nextRowGroupSize = Math.min( - parquetFileWriter.getNextRowGroupSize(), - rowGroupSizeThreshold); + parquetFileWriter.getNextRowGroupSize(), + rowGroupSizeThreshold); } columnStore = null; diff --git a/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/ParquetOutputFormat.java b/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/ParquetOutputFormat.java index 04cbd15c0b..8529eeeecf 100644 --- a/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/ParquetOutputFormat.java +++ b/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/ParquetOutputFormat.java @@ -145,6 +145,8 @@ public static enum JobSummaryLevel { public static final String ESTIMATE_PAGE_SIZE_CHECK = "parquet.page.size.check.estimate"; public static final String COLUMN_INDEX_TRUNCATE_LENGTH = "parquet.columnindex.truncate.length"; public static final String PAGE_ROW_COUNT_LIMIT = "parquet.page.row.count.limit"; + public static final String PAGE_ROW_COUNT_CHECK_ENABLE = "parquet.page.size.customer.check.enable"; + public static final String PAGE_SINGLE_ROW_SIZE_LIMIT = "parquet.single.row.size.limit"; public static JobSummaryLevel getJobSummaryLevel(Configuration conf) { String level = conf.get(JOB_SUMMARY_LEVEL); @@ -338,6 +340,22 @@ private static int getPageRowCountLimit(Configuration conf) { return conf.getInt(PAGE_ROW_COUNT_LIMIT, ParquetProperties.DEFAULT_PAGE_ROW_COUNT_LIMIT); } + public static boolean getPageRowCountCheckEnable(Configuration conf) { + return conf.getBoolean(PAGE_ROW_COUNT_CHECK_ENABLE, ParquetProperties.DEFAULT_PAGE_ROW_COUNT_CHECK_ENABLE); + } + + public static void setPageRowCountCheckEnable(Configuration conf, boolean enable) { + conf.setBoolean(PAGE_ROW_COUNT_CHECK_ENABLE, enable); + } + + public static long getSingleRowSizeLimit(Configuration conf) { + return conf.getLong(PAGE_SINGLE_ROW_SIZE_LIMIT, ParquetProperties.DEFAULT_SINGLE_ROW_SIZE_LIMIT); + } + + public static void setPageSingleRowSizeLimit(Configuration conf, Long maxSingleRowSize) { + conf.setLong(PAGE_SINGLE_ROW_SIZE_LIMIT, maxSingleRowSize); + } + private WriteSupport writeSupport; private ParquetOutputCommitter committer; @@ -394,6 +412,8 @@ public RecordWriter getRecordWriter(Configuration conf, Path file, Comp .withMaxRowCountForPageSizeCheck(getMaxRowCountForPageSizeCheck(conf)) .withColumnIndexTruncateLength(getColumnIndexTruncateLength(conf)) .withPageRowCountLimit(getPageRowCountLimit(conf)) + .withPageRowCheckEnable(getPageRowCountCheckEnable(conf)) + .withPageRowCountLimit(getPageRowCountLimit(conf)) .build(); long blockSize = getLongBlockSize(conf); @@ -404,6 +424,7 @@ public RecordWriter getRecordWriter(Configuration conf, Path file, Comp LOG.info("Parquet block size to {}", blockSize); LOG.info("Parquet page size to {}", props.getPageSizeThreshold()); LOG.info("Parquet dictionary page size to {}", props.getDictionaryPageSizeThreshold()); + LOG.info("Parquet page size customer check is {}", (props.isPageSizeCheckEnable() ? "on" : "off")); LOG.info("Dictionary is {}", (props.isEnableDictionary() ? "on" : "off")); LOG.info("Validation is {}", (validating ? "on" : "off")); LOG.info("Writer version is: {}", props.getWriterVersion()); @@ -413,6 +434,7 @@ public RecordWriter getRecordWriter(Configuration conf, Path file, Comp LOG.info("Max row count for page size check is: {}", props.getMaxRowCountForPageSizeCheck()); LOG.info("Truncate length for column indexes is: {}", props.getColumnIndexTruncateLength()); LOG.info("Page row count limit to {}", props.getPageRowCountLimit()); + LOG.info("Single row size limit to {}", props.getSingleRowSizeLimit()); } WriteContext init = writeSupport.init(conf); diff --git a/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/ParquetWriter.java b/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/ParquetWriter.java index 1ed5e32ca7..e7c5eb5f96 100644 --- a/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/ParquetWriter.java +++ b/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/ParquetWriter.java @@ -232,6 +232,7 @@ public ParquetWriter( .withDictionaryPageSize(dictionaryPageSize) .withDictionaryEncoding(enableDictionary) .withWriterVersion(writerVersion) + .withMinRowCountForPageSizeCheck(Integer.valueOf(conf.get(ParquetOutputFormat.MIN_ROW_COUNT_FOR_PAGE_SIZE_CHECK, "100"))) .build()); } diff --git a/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/api/WriteSupport.java b/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/api/WriteSupport.java index c08882f8ca..b4ba6be165 100644 --- a/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/api/WriteSupport.java +++ b/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/api/WriteSupport.java @@ -112,6 +112,15 @@ public Map getExtraMetaData() { */ public abstract void write(T record); + /** + * called once per record + * @param record one record to write to the previously provided record consumer + */ + public long writeAndFetchRecordSize(T record) { + write(record); + return Long.MIN_VALUE; + }; + /** * Called to get a name to identify the WriteSupport object model. * If not null, this is added to the file footer metadata. diff --git a/parquet-hive-bundle/pom.xml b/parquet-hive-bundle/pom.xml index d11863567e..c9ccdfdfa8 100644 --- a/parquet-hive-bundle/pom.xml +++ b/parquet-hive-bundle/pom.xml @@ -21,7 +21,7 @@ org.apache.parquet parquet ../pom.xml - 1.12.0-kylin-r2 + 1.12.0-kylin-r3 4.0.0 diff --git a/parquet-hive/parquet-hive-binding/parquet-hive-0.10-binding/pom.xml b/parquet-hive/parquet-hive-binding/parquet-hive-0.10-binding/pom.xml index bb20ea55b4..302b6d5227 100644 --- a/parquet-hive/parquet-hive-binding/parquet-hive-0.10-binding/pom.xml +++ b/parquet-hive/parquet-hive-binding/parquet-hive-0.10-binding/pom.xml @@ -21,7 +21,7 @@ org.apache.parquet parquet-hive-binding ../pom.xml - 1.12.0-kylin-r2 + 1.12.0-kylin-r3 4.0.0 diff --git a/parquet-hive/parquet-hive-binding/parquet-hive-0.12-binding/pom.xml b/parquet-hive/parquet-hive-binding/parquet-hive-0.12-binding/pom.xml index 93bb709ff8..2c958be476 100644 --- a/parquet-hive/parquet-hive-binding/parquet-hive-0.12-binding/pom.xml +++ b/parquet-hive/parquet-hive-binding/parquet-hive-0.12-binding/pom.xml @@ -21,7 +21,7 @@ org.apache.parquet parquet-hive-binding ../pom.xml - 1.12.0-kylin-r2 + 1.12.0-kylin-r3 4.0.0 diff --git a/parquet-hive/parquet-hive-binding/parquet-hive-binding-bundle/pom.xml b/parquet-hive/parquet-hive-binding/parquet-hive-binding-bundle/pom.xml index 693481f364..784cd127c3 100644 --- a/parquet-hive/parquet-hive-binding/parquet-hive-binding-bundle/pom.xml +++ b/parquet-hive/parquet-hive-binding/parquet-hive-binding-bundle/pom.xml @@ -23,7 +23,7 @@ org.apache.parquet parquet-hive-binding ../pom.xml - 1.12.0-kylin-r2 + 1.12.0-kylin-r3 4.0.0 diff --git a/parquet-hive/parquet-hive-binding/parquet-hive-binding-factory/pom.xml b/parquet-hive/parquet-hive-binding/parquet-hive-binding-factory/pom.xml index 99d7cb0771..60b78ebc76 100644 --- a/parquet-hive/parquet-hive-binding/parquet-hive-binding-factory/pom.xml +++ b/parquet-hive/parquet-hive-binding/parquet-hive-binding-factory/pom.xml @@ -21,7 +21,7 @@ org.apache.parquet parquet-hive-binding ../pom.xml - 1.12.0-kylin-r2 + 1.12.0-kylin-r3 4.0.0 diff --git a/parquet-hive/parquet-hive-binding/parquet-hive-binding-interface/pom.xml b/parquet-hive/parquet-hive-binding/parquet-hive-binding-interface/pom.xml index 35725e2f92..1b02e635a3 100644 --- a/parquet-hive/parquet-hive-binding/parquet-hive-binding-interface/pom.xml +++ b/parquet-hive/parquet-hive-binding/parquet-hive-binding-interface/pom.xml @@ -21,7 +21,7 @@ org.apache.parquet parquet-hive-binding ../pom.xml - 1.12.0-kylin-r2 + 1.12.0-kylin-r3 4.0.0 diff --git a/parquet-hive/parquet-hive-binding/pom.xml b/parquet-hive/parquet-hive-binding/pom.xml index c90d1098b1..eb04d1b1a6 100644 --- a/parquet-hive/parquet-hive-binding/pom.xml +++ b/parquet-hive/parquet-hive-binding/pom.xml @@ -21,7 +21,7 @@ org.apache.parquet parquet-hive ../pom.xml - 1.12.0-kylin-r2 + 1.12.0-kylin-r3 4.0.0 diff --git a/parquet-hive/parquet-hive-storage-handler/pom.xml b/parquet-hive/parquet-hive-storage-handler/pom.xml index 66b77e1b22..254cb5daf0 100644 --- a/parquet-hive/parquet-hive-storage-handler/pom.xml +++ b/parquet-hive/parquet-hive-storage-handler/pom.xml @@ -21,7 +21,7 @@ org.apache.parquet parquet-hive ../pom.xml - 1.12.0-kylin-r2 + 1.12.0-kylin-r3 4.0.0 diff --git a/parquet-hive/pom.xml b/parquet-hive/pom.xml index 6f6d381a44..f7ffbeaed8 100644 --- a/parquet-hive/pom.xml +++ b/parquet-hive/pom.xml @@ -21,7 +21,7 @@ org.apache.parquet parquet ../pom.xml - 1.12.0-kylin-r2 + 1.12.0-kylin-r3 4.0.0 diff --git a/parquet-jackson/pom.xml b/parquet-jackson/pom.xml index 44ede63723..b02bdaf10a 100644 --- a/parquet-jackson/pom.xml +++ b/parquet-jackson/pom.xml @@ -21,7 +21,7 @@ org.apache.parquet parquet ../pom.xml - 1.12.0-kylin-r2 + 1.12.0-kylin-r3 4.0.0 diff --git a/parquet-pig-bundle/pom.xml b/parquet-pig-bundle/pom.xml index 48bffe8ca2..55db953fa7 100644 --- a/parquet-pig-bundle/pom.xml +++ b/parquet-pig-bundle/pom.xml @@ -21,7 +21,7 @@ org.apache.parquet parquet ../pom.xml - 1.12.0-kylin-r2 + 1.12.0-kylin-r3 4.0.0 diff --git a/parquet-pig/pom.xml b/parquet-pig/pom.xml index 897527ed83..f6859994d7 100644 --- a/parquet-pig/pom.xml +++ b/parquet-pig/pom.xml @@ -21,7 +21,7 @@ org.apache.parquet parquet ../pom.xml - 1.12.0-kylin-r2 + 1.12.0-kylin-r3 4.0.0 diff --git a/parquet-protobuf/pom.xml b/parquet-protobuf/pom.xml index 82a2f0fc70..4d4b26dd7b 100644 --- a/parquet-protobuf/pom.xml +++ b/parquet-protobuf/pom.xml @@ -21,7 +21,7 @@ org.apache.parquet parquet ../pom.xml - 1.12.0-kylin-r2 + 1.12.0-kylin-r3 4.0.0 diff --git a/parquet-scala/pom.xml b/parquet-scala/pom.xml index 344bf475a2..418e19ee2b 100644 --- a/parquet-scala/pom.xml +++ b/parquet-scala/pom.xml @@ -21,7 +21,7 @@ org.apache.parquet parquet ../pom.xml - 1.12.0-kylin-r2 + 1.12.0-kylin-r3 4.0.0 diff --git a/parquet-scrooge/pom.xml b/parquet-scrooge/pom.xml index 887be5c424..66665775e9 100644 --- a/parquet-scrooge/pom.xml +++ b/parquet-scrooge/pom.xml @@ -21,7 +21,7 @@ org.apache.parquet parquet ../pom.xml - 1.12.0-kylin-r2 + 1.12.0-kylin-r3 4.0.0 diff --git a/parquet-thrift/pom.xml b/parquet-thrift/pom.xml index ae78bedba1..85b036c968 100644 --- a/parquet-thrift/pom.xml +++ b/parquet-thrift/pom.xml @@ -21,7 +21,7 @@ org.apache.parquet parquet ../pom.xml - 1.12.0-kylin-r2 + 1.12.0-kylin-r3 4.0.0 diff --git a/parquet-tools/pom.xml b/parquet-tools/pom.xml index 542c4aaa37..2762beb235 100644 --- a/parquet-tools/pom.xml +++ b/parquet-tools/pom.xml @@ -21,7 +21,7 @@ org.apache.parquet parquet ../pom.xml - 1.12.0-kylin-r2 + 1.12.0-kylin-r3 4.0.0 diff --git a/pom.xml b/pom.xml index 24ed4b5a0c..7c061dcb2f 100644 --- a/pom.xml +++ b/pom.xml @@ -9,7 +9,7 @@ org.apache.parquet parquet - 1.12.0-kylin-r2 + 1.12.0-kylin-r3 pom Apache Parquet MR