[parquet-cpp] branch master updated: PARQUET-1177: Add PARQUET_BUILD_WARNING_LEVEL option and more rigorous Clang warnings

2017-12-13 Thread wesm
This is an automated email from the ASF dual-hosted git repository.

wesm pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/parquet-cpp.git


The following commit(s) were added to refs/heads/master by this push:
 new 6ab16f3  PARQUET-1177: Add PARQUET_BUILD_WARNING_LEVEL option and more 
rigorous Clang warnings
6ab16f3 is described below

commit 6ab16f3ae8e4a76ea28a704d88267bb342ba407b
Author: Wes McKinney 
AuthorDate: Wed Dec 13 17:19:22 2017 -0500

PARQUET-1177: Add PARQUET_BUILD_WARNING_LEVEL option and more rigorous 
Clang warnings

These warnings will catch a number of things that have bitten us in the 
past, like missing virtual destructors. This brings Parquet's compiler warnings 
up to the same quality as Arrow's

Author: Wes McKinney 
Author: Wes McKinney 

Closes #425 from wesm/PARQUET-1177 and squashes the following commits:

3769a8c [Wes McKinney] Add -Wno-missing-noreturn
5b6cd80 [Wes McKinney] Compile with /bigobj in MSVC
cc5bca0 [Wes McKinney] Add noreturn to static methods in ParquetException
e3ffb71 [Wes McKinney] Fix -Wconversion warnings in decode_benchmark.cc
758a216 [Wes McKinney] Fix warnings on macOS Clang
3aef3b4 [Wes McKinney] Do not pass -Werror via PARQUET_CXXFLAGS
5a98e81 [Wes McKinney] Fix usage of PrimitiveArray::raw_values
c848855 [Wes McKinney] Fix compiler warnings with gcc 4.9
ca9a374 [Wes McKinney] Add SetupCxxFlags.cmake from Apache Arrow. Add 
PARQUET_BUILD_WARNING_LEVEL flag. Fix Clang compiler warnings
---
 .travis.yml   |   9 +-
 CMakeLists.txt|  84 ++---
 benchmarks/decode_benchmark.cc|  58 ---
 ci/before_script_travis.sh|   2 +
 cmake_modules/CompilerInfo.cmake  |   2 +-
 cmake_modules/SetupCxxFlags.cmake | 241 ++
 examples/reader-writer.cc |  16 +-
 src/parquet/arrow/arrow-reader-writer-test.cc |   9 +-
 src/parquet/arrow/arrow-schema-test.cc|   4 +-
 src/parquet/arrow/reader.cc   |  10 +-
 src/parquet/arrow/record_reader.cc|   4 +-
 src/parquet/arrow/schema.cc   |   5 +-
 src/parquet/arrow/writer.cc   |  43 ++---
 src/parquet/column-io-benchmark.cc|  19 +-
 src/parquet/column_reader.cc  |   2 +-
 src/parquet/column_writer-test.cc |  16 +-
 src/parquet/column_writer.cc  |   2 +-
 src/parquet/column_writer.h   |   2 +
 src/parquet/encoding-benchmark.cc |  28 +--
 src/parquet/encoding-internal.h   |  12 +-
 src/parquet/encoding-test.cc  |   2 +-
 src/parquet/exception.cc  |  10 +-
 src/parquet/exception.h   |   7 +-
 src/parquet/file_reader.cc|  18 +-
 src/parquet/file_writer.h |   2 +-
 src/parquet/public-api-test.cc|   4 +-
 src/parquet/statistics-test.cc|   8 +-
 src/parquet/test-specialization.h |   2 +-
 src/parquet/types.cc  |  25 ---
 src/parquet/util/macros.h |  17 ++
 src/parquet/util/memory-test.cc   |   2 +-
 src/parquet/util/memory.h |   6 +-
 src/parquet/util/test-common.h|   6 +-
 tools/parquet-scan.cc |   3 +-
 34 files changed, 440 insertions(+), 240 deletions(-)

diff --git a/.travis.yml b/.travis.yml
index ae24cfe..7918b89 100644
--- a/.travis.yml
+++ b/.travis.yml
@@ -41,24 +41,23 @@ matrix:
   - compiler: gcc
 os: linux
 before_script:
-- export PARQUET_CXXFLAGS="-Werror -DARROW_NO_DEPRECATED_API"
+- export PARQUET_CXXFLAGS="-DARROW_NO_DEPRECATED_API"
 - source $TRAVIS_BUILD_DIR/ci/before_script_travis.sh
   - compiler: gcc
 os: linux
 before_script:
-- export PARQUET_CXXFLAGS="-Werror"
 - source $TRAVIS_BUILD_DIR/ci/before_script_travis.sh
   - compiler: clang
 os: linux
 before_script:
-- export PARQUET_CXXFLAGS="-Werror -DARROW_NO_DEPRECATED_API"
+- export PARQUET_CXXFLAGS="-DARROW_NO_DEPRECATED_API"
 - source $TRAVIS_BUILD_DIR/ci/before_script_travis.sh
   - compiler: clang
 os: osx
 osx_image: xcode6.4
 addons:
 before_script:
-- export PARQUET_CXXFLAGS="-Werror -DARROW_NO_DEPRECATED_API"
+- export PARQUET_CXXFLAGS="-DARROW_NO_DEPRECATED_API"
 - source $TRAVIS_BUILD_DIR/ci/before_script_travis.sh
 before_install:
 - mkdir $TRAVIS_BUILD_DIR/parquet-build
@@ -68,7 +67,7 @@ matrix:
 env: PARQUET_BUILD_GROUP=toolchain
 before_script:
 script:
-- export PARQUET_CXXFLAGS="-Werror -DARROW_NO_DEPRECATED_API"
+- export PARQUET_CXXFLAGS="-DARROW_NO_DEPRECATED_API"
 - 

[4/4] parquet-mr git commit: PARQUET-1142: Add alternatives to Hadoop classes in the API

2017-12-13 Thread blue
PARQUET-1142: Add alternatives to Hadoop classes in the API

This updates the read and write paths to avoid using Hadoop classes where 
possible.

* Adds a generic compression interface, `CompressionCodecFactory`
* Adds `OutputFile` and `PositionOutputStream`
* Adds classes to help implementations wrap input and output streams: 
`DelegatingSeekableInputStream` and `DelegatingPositionOutputStream`
* Adds `ParquetReadOptions` to avoid passing options with `Configuration`
* Updates the read and write APIs to use new abstractions instead of Hadoop

Author: Ryan Blue 

Closes #429 from rdblue/PARQUET-1142-add-hadoop-alternatives and squashes the 
following commits:

21500337b [Ryan Blue] PARQUET-1142: Fix NPE when not filtering with new read 
API.
35eddd735 [Ryan Blue] PARQUET-1142: Fix problems from Gabor's review.
da391b0d4 [Ryan Blue] PARQUET-1142: Fix binary incompatibilities.
2e3d693ab [Ryan Blue] PARQUET-1142: Update the read and write paths to use new 
files and streams.
8d57e089f [Ryan Blue] PARQUET-1142: Add OutputFile and PositionOutputStream.
42908a95e [Ryan Blue] PARQUET-1142: Extract non-Hadoop API from CodecFactory.


Project: http://git-wip-us.apache.org/repos/asf/parquet-mr/repo
Commit: http://git-wip-us.apache.org/repos/asf/parquet-mr/commit/8bfd9b4d
Tree: http://git-wip-us.apache.org/repos/asf/parquet-mr/tree/8bfd9b4d
Diff: http://git-wip-us.apache.org/repos/asf/parquet-mr/diff/8bfd9b4d

Branch: refs/heads/master
Commit: 8bfd9b4d8f4fb0a2b522c9328f67eb642066306b
Parents: 81f4801
Author: Ryan Blue 
Authored: Wed Dec 13 11:27:54 2017 -0800
Committer: Ryan Blue 
Committed: Wed Dec 13 11:27:54 2017 -0800

--
 parquet-common/pom.xml  |   6 +
 .../org/apache/parquet/bytes/BytesInput.java| 486 +++
 .../bytes/CapacityByteArrayOutputStream.java| 337 
 .../bytes/ConcatenatingByteArrayCollector.java  |  63 ++
 .../bytes/LittleEndianDataInputStream.java  | 424 +
 .../bytes/LittleEndianDataOutputStream.java | 220 +
 .../compression/CompressionCodecFactory.java|  47 +
 .../CompressionCodecNotSupportedException.java  |  38 +
 .../hadoop/metadata/CompressionCodecName.java   |  98 +++
 .../io/DelegatingPositionOutputStream.java  |  63 ++
 .../io/DelegatingSeekableInputStream.java   | 171 
 .../java/org/apache/parquet/io/InputFile.java   |   9 +-
 .../java/org/apache/parquet/io/OutputFile.java  |  34 +
 .../apache/parquet/io/PositionOutputStream.java |  39 +
 .../org/apache/parquet/io/MockInputStream.java  |  56 ++
 .../io/TestDelegatingSeekableInputStream.java   | 861 +++
 .../org/apache/parquet/bytes/BytesInput.java| 486 ---
 .../bytes/CapacityByteArrayOutputStream.java| 337 
 .../bytes/ConcatenatingByteArrayCollector.java  |  63 --
 .../bytes/LittleEndianDataInputStream.java  | 424 -
 .../bytes/LittleEndianDataOutputStream.java | 220 -
 .../org/apache/parquet/HadoopReadOptions.java   |  98 +++
 .../org/apache/parquet/ParquetReadOptions.java  | 232 +
 .../parquet/filter2/compat/RowGroupFilter.java  |   4 +
 .../converter/ParquetMetadataConverter.java |  22 +-
 .../org/apache/parquet/hadoop/CodecFactory.java |  26 +-
 .../hadoop/ColumnChunkPageReadStore.java|   6 +-
 .../parquet/hadoop/DirectCodecFactory.java  |  12 +-
 .../hadoop/InternalParquetRecordReader.java |  34 +-
 .../parquet/hadoop/ParquetFileReader.java   | 254 +++---
 .../parquet/hadoop/ParquetFileWriter.java   | 147 ++--
 .../parquet/hadoop/ParquetInputFormat.java  |   3 -
 .../parquet/hadoop/ParquetOutputFormat.java |   5 +-
 .../apache/parquet/hadoop/ParquetReader.java| 174 +++-
 .../parquet/hadoop/ParquetRecordReader.java |  26 +-
 .../apache/parquet/hadoop/ParquetWriter.java|  50 +-
 .../hadoop/UnmaterializableRecordCounter.java   |  15 +
 .../CompressionCodecNotSupportedException.java  |  36 -
 .../hadoop/metadata/CompressionCodecName.java   |  98 ---
 .../hadoop/util/H1SeekableInputStream.java  | 101 +--
 .../hadoop/util/H2SeekableInputStream.java  |  20 +-
 .../parquet/hadoop/util/HadoopCodecs.java   |  39 +
 .../parquet/hadoop/util/HadoopOutputFile.java   | 100 +++
 .../hadoop/util/HadoopPositionOutputStream.java |  66 ++
 .../parquet/hadoop/util/HadoopStreams.java  |  15 +
 .../TestInputOutputFormatWithPadding.java   |   6 +-
 .../parquet/hadoop/TestParquetFileWriter.java   |   1 +
 .../hadoop/util/MockHadoopInputStream.java  |  87 ++
 .../parquet/hadoop/util/MockInputStream.java|  87 --
 .../hadoop/util/TestHadoop1ByteBufferReads.java | 761 
 .../hadoop/util/TestHadoop2ByteBufferReads.java |  30 +-
 .../parquet/tools/command/MergeCommand.java |   3 +-
 pom.xml |   9 +-
 53 files changed, 4158 insertions(+), 2891 deletions(-)

[2/4] parquet-mr git commit: PARQUET-1142: Add alternatives to Hadoop classes in the API

2017-12-13 Thread blue
http://git-wip-us.apache.org/repos/asf/parquet-mr/blob/8bfd9b4d/parquet-hadoop/src/main/java/org/apache/parquet/HadoopReadOptions.java
--
diff --git 
a/parquet-hadoop/src/main/java/org/apache/parquet/HadoopReadOptions.java 
b/parquet-hadoop/src/main/java/org/apache/parquet/HadoopReadOptions.java
new file mode 100644
index 000..87c8ac9
--- /dev/null
+++ b/parquet-hadoop/src/main/java/org/apache/parquet/HadoopReadOptions.java
@@ -0,0 +1,98 @@
+/*
+ *  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.parquet;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.parquet.bytes.ByteBufferAllocator;
+import org.apache.parquet.compression.CompressionCodecFactory;
+import org.apache.parquet.filter2.compat.FilterCompat;
+import 
org.apache.parquet.format.converter.ParquetMetadataConverter.MetadataFilter;
+import org.apache.parquet.hadoop.util.HadoopCodecs;
+
+import java.util.Map;
+
+import static 
org.apache.parquet.hadoop.ParquetInputFormat.DICTIONARY_FILTERING_ENABLED;
+import static 
org.apache.parquet.hadoop.ParquetInputFormat.RECORD_FILTERING_ENABLED;
+import static 
org.apache.parquet.hadoop.ParquetInputFormat.STATS_FILTERING_ENABLED;
+import static org.apache.parquet.hadoop.ParquetInputFormat.getFilter;
+import static 
org.apache.parquet.hadoop.UnmaterializableRecordCounter.BAD_RECORD_THRESHOLD_CONF_KEY;
+
+public class HadoopReadOptions extends ParquetReadOptions {
+  private final Configuration conf;
+
+  private HadoopReadOptions(boolean useSignedStringMinMax,
+boolean useStatsFilter,
+boolean useDictionaryFilter,
+boolean useRecordFilter,
+FilterCompat.Filter recordFilter,
+MetadataFilter metadataFilter,
+CompressionCodecFactory codecFactory,
+ByteBufferAllocator allocator,
+Map properties,
+Configuration conf) {
+super(
+useSignedStringMinMax, useStatsFilter, useDictionaryFilter, 
useRecordFilter, recordFilter,
+metadataFilter, codecFactory, allocator, properties
+);
+this.conf = conf;
+  }
+
+  @Override
+  public String getProperty(String property) {
+String value = super.getProperty(property);
+if (value != null) {
+  return value;
+}
+return conf.get(property);
+  }
+
+  public Configuration getConf() {
+return conf;
+  }
+
+  public static Builder builder(Configuration conf) {
+return new Builder(conf);
+  }
+
+  public static class Builder extends ParquetReadOptions.Builder {
+private final Configuration conf;
+
+public Builder(Configuration conf) {
+  this.conf = conf;
+  
useSignedStringMinMax(conf.getBoolean("parquet.strings.signed-min-max.enabled", 
false));
+  useDictionaryFilter(conf.getBoolean(STATS_FILTERING_ENABLED, true));
+  useStatsFilter(conf.getBoolean(DICTIONARY_FILTERING_ENABLED, true));
+  useRecordFilter(conf.getBoolean(RECORD_FILTERING_ENABLED, true));
+  withCodecFactory(HadoopCodecs.newFactory(conf, 0));
+  withRecordFilter(getFilter(conf));
+  String badRecordThresh = conf.get(BAD_RECORD_THRESHOLD_CONF_KEY);
+  if (badRecordThresh != null) {
+set(BAD_RECORD_THRESHOLD_CONF_KEY, badRecordThresh);
+  }
+}
+
+@Override
+public ParquetReadOptions build() {
+  return new HadoopReadOptions(
+  useSignedStringMinMax, useStatsFilter, useDictionaryFilter, 
useRecordFilter,
+  recordFilter, metadataFilter, codecFactory, allocator, properties, 
conf);
+}
+  }
+}

http://git-wip-us.apache.org/repos/asf/parquet-mr/blob/8bfd9b4d/parquet-hadoop/src/main/java/org/apache/parquet/ParquetReadOptions.java
--
diff --git 
a/parquet-hadoop/src/main/java/org/apache/parquet/ParquetReadOptions.java 
b/parquet-hadoop/src/main/java/org/apache/parquet/ParquetReadOptions.java
new file mode 100644
index 000..5f2f0a8
--- /dev/null
+++ 

[1/4] parquet-mr git commit: PARQUET-1142: Add alternatives to Hadoop classes in the API

2017-12-13 Thread blue
Repository: parquet-mr
Updated Branches:
  refs/heads/master 81f480149 -> 8bfd9b4d8


http://git-wip-us.apache.org/repos/asf/parquet-mr/blob/8bfd9b4d/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/ParquetWriter.java
--
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 9512b93..bdde70e 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
@@ -28,6 +28,8 @@ import org.apache.parquet.column.ParquetProperties;
 import org.apache.parquet.column.ParquetProperties.WriterVersion;
 import org.apache.parquet.hadoop.api.WriteSupport;
 import org.apache.parquet.hadoop.metadata.CompressionCodecName;
+import org.apache.parquet.hadoop.util.HadoopOutputFile;
+import org.apache.parquet.io.OutputFile;
 import org.apache.parquet.schema.MessageType;
 
 /**
@@ -219,7 +221,8 @@ public class ParquetWriter implements Closeable {
   boolean validating,
   WriterVersion writerVersion,
   Configuration conf) throws IOException {
-this(file, mode, writeSupport, compressionCodecName, blockSize,
+this(HadoopOutputFile.fromPath(file, conf),
+mode, writeSupport, compressionCodecName, blockSize,
 validating, conf, MAX_PADDING_SIZE_DEFAULT,
 ParquetProperties.builder()
 .withPageSize(pageSize)
@@ -257,11 +260,11 @@ public class ParquetWriter implements Closeable {
   }
 
   ParquetWriter(
-  Path file,
+  OutputFile file,
   ParquetFileWriter.Mode mode,
   WriteSupport writeSupport,
   CompressionCodecName compressionCodecName,
-  int blockSize,
+  int rowGroupSize,
   boolean validating,
   Configuration conf,
   int maxPaddingSize,
@@ -271,7 +274,7 @@ public class ParquetWriter implements Closeable {
 MessageType schema = writeContext.getSchema();
 
 ParquetFileWriter fileWriter = new ParquetFileWriter(
-conf, schema, file, mode, blockSize, maxPaddingSize);
+file, schema, mode, rowGroupSize, maxPaddingSize);
 fileWriter.start();
 
 this.codecFactory = new CodecFactory(conf, 
encodingProps.getPageSizeThreshold());
@@ -281,7 +284,7 @@ public class ParquetWriter implements Closeable {
 writeSupport,
 schema,
 writeContext.getExtraMetaData(),
-blockSize,
+rowGroupSize,
 compressor,
 validating,
 encodingProps);
@@ -324,7 +327,8 @@ public class ParquetWriter implements Closeable {
* @param  The type of this builder that is returned by builder methods
*/
   public abstract static class Builder> {
-private final Path file;
+private OutputFile file = null;
+private Path path = null;
 private Configuration conf = new Configuration();
 private ParquetFileWriter.Mode mode;
 private CompressionCodecName codecName = DEFAULT_COMPRESSION_CODEC_NAME;
@@ -334,8 +338,12 @@ public class ParquetWriter implements Closeable {
 private ParquetProperties.Builder encodingPropsBuilder =
 ParquetProperties.builder();
 
-protected Builder(Path file) {
-  this.file = file;
+protected Builder(Path path) {
+  this.path = path;
+}
+
+protected Builder(OutputFile path) {
+  this.file = path;
 }
 
 /**
@@ -485,15 +493,35 @@ public class ParquetWriter implements Closeable {
 }
 
 /**
+ * Set a property that will be available to the read path. For writers 
that use a Hadoop
+ * configuration, this is the recommended way to add configuration values.
+ *
+ * @param property a String property name
+ * @param value a String property value
+ * @return this builder for method chaining.
+ */
+public SELF config(String property, String value) {
+  conf.set(property, value);
+  return self();
+}
+
+/**
  * Build a {@link ParquetWriter} with the accumulated configuration.
  *
  * @return a configured {@code ParquetWriter} instance.
  * @throws IOException
  */
 public ParquetWriter build() throws IOException {
-  return new ParquetWriter(file, mode, getWriteSupport(conf), codecName,
-  rowGroupSize, enableValidation, conf, maxPaddingSize,
-  encodingPropsBuilder.build());
+  if (file != null) {
+return new ParquetWriter<>(file,
+mode, getWriteSupport(conf), codecName, rowGroupSize, 
enableValidation, conf,
+maxPaddingSize, encodingPropsBuilder.build());
+  } else {
+return new ParquetWriter<>(HadoopOutputFile.fromPath(path, conf),
+mode, getWriteSupport(conf), codecName,
+rowGroupSize, enableValidation, conf, maxPaddingSize,
+encodingPropsBuilder.build());
+  }
 }
   }
 }