kl0u commented on a change in pull request #11474:
URL: https://github.com/apache/flink/pull/11474#discussion_r411208175



##########
File path: 
flink-formats/flink-orc/src/main/java/org/apache/flink/orc/vector/Vectorizer.java
##########
@@ -0,0 +1,93 @@
+/*
+ * 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.flink.orc.vector;
+
+import org.apache.flink.annotation.PublicEvolving;
+
+import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch;
+import org.apache.orc.TypeDescription;
+
+import java.io.IOException;
+import java.io.Serializable;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * This class provides an abstracted set of methods to handle the lifecycle of 
{@link VectorizedRowBatch}.
+ *
+ * <p>Developers have to extend this class and override the vectorize() method 
with the logic
+ * to transform the element to a {@link VectorizedRowBatch}.
+ *
+ * @param <T> The type of the element
+ */
+@PublicEvolving
+public abstract class Vectorizer<T> implements Serializable {
+
+       protected transient VectorizedRowBatch rowBatch;
+
+       private final TypeDescription schema;
+
+       public Vectorizer(final String schema) {
+               checkNotNull(schema);
+               this.schema = TypeDescription.fromString(schema);
+               this.rowBatch = this.schema.createRowBatch();
+       }
+

Review comment:
       Why exposing this to the user? The lifecycle of the `VectorizedRowBatch` 
should be in our hands. 

##########
File path: 
flink-formats/flink-orc/src/main/java/org/apache/flink/orc/vector/Vectorizer.java
##########
@@ -0,0 +1,93 @@
+/*
+ * 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.flink.orc.vector;
+
+import org.apache.flink.annotation.PublicEvolving;
+
+import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch;
+import org.apache.orc.TypeDescription;
+
+import java.io.IOException;
+import java.io.Serializable;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * This class provides an abstracted set of methods to handle the lifecycle of 
{@link VectorizedRowBatch}.
+ *
+ * <p>Developers have to extend this class and override the vectorize() method 
with the logic
+ * to transform the element to a {@link VectorizedRowBatch}.
+ *
+ * @param <T> The type of the element
+ */
+@PublicEvolving
+public abstract class Vectorizer<T> implements Serializable {
+
+       protected transient VectorizedRowBatch rowBatch;
+
+       private final TypeDescription schema;
+
+       public Vectorizer(final String schema) {
+               checkNotNull(schema);
+               this.schema = TypeDescription.fromString(schema);
+               this.rowBatch = this.schema.createRowBatch();
+       }
+
+       public void initBatch() {
+               this.rowBatch = this.schema.createRowBatch();
+       }
+
+       /**
+        * Provides the ORC schema.
+        *
+        * @return the ORC schema
+        */
+       public TypeDescription getSchema() {
+               return this.schema;
+       }
+
+       /**
+        * Provides the VectorizedRowBatch containing the
+        * ColumnVectors of the input elements.
+        *
+        * @return vectorized row batch
+        */
+       public VectorizedRowBatch getRowBatch() {
+               return this.rowBatch;
+       }
+
+       /**
+        * Calls reset on the VectorizedRowBatch instance.
+        */
+       public void reset() {
+               if (this.rowBatch != null) {
+                       this.rowBatch.reset();
+               }
+       }
+
+       /**
+        * Transforms the provided element to ColumnVectors and
+        * sets them in the exposed VectorizedRowBatch.
+        *
+        * @param element The input element
+        * @throws IOException if there is an error while transforming the 
input.
+        */
+       public abstract void vectorize(T element) throws IOException;

Review comment:
       Here we should explicitly pass the `VectorizedRowBatch` as an argument. 
If not, an implementor of a custom `Vectorizer` should know about internals of 
the abstract class, which he does not need to.

##########
File path: 
flink-formats/flink-orc/src/main/java/org/apache/flink/orc/writer/OrcBulkWriter.java
##########
@@ -0,0 +1,70 @@
+/*
+ * 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.flink.orc.writer;
+
+import org.apache.flink.annotation.PublicEvolving;
+import org.apache.flink.api.common.serialization.BulkWriter;
+import org.apache.flink.orc.vector.Vectorizer;
+
+import org.apache.orc.OrcProto;
+import org.apache.orc.Writer;
+
+import java.io.IOException;
+import java.util.List;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * A {@link BulkWriter} implementation that writes data in ORC format.
+ *
+ * @param <T> The type of element written.
+ */
+@PublicEvolving
+public class OrcBulkWriter<T> implements BulkWriter<T> {
+
+       private final Writer writer;
+       private final Vectorizer<T> vectorizer;
+

Review comment:
       The `VectorizedRowBatch` can be a member of this class, and we can 
initialize it in the constructor, `reset()` it in `finish()` and pass it to the 
`vectorize()` in the `addElement`. 

##########
File path: 
flink-formats/flink-orc/src/main/java/org/apache/flink/orc/writer/OrcBulkWriterFactory.java
##########
@@ -0,0 +1,139 @@
+/*
+ * 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.flink.orc.writer;
+
+import org.apache.flink.annotation.PublicEvolving;
+import org.apache.flink.api.common.serialization.BulkWriter;
+import org.apache.flink.core.fs.FSDataOutputStream;
+import org.apache.flink.orc.vector.Vectorizer;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.orc.OrcFile;
+import org.apache.orc.OrcProto;
+import org.apache.orc.impl.WriterImpl;
+
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Properties;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * A factory that creates an ORC {@link BulkWriter}. The factory takes a user
+ * supplied {@link Vectorizer} implementation to convert the element into an
+ * {@link org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch}.
+ *
+ * @param <T> The type of element to write.
+ */
+@PublicEvolving
+public class OrcBulkWriterFactory<T> implements BulkWriter.Factory<T> {
+
+       /*
+       A dummy Hadoop Path to work around the current implementation of ORC 
WriterImpl which
+       works on the basis of a Hadoop FileSystem and Hadoop Path but since we 
use a customised
+       ORC PhysicalWriter implementation that uses Flink's own 
FSDataOutputStream as the
+       underlying/internal stream instead of Hadoop's FSDataOutputStream, we 
don't have to worry
+       about this usage.
+        */
+       private static final Path FIXED_PATH = new Path(".");
+
+       private final Vectorizer<T> vectorizer;
+       private final Properties writerProperties;
+       private final Map<String, String> confMap;
+
+       private OrcFile.WriterOptions writerOptions;
+       private List<OrcProto.UserMetadataItem> userMetadata;

Review comment:
       If we agree on my previous comment, then this should go away.

##########
File path: 
flink-formats/flink-orc/src/main/java/org/apache/flink/orc/writer/OrcBulkWriter.java
##########
@@ -0,0 +1,70 @@
+/*
+ * 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.flink.orc.writer;
+
+import org.apache.flink.annotation.PublicEvolving;
+import org.apache.flink.api.common.serialization.BulkWriter;
+import org.apache.flink.orc.vector.Vectorizer;
+
+import org.apache.orc.OrcProto;
+import org.apache.orc.Writer;
+
+import java.io.IOException;
+import java.util.List;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * A {@link BulkWriter} implementation that writes data in ORC format.
+ *
+ * @param <T> The type of element written.
+ */
+@PublicEvolving
+public class OrcBulkWriter<T> implements BulkWriter<T> {
+
+       private final Writer writer;
+       private final Vectorizer<T> vectorizer;
+
+       private transient List<OrcProto.UserMetadataItem> userMetadata;

Review comment:
       I do not think this should be here. I was referring to the method 
   
   ```
   /**
      * Add a row batch to the ORC file.
      * @param batch the rows to add
      */
     void addRowBatch(VectorizedRowBatch batch) throws IOException;
   ```
   
   of the `Writer` and _not_ the `appendUserMetadata()`. The former seems to be 
aimed to be exposed to the user, while the latter does not (from the Javadoc).
   
   In addition, this method seems to be intended to be called at any time, and 
not at the instantiation of the `OrcBulkWriter`. The user can add metadata at 
any point, they will be appended to a list, and the list will be appended to 
the output file when it is finally written. This is why in my branch I just set 
the `Writer` to the `vectorizer` and let the user use it whenever he/she wants. 
I still believe that that is the correct way, correct me if I am wrong.

##########
File path: 
flink-formats/flink-orc/src/main/java/org/apache/flink/orc/vector/Vectorizer.java
##########
@@ -0,0 +1,93 @@
+/*
+ * 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.flink.orc.vector;
+
+import org.apache.flink.annotation.PublicEvolving;
+
+import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch;
+import org.apache.orc.TypeDescription;
+
+import java.io.IOException;
+import java.io.Serializable;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * This class provides an abstracted set of methods to handle the lifecycle of 
{@link VectorizedRowBatch}.
+ *
+ * <p>Developers have to extend this class and override the vectorize() method 
with the logic
+ * to transform the element to a {@link VectorizedRowBatch}.
+ *
+ * @param <T> The type of the element
+ */
+@PublicEvolving
+public abstract class Vectorizer<T> implements Serializable {
+
+       protected transient VectorizedRowBatch rowBatch;

Review comment:
       I think that this should not be a field in this class, but (potentially) 
part of the `OrcBulkWriter`.  It is the writer that will handle the lifecycle 
of the batch in any case.




----------------------------------------------------------------
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]


Reply via email to