-
Notifications
You must be signed in to change notification settings - Fork 3.5k
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
Changes from all commits
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -23,6 +23,7 @@ | |
public enum FileFormat { | ||
PARQUET(0), | ||
ARROW_IPC(1), | ||
ORC(2), | ||
NONE(-1); | ||
|
||
private final int id; | ||
|
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 |
---|---|---|
|
@@ -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; | ||
|
@@ -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; | ||
|
@@ -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(), | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. LGTM, only have this comment:
There was a problem hiding this comment. Choose a reason for hiding this commentThe 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 There was a problem hiding this comment. Choose a reason for hiding this commentThe 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.) There was a problem hiding this comment. Choose a reason for hiding this commentThe 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(); | ||
|
There was a problem hiding this comment.
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
There was a problem hiding this comment.
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
There was a problem hiding this comment.
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.