[jira] [Commented] (PARQUET-1380) move Bloom filter test binary to parquet-testing repo
[ https://issues.apache.org/jira/browse/PARQUET-1380?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16581758#comment-16581758 ] Junjie Chen commented on PARQUET-1380: -- Hi [~wesmckinn], I created this to track following thing after moving binary. Do you want a new PR against parquet-testing or you will help copy binary and organize directory? > move Bloom filter test binary to parquet-testing repo > - > > Key: PARQUET-1380 > URL: https://issues.apache.org/jira/browse/PARQUET-1380 > Project: Parquet > Issue Type: Sub-task > Components: parquet-cpp >Reporter: Junjie Chen >Assignee: Junjie Chen >Priority: Minor > -- This message was sent by Atlassian JIRA (v7.6.3#76005)
[jira] [Created] (PARQUET-1381) Add merge blocks command to parquet-tools
Ekaterina Galieva created PARQUET-1381: -- Summary: Add merge blocks command to parquet-tools Key: PARQUET-1381 URL: https://issues.apache.org/jira/browse/PARQUET-1381 Project: Parquet Issue Type: New Feature Components: parquet-mr Affects Versions: 1.10.0 Reporter: Ekaterina Galieva Fix For: 1.10.1 Current implementation of merge command in parquet-tools doesn't merge row groups, just places one after the other. Add API and command option to be able to merge small blocks into larger ones up to specified size limit. h6. Implementation details: Blocks are not reordered not to break possible initial predicate pushdown optimizations. Blocks are not divided to fit upper bound perfectly. This is an intentional performance optimization. This gives an opportunity to form new blocks by coping full content of smaller blocks by column, not by row. h6. Examples: # Input files with blocks sizes: {code:java} [128 | 35], [128 | 40], [120]{code} Expected output file blocks sizes: {{merge }} {code:java} [128 | 35 | 128 | 40 | 120] {code} {{merge -b}} {code:java} [128 | 35 | 128 | 40 | 120] {code} {{merge -b -l 256 }} {code:java} [163 | 168 | 120] {code} # Input files with blocks sizes: {code:java} [128 | 35], [40], [120], [6] {code} Expected output file blocks sizes: {{merge}} {code:java} [128 | 35 | 40 | 120 | 6] {code} {{merge -b}} {code:java} [128 | 75 | 126] {code} {{merge -b -l 256}} {code:java} [203 | 126]{code} -- This message was sent by Atlassian JIRA (v7.6.3#76005)
[VOTE] Moving Apache Parquet C++ development process to a monorepo structure with Apache Arrow C++
hi all, As discussed on the mailing list [1] I am proposing to undertake a restructuring of the development process for parquet-cpp and its consumption in the Arrow ecosystem to benefit the developers and users of both communities. The specific actions we would take would be: 1) Move the source code currently located at src/ in the apache/parquet-cpp repository [2] to the cpp/src/ directory located in apache/arrow [3] 2) The parquet code tree would remain separate from the Arrow code tree, though the two projects will continue to share code as they do now 3) The build system in apache/parquet-cpp would be effectively deprecated and can be mostly discarded, as it is largely redundant and duplicated from the build system in apache/arrow 4) The Parquet and Arrow C++ communities will collaborate to provide development workflows to enable contributors working exclusively on the Parquet core functionality to be able to work unencumbered with unnecessary build or test dependencies from the rest of the Arrow codebase. Note that parquet-cpp already builds a significant portion of Apache Arrow en route to creating its libraries 5) The Parquet community can create scripts to "cut" Parquet C++ releases by packaging up the appropriate components and ensuring that they can be built and installed independently as now 6) The CI processes would be merged -- since we already build the Parquet libraries in Arrow's CI workflow, this would amount to building the Parquet unit tests and running them. 7) Patches contributed that do not involve Arrow-related functionality could use the PARQUET- marking, though some ARROW- patches may span both codebases 8) Parquet C++ committers can be given push rights on apache/arrow subject to ongoing good citizenry (e.g. not merging patches that break builds). The Arrow PMC may need to vote on the procedure for offering pass-through commit rights to anyone who has been invited to be a committer for Apache Parquet 9) The contributors who work on both Arrow and Parquet will work in good faith to ensure that that needs of Parquet-only developers (i.e. who consume Parquet files in some way unrelated to the Arrow columnar standard) are accommodated There are a number of particular details we will need to discuss further (such as the specific logistics of the codebase surgery; e.g. how to manage the commit history in apache/parquet-cpp -- do we care about git blame?) This vote is to determine if the Parquet PMC is in favor of working in good faith to execute on the above plan. I will inquire with the Arrow PMC to see if we need to have a corresponding vote there, and also how to handle the management of commit rights. [ ] +1: In favor of implementing the proposed monorepo plan [ ] +0: . . . [ ] -1: Not in favor because . . . Here is my vote: +1. Thank you, Wes [1]: https://lists.apache.org/thread.html/4bc135b4e933b959602df48bc3d5978ab7a4299d83d4295da9f498ac@%3Cdev.parquet.apache.org%3E [2]: https://github.com/apache/parquet-cpp/tree/master/src/parquet [3]: https://github.com/apache/arrow/tree/master/cpp/src
[jira] [Commented] (PARQUET-1148) [C++] Code coverage has been broken since June 23
[ https://issues.apache.org/jira/browse/PARQUET-1148?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16581398#comment-16581398 ] ASF GitHub Bot commented on PARQUET-1148: - wesm closed pull request #415: PARQUET-1148: Revert coverage changes from PARQUET-1038 URL: https://github.com/apache/parquet-cpp/pull/415 This is a PR merged from a forked repository. As GitHub hides the original diff on merge, it is displayed below for the sake of provenance: As this is a foreign pull request (from a fork), the diff is supplied below (as it won't show otherwise due to GitHub magic): diff --git a/ci/upload_coverage.sh b/ci/upload_coverage.sh index 73b37c26..81e7565b 100755 --- a/ci/upload_coverage.sh +++ b/ci/upload_coverage.sh @@ -26,19 +26,11 @@ ls -l echo $PARQUET_ROOT -coveralls --gcov $(which gcov-4.9) \ ---gcov-options '\-l' --root '' \ ---include $PARQUET_ROOT \ ---exclude $PARQUET_ROOT/parquet-build/thirdparty \ ---exclude $PARQUET_ROOT/parquet-build/arrow_ep \ ---exclude $PARQUET_ROOT/parquet-build/brotli_ep \ ---exclude $PARQUET_ROOT/parquet-build/brotli_ep-prefix \ ---exclude $PARQUET_ROOT/parquet-build/gbenchmark_ep \ ---exclude $PARQUET_ROOT/parquet-build/googletest_ep-prefix \ ---exclude $PARQUET_ROOT/parquet-build/snappy_ep \ ---exclude $PARQUET_ROOT/parquet-build/snappy_ep-prefix \ ---exclude $PARQUET_ROOT/parquet-build/zlib_ep \ ---exclude $PARQUET_ROOT/parquet-build/zlib_ep-prefix \ ---exclude $PARQUET_ROOT/build \ ---exclude $PARQUET_ROOT/src/parquet/thrift \ ---exclude /usr +coveralls --gcov 'gcov-4.9' --gcov-options '\-lp' \ + -r $PARQUET_ROOT \ + -i '/src' \ + -i '$PARQUET_ROOT/src' \ + -e '/parquet-build' \ + -e '$PARQUET_ROOT/parquet-build' \ + -e '/usr' \ + -e '$PARQUET_ROOT/src/parquet/thrift' This is an automated message from the Apache Git Service. To respond to the message, please log on GitHub and use the URL above to go to the specific comment. For queries about this service, please contact Infrastructure at: us...@infra.apache.org > [C++] Code coverage has been broken since June 23 > - > > Key: PARQUET-1148 > URL: https://issues.apache.org/jira/browse/PARQUET-1148 > Project: Parquet > Issue Type: Bug > Components: parquet-cpp >Reporter: Wes McKinney >Priority: Major > Labels: pull-request-available > Fix For: cpp-1.5.0 > > > PR #357 is the smoking gun, but we'll need to investigate -- This message was sent by Atlassian JIRA (v7.6.3#76005)
[jira] [Updated] (PARQUET-1148) [C++] Code coverage has been broken since June 23
[ https://issues.apache.org/jira/browse/PARQUET-1148?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] ASF GitHub Bot updated PARQUET-1148: Labels: pull-request-available (was: ) > [C++] Code coverage has been broken since June 23 > - > > Key: PARQUET-1148 > URL: https://issues.apache.org/jira/browse/PARQUET-1148 > Project: Parquet > Issue Type: Bug > Components: parquet-cpp >Reporter: Wes McKinney >Priority: Major > Labels: pull-request-available > Fix For: cpp-1.5.0 > > > PR #357 is the smoking gun, but we'll need to investigate -- This message was sent by Atlassian JIRA (v7.6.3#76005)
[jira] [Commented] (PARQUET-632) Parquet file in invalid state while writing to S3 from EMR
[ https://issues.apache.org/jira/browse/PARQUET-632?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16581275#comment-16581275 ] Ryan Blue commented on PARQUET-632: --- [~pkgajulapalli], can you go ahead and post the stack trace? I thought you said you were using 2.2.0. These classes definitely changed. > Parquet file in invalid state while writing to S3 from EMR > -- > > Key: PARQUET-632 > URL: https://issues.apache.org/jira/browse/PARQUET-632 > Project: Parquet > Issue Type: Bug >Affects Versions: 1.7.0 >Reporter: Peter Halliday >Priority: Blocker > > I'm writing parquet to S3 from Spark 1.6.1 on EMR. And when it got to the > last few files to write to S3, I received this stacktrace in the log with no > other errors before or after it. It's very consistent. This particular > batch keeps erroring the same way. > {noformat} > 2016-06-10 01:46:05,282] WARN org.apache.spark.scheduler.TaskSetManager > [task-result-getter-2hread] - Lost task 3737.0 in stage 2.0 (TID 10585, > ip-172-16-96-32.ec2.internal): org.apache.spark.SparkException: Task failed > while writing rows. > at > org.apache.spark.sql.execution.datasources.DynamicPartitionWriterContainer.writeRows(WriterContainer.scala:414) > at > org.apache.spark.sql.execution.datasources.InsertIntoHadoopFsRelation$$anonfun$run$1$$anonfun$apply$mcV$sp$3.apply(InsertIntoHadoopFsRelation.scala:150) > at > org.apache.spark.sql.execution.datasources.InsertIntoHadoopFsRelation$$anonfun$run$1$$anonfun$apply$mcV$sp$3.apply(InsertIntoHadoopFsRelation.scala:150) > at org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:66) > at org.apache.spark.scheduler.Task.run(Task.scala:89) > at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:214) > at > java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145) > at > java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615) > at java.lang.Thread.run(Thread.java:745) > Caused by: java.io.IOException: The file being written is in an invalid > state. Probably caused by an error thrown previously. Current state: COLUMN > at > org.apache.parquet.hadoop.ParquetFileWriter$STATE.error(ParquetFileWriter.java:146) > at > org.apache.parquet.hadoop.ParquetFileWriter$STATE.startBlock(ParquetFileWriter.java:138) > at > org.apache.parquet.hadoop.ParquetFileWriter.startBlock(ParquetFileWriter.java:195) > at > org.apache.parquet.hadoop.InternalParquetRecordWriter.flushRowGroupToStore(InternalParquetRecordWriter.java:153) > at > org.apache.parquet.hadoop.InternalParquetRecordWriter.close(InternalParquetRecordWriter.java:113) > at > org.apache.parquet.hadoop.ParquetRecordWriter.close(ParquetRecordWriter.java:112) > at > org.apache.spark.sql.execution.datasources.parquet.ParquetOutputWriter.close(ParquetRelation.scala:101) > at > org.apache.spark.sql.execution.datasources.DynamicPartitionWriterContainer.writeRows(WriterContainer.scala:405) > ... 8 more > {noformat} -- This message was sent by Atlassian JIRA (v7.6.3#76005)
[jira] [Updated] (PARQUET-1376) Data obfuscation layer for encryption
[ https://issues.apache.org/jira/browse/PARQUET-1376?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Gidon Gershinsky updated PARQUET-1376: -- Summary: Data obfuscation layer for encryption (was: Anonymity layer for encryption) > Data obfuscation layer for encryption > - > > Key: PARQUET-1376 > URL: https://issues.apache.org/jira/browse/PARQUET-1376 > Project: Parquet > Issue Type: New Feature >Reporter: Gidon Gershinsky >Assignee: Gidon Gershinsky >Priority: Major > > Anonymity layer for hidden columns > # Different data masking options > ** per-cell > ** aggregated (average, etc) > # Reader notification on data access status > # Providing readers with a choice of masking options (if available) -- This message was sent by Atlassian JIRA (v7.6.3#76005)
[jira] [Created] (PARQUET-1380) move Bloom filter test binary to parquet-testing repo
Junjie Chen created PARQUET-1380: Summary: move Bloom filter test binary to parquet-testing repo Key: PARQUET-1380 URL: https://issues.apache.org/jira/browse/PARQUET-1380 Project: Parquet Issue Type: Sub-task Components: parquet-cpp Reporter: Junjie Chen Assignee: Junjie Chen -- This message was sent by Atlassian JIRA (v7.6.3#76005)
[jira] [Commented] (PARQUET-1378) [c++] Allow RowGroups with zero rows to be written
[ https://issues.apache.org/jira/browse/PARQUET-1378?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16581045#comment-16581045 ] ASF GitHub Bot commented on PARQUET-1378: - wesm closed pull request #485: PARQUET-1378: Allow RowGroups with zero rows to be written URL: https://github.com/apache/parquet-cpp/pull/485 This is a PR merged from a forked repository. As GitHub hides the original diff on merge, it is displayed below for the sake of provenance: As this is a foreign pull request (from a fork), the diff is supplied below (as it won't show otherwise due to GitHub magic): diff --git a/src/parquet/column_writer.cc b/src/parquet/column_writer.cc index 48fba555..934530c0 100644 --- a/src/parquet/column_writer.cc +++ b/src/parquet/column_writer.cc @@ -432,12 +432,13 @@ int64_t ColumnWriter::Close() { FlushBufferedDataPages(); EncodedStatistics chunk_statistics = GetChunkStatistics(); +// Write stats only if the column has atleast one row written // From parquet-mr // Don't write stats larger than the max size rather than truncating. The // rationale is that some engines may use the minimum value in the page as // the true minimum for aggregations and there is no way to mark that a // value has been truncated and is a lower bound and not in the page. -if (chunk_statistics.is_set() && +if (rows_written_ > 0 && chunk_statistics.is_set() && chunk_statistics.max_stat_length() <= properties_->max_statistics_size(descr_->path())) { metadata_->SetStatistics(SortOrder::SIGNED == descr_->sort_order(), diff --git a/src/parquet/file-serialize-test.cc b/src/parquet/file-serialize-test.cc index 31d2bd46..19934041 100644 --- a/src/parquet/file-serialize-test.cc +++ b/src/parquet/file-serialize-test.cc @@ -176,6 +176,27 @@ class TestSerialize : public PrimitiveTypedTest { column_writer->Close(); } } + + void ZeroRowsRowGroup() { +std::shared_ptr sink(new InMemoryOutputStream()); +auto gnode = std::static_pointer_cast(this->node_); + +std::shared_ptr props = WriterProperties::Builder().build(); + +auto file_writer = ParquetFileWriter::Open(sink, gnode, props); + +RowGroupWriter* row_group_writer; +row_group_writer = file_writer->AppendRowGroup(); + +for (int col = 0; col < num_columns_; ++col) { + auto column_writer = + static_cast*>(row_group_writer->NextColumn()); + column_writer->Close(); +} + +row_group_writer->Close(); +file_writer->Close(); + } }; typedef ::testing::TypesUnequalNumRows(101, num_rows), ParquetException); } +TYPED_TEST(TestSerialize, ZeroRows) { ASSERT_NO_THROW(this->ZeroRowsRowGroup()); } + TYPED_TEST(TestSerialize, RepeatedTooFewRows) { ASSERT_THROW(this->RepeatedUnequalRows(), ParquetException); } diff --git a/src/parquet/metadata.cc b/src/parquet/metadata.cc index 39dee638..1cab51f0 100644 --- a/src/parquet/metadata.cc +++ b/src/parquet/metadata.cc @@ -731,7 +731,7 @@ class RowGroupMetaDataBuilder::RowGroupMetaDataBuilderImpl { int64_t total_byte_size = 0; for (int i = 0; i < schema_->num_columns(); i++) { - if (!(row_group_->columns[i].file_offset > 0)) { + if (!(row_group_->columns[i].file_offset >= 0)) { std::stringstream ss; ss << "Column " << i << " is not complete."; throw ParquetException(ss.str()); This is an automated message from the Apache Git Service. To respond to the message, please log on GitHub and use the URL above to go to the specific comment. For queries about this service, please contact Infrastructure at: us...@infra.apache.org > [c++] Allow RowGroups with zero rows to be written > -- > > Key: PARQUET-1378 > URL: https://issues.apache.org/jira/browse/PARQUET-1378 > Project: Parquet > Issue Type: Improvement >Reporter: Deepak Majeti >Assignee: Deepak Majeti >Priority: Major > Labels: pull-request-available > Fix For: cpp-1.5.0 > > > Currently, the reader-writer.cc example fails when zero rows are written. -- This message was sent by Atlassian JIRA (v7.6.3#76005)
[jira] [Resolved] (PARQUET-1378) [c++] Allow RowGroups with zero rows to be written
[ https://issues.apache.org/jira/browse/PARQUET-1378?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Wes McKinney resolved PARQUET-1378. --- Resolution: Fixed Fix Version/s: (was: 1.5.0) cpp-1.5.0 Issue resolved by pull request 485 [https://github.com/apache/parquet-cpp/pull/485] > [c++] Allow RowGroups with zero rows to be written > -- > > Key: PARQUET-1378 > URL: https://issues.apache.org/jira/browse/PARQUET-1378 > Project: Parquet > Issue Type: Improvement >Reporter: Deepak Majeti >Assignee: Deepak Majeti >Priority: Major > Labels: pull-request-available > Fix For: cpp-1.5.0 > > > Currently, the reader-writer.cc example fails when zero rows are written. -- This message was sent by Atlassian JIRA (v7.6.3#76005)
[jira] [Commented] (PARQUET-1332) [C++] Add bloom filter utility class
[ https://issues.apache.org/jira/browse/PARQUET-1332?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16581039#comment-16581039 ] ASF GitHub Bot commented on PARQUET-1332: - wesm closed pull request #432: PARQUET-1332: Add bloom filter for parquet URL: https://github.com/apache/parquet-cpp/pull/432 This is a PR merged from a forked repository. As GitHub hides the original diff on merge, it is displayed below for the sake of provenance: As this is a foreign pull request (from a fork), the diff is supplied below (as it won't show otherwise due to GitHub magic): diff --git a/CMakeLists.txt b/CMakeLists.txt index b53f5980..f03c53f4 100644 --- a/CMakeLists.txt +++ b/CMakeLists.txt @@ -674,6 +674,7 @@ set(LIBPARQUET_SRCS src/parquet/arrow/record_reader.cc src/parquet/arrow/schema.cc src/parquet/arrow/writer.cc + src/parquet/bloom_filter.cc src/parquet/column_reader.cc src/parquet/column_scanner.cc src/parquet/column_writer.cc @@ -681,6 +682,7 @@ set(LIBPARQUET_SRCS src/parquet/file_reader.cc src/parquet/file_writer.cc src/parquet/metadata.cc + src/parquet/murmur3.cc src/parquet/parquet_constants.cpp src/parquet/parquet_types.cpp src/parquet/printer.cc diff --git a/data/bloom_filter.bin b/data/bloom_filter.bin new file mode 100644 index ..c0e30ce7 Binary files /dev/null and b/data/bloom_filter.bin differ diff --git a/src/parquet/CMakeLists.txt b/src/parquet/CMakeLists.txt index bc16d8bd..93a242c6 100644 --- a/src/parquet/CMakeLists.txt +++ b/src/parquet/CMakeLists.txt @@ -17,6 +17,7 @@ # Headers: top level install(FILES + bloom_filter.h column_reader.h column_page.h column_scanner.h @@ -25,7 +26,9 @@ install(FILES exception.h file_reader.h file_writer.h + hasher.h metadata.h + murmur3.h printer.h properties.h schema.h @@ -50,6 +53,7 @@ install(FILES "${CMAKE_CURRENT_BINARY_DIR}/parquet.pc" DESTINATION "${CMAKE_INSTALL_LIBDIR}/pkgconfig/") +ADD_PARQUET_TEST(bloom_filter-test) ADD_PARQUET_TEST(column_reader-test) ADD_PARQUET_TEST(column_scanner-test) ADD_PARQUET_TEST(column_writer-test) diff --git a/src/parquet/README b/src/parquet/README new file mode 100644 index ..fc16a46c --- /dev/null +++ b/src/parquet/README @@ -0,0 +1,10 @@ +The CompatibilityTest of bloom_filter-test.cc is used to test cross compatibility of +Bloom filters between parquet-mr and parquet-cpp. It reads the Bloom filter binary +generated by the Bloom filter class in the parquet-mr project and tests whether the +values inserted before could be filtered or not. + +The Bloom filter binary is generated by three steps from Parquet-mr: +Step 1: Construct a Bloom filter with 1024 bytes of bitset. +Step 2: Insert hashes of "hello", "parquet", "bloom", "filter" strings to Bloom filter +by calling hash and insert APIs. +Step 3: Call writeTo API to write to File. diff --git a/src/parquet/bloom_filter-test.cc b/src/parquet/bloom_filter-test.cc new file mode 100644 index ..dbef8c8b --- /dev/null +++ b/src/parquet/bloom_filter-test.cc @@ -0,0 +1,241 @@ +// 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. + +#include + +#include +#include +#include + +#include "arrow/io/file.h" +#include "parquet/bloom_filter.h" +#include "parquet/murmur3.h" +#include "parquet/util/memory.h" +#include "parquet/util/test-common.h" + +namespace parquet { +namespace test { +TEST(Murmur3Test, TestBloomFilter) { + uint64_t result; + const uint8_t bitset[8] = {0x0, 0x1, 0x2, 0x3, 0x4, 0x5, 0x6, 0x7}; + ByteArray byteArray(8, bitset); + MurmurHash3 murmur3; + result = murmur3.Hash(); + EXPECT_EQ(result, UINT64_C(913737700387071329)); +} + +TEST(ConstructorTest, TestBloomFilter) { + BlockSplitBloomFilter bloom_filter; + EXPECT_NO_THROW(bloom_filter.Init(1000)); + + // It throws because the length cannot be zero + std::unique_ptr bitset1(new uint8_t[1024]()); + EXPECT_THROW(bloom_filter.Init(bitset1.get(), 0), ParquetException); + + // It throws because the number of bytes of Bloom filter bitset must be a power of 2. + std::unique_ptr bitset2(new uint8_t[1024]()); + EXPECT_THROW(bloom_filter.Init(bitset2.get(), 1023),
[jira] [Updated] (PARQUET-1332) [C++] Add bloom filter utility class
[ https://issues.apache.org/jira/browse/PARQUET-1332?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] ASF GitHub Bot updated PARQUET-1332: Labels: pull-request-available (was: ) > [C++] Add bloom filter utility class > > > Key: PARQUET-1332 > URL: https://issues.apache.org/jira/browse/PARQUET-1332 > Project: Parquet > Issue Type: Sub-task > Components: parquet-cpp >Reporter: Junjie Chen >Assignee: Junjie Chen >Priority: Minor > Labels: pull-request-available > Fix For: cpp-1.5.0 > > > This is subtask of PARQUET-41, this JIRA is used to track adding bloom filter > class to paruqet-mr and parquet-cpp. -- This message was sent by Atlassian JIRA (v7.6.3#76005)
[jira] [Updated] (PARQUET-1379) Incorrect check for ASCENDING/DESCENDING at column index write path
[ https://issues.apache.org/jira/browse/PARQUET-1379?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Gabor Szadovszky updated PARQUET-1379: -- Issue Type: Sub-task (was: Bug) Parent: PARQUET-1201 > Incorrect check for ASCENDING/DESCENDING at column index write path > --- > > Key: PARQUET-1379 > URL: https://issues.apache.org/jira/browse/PARQUET-1379 > Project: Parquet > Issue Type: Sub-task > Components: parquet-mr >Reporter: Gabor Szadovszky >Assignee: Gabor Szadovszky >Priority: Major > > On the branch _column-index_ the check of ASCENDING/DESCENDING orders at > column index write path does not check if {{min[i] <= max[i]}}. -- This message was sent by Atlassian JIRA (v7.6.3#76005)
[jira] [Created] (PARQUET-1379) Incorrect check for ASCENDING/DESCENDING at column index write path
Gabor Szadovszky created PARQUET-1379: - Summary: Incorrect check for ASCENDING/DESCENDING at column index write path Key: PARQUET-1379 URL: https://issues.apache.org/jira/browse/PARQUET-1379 Project: Parquet Issue Type: Bug Components: parquet-mr Reporter: Gabor Szadovszky Assignee: Gabor Szadovszky On the branch _column-index_ the check of ASCENDING/DESCENDING orders at column index write path does not check if {{min[i] <= max[i]}}. -- This message was sent by Atlassian JIRA (v7.6.3#76005)
[jira] [Commented] (PARQUET-632) Parquet file in invalid state while writing to S3 from EMR
[ https://issues.apache.org/jira/browse/PARQUET-632?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16580761#comment-16580761 ] Gajulapalli Praveen Kumar commented on PARQUET-632: --- [~rdblue], the schema of dataframe is pretty simple. {quote}scala> df.schema res2: org.apache.spark.sql.types.StructType = StructType(StructField(col1,StringType,true), StructField(col2,IntegerType,true), StructField(col3,DateType,true), StructField(col4,StringType,true), StructField(col5,DecimalType(18,4),true)) {quote} The stack trace is same as in description. > Parquet file in invalid state while writing to S3 from EMR > -- > > Key: PARQUET-632 > URL: https://issues.apache.org/jira/browse/PARQUET-632 > Project: Parquet > Issue Type: Bug >Affects Versions: 1.7.0 >Reporter: Peter Halliday >Priority: Blocker > > I'm writing parquet to S3 from Spark 1.6.1 on EMR. And when it got to the > last few files to write to S3, I received this stacktrace in the log with no > other errors before or after it. It's very consistent. This particular > batch keeps erroring the same way. > {noformat} > 2016-06-10 01:46:05,282] WARN org.apache.spark.scheduler.TaskSetManager > [task-result-getter-2hread] - Lost task 3737.0 in stage 2.0 (TID 10585, > ip-172-16-96-32.ec2.internal): org.apache.spark.SparkException: Task failed > while writing rows. > at > org.apache.spark.sql.execution.datasources.DynamicPartitionWriterContainer.writeRows(WriterContainer.scala:414) > at > org.apache.spark.sql.execution.datasources.InsertIntoHadoopFsRelation$$anonfun$run$1$$anonfun$apply$mcV$sp$3.apply(InsertIntoHadoopFsRelation.scala:150) > at > org.apache.spark.sql.execution.datasources.InsertIntoHadoopFsRelation$$anonfun$run$1$$anonfun$apply$mcV$sp$3.apply(InsertIntoHadoopFsRelation.scala:150) > at org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:66) > at org.apache.spark.scheduler.Task.run(Task.scala:89) > at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:214) > at > java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145) > at > java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615) > at java.lang.Thread.run(Thread.java:745) > Caused by: java.io.IOException: The file being written is in an invalid > state. Probably caused by an error thrown previously. Current state: COLUMN > at > org.apache.parquet.hadoop.ParquetFileWriter$STATE.error(ParquetFileWriter.java:146) > at > org.apache.parquet.hadoop.ParquetFileWriter$STATE.startBlock(ParquetFileWriter.java:138) > at > org.apache.parquet.hadoop.ParquetFileWriter.startBlock(ParquetFileWriter.java:195) > at > org.apache.parquet.hadoop.InternalParquetRecordWriter.flushRowGroupToStore(InternalParquetRecordWriter.java:153) > at > org.apache.parquet.hadoop.InternalParquetRecordWriter.close(InternalParquetRecordWriter.java:113) > at > org.apache.parquet.hadoop.ParquetRecordWriter.close(ParquetRecordWriter.java:112) > at > org.apache.spark.sql.execution.datasources.parquet.ParquetOutputWriter.close(ParquetRelation.scala:101) > at > org.apache.spark.sql.execution.datasources.DynamicPartitionWriterContainer.writeRows(WriterContainer.scala:405) > ... 8 more > {noformat} -- This message was sent by Atlassian JIRA (v7.6.3#76005)