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

ARROW-17525: [Java] Read ORC files using NativeDatasetFactory #13973

Merged
merged 2 commits into from
Sep 7, 2022
Merged
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
32 changes: 32 additions & 0 deletions java/dataset/pom.xml
Original file line number Diff line number Diff line change
Expand Up @@ -109,6 +109,38 @@
<artifactId>jackson-databind</artifactId>
<scope>test</scope>
</dependency>
<dependency>
<groupId>org.apache.arrow.orc</groupId>
<artifactId>arrow-orc</artifactId>
Copy link
Member

Choose a reason for hiding this comment

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

Is this dependency actually used? I see the ORC library but not the Arrow ORC adapter

Copy link
Contributor Author

Choose a reason for hiding this comment

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

@lidavidm sure! If comment this dependency: java.lang.NoClassDefFoundError: org/apache/hadoop/hdfs/client/HdfsDataOutputStream$SyncFlag
image

Copy link
Member

Choose a reason for hiding this comment

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

It seems like that's just a transitive dependency that needs to get pulled in…but since this is a test-only dependency it's ok then.

<version>${project.version}</version>
<scope>test</scope>
</dependency>
<dependency>
<groupId>org.apache.orc</groupId>
<artifactId>orc-core</artifactId>
<version>1.7.6</version>
<scope>test</scope>
<exclusions>
<exclusion>
<groupId>log4j</groupId>
<artifactId>log4j</artifactId>
</exclusion>
<exclusion>
<groupId>org.slf4j</groupId>
<artifactId>slf4j-log4j12</artifactId>
</exclusion>
<exclusion>
<groupId>commons-logging</groupId>
<artifactId>commons-logging</artifactId>
</exclusion>
</exclusions>
</dependency>
<dependency>
<groupId>org.apache.hive</groupId>
<artifactId>hive-storage-api</artifactId>
<version>2.8.1</version>
<scope>test</scope>
</dependency>
</dependencies>
<build>
<resources>
Expand Down
2 changes: 2 additions & 0 deletions java/dataset/src/main/cpp/jni_wrapper.cc
Original file line number Diff line number Diff line change
Expand Up @@ -91,6 +91,8 @@ arrow::Result<std::shared_ptr<arrow::dataset::FileFormat>> GetFileFormat(
return std::make_shared<arrow::dataset::ParquetFileFormat>();
case 1:
return std::make_shared<arrow::dataset::IpcFileFormat>();
case 2:
return std::make_shared<arrow::dataset::OrcFileFormat>();
default:
std::string error_message =
"illegal file format id: " + std::to_string(file_format_id);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -23,6 +23,7 @@
public enum FileFormat {
PARQUET(0),
ARROW_IPC(1),
ORC(2),
NONE(-1);

private final int id;
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,42 @@
/*
* 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.arrow.dataset;

import java.io.IOException;

import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hive.ql.exec.vector.LongColumnVector;
import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch;
import org.apache.orc.OrcFile;
import org.apache.orc.TypeDescription;
import org.apache.orc.Writer;

public class OrcWriteSupport {
public static void writeTempFile(TypeDescription orcSchema, Path path, Integer[] values) throws IOException {
Writer writer = OrcFile.createWriter(path, OrcFile.writerOptions(new Configuration()).setSchema(orcSchema));
VectorizedRowBatch batch = orcSchema.createRowBatch();
LongColumnVector longColumnVector = (LongColumnVector) batch.cols[0];
for (int idx = 0; idx < values.length; idx++) {
longColumnVector.vector[idx] = values[idx];
}
batch.size = values.length;
writer.addRowBatch(batch);
writer.close();
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -37,6 +37,7 @@
import java.util.concurrent.Executors;
import java.util.stream.Collectors;

import org.apache.arrow.dataset.OrcWriteSupport;
import org.apache.arrow.dataset.ParquetWriteSupport;
import org.apache.arrow.dataset.jni.NativeDataset;
import org.apache.arrow.dataset.jni.NativeInstanceReleasedException;
Expand All @@ -59,6 +60,8 @@
import org.apache.arrow.vector.types.pojo.Schema;
import org.apache.avro.generic.GenericRecord;
import org.apache.avro.generic.GenericRecordBuilder;
import org.apache.hadoop.fs.Path;
import org.apache.orc.TypeDescription;
import org.junit.Assert;
import org.junit.ClassRule;
import org.junit.Test;
Expand Down Expand Up @@ -357,6 +360,34 @@ public void testBaseArrowIpcRead() throws Exception {
AutoCloseables.close(factory);
}

@Test
public void testBaseOrcRead() throws Exception {
String dataName = "test-orc";
String basePath = TMP.getRoot().getAbsolutePath();

TypeDescription orcSchema = TypeDescription.fromString("struct<ints:int>");
Path path = new Path(basePath, dataName);
OrcWriteSupport.writeTempFile(orcSchema, path, new Integer[]{Integer.MIN_VALUE, Integer.MAX_VALUE});

String orcDatasetUri = new File(basePath, dataName).toURI().toString();
FileSystemDatasetFactory factory = new FileSystemDatasetFactory(rootAllocator(), NativeMemoryPool.getDefault(),
Copy link
Contributor

Choose a reason for hiding this comment

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

LGTM, only have this comment:

  1. There is a Jira ticket in case to use NativeMemoryPool.createListenable for big size data. Do you know if there are some limitation/restriction for big ORC files also?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

I don't know any things related to NativeMemoryPool.createListenable. It looks like common & separate Java Dataset API issue not introduced by this PR

Copy link
Member

Choose a reason for hiding this comment

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

That ticket is entirely unrelated, I agree. (I added some comments on Jira.)

Copy link
Contributor

Choose a reason for hiding this comment

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

Thanks for the clarification

FileFormat.ORC, orcDatasetUri);
ScanOptions options = new ScanOptions(100);
Schema schema = inferResultSchemaFromFactory(factory, options);
List<ArrowRecordBatch> datum = collectResultFromFactory(factory, options);

assertSingleTaskProduced(factory, options);
assertEquals(1, datum.size());
assertEquals(1, schema.getFields().size());
assertEquals("ints", schema.getFields().get(0).getName());

String expectedJsonUnordered = "[[2147483647], [-2147483648]]";
checkParquetReadResult(schema, expectedJsonUnordered, datum);

AutoCloseables.close(datum);
AutoCloseables.close(factory);
}

private void checkParquetReadResult(Schema schema, String expectedJson, List<ArrowRecordBatch> actual)
throws IOException {
final ObjectMapper json = new ObjectMapper();
Expand Down