Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

[FLINK-27805][Connectors/ORC] bump orc version to 1.7.5 #19844

Closed
wants to merge 4 commits into from
Closed
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension


Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
12 changes: 12 additions & 0 deletions flink-formats/flink-orc-nohive/pom.xml
Original file line number Diff line number Diff line change
Expand Up @@ -82,6 +82,18 @@ under the License.
</exclusions>
</dependency>

<dependency>
<groupId>org.apache.hadoop</groupId>
<artifactId>hadoop-common</artifactId>
<scope>provided</scope>
</dependency>

<dependency>
<groupId>org.apache.hadoop</groupId>
<artifactId>hadoop-hdfs</artifactId>
<scope>provided</scope>
</dependency>

<!-- Tests -->

<dependency>
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -20,7 +20,7 @@

import org.apache.flink.api.common.serialization.BulkWriter;
import org.apache.flink.core.fs.FSDataOutputStream;
import org.apache.flink.orc.nohive.writer.NoHivePhysicalWriterImpl;
import org.apache.flink.orc.writer.HadoopNoCloseStream;
import org.apache.flink.table.data.RowData;
import org.apache.flink.table.types.logical.DecimalType;
import org.apache.flink.table.types.logical.LocalZonedTimestampType;
Expand All @@ -31,7 +31,9 @@
import org.apache.hadoop.fs.Path;
import org.apache.orc.OrcFile;
import org.apache.orc.TypeDescription;
import org.apache.orc.impl.PhysicalFsWriter;
import org.apache.orc.impl.WriterImpl;
import org.apache.orc.impl.writer.WriterEncryptionVariant;
import org.apache.orc.storage.common.type.HiveDecimal;
import org.apache.orc.storage.ql.exec.vector.BytesColumnVector;
import org.apache.orc.storage.ql.exec.vector.ColumnVector;
Expand Down Expand Up @@ -65,7 +67,10 @@ public BulkWriter<RowData> create(FSDataOutputStream out) throws IOException {
OrcFile.WriterOptions opts = OrcFile.writerOptions(new Properties(), conf);
TypeDescription description = TypeDescription.fromString(schema);
opts.setSchema(description);
opts.physicalWriter(new NoHivePhysicalWriterImpl(out, opts));

HadoopNoCloseStream hadoopOutputStream = new HadoopNoCloseStream(out, null);
opts.physicalWriter(
new PhysicalFsWriter(hadoopOutputStream, opts, new WriterEncryptionVariant[0]));
WriterImpl writer = new WriterImpl(null, new Path("."), opts);

VectorizedRowBatch rowBatch = description.createRowBatch();
Expand Down

This file was deleted.

Original file line number Diff line number Diff line change
Expand Up @@ -47,11 +47,11 @@ protected void prepareReadFileWithTypes(String file, int rowSize) throws IOExcep
TypeDescription schema =
TypeDescription.fromString(
"struct<"
+ "f0:float,"
+ "f1:double,"
+ "f2:timestamp,"
+ "f3:tinyint,"
+ "f4:smallint"
+ "_col0:float,"
+ "_col1:double,"
+ "_col2:timestamp,"
+ "_col3:tinyint,"
+ "_col4:smallint"
+ ">");

org.apache.hadoop.fs.Path filePath = new org.apache.hadoop.fs.Path(file);
Expand Down Expand Up @@ -105,7 +105,9 @@ protected OrcColumnarRowSplitReader createReader(
throws IOException {
return OrcNoHiveSplitReaderUtil.genPartColumnarRowReader(
new Configuration(),
IntStream.range(0, fullTypes.length).mapToObj(i -> "f" + i).toArray(String[]::new),
IntStream.range(0, fullTypes.length)
.mapToObj(i -> "_col" + i)
.toArray(String[]::new),
fullTypes,
partitionSpec,
selectedFields,
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,43 @@
/*
* 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.hadoop.fs.FSDataOutputStream;
import org.apache.hadoop.fs.FileSystem;

import java.io.IOException;
import java.io.OutputStream;

/**
* This class is designed to not close the underlying flink stream to avoid exceptions when
* checkpointing.
*/
public class HadoopNoCloseStream extends FSDataOutputStream {
Copy link

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

This looks a bit scary, @gyfora?

Copy link

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@liujiawinds could you please clarify why we need the HadoopNoCloseStream here?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@morhidi Because BulkWriter needs to rely on a flink FSDataOutputStream, and ORC writer uses hadoop FSDataOutputStream. So I wrapped flink FSDataOutputStream.
Additionally, BulkWriter closes the underlying ORC writer stream at checkpoint, which will cause flink to throw a ClosedChannelException if the close action is passed to flink FSDataOutputStream.

Copy link

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

qq: Is this covered in a test case somewhere?


public HadoopNoCloseStream(OutputStream out, FileSystem.Statistics stats) throws IOException {
super(out, stats);
}

@Override
public void close() throws IOException {
// Don't close the internal stream here to avoid
// Stream Closed or ClosedChannelException when Flink performs checkpoint.
// noop
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -27,7 +27,9 @@
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.Path;
import org.apache.orc.OrcFile;
import org.apache.orc.impl.PhysicalFsWriter;
import org.apache.orc.impl.WriterImpl;
import org.apache.orc.impl.writer.WriterEncryptionVariant;

import java.io.IOException;
import java.util.HashMap;
Expand Down Expand Up @@ -96,7 +98,9 @@ public OrcBulkWriterFactory(
@Override
public BulkWriter<T> create(FSDataOutputStream out) throws IOException {
OrcFile.WriterOptions opts = getWriterOptions();
opts.physicalWriter(new PhysicalWriterImpl(out, opts));
HadoopNoCloseStream hadoopOutputStream = new HadoopNoCloseStream(out, null);
opts.physicalWriter(
new PhysicalFsWriter(hadoopOutputStream, opts, new WriterEncryptionVariant[0]));

// The path of the Writer is not used to indicate the destination file
// in this case since we have used a dedicated physical writer to write
Expand Down
Loading