[
https://issues.apache.org/jira/browse/FLINK-9407?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16536571#comment-16536571
]
ASF GitHub Bot commented on FLINK-9407:
---------------------------------------
Github user zhangminglei commented on a diff in the pull request:
https://github.com/apache/flink/pull/6075#discussion_r200890648
--- Diff:
flink-connectors/flink-orc/src/main/java/org/apache/flink/orc/OrcFileWriter.java
---
@@ -0,0 +1,269 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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;
+
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.streaming.connectors.fs.StreamWriterBase;
+import org.apache.flink.streaming.connectors.fs.Writer;
+import org.apache.flink.table.api.TableSchema;
+import org.apache.flink.types.Row;
+
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hive.ql.exec.vector.BytesColumnVector;
+import org.apache.hadoop.hive.ql.exec.vector.DoubleColumnVector;
+import org.apache.hadoop.hive.ql.exec.vector.LongColumnVector;
+import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch;
+import org.apache.orc.CompressionKind;
+import org.apache.orc.OrcFile;
+import org.apache.orc.TypeDescription;
+
+import java.io.IOException;
+import java.nio.charset.StandardCharsets;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+import java.util.Objects;
+import java.util.stream.IntStream;
+
+import static org.apache.flink.orc.OrcBatchReader.schemaToTypeInfo;
+
+/**
+ * A {@link Writer} that writes the bucket files as Hadoop {@link OrcFile}.
+ *
+ * @param <T> The type of the elements that are being written by the sink.
+ */
+public class OrcFileWriter<T extends Row> extends StreamWriterBase<T> {
+
+ private static final long serialVersionUID = 3L;
+
+ /**
+ * The description of the types in an ORC file.
+ */
+ private TypeDescription schema;
+
+ /**
+ * The schema of an ORC file.
+ */
+ private String metaSchema;
+
+ /**
+ * A row batch that will be written to the ORC file.
+ */
+ private VectorizedRowBatch rowBatch;
+
+ /**
+ * The writer that fill the records into the batch.
+ */
+ private OrcBatchWriter orcBatchWriter;
+
+ private transient org.apache.orc.Writer writer;
+
+ private CompressionKind compressionKind;
+
+ /**
+ * The number of rows that currently being written.
+ */
+ private long writedRowSize;
+
+ /**
+ * Creates a new {@code OrcFileWriter} that writes orc files without
compression.
+ *
+ * @param metaSchema The orc schema.
+ */
+ public OrcFileWriter(String metaSchema) {
+ this(metaSchema, CompressionKind.NONE);
+ }
+
+ /**
+ * Create a new {@code OrcFileWriter} that writes orc file with the
gaven
+ * schema and compression kind.
+ *
+ * @param metaSchema The schema of an orc file.
+ * @param compressionKind The compression kind to use.
+ */
+ public OrcFileWriter(String metaSchema, CompressionKind
compressionKind) {
+ this.metaSchema = metaSchema;
+ this.schema = TypeDescription.fromString(metaSchema);
+ this.compressionKind = compressionKind;
+ }
+
+ @Override
+ public void open(FileSystem fs, Path path) throws IOException {
+ writer = OrcFile.createWriter(path,
OrcFile.writerOptions(fs.getConf()).setSchema(schema).compress(compressionKind));
+ rowBatch = schema.createRowBatch();
+ orcBatchWriter = new
OrcBatchWriter(Arrays.asList(orcSchemaToTableSchema(schema).getTypes()));
+ }
+
+ private TableSchema orcSchemaToTableSchema(TypeDescription orcSchema) {
+ List<String> fieldNames = orcSchema.getFieldNames();
+ List<TypeDescription> typeDescriptions =
orcSchema.getChildren();
+ List<TypeInformation> typeInformations = new ArrayList<>();
+
+ typeDescriptions.forEach(typeDescription -> {
+ typeInformations.add(schemaToTypeInfo(typeDescription));
+ });
+
+ return new TableSchema(
+ fieldNames.toArray(new String[fieldNames.size()]),
+ typeInformations.toArray(new
TypeInformation[typeInformations.size()]));
+ }
+
+ @Override
+ public void write(T element) throws IOException {
+ Boolean isFill = orcBatchWriter.fill(rowBatch, element);
+ if (!isFill) {
+ writer.addRowBatch(rowBatch);
+ writedRowSize += writedRowSize + rowBatch.size;
+ rowBatch.reset();
+ }
+ }
+
+ @Override
+ public long flush() throws IOException {
+ writer.addRowBatch(rowBatch);
+ writedRowSize += rowBatch.size;
+ rowBatch.reset();
+ return writedRowSize;
+ }
+
+ @Override
+ public long getPos() throws IOException {
+ return writedRowSize;
+ }
+
+ @Override
+ public Writer<T> duplicate() {
+ return new OrcFileWriter<>(metaSchema, compressionKind);
+ }
+
+ @Override
+ public void close() throws IOException {
+ flush();
+ if (rowBatch.size != 0) {
+ writer.addRowBatch(rowBatch);
+ rowBatch.reset();
+ }
+ writer.close();
+ super.close();
+ }
+
+ @Override
+ public int hashCode() {
+ return Objects.hash(super.hashCode(), schema, metaSchema,
rowBatch, orcBatchWriter);
+ }
+
+ @Override
+ public boolean equals(Object other) {
+ if (this == other) {
+ return true;
+ }
+ if (other == null) {
+ return false;
+ }
+ if (getClass() != other.getClass()) {
+ return false;
+ }
+ OrcFileWriter<T> writer = (OrcFileWriter<T>) other;
+ return Objects.equals(schema, writer.schema)
+ && Objects.equals(metaSchema, writer.metaSchema)
+ && Objects.equals(rowBatch, writer.rowBatch)
+ && Objects.equals(orcBatchWriter,
writer.orcBatchWriter);
+ }
+
+ /**
+ * The writer that fill the records into the batch.
+ */
+ private class OrcBatchWriter {
+
+ private List<TypeInformation> typeInfos;
+
+ public OrcBatchWriter(List<TypeInformation> typeInfos) {
+ this.typeInfos = typeInfos;
+ }
+
+ /**
+ * Fill the record into {@code VectorizedRowBatch}, return
false if batch is full.
+ *
+ * @param batch An reusable orc VectorizedRowBatch.
+ * @param record Input record.
+ * @return Return false if batch is full.
+ */
+ private boolean fill(VectorizedRowBatch batch, T record) {
+ // If the batch is full, write it out and start over.
+ if (batch.size == batch.getMaxSize()) {
+ return false;
+ } else {
+ IntStream.range(0, typeInfos.size()).forEach(
+ index -> {
+
setColumnVectorValueByType(typeInfos.get(index), batch, index, batch.size,
record);
--- End diff --
What do you mean "first level schema" ? This PR does not support complex
data structure at the moment.
> Support orc rolling sink writer
> -------------------------------
>
> Key: FLINK-9407
> URL: https://issues.apache.org/jira/browse/FLINK-9407
> Project: Flink
> Issue Type: New Feature
> Components: filesystem-connector
> Reporter: zhangminglei
> Assignee: zhangminglei
> Priority: Major
> Labels: pull-request-available
>
> Currently, we only support {{StringWriter}}, {{SequenceFileWriter}} and
> {{AvroKeyValueSinkWriter}}. I would suggest add an orc writer for rolling
> sink.
> Below, FYI.
> I tested the PR and verify the results with spark sql. Obviously, we can get
> the results of what we had written down before. But I will give more tests in
> the next couple of days. Including the performance under compression with
> short checkpoint intervals. And more UTs.
> {code:java}
> scala> spark.read.orc("hdfs://10.199.196.0:9000/data/hive/man/2018-07-06--21")
> res1: org.apache.spark.sql.DataFrame = [name: string, age: int ... 1 more
> field]
> scala>
> scala> res1.registerTempTable("tablerice")
> warning: there was one deprecation warning; re-run with -deprecation for
> details
> scala> spark.sql("select * from tablerice")
> res3: org.apache.spark.sql.DataFrame = [name: string, age: int ... 1 more
> field]
> scala> res3.show(3)
> +-----+---+-------+
> | name|age|married|
> +-----+---+-------+
> |Sagar| 26| false|
> |Sagar| 30| false|
> |Sagar| 34| false|
> +-----+---+-------+
> only showing top 3 rows
> {code}
--
This message was sent by Atlassian JIRA
(v7.6.3#76005)