iemejia commented on a change in pull request #13646:
URL: https://github.com/apache/beam/pull/13646#discussion_r551963670
##########
File path: sdks/java/core/src/main/java/org/apache/beam/sdk/io/AvroIO.java
##########
@@ -569,16 +570,7 @@
@Experimental(Kind.SCHEMAS)
private static <T> PCollection<T> setBeamSchema(
PCollection<T> pc, Class<T> clazz, @Nullable Schema schema) {
- org.apache.beam.sdk.schemas.Schema beamSchema =
- org.apache.beam.sdk.schemas.utils.AvroUtils.getSchema(clazz, schema);
- if (beamSchema != null) {
- pc.setSchema(
- beamSchema,
- TypeDescriptor.of(clazz),
- org.apache.beam.sdk.schemas.utils.AvroUtils.getToRowFunction(clazz,
schema),
-
org.apache.beam.sdk.schemas.utils.AvroUtils.getFromRowFunction(clazz));
- }
- return pc;
+ return pc.setCoder(AvroUtils.schemaCoder(clazz, schema));
Review comment:
:+1:
##########
File path:
sdks/java/io/parquet/src/main/java/org/apache/beam/sdk/io/parquet/GenericRecordReadConverter.java
##########
@@ -1,64 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.beam.sdk.io.parquet;
-
-import com.google.auto.value.AutoValue;
-import java.io.Serializable;
-import org.apache.avro.generic.GenericRecord;
-import org.apache.beam.sdk.schemas.Schema;
-import org.apache.beam.sdk.schemas.utils.AvroUtils;
-import org.apache.beam.sdk.transforms.DoFn;
-import org.apache.beam.sdk.transforms.PTransform;
-import org.apache.beam.sdk.transforms.ParDo;
-import org.apache.beam.sdk.values.PCollection;
-import org.apache.beam.sdk.values.Row;
-
-/** A {@link PTransform} to convert {@link GenericRecord} to {@link Row}. */
-@AutoValue
-abstract class GenericRecordReadConverter
Review comment:
Thanks for removing this one, I had not even thought that this one was
repeated code now :+1:
##########
File path:
sdks/java/io/parquet/src/main/java/org/apache/beam/sdk/io/parquet/ParquetIO.java
##########
@@ -1165,6 +1178,17 @@ public GenericRecord apply(GenericRecord input) {
private GenericRecordPassthroughFn() {}
}
+ /** Encapsulates checked exceptions with Parquet file operation as
RuntimeException. */
+ public static class ParquetIoException extends RuntimeException {
Review comment:
I know this is going to sound a bit odd but can we just remove this and
let exception handling as it was before. Beam rarely does this kind of wrapping
and I don't want ParquetIO to differ from how others IOs behave.
##########
File path:
sdks/java/io/parquet/src/main/java/org/apache/beam/sdk/io/parquet/ParquetIO.java
##########
@@ -1103,14 +1103,14 @@ public long defaultBlockSize() {
private static class BeamOutputStream extends PositionOutputStream {
private long position = 0;
- private OutputStream outputStream;
+ private final OutputStream outputStream;
private BeamOutputStream(OutputStream outputStream) {
this.outputStream = outputStream;
}
@Override
- public long getPos() throws IOException {
+ public long getPos() {
Review comment:
:+1:
##########
File path: sdks/java/core/src/main/java/org/apache/beam/sdk/io/AvroIO.java
##########
@@ -197,7 +198,7 @@
*
* <pre>{@code
* PCollection<AvroAutoGenClass> records =
- * p.apply(AvroIO.read(...).from(...).withBeamSchemas(true);
+ * p.apply(AvroIO.read(...).from(...).withBeamSchemas(true));
Review comment:
:+1:
##########
File path:
sdks/java/io/parquet/src/test/java/org/apache/beam/sdk/io/parquet/ParquetIOTest.java
##########
@@ -337,7 +337,7 @@ public void
testReadFilesUnknownSchemaFilesForGenericRecordThrowException() {
public void testReadDisplayData() {
DisplayData displayData =
DisplayData.from(ParquetIO.read(SCHEMA).from("foo.parquet"));
- Assert.assertThat(displayData, hasDisplayItem("filePattern",
"foo.parquet"));
+ MatcherAssert.assertThat(displayData, hasDisplayItem("filePattern",
"foo.parquet"));
Review comment:
why the hamcrest one? the other one misses something? otherwise prefer
the junit one.
##########
File path:
sdks/java/io/parquet/src/main/java/org/apache/beam/sdk/io/parquet/ParquetIO.java
##########
@@ -1143,6 +1143,19 @@ public void close() throws IOException {
}
}
+ /** Returns a model object created using provided modelClass or null. */
+ private static GenericData buildModelObject(@Nullable Class<? extends
GenericData> modelClass) {
+ try {
+ if (modelClass == null) {
Review comment:
please make this a return with a ternary operator
----------------------------------------------------------------
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.
For queries about this service, please contact Infrastructure at:
[email protected]