You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@parquet.apache.org by GitBox <gi...@apache.org> on 2022/09/23 06:18:38 UTC

[GitHub] [parquet-mr] wgtmac opened a new pull request, #998: PARQUET-2195: Add scan command to parquet-cli

wgtmac opened a new pull request, #998:
URL: https://github.com/apache/parquet-mr/pull/998

   This PR enhances parquet-cli by adding a scan command to go through all records without printing them. This is useful when users need to verify if the parquet file is corrupted.
   
   No additional unit tests are added. Test it manually with local parquet files.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: dev-unsubscribe@parquet.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [parquet-mr] wgtmac commented on pull request #998: PARQUET-2195: Add scan command to parquet-cli

Posted by GitBox <gi...@apache.org>.
wgtmac commented on PR #998:
URL: https://github.com/apache/parquet-mr/pull/998#issuecomment-1255836576

   @shangxinli Please take a look when you have time. Thanks!


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: dev-unsubscribe@parquet.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [parquet-mr] shangxinli commented on a diff in pull request #998: PARQUET-2195: Add scan command to parquet-cli

Posted by GitBox <gi...@apache.org>.
shangxinli commented on code in PR #998:
URL: https://github.com/apache/parquet-mr/pull/998#discussion_r990827069


##########
parquet-cli/src/main/java/org/apache/parquet/cli/commands/ScanCommand.java:
##########
@@ -0,0 +1,94 @@
+/*
+ * 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.cli.commands;
+
+import com.beust.jcommander.Parameter;
+import com.beust.jcommander.Parameters;
+import com.google.common.base.Preconditions;
+import com.google.common.collect.Lists;
+import com.google.common.io.Closeables;
+import org.apache.avro.Schema;
+import org.apache.parquet.cli.BaseCommand;
+import org.apache.parquet.cli.util.Expressions;
+import org.slf4j.Logger;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.util.List;
+
+@Parameters(commandDescription = "Scan all records from a file")
+public class ScanCommand extends BaseCommand {
+
+  @Parameter(description = "<file>")
+  List<String> sourceFiles;
+
+  @Parameter(
+    names = {"-c", "--column", "--columns"},
+    description = "List of columns")
+  List<String> columns;
+
+  public ScanCommand(Logger console) {
+    super(console);
+  }
+
+  @Override
+  public int run() throws IOException {
+    Preconditions.checkArgument(
+      sourceFiles != null && !sourceFiles.isEmpty(),
+      "Missing file name");
+    Preconditions.checkArgument(sourceFiles.size() == 1,
+      "Only one file can be given");
+
+    final String source = sourceFiles.get(0);
+    Schema schema = getAvroSchema(source);
+    Schema projection = Expressions.filterSchema(schema, columns);

Review Comment:
   What do we do if the file that doesn't have Avro Schema, 



##########
parquet-cli/src/main/java/org/apache/parquet/cli/commands/ScanCommand.java:
##########
@@ -0,0 +1,94 @@
+/*
+ * 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.cli.commands;
+
+import com.beust.jcommander.Parameter;
+import com.beust.jcommander.Parameters;
+import com.google.common.base.Preconditions;
+import com.google.common.collect.Lists;
+import com.google.common.io.Closeables;
+import org.apache.avro.Schema;
+import org.apache.parquet.cli.BaseCommand;
+import org.apache.parquet.cli.util.Expressions;
+import org.slf4j.Logger;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.util.List;
+
+@Parameters(commandDescription = "Scan all records from a file")
+public class ScanCommand extends BaseCommand {
+
+  @Parameter(description = "<file>")
+  List<String> sourceFiles;
+
+  @Parameter(
+    names = {"-c", "--column", "--columns"},
+    description = "List of columns")
+  List<String> columns;
+
+  public ScanCommand(Logger console) {
+    super(console);
+  }
+
+  @Override
+  public int run() throws IOException {
+    Preconditions.checkArgument(
+      sourceFiles != null && !sourceFiles.isEmpty(),
+      "Missing file name");
+    Preconditions.checkArgument(sourceFiles.size() == 1,
+      "Only one file can be given");
+
+    final String source = sourceFiles.get(0);
+    Schema schema = getAvroSchema(source);
+    Schema projection = Expressions.filterSchema(schema, columns);

Review Comment:
   What do we do if the file that doesn't have Avro Schema?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: dev-unsubscribe@parquet.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [parquet-mr] wgtmac commented on a diff in pull request #998: PARQUET-2195: Add scan command to parquet-cli

Posted by GitBox <gi...@apache.org>.
wgtmac commented on code in PR #998:
URL: https://github.com/apache/parquet-mr/pull/998#discussion_r992431971


##########
parquet-cli/src/main/java/org/apache/parquet/cli/commands/ScanCommand.java:
##########
@@ -0,0 +1,94 @@
+/*
+ * 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.cli.commands;
+
+import com.beust.jcommander.Parameter;
+import com.beust.jcommander.Parameters;
+import com.google.common.base.Preconditions;
+import com.google.common.collect.Lists;
+import com.google.common.io.Closeables;
+import org.apache.avro.Schema;
+import org.apache.parquet.cli.BaseCommand;
+import org.apache.parquet.cli.util.Expressions;
+import org.slf4j.Logger;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.util.List;
+
+@Parameters(commandDescription = "Scan all records from a file")
+public class ScanCommand extends BaseCommand {
+
+  @Parameter(description = "<file>")
+  List<String> sourceFiles;
+
+  @Parameter(
+    names = {"-c", "--column", "--columns"},
+    description = "List of columns")
+  List<String> columns;
+
+  public ScanCommand(Logger console) {
+    super(console);
+  }
+
+  @Override
+  public int run() throws IOException {
+    Preconditions.checkArgument(
+      sourceFiles != null && !sourceFiles.isEmpty(),
+      "Missing file name");
+    Preconditions.checkArgument(sourceFiles.size() == 1,
+      "Only one file can be given");
+
+    final String source = sourceFiles.get(0);
+    Schema schema = getAvroSchema(source);
+    Schema projection = Expressions.filterSchema(schema, columns);

Review Comment:
   I believe its naming is a little bit confusing. It supports getting schema from parquet, avro and avsc files. 
   
   Please check here for detail: https://github.com/apache/parquet-mr/blob/master/parquet-cli/src/main/java/org/apache/parquet/cli/BaseCommand.java#L397



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: dev-unsubscribe@parquet.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [parquet-mr] wgtmac commented on a diff in pull request #998: PARQUET-2195: Add scan command to parquet-cli

Posted by GitBox <gi...@apache.org>.
wgtmac commented on code in PR #998:
URL: https://github.com/apache/parquet-mr/pull/998#discussion_r992446760


##########
parquet-cli/src/main/java/org/apache/parquet/cli/commands/ScanCommand.java:
##########
@@ -0,0 +1,94 @@
+/*
+ * 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.cli.commands;
+
+import com.beust.jcommander.Parameter;
+import com.beust.jcommander.Parameters;
+import com.google.common.base.Preconditions;
+import com.google.common.collect.Lists;
+import com.google.common.io.Closeables;
+import org.apache.avro.Schema;
+import org.apache.parquet.cli.BaseCommand;
+import org.apache.parquet.cli.util.Expressions;
+import org.slf4j.Logger;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.util.List;
+
+@Parameters(commandDescription = "Scan all records from a file")
+public class ScanCommand extends BaseCommand {
+
+  @Parameter(description = "<file>")
+  List<String> sourceFiles;
+
+  @Parameter(
+    names = {"-c", "--column", "--columns"},
+    description = "List of columns")
+  List<String> columns;
+
+  public ScanCommand(Logger console) {
+    super(console);
+  }
+
+  @Override
+  public int run() throws IOException {
+    Preconditions.checkArgument(
+      sourceFiles != null && !sourceFiles.isEmpty(),
+      "Missing file name");
+    Preconditions.checkArgument(sourceFiles.size() == 1,
+      "Only one file can be given");

Review Comment:
   Fixed



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: dev-unsubscribe@parquet.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [parquet-mr] shangxinli commented on a diff in pull request #998: PARQUET-2195: Add scan command to parquet-cli

Posted by GitBox <gi...@apache.org>.
shangxinli commented on code in PR #998:
URL: https://github.com/apache/parquet-mr/pull/998#discussion_r990827325


##########
parquet-cli/src/main/java/org/apache/parquet/cli/commands/ScanCommand.java:
##########
@@ -0,0 +1,94 @@
+/*
+ * 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.cli.commands;
+
+import com.beust.jcommander.Parameter;
+import com.beust.jcommander.Parameters;
+import com.google.common.base.Preconditions;
+import com.google.common.collect.Lists;
+import com.google.common.io.Closeables;
+import org.apache.avro.Schema;
+import org.apache.parquet.cli.BaseCommand;
+import org.apache.parquet.cli.util.Expressions;
+import org.slf4j.Logger;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.util.List;
+
+@Parameters(commandDescription = "Scan all records from a file")
+public class ScanCommand extends BaseCommand {
+
+  @Parameter(description = "<file>")
+  List<String> sourceFiles;
+
+  @Parameter(
+    names = {"-c", "--column", "--columns"},
+    description = "List of columns")
+  List<String> columns;
+
+  public ScanCommand(Logger console) {
+    super(console);
+  }
+
+  @Override
+  public int run() throws IOException {
+    Preconditions.checkArgument(
+      sourceFiles != null && !sourceFiles.isEmpty(),
+      "Missing file name");
+    Preconditions.checkArgument(sourceFiles.size() == 1,
+      "Only one file can be given");
+
+    final String source = sourceFiles.get(0);
+    Schema schema = getAvroSchema(source);
+    Schema projection = Expressions.filterSchema(schema, columns);
+
+    long startTime = System.currentTimeMillis();
+    Iterable<Object> reader = openDataFile(source, projection);
+    boolean threw = true;
+    long count = 0;
+    try {
+      for (Object record : reader) {
+        count += 1;

Review Comment:
   If your goal is only to get count, why not get it from the metadata? Iterate each record is an expensive operation. 



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: dev-unsubscribe@parquet.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [parquet-mr] wgtmac commented on a diff in pull request #998: PARQUET-2195: Add scan command to parquet-cli

Posted by GitBox <gi...@apache.org>.
wgtmac commented on code in PR #998:
URL: https://github.com/apache/parquet-mr/pull/998#discussion_r992436297


##########
parquet-cli/src/main/java/org/apache/parquet/cli/commands/ScanCommand.java:
##########
@@ -0,0 +1,94 @@
+/*
+ * 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.cli.commands;
+
+import com.beust.jcommander.Parameter;
+import com.beust.jcommander.Parameters;
+import com.google.common.base.Preconditions;
+import com.google.common.collect.Lists;
+import com.google.common.io.Closeables;
+import org.apache.avro.Schema;
+import org.apache.parquet.cli.BaseCommand;
+import org.apache.parquet.cli.util.Expressions;
+import org.slf4j.Logger;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.util.List;
+
+@Parameters(commandDescription = "Scan all records from a file")
+public class ScanCommand extends BaseCommand {
+
+  @Parameter(description = "<file>")
+  List<String> sourceFiles;
+
+  @Parameter(
+    names = {"-c", "--column", "--columns"},
+    description = "List of columns")
+  List<String> columns;
+
+  public ScanCommand(Logger console) {
+    super(console);
+  }
+
+  @Override
+  public int run() throws IOException {
+    Preconditions.checkArgument(
+      sourceFiles != null && !sourceFiles.isEmpty(),
+      "Missing file name");
+    Preconditions.checkArgument(sourceFiles.size() == 1,
+      "Only one file can be given");
+
+    final String source = sourceFiles.get(0);
+    Schema schema = getAvroSchema(source);
+    Schema projection = Expressions.filterSchema(schema, columns);
+
+    long startTime = System.currentTimeMillis();
+    Iterable<Object> reader = openDataFile(source, projection);
+    boolean threw = true;
+    long count = 0;
+    try {
+      for (Object record : reader) {
+        count += 1;

Review Comment:
   It serves the same purpose as https://github.com/apache/arrow/blob/master/cpp/tools/parquet/parquet_scan.cc to validate the data integrity of a parquet file.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: dev-unsubscribe@parquet.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [parquet-mr] shangxinli commented on pull request #998: PARQUET-2195: Add scan command to parquet-cli

Posted by GitBox <gi...@apache.org>.
shangxinli commented on PR #998:
URL: https://github.com/apache/parquet-mr/pull/998#issuecomment-1305941081

   lgtm


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: dev-unsubscribe@parquet.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [parquet-mr] shangxinli commented on a diff in pull request #998: PARQUET-2195: Add scan command to parquet-cli

Posted by GitBox <gi...@apache.org>.
shangxinli commented on code in PR #998:
URL: https://github.com/apache/parquet-mr/pull/998#discussion_r990826841


##########
parquet-cli/src/main/java/org/apache/parquet/cli/commands/ScanCommand.java:
##########
@@ -0,0 +1,94 @@
+/*
+ * 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.cli.commands;
+
+import com.beust.jcommander.Parameter;
+import com.beust.jcommander.Parameters;
+import com.google.common.base.Preconditions;
+import com.google.common.collect.Lists;
+import com.google.common.io.Closeables;
+import org.apache.avro.Schema;
+import org.apache.parquet.cli.BaseCommand;
+import org.apache.parquet.cli.util.Expressions;
+import org.slf4j.Logger;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.util.List;
+
+@Parameters(commandDescription = "Scan all records from a file")
+public class ScanCommand extends BaseCommand {
+
+  @Parameter(description = "<file>")
+  List<String> sourceFiles;
+
+  @Parameter(
+    names = {"-c", "--column", "--columns"},
+    description = "List of columns")
+  List<String> columns;
+
+  public ScanCommand(Logger console) {
+    super(console);
+  }
+
+  @Override
+  public int run() throws IOException {
+    Preconditions.checkArgument(
+      sourceFiles != null && !sourceFiles.isEmpty(),
+      "Missing file name");
+    Preconditions.checkArgument(sourceFiles.size() == 1,
+      "Only one file can be given");

Review Comment:
   Why not define it as String instead of List<Striing>? 



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: dev-unsubscribe@parquet.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [parquet-mr] wgtmac commented on a diff in pull request #998: PARQUET-2195: Add scan command to parquet-cli

Posted by GitBox <gi...@apache.org>.
wgtmac commented on code in PR #998:
URL: https://github.com/apache/parquet-mr/pull/998#discussion_r992437914


##########
parquet-cli/src/main/java/org/apache/parquet/cli/commands/ScanCommand.java:
##########
@@ -0,0 +1,94 @@
+/*
+ * 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.cli.commands;
+
+import com.beust.jcommander.Parameter;
+import com.beust.jcommander.Parameters;
+import com.google.common.base.Preconditions;
+import com.google.common.collect.Lists;
+import com.google.common.io.Closeables;
+import org.apache.avro.Schema;
+import org.apache.parquet.cli.BaseCommand;
+import org.apache.parquet.cli.util.Expressions;
+import org.slf4j.Logger;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.util.List;
+
+@Parameters(commandDescription = "Scan all records from a file")
+public class ScanCommand extends BaseCommand {
+
+  @Parameter(description = "<file>")
+  List<String> sourceFiles;
+
+  @Parameter(
+    names = {"-c", "--column", "--columns"},
+    description = "List of columns")
+  List<String> columns;
+
+  public ScanCommand(Logger console) {
+    super(console);
+  }
+
+  @Override
+  public int run() throws IOException {
+    Preconditions.checkArgument(
+      sourceFiles != null && !sourceFiles.isEmpty(),
+      "Missing file name");
+    Preconditions.checkArgument(sourceFiles.size() == 1,
+      "Only one file can be given");
+
+    final String source = sourceFiles.get(0);
+    Schema schema = getAvroSchema(source);
+    Schema projection = Expressions.filterSchema(schema, columns);
+
+    long startTime = System.currentTimeMillis();
+    Iterable<Object> reader = openDataFile(source, projection);
+    boolean threw = true;
+    long count = 0;
+    try {
+      for (Object record : reader) {
+        count += 1;

Review Comment:
   This is very handy when we want to quickly check if any file is corrupted in production.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: dev-unsubscribe@parquet.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [parquet-mr] shangxinli merged pull request #998: PARQUET-2195: Add scan command to parquet-cli

Posted by GitBox <gi...@apache.org>.
shangxinli merged PR #998:
URL: https://github.com/apache/parquet-mr/pull/998


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: dev-unsubscribe@parquet.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org