x-tong commented on code in PR #1930:
URL: https://github.com/apache/auron/pull/1930#discussion_r2708816549


##########
auron-flink-extension/auron-flink-runtime/src/main/java/org/apache/auron/flink/arrow/FlinkArrowWriter.java:
##########
@@ -0,0 +1,245 @@
+/*
+ * 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.auron.flink.arrow;
+
+import java.util.List;
+import org.apache.arrow.memory.BufferAllocator;
+import org.apache.arrow.vector.FieldVector;
+import org.apache.arrow.vector.ValueVector;
+import org.apache.arrow.vector.VectorSchemaRoot;
+import org.apache.arrow.vector.complex.ListVector;
+import org.apache.arrow.vector.complex.MapVector;
+import org.apache.arrow.vector.complex.StructVector;
+import org.apache.arrow.vector.types.pojo.Schema;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.types.logical.ArrayType;
+import org.apache.flink.table.types.logical.BigIntType;
+import org.apache.flink.table.types.logical.BinaryType;
+import org.apache.flink.table.types.logical.BooleanType;
+import org.apache.flink.table.types.logical.CharType;
+import org.apache.flink.table.types.logical.DateType;
+import org.apache.flink.table.types.logical.DecimalType;
+import org.apache.flink.table.types.logical.DoubleType;
+import org.apache.flink.table.types.logical.FloatType;
+import org.apache.flink.table.types.logical.IntType;
+import org.apache.flink.table.types.logical.LocalZonedTimestampType;
+import org.apache.flink.table.types.logical.LogicalType;
+import org.apache.flink.table.types.logical.MapType;
+import org.apache.flink.table.types.logical.NullType;
+import org.apache.flink.table.types.logical.RowType;
+import org.apache.flink.table.types.logical.SmallIntType;
+import org.apache.flink.table.types.logical.TimeType;
+import org.apache.flink.table.types.logical.TimestampType;
+import org.apache.flink.table.types.logical.TinyIntType;
+import org.apache.flink.table.types.logical.VarBinaryType;
+import org.apache.flink.table.types.logical.VarCharType;
+
+/** Writer that converts Flink RowData to Arrow VectorSchemaRoot. */
+public class FlinkArrowWriter {
+
+    private final VectorSchemaRoot root;
+    private final FlinkArrowFieldWriter[] fieldWriters;
+
+    private FlinkArrowWriter(VectorSchemaRoot root, FlinkArrowFieldWriter[] 
fieldWriters) {
+        this.root = root;
+        this.fieldWriters = fieldWriters;
+    }
+
+    /**
+     * Creates a FlinkArrowWriter from a Flink RowType.
+     *
+     * @param rowType The Flink row type
+     * @return A new FlinkArrowWriter instance
+     */
+    public static FlinkArrowWriter create(RowType rowType) {
+        Schema arrowSchema = FlinkArrowUtils.toArrowSchema(rowType);
+        VectorSchemaRoot root = VectorSchemaRoot.create(arrowSchema, 
FlinkArrowUtils.ROOT_ALLOCATOR);
+        return create(root, rowType);
+    }

Review Comment:
   Spark's ArrowWriter does not implement AutoCloseable either; this is a 
design choice: the Writer is just a data converter, does not own resources; the 
resources (VectorSchemaRoot) are managed by their owner (Exporter).



-- 
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: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]

Reply via email to