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-17303: [Java][Dataset] Read Arrow IPC files by NativeDatasetFactory (#13760) #13811

Merged
merged 5 commits into from
Aug 8, 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
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 @@ -89,6 +89,8 @@ arrow::Result<std::shared_ptr<arrow::dataset::FileFormat>> GetFileFormat(
switch (file_format_id) {
case 0:
return std::make_shared<arrow::dataset::ParquetFileFormat>();
case 1:
return std::make_shared<arrow::dataset::IpcFileFormat>();
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 @@ -22,6 +22,7 @@
*/
public enum FileFormat {
PARQUET(0),
ARROW_IPC(1),
NONE(-1);

private final int id;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -23,6 +23,7 @@
import static org.junit.jupiter.api.Assertions.assertThrows;

import java.io.File;
import java.io.FileOutputStream;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Collections;
Expand All @@ -46,11 +47,15 @@
import org.apache.arrow.dataset.scanner.ScanOptions;
import org.apache.arrow.util.AutoCloseables;
import org.apache.arrow.vector.FieldVector;
import org.apache.arrow.vector.IntVector;
import org.apache.arrow.vector.VectorLoader;
import org.apache.arrow.vector.VectorSchemaRoot;
import org.apache.arrow.vector.ipc.ArrowFileWriter;
import org.apache.arrow.vector.ipc.ArrowReader;
import org.apache.arrow.vector.ipc.message.ArrowRecordBatch;
import org.apache.arrow.vector.types.Types;
import org.apache.arrow.vector.types.pojo.ArrowType;
import org.apache.arrow.vector.types.pojo.Field;
import org.apache.arrow.vector.types.pojo.Schema;
import org.apache.avro.generic.GenericRecord;
import org.apache.avro.generic.GenericRecordBuilder;
Expand Down Expand Up @@ -316,6 +321,42 @@ public void testErrorThrownWhenIterateOnIteratorAfterTaskClose() throws Exceptio
AutoCloseables.close(factory);
}

@Test
public void testBaseArrowIpcRead() throws Exception {
File dataFile = TMP.newFile();
Schema sourceSchema = new Schema(Collections.singletonList(Field.nullable("ints", new ArrowType.Int(32, true))));
try (VectorSchemaRoot root = VectorSchemaRoot.create(sourceSchema, rootAllocator());
FileOutputStream sink = new FileOutputStream(dataFile);
ArrowFileWriter writer = new ArrowFileWriter(root, /*dictionaryProvider=*/null, sink.getChannel())) {
IntVector ints = (IntVector) root.getVector(0);
ints.setSafe(0, 0);
ints.setSafe(1, 1024);
ints.setSafe(2, Integer.MAX_VALUE);
root.setRowCount(3);
writer.start();
writer.writeBatch();
writer.end();
}

String arrowDataURI = dataFile.toURI().toString();
FileSystemDatasetFactory factory = new FileSystemDatasetFactory(rootAllocator(), NativeMemoryPool.getDefault(),
FileFormat.ARROW_IPC, arrowDataURI);
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 = String.format("[[0],[1024],[%d]]", Integer.MAX_VALUE);
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