diff --git a/java/hive/README.rst b/java/hive/README.rst index 59dbde8ba..a579a4c26 100644 --- a/java/hive/README.rst +++ b/java/hive/README.rst @@ -32,7 +32,7 @@ In this repo, we set ```hive.default.fileformat``` as ```Vineyard``` and set ``` If you want to use local file system or HDFS, you need to change the configuration or point out the storage format when creating table. -- Create table and insert some data: +- Create table as textfile and insert some data: .. code:: sql @@ -41,7 +41,7 @@ creating table. a string, b int) stored as TEXTFILE - LOCATION "file:///opt/hive/data/warehouse/hive_example"; + location "file:///opt/hive/data/warehouse/hive_example"; insert into hive_example values('a', 1), ('a', 2), ('b',3); select count(distinct a) from hive_example; @@ -56,7 +56,16 @@ creating table. Hive and Vineyard ----------------- -- Create hive table on vineyard (using :code:`file:///` is enough as we won't touch filesystem input/output format): +- Start vineyard server: + + The socket file must be placed in the correct directory. Please refer to the docker-compose.yml file for details. + You can change the socket file path as you like and change the docker-compose.yml file accordingly. + + .. code:: bash + + vineyardd --socket=./vineyard/vineyard.sock --meta=local + +- Create hive table on vineyard: .. code:: sql @@ -64,6 +73,7 @@ Hive and Vineyard a string, b int); describe formatted hive_example; + drop table hive_example; - Create table and select @@ -76,8 +86,9 @@ Hive and Vineyard select * from hive_example2; explain vectorization only select * from hive_example2; + drop table hive_example2; -- Vectorized Input (and output, currently unavaliabe): +- Vectorized input and output(Currently unavaliabe): .. code:: sql @@ -107,6 +118,7 @@ Hive and Vineyard explain vectorization select * from hive_example; insert into hive_example values(1, 1), (2, 2), (3,3); + drop table hive_example; - Test large data sets: @@ -125,19 +137,9 @@ Hive and Vineyard row format serde 'org.apache.hadoop.hive.serde2.OpenCSVSerde' stored as textfile; load data local inpath "file:///opt/hive/data/warehouse/soc-livejournal.csv" into table hive_test_data_livejournal; - insert into hive_example3 select * from hive_test_data_livejournal; - -- Test output format: - - .. code:: sql - - create table hive_example_orc( - field_1 int, - field_2 int) - stored as orc - LOCATION "file:///opt/hive/data/warehouse/hive_example_orc"; - insert into hive_example values(1, 1), (2, 2), (3, 3); - explain vectorization select * from hive_example_orc; + insert into hive_example3 select * from hive_test_data_livejournal; + drop table hive_test_data_livejournal; + select * from hive_example3; - Test static partition: @@ -153,6 +155,7 @@ Hive and Vineyard select * from hive_static_partition; select * from hive_static_partition where value=666; select * from hive_static_partition where value=114514; + drop table hive_static_partition; - Test dynamic partition: @@ -163,7 +166,7 @@ Hive and Vineyard dst_id int, year int) stored as TEXTFILE - LOCATION "file:///opt/hive/data/warehouse/hive_dynamic_partition_data"; + location "file:///opt/hive/data/warehouse/hive_dynamic_partition_data"; insert into table hive_dynamic_partition_data values (1, 2, 2018),(3, 4, 2018),(1, 2, 2017); create table hive_dynamic_partition_test @@ -173,6 +176,73 @@ Hive and Vineyard )partitioned by(mounth int, year int); insert into table hive_dynamic_partition_test partition(mounth=1, year) select src_id,dst_id,year from hive_dynamic_partition_data; select * from hive_dynamic_partition_test; + drop table hive_dynamic_partition_test; + drop table hive_dynamic_partition_data; + +- Test all primitive types: + + Now vineyard support to store tinyint, smallint, int, bigint, boolean, string, float, double, date, timestamp, binary and decimal. + + .. code:: sql + + create table test_all_primitive_types ( + field_1 tinyint, + field_2 smallint, + field_3 bigint, + field_4 int, + field_5 double, + field_6 float, + field_7 string, + field_9 varchar(10), + field_10 char(10), + field_8 binary, + field_11 date, + field_12 boolean, + field_13 timestamp, + field_14 decimal(6, 2) + ); + + insert into test_all_primitive_types select + tinyint(1), + smallint(1), + 42, + bigint(1), + double(2.0), + float(1.0), + 'hello world1!', + 'hello world2!', + 'hello world3!', + cast('hello world4!' as binary), + date('2023-12-31'), + true, + timestamp('2023-12-31 23:59:59'), + cast(1234.56 as decimal); + + select * from test_all_primitive_types; + drop table test_all_primitive_types; + +- Test nested types: + + Now vineyard support to store array, map and struct. + + .. code:: sql + + CREATE TABLE nested_table ( + field_1 map>> + ); + + insert INTO nested_table select + map( + 42, + array(named_struct('field_1', 1, + 'field_2', 'hello'), + named_struct('field_1', 2, + 'field_2', 'world!'))); + + select * from nested_table; + drop table nested_table; Connect to Hive from Spark -------------------------- diff --git a/java/hive/src/main/java/io/v6d/hadoop/fs/FileSystem.java b/java/hive/src/main/java/io/v6d/hadoop/fs/FileSystem.java index d5e3da723..761e2027b 100644 --- a/java/hive/src/main/java/io/v6d/hadoop/fs/FileSystem.java +++ b/java/hive/src/main/java/io/v6d/hadoop/fs/FileSystem.java @@ -15,12 +15,12 @@ package io.v6d.hadoop.fs; import com.google.common.base.StopwatchContext; -import com.google.common.jimfs.Jimfs; import io.v6d.core.client.Context; import io.v6d.core.client.IPCClient; import io.v6d.core.client.ds.ObjectFactory; import io.v6d.core.client.ds.ObjectMeta; import io.v6d.core.common.util.ObjectID; +import io.v6d.core.common.util.VineyardException.ObjectNotExists; import io.v6d.hive.ql.io.CloseableReentrantLock; import io.v6d.modules.basic.arrow.SchemaBuilder; import io.v6d.modules.basic.arrow.Table; @@ -31,9 +31,6 @@ import java.nio.ByteBuffer; import java.nio.channels.FileChannel; import java.nio.charset.StandardCharsets; -import java.nio.file.DirectoryStream; -import java.nio.file.Files; -import java.nio.file.StandardOpenOption; import java.util.ArrayList; import java.util.Collection; import java.util.EnumSet; @@ -45,7 +42,6 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.*; import org.apache.hadoop.fs.permission.FsPermission; -import org.apache.hadoop.hive.conf.HiveConf; import org.apache.hadoop.io.DataOutputBuffer; import org.apache.hadoop.util.Progressable; import org.slf4j.Logger; @@ -128,8 +124,9 @@ public class FileSystem extends org.apache.hadoop.fs.FileSystem { static final CloseableReentrantLock lock = new CloseableReentrantLock(); private Configuration conf = null; - static java.nio.file.FileSystem jimfs = null; + static RawLocalFileSystem fs = null; static boolean enablePrintAllFiles = false; + static boolean enablePrintAllObjects = false; Path workingDir = new Path("vineyard:/"); @@ -137,30 +134,45 @@ public FileSystem() { super(); } - public static void printAllFiles(java.nio.file.Path root, java.nio.file.FileSystem fs) - throws IOException { - DirectoryStream stream = Files.newDirectoryStream(root); - Queue queue = new java.util.LinkedList(); - for (java.nio.file.Path p : stream) { - queue.add(p); + private static void printAllFiles(Path p) throws IOException { + FileStatus[] status = fs.listStatus(p); + Queue queue = new java.util.LinkedList(); + for (FileStatus s : status) { + queue.add(s); } while (!queue.isEmpty()) { - java.nio.file.Path p = queue.poll(); - Context.println(p.toString()); - if (Files.isDirectory(p)) { - DirectoryStream streamTemp = Files.newDirectoryStream(p); - for (java.nio.file.Path p1 : streamTemp) { - queue.add(p1); + FileStatus p1 = queue.poll(); + Context.println(p1.getPath().toString()); + if (p1.isDirectory()) { + FileStatus[] statusTemp = fs.listStatus(p1.getPath()); + for (FileStatus s : statusTemp) { + queue.add(s); } - streamTemp.close(); } } - stream.close(); + } + + private void printAllObjectsWithName() throws IOException { + if (enablePrintAllObjects) { + IPCClient client = Context.getClient(); + Context.println("print all objects with name"); + Context.println("===================================="); + Map objects = client.listNames(".*", true, 255); + for (val object : objects.entrySet()) { + Context.println( + "object name:" + + object.getKey() + + ", object id:" + + object.getValue().value()); + } + Context.println("===================================="); + } } private static void printAllFiles() throws IOException { if (enablePrintAllFiles) { - printAllFiles(jimfs.getPath("/"), jimfs); + Context.println("------------------"); + printAllFiles(new Path("/opt/hive/data/warehouse")); } } @@ -185,27 +197,21 @@ protected URI canonicalizeUri(URI uri) { @Override public void initialize(URI name, Configuration conf) throws IOException { + Context.println("Initialize vineyard file system: " + name.toString()); super.initialize(name, conf); this.conf = conf; this.uri = name; - try { - if (jimfs == null) { - jimfs = Jimfs.newFileSystem(com.google.common.jimfs.Configuration.unix()); - } - } catch (Exception e) { - Context.println("Exception: " + e.getMessage()); - throw e; - } - mkdirs(new Path(uri.toString().replaceAll("///", "/"))); + + fs = new RawLocalFileSystem(); + fs.initialize(URI.create("file:///"), conf); + mkdirs(new Path(uri.toString().replaceAll("/+", "/"))); } @Override public FSDataInputStream open(Path path, int i) throws IOException { - FileChannel channel = - FileChannel.open( - jimfs.getPath(path.toString().substring(path.toString().indexOf(":") + 1)), - StandardOpenOption.READ); - return new FSDataInputStream(new VineyardInputStream(channel)); + Path newPath = new Path(path.toString().replaceAll("vineyard", "file")); + FSDataInputStream result = fs.open(newPath); + return result; } @Override @@ -239,16 +245,29 @@ private FSDataOutputStream createInternal( long blockSize, Progressable progressable) throws IOException { - java.nio.file.Path nioFilePath = - jimfs.getPath(path.toString().substring(path.toString().indexOf(":") + 1)); - java.nio.file.Path nioParentDirPath = nioFilePath.getParent(); - if (nioParentDirPath != null) { - Files.createDirectories(nioParentDirPath); + Path newPath = new Path(path.toString().replaceAll("vineyard", "file")); + Path parentPath = newPath.getParent(); + try { + FileStatus parentStatus = fs.getFileStatus(parentPath); + if (!parentStatus.isDirectory()) { + throw new IOException("Parent path is not a directory:" + parentPath.toString()); + } + } catch (FileNotFoundException e) { + // parent path not exist + Context.println("Parent dir not exists. Create parent dir first!"); + fs.mkdirs(parentPath); } - Files.createFile(nioFilePath); - FileChannel channel = FileChannel.open(nioFilePath, StandardOpenOption.WRITE); printAllFiles(); - return new FSDataOutputStream(new VineyardOutputStream(channel), null); + FSDataOutputStream result = + fs.create( + newPath, + fsPermission, + overwrite, + bufferSize, + replication, + blockSize, + progressable); + return result; } @Override @@ -260,36 +279,53 @@ public FSDataOutputStream append(Path path, int i, Progressable progressable) @Override public boolean delete(Path path, boolean b) throws IOException { try (val lock = this.lock.open()) { - return this.deleteInternal( - jimfs.getPath(path.toString().substring(path.toString().indexOf(":") + 1)), b); + Path newPath = new Path(path.toString().replaceAll("vineyard", "file")); + return this.deleteInternal(newPath, b); } } - private boolean deleteInternal(java.nio.file.Path path, boolean b) throws IOException { - java.nio.file.Path nioFilePath = - jimfs.getPath(path.toString().substring(path.toString().indexOf(":") + 1)); - - // check if the path is a directory - if (Files.isDirectory(nioFilePath)) { - DirectoryStream stream = Files.newDirectoryStream(nioFilePath); - for (java.nio.file.Path p : stream) { - deleteInternal(p, b); - } - stream.close(); - } else { - // drop name - String name = nioFilePath.getFileName().toString(); - IPCClient client = Context.getClient(); + public void cleanObjectInVineyard(Path filePath) throws IOException { + IPCClient client = Context.getClient(); + Queue queue = new java.util.LinkedList(); + Collection objectIDs = new ArrayList(); + queue.add(filePath); + while (!queue.isEmpty()) { + Path path = queue.peek(); try { - client.dropName(name); - } catch (Exception e) { - Context.println("Failed to drop name from vineyard: " + e.getMessage()); + FileStatus fileStatus = fs.getFileStatus(path); + if (fileStatus.isDirectory()) { + FileStatus[] fileStatusArray = fs.listStatus(path); + for (FileStatus s : fileStatusArray) { + if (s.getPath().toString().compareTo(filePath.toString()) == 0) { + continue; + } + queue.add(s.getPath()); + } + } + + String objectName = path.toString().substring(path.toString().indexOf(":") + 1); + ObjectID objectID = client.getName(objectName); + objectIDs.add(objectID); + client.dropName(objectName); + } catch (FileNotFoundException e) { + // file not exist, skip + Context.println("File: " + path.toString() + " not exist."); + continue; + } catch (ObjectNotExists e) { + // object not exist + Context.println("Object of file: " + path.toString() + " not exist."); + continue; + } finally { + queue.poll(); } } - Files.deleteIfExists(nioFilePath); + client.delete(objectIDs, false, false); + printAllObjectsWithName(); + } - printAllFiles(); - return false; + private boolean deleteInternal(Path path, boolean b) throws IOException { + cleanObjectInVineyard(path); + return fs.delete(path, b); } @Override @@ -297,23 +333,24 @@ public boolean rename(Path src, Path dst) throws IOException { try (val lock = this.lock.open()) { val watch = StopwatchContext.create(); val renamed = this.renameInternal(src, dst); - Context.println("filesystem rename uses: " + watch.stop()); + Context.println("Filesystem rename uses: " + watch.stop()); return renamed; } } - private void mergeFile(java.nio.file.Path src, java.nio.file.Path dst) throws IOException { - FileChannel channelSrc = FileChannel.open(src, StandardOpenOption.READ); - FileChannel channelDst = FileChannel.open(dst, StandardOpenOption.READ); + private void mergeFile(Path src, Path dst) throws IOException { + FSDataInputStream srcInput = fs.open(src); + FSDataInputStream dstInput = fs.open(dst); + byte[] objectIDByteArray = new byte[255]; - ByteBuffer bytes = ByteBuffer.allocate(255); - int len = channelSrc.read(bytes); + int len = srcInput.read(objectIDByteArray); String srcObjectIDStr = - new String(bytes.array(), 0, len, StandardCharsets.UTF_8).replaceAll("\n", ""); - bytes = ByteBuffer.allocate(255); - len = channelDst.read(bytes); + new String(objectIDByteArray, 0, len, StandardCharsets.UTF_8).replaceAll("\n", ""); + + objectIDByteArray = new byte[255]; + len = dstInput.read(objectIDByteArray); String dstObjectIDStr = - new String(bytes.array(), 0, len, StandardCharsets.UTF_8).replaceAll("\n", ""); + new String(objectIDByteArray, 0, len, StandardCharsets.UTF_8).replaceAll("\n", ""); ObjectID mergedTableObjectID = null; try { @@ -339,11 +376,10 @@ private void mergeFile(java.nio.file.Path src, java.nio.file.Path dst) throws IO } ObjectMeta meta = mergedTableBuilder.seal(client); - Context.println("record batch size:" + mergedTableBuilder.getBatchSize()); Context.println("Table id in vineyard:" + meta.getId().value()); client.persist(meta.getId()); Context.println("Table persisted, name:" + dst); - client.putName(meta.getId(), dst.toString()); + client.putName(meta.getId(), dst.toString().substring(dst.toString().indexOf(":") + 1)); client.dropName(src.toString()); mergedTableObjectID = meta.getId(); @@ -352,82 +388,95 @@ private void mergeFile(java.nio.file.Path src, java.nio.file.Path dst) throws IO ids.add(srcObjectID); ids.add(dstObjectID); client.delete(ids, false, false); + } catch (ObjectNotExists e) { + // Skip invalid file. } finally { - channelSrc.close(); - channelDst.close(); + srcInput.close(); + dstInput.close(); if (mergedTableObjectID != null) { - channelDst = FileChannel.open(dst, StandardOpenOption.WRITE); + FSDataOutputStream out = fs.create(dst); String mergedTableIDStr = mergedTableObjectID.toString() + "\n"; - bytes = - ByteBuffer.allocate( - mergedTableIDStr.getBytes(StandardCharsets.UTF_8).length); - channelDst.write( - ByteBuffer.wrap(mergedTableIDStr.getBytes(StandardCharsets.UTF_8))); + out.write((mergedTableIDStr + "\n").getBytes(StandardCharsets.UTF_8)); + out.close(); } } } public boolean renameInternal(Path src, Path dst) throws IOException { - // now we create new file and delete old file to simulate rename - java.nio.file.Path srcNioFilePath = - jimfs.getPath(src.toString().substring(src.toString().indexOf(":") + 1)); - java.nio.file.Path dstNioFilePath = - jimfs.getPath(dst.toString().substring(dst.toString().indexOf(":") + 1)); - java.nio.file.Path dstNioParentDirPath = dstNioFilePath.getParent(); - Files.createDirectories(dstNioParentDirPath); - - if (Files.isDirectory(srcNioFilePath)) { - DirectoryStream stream = Files.newDirectoryStream(srcNioFilePath); - for (java.nio.file.Path p : stream) { + + Path newSrc = new Path(src.toString().replaceAll("vineyard", "file")); + Path newDst = new Path(dst.toString().replaceAll("vineyard", "file")); + String newTableName = + dst.toString().substring(dst.toString().indexOf(":") + 1).replaceAll("/+", "/"); + String oldTableName = + src.toString().substring(src.toString().indexOf(":") + 1).replaceAll("/+", "/"); + FileStatus srcStatus; + try { + srcStatus = fs.getFileStatus(newSrc); + } catch (FileNotFoundException e) { + // src file not exist + Context.println("Src file not exist"); + return false; + } + + if (srcStatus.isDirectory()) { + FileStatus[] status = fs.listStatus(newSrc); + for (FileStatus s : status) { renameInternal( - new Path(SCHEME + ":/" + p.toString()), - new Path( - SCHEME + ":/" + dstNioFilePath.toString() + "/" + p.getFileName())); + s.getPath(), new Path(newDst.toString() + "/" + s.getPath().getName())); } - stream.close(); - Files.delete(srcNioFilePath); + fs.delete(newSrc, true); + return true; } else { - if (Files.exists(dstNioFilePath)) { + try { + fs.getFileStatus(newDst); + } catch (FileNotFoundException e) { + // dst file not exist + fs.rename(newSrc, newDst); + FSDataInputStream in = fs.open(newDst); + byte[] objectIDByteArray = new byte[255]; + int len = in.read(objectIDByteArray); + if (len > 0) { + String objectIDStr = + new String(objectIDByteArray, 0, len, StandardCharsets.UTF_8) + .replaceAll("\n", ""); + IPCClient client = Context.getClient(); + try { + client.putName(ObjectID.fromString(objectIDStr), newTableName); + client.dropName(oldTableName); + } catch (Exception e1) { + // Skip some invalid file. + // File content may be not a valid object id. + Context.println("Failed to put name to vineyard: " + e1.getMessage()); + } + } printAllFiles(); - mergeFile(srcNioFilePath, dstNioFilePath); - Files.delete(srcNioFilePath); - } else { - Files.move(srcNioFilePath, dstNioFilePath); - ByteBuffer bytes = ByteBuffer.allocate(255); - FileChannel channel = FileChannel.open(dstNioFilePath, StandardOpenOption.READ); - int len = channel.read(bytes); - String objectIDStr = - new String(bytes.array(), 0, len, StandardCharsets.UTF_8) - .replaceAll("\n", ""); - IPCClient client = Context.getClient(); - client.putName(ObjectID.fromString(objectIDStr), dstNioFilePath.toString()); - client.dropName(srcNioFilePath.toString()); + return true; } - } - printAllFiles(); + // dst file exist + mergeFile(newSrc, newDst); + deleteInternal(newSrc, true); - return true; + printAllFiles(); + return true; + } } - private void syncWithVineyard(String prefix) throws IOException { + public void syncWithVineyard(String prefix) throws IOException { IPCClient client = Context.getClient(); - try { - String reg = "^" + prefix + ".*"; - Map objects = client.listNames(reg, true, 255); - for (val object : objects.entrySet()) { - if (Files.exists(jimfs.getPath(object.getKey()))) { - continue; - } - Files.createFile(jimfs.getPath(object.getKey())); - FileChannel channel = - FileChannel.open(jimfs.getPath(object.getKey()), StandardOpenOption.WRITE); + String reg = "^" + prefix + ".*"; + Map objects = client.listNames(reg, true, 255); + for (val object : objects.entrySet()) { + try { + fs.getFileStatus(new Path("file://" + object.getKey())); + } catch (FileNotFoundException e) { + // file not exist + Path path = new Path("file://" + object.getKey()); + FSDataOutputStream out = fs.create(path); ObjectID id = object.getValue(); - channel.write( - ByteBuffer.wrap((id.toString() + "\n").getBytes(StandardCharsets.UTF_8))); - channel.close(); + out.write((id.toString() + "\n").getBytes(StandardCharsets.UTF_8)); + out.close(); } - } catch (Exception e) { - Context.println("Exception: " + e.getMessage()); } } @@ -435,26 +484,44 @@ private void syncWithVineyard(String prefix) throws IOException { public FileStatus[] listStatus(Path path) throws FileNotFoundException, IOException { List result = new ArrayList(); try (val lock = this.lock.open()) { - java.nio.file.Path nioFilePath = - jimfs.getPath(path.toString().substring(path.toString().indexOf(":") + 1)); - syncWithVineyard(nioFilePath.toString()); - if (Files.isDirectory(nioFilePath)) { - DirectoryStream stream = Files.newDirectoryStream(nioFilePath); - for (java.nio.file.Path p : stream) { - result.add( + String prefix = + path.toString() + .substring(path.toString().indexOf(":") + 1) + .replaceAll("/+", "/"); + syncWithVineyard(prefix); + try { + FileStatus status = + fs.getFileStatus(new Path(path.toString().replaceAll("vineyard", "file"))); + FileStatus[] statusArray = + fs.listStatus(new Path(path.toString().replaceAll("vineyard", "file"))); + for (FileStatus s : statusArray) { + FileStatus temp = new FileStatus( - Files.size(p), - Files.isDirectory(p), - 1, - 1, - 0, - 0, + s.getLen(), + s.isDirectory(), + s.getReplication(), + s.getBlockSize(), + s.getModificationTime(), + s.getAccessTime(), new FsPermission((short) 777), - null, - null, - new Path(SCHEME + ":/" + p.toString()))); + s.getOwner(), + s.getGroup(), + new Path( + SCHEME + + ":///" + + s.getPath() + .toString() + .substring( + s.getPath() + .toString() + .indexOf(":") + + 1) + .replaceAll("/+", "/"))); + result.add(temp); } - stream.close(); + } catch (FileNotFoundException e) { + // file not exist + return new FileStatus[0]; } } printAllFiles(); @@ -479,11 +546,17 @@ public boolean mkdirs(Path path, FsPermission fsPermission) throws IOException { } private boolean mkdirsInternal(Path path, FsPermission fsPermission) throws IOException { - java.nio.file.Path nioDirPath = - jimfs.getPath(path.toString().substring(path.toString().indexOf(":") + 1)); - Files.createDirectories(nioDirPath); - printAllFiles(); - return true; + + Path newPath = new Path(path.toString().replaceAll("vineyard", "file")); + try { + fs.getFileStatus(newPath); + } catch (FileNotFoundException e) { + // file not exist + boolean result = fs.mkdirs(newPath); + printAllFiles(); + return result; + } + return false; } @Override @@ -494,34 +567,29 @@ public FileStatus getFileStatus(Path path) throws IOException { } public FileStatus getFileStatusInternal(Path path) throws IOException { - String pathStr = path.toString().substring(path.toString().indexOf(":") + 1); - java.nio.file.Path nioFilePath = jimfs.getPath(pathStr); - if (Files.exists(nioFilePath)) { - printAllFiles(); - return new FileStatus( - Files.size(nioFilePath), - Files.isDirectory(nioFilePath), - 1, - 1, - 0, - 0, - new FsPermission((short) 777), - null, - null, - path); - } - - pathStr = pathStr.replaceAll("//", "/"); - int stageDirIndex = pathStr.indexOf(HiveConf.getVar(conf, HiveConf.ConfVars.STAGINGDIR)); - if (stageDirIndex >= 0 && pathStr.substring(stageDirIndex).split("/").length == 1) { - Context.println("Staging dir not exists, create file as dir!"); - Files.createDirectories(nioFilePath); - printAllFiles(); - return new FileStatus( - 1, true, 1, 1, 0, 0, new FsPermission((short) 777), null, null, path); - } printAllFiles(); - throw new FileNotFoundException(); + FileStatus temp = + fs.getFileStatus(new Path(path.toString().replaceAll("vineyard", "file"))); + FileStatus result = + new FileStatus( + temp.getLen(), + temp.isDirectory(), + temp.getReplication(), + temp.getBlockSize(), + temp.getModificationTime(), + temp.getAccessTime(), + new FsPermission((short) 777), + temp.getOwner(), + temp.getGroup(), + new Path( + SCHEME + + ":///" + + temp.getPath() + .toString() + .substring( + temp.getPath().toString().indexOf(":") + 1) + .replaceAll("/+", "/"))); + return result; } @Override @@ -537,6 +605,13 @@ public void moveFromLocalFile(Path src, Path dst) throws IOException { @Override public void copyFromLocalFile(boolean delSrc, Path src, Path dst) throws IOException { + Context.println( + "copyFromLocalFile: " + + src.toString() + + " to " + + dst.toString() + + " delSrc: " + + delSrc); throw new UnsupportedOperationException( "Vineyard file system not support copyFromLocalFile."); } diff --git a/java/hive/src/main/java/io/v6d/hive/ql/io/HiveTypeResolver.java b/java/hive/src/main/java/io/v6d/hive/ql/io/HiveTypeResolver.java new file mode 100644 index 000000000..4658ddb0e --- /dev/null +++ b/java/hive/src/main/java/io/v6d/hive/ql/io/HiveTypeResolver.java @@ -0,0 +1,28 @@ +/** Copyright 2020-2023 Alibaba Group Holding Limited. + * + * Licensed 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 io.v6d.hive.ql.io; + +import io.v6d.modules.basic.arrow.util.ObjectResolver; +import java.math.BigDecimal; +import org.apache.hadoop.hive.common.type.HiveDecimal; + +public class HiveTypeResolver extends ObjectResolver { + public HiveTypeResolver() {} + + @Override + public final Object resolveDecimal(BigDecimal object, int precision, int scale, int bitWidth) { + return HiveDecimal.create(object); + } +} diff --git a/java/hive/src/main/java/io/v6d/hive/ql/io/HiveTypeTransformer.java b/java/hive/src/main/java/io/v6d/hive/ql/io/HiveTypeTransformer.java new file mode 100644 index 000000000..b03aab6d6 --- /dev/null +++ b/java/hive/src/main/java/io/v6d/hive/ql/io/HiveTypeTransformer.java @@ -0,0 +1,32 @@ +/** Copyright 2020-2023 Alibaba Group Holding Limited. + * + * Licensed 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 io.v6d.hive.ql.io; + +import io.v6d.modules.basic.arrow.util.ObjectTransformer; +import java.math.BigDecimal; +import org.apache.hadoop.hive.common.type.HiveDecimal; + +public class HiveTypeTransformer extends ObjectTransformer { + public HiveTypeTransformer() {} + + @Override + public final BigDecimal transformDecimal( + Object object, int precision, int scale, int bitWidth) { + HiveDecimal hiveDecimal = (HiveDecimal) (object); + BigDecimal bigDecimal = hiveDecimal.bigDecimalValue(); + bigDecimal = bigDecimal.setScale(scale); + return bigDecimal; + } +} diff --git a/java/hive/src/main/java/io/v6d/hive/ql/io/RecordWrapperWritable.java b/java/hive/src/main/java/io/v6d/hive/ql/io/RecordWrapperWritable.java index f0542079d..d15282734 100644 --- a/java/hive/src/main/java/io/v6d/hive/ql/io/RecordWrapperWritable.java +++ b/java/hive/src/main/java/io/v6d/hive/ql/io/RecordWrapperWritable.java @@ -14,150 +14,68 @@ */ package io.v6d.hive.ql.io; -import io.v6d.modules.basic.arrow.Arrow; import io.v6d.modules.basic.columnar.ColumnarData; import java.io.DataInput; import java.io.DataOutput; import java.util.ArrayList; import java.util.Arrays; import java.util.List; -import java.util.function.BiConsumer; import lombok.val; import org.apache.arrow.vector.types.pojo.Schema; import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector; +import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorFactory; import org.apache.hadoop.hive.serde2.objectinspector.SettableStructObjectInspector; import org.apache.hadoop.hive.serde2.objectinspector.StructField; +import org.apache.hadoop.hive.serde2.objectinspector.primitive.JavaHiveCharObjectInspector; +import org.apache.hadoop.hive.serde2.objectinspector.primitive.JavaHiveDecimalObjectInspector; +import org.apache.hadoop.hive.serde2.objectinspector.primitive.JavaHiveVarcharObjectInspector; import org.apache.hadoop.hive.serde2.objectinspector.primitive.PrimitiveObjectInspectorFactory; +import org.apache.hadoop.hive.serde2.typeinfo.CharTypeInfo; +import org.apache.hadoop.hive.serde2.typeinfo.DecimalTypeInfo; +import org.apache.hadoop.hive.serde2.typeinfo.ListTypeInfo; +import org.apache.hadoop.hive.serde2.typeinfo.MapTypeInfo; import org.apache.hadoop.hive.serde2.typeinfo.PrimitiveTypeInfo; import org.apache.hadoop.hive.serde2.typeinfo.StructTypeInfo; import org.apache.hadoop.hive.serde2.typeinfo.TypeInfo; +import org.apache.hadoop.hive.serde2.typeinfo.VarcharTypeInfo; import org.apache.hadoop.io.*; public class RecordWrapperWritable implements WritableComparable { - // output format: raw java objects, e.g., integer, string - // input format: writable private Object[] values; - // for input format - private boolean[] nullIndicators; - private BiConsumer[] setters; - - // for output format public RecordWrapperWritable() {} - // for input format public RecordWrapperWritable(Schema schema) { - this.values = new Writable[schema.getFields().size()]; - this.nullIndicators = new boolean[schema.getFields().size()]; - this.setters = new BiConsumer[schema.getFields().size()]; - for (int i = 0; i < schema.getFields().size(); i++) { - val dtype = schema.getFields().get(i).getFieldType().getType(); - if (Arrow.Type.Boolean.equals(dtype)) { - this.values[i] = new BooleanWritable(); - this.setters[i] = RecordWrapperWritable::setBool; - } else if (Arrow.Type.Int.equals(dtype) || Arrow.Type.UInt.equals(dtype)) { - this.values[i] = new IntWritable(); - this.setters[i] = RecordWrapperWritable::setInt; - } else if (Arrow.Type.Int64.equals(dtype) || Arrow.Type.UInt64.equals(dtype)) { - this.values[i] = new LongWritable(); - this.setters[i] = RecordWrapperWritable::setLong; - } else if (Arrow.Type.Float.equals(dtype)) { - this.values[i] = new FloatWritable(); - this.setters[i] = RecordWrapperWritable::setFloat; - } else if (Arrow.Type.Double.equals(dtype)) { - this.values[i] = new DoubleWritable(); - this.setters[i] = RecordWrapperWritable::setDouble; - } else if (Arrow.Type.VarChar.equals(dtype) - || Arrow.Type.LargeVarChar.equals(dtype) - || Arrow.Type.VarBinary.equals(dtype) - || Arrow.Type.LargeVarBinary.equals(dtype)) { - this.values[i] = new Text(); - this.setters[i] = RecordWrapperWritable::setString; - } else { - throw new UnsupportedOperationException("Unsupported type: " + dtype); - } - } + this.values = new Object[schema.getFields().size()]; } - // for input format public RecordWrapperWritable(List fieldTypes) { - this.values = new Writable[fieldTypes.size()]; - this.nullIndicators = new boolean[fieldTypes.size()]; - this.setters = new BiConsumer[fieldTypes.size()]; - for (int i = 0; i < fieldTypes.size(); i++) { - val info = fieldTypes.get(i); - if (info.getCategory() != ObjectInspector.Category.PRIMITIVE) { - throw new UnsupportedOperationException("Unsupported type: " + info); - } - switch (((PrimitiveTypeInfo) info).getPrimitiveCategory()) { - case BOOLEAN: - this.values[i] = new BooleanWritable(); - this.setters[i] = RecordWrapperWritable::setBool; - break; - case BYTE: - this.values[i] = new ByteWritable(); - this.setters[i] = RecordWrapperWritable::setByte; - break; - case SHORT: - this.values[i] = new ShortWritable(); - this.setters[i] = RecordWrapperWritable::setShort; - break; - case INT: - this.values[i] = new IntWritable(); - this.setters[i] = RecordWrapperWritable::setInt; - break; - case LONG: - this.values[i] = new LongWritable(); - this.setters[i] = RecordWrapperWritable::setLong; - break; - case FLOAT: - this.values[i] = new FloatWritable(); - this.setters[i] = RecordWrapperWritable::setFloat; - break; - case DOUBLE: - this.values[i] = new DoubleWritable(); - this.setters[i] = RecordWrapperWritable::setDouble; - break; - case STRING: - this.values[i] = new Text(); - this.setters[i] = RecordWrapperWritable::setString; - break; - default: - throw new UnsupportedOperationException("Unsupported type: " + info); - } - } + this.values = new Object[fieldTypes.size()]; } - public Object[] getValues() { - return values; + public void setValue(int index, Object value) { + values[index] = value; } - // for output format public void setValues(Object[] values) { this.values = values; } - // for input format - public void setWritables(ColumnarData[] columns, int index) { + public void setValues(ColumnarData[] columns, int index) { for (int i = 0; i < columns.length; i++) { - setters[i].accept((Writable) values[i], columns[i].getObject(index)); + values[i] = columns[i].getObject(index); } } public Object getValue(int index) { + if (index >= values.length) { + return null; + } return values[index]; } - // for output format - public void setValue(int index, Object value) { - values[index] = value; - } - - // for input format - public void setWritable(int index, Writable value) { - // n.b.: need to use setters, as values from "setStructFieldData" - // are already writables. - values[index] = value; + public Object[] getValues() { + return values; } @Override @@ -180,43 +98,6 @@ public boolean equals(Object o) { return true; } - private BooleanWritable makeWritable(boolean value) { - return new BooleanWritable(value); - } - - private static void setBool(Writable w, Object value) { - ((BooleanWritable) w).set((boolean) value); - } - - private static void setByte(Writable w, Object value) { - ((ByteWritable) w).set((byte) value); - } - - private static void setShort(Writable w, Object value) { - ((ShortWritable) w).set((short) value); - } - - private static void setInt(Writable w, Object value) { - ((IntWritable) w).set((int) value); - } - - private static void setLong(Writable w, Object value) { - ((LongWritable) w).set((long) value); - } - - private static void setFloat(Writable w, Object value) { - ((FloatWritable) w).set((float) value); - } - - private static void setDouble(Writable w, Object value) { - ((DoubleWritable) w).set((double) value); - } - - private static void setString(Writable w, Object value) { - // keep the casting as a type sanity check - ((Text) w).set(((org.apache.arrow.vector.util.Text) value).toString()); - } - static class Field implements StructField { private final String name; @@ -284,7 +165,7 @@ public Object create() { public Object setStructFieldData(Object struct, StructField field, Object fieldValue) { int offset = ((Field) field).offset; RecordWrapperWritable writable = (RecordWrapperWritable) struct; - writable.setWritable(offset, (Writable) fieldValue); + writable.setValue(offset, fieldValue); return struct; } @@ -344,31 +225,68 @@ static ObjectInspector createObjectInspector(StructTypeInfo info) { } static ObjectInspector createObjectInspector(TypeInfo info) { - if (info.getCategory() == ObjectInspector.Category.STRUCT) { - return createObjectInspector((StructTypeInfo) info); - } + return createObjectInspector(info, 0); + } + + static ObjectInspector createObjectInspector(TypeInfo info, int level) { if (info.getCategory() != ObjectInspector.Category.PRIMITIVE) { - throw new UnsupportedOperationException("Unsupported type: " + info); - } - switch (((PrimitiveTypeInfo) info).getPrimitiveCategory()) { - case BOOLEAN: - return PrimitiveObjectInspectorFactory.writableBooleanObjectInspector; - case BYTE: - return PrimitiveObjectInspectorFactory.writableByteObjectInspector; - case SHORT: - return PrimitiveObjectInspectorFactory.writableShortObjectInspector; - case INT: - return PrimitiveObjectInspectorFactory.writableIntObjectInspector; - case LONG: - return PrimitiveObjectInspectorFactory.writableLongObjectInspector; - case FLOAT: - return PrimitiveObjectInspectorFactory.writableFloatObjectInspector; - case DOUBLE: - return PrimitiveObjectInspectorFactory.writableDoubleObjectInspector; - case STRING: - return PrimitiveObjectInspectorFactory.writableStringObjectInspector; - default: - throw new UnsupportedOperationException("Unsupported type: " + info); + switch (info.getCategory()) { + case LIST: + TypeInfo elementInfo = ((ListTypeInfo) info).getListElementTypeInfo(); + return ObjectInspectorFactory.getStandardListObjectInspector( + createObjectInspector(elementInfo, level + 1)); + case MAP: + TypeInfo keyTypeInfo = ((MapTypeInfo) info).getMapKeyTypeInfo(); + TypeInfo valueTypeInfo = ((MapTypeInfo) info).getMapValueTypeInfo(); + return ObjectInspectorFactory.getStandardMapObjectInspector( + createObjectInspector(keyTypeInfo, level + 1), + createObjectInspector(valueTypeInfo, level + 1)); + case STRUCT: + List elemTypes = ((StructTypeInfo) info).getAllStructFieldTypeInfos(); + List elemNames = ((StructTypeInfo) info).getAllStructFieldNames(); + List elemInspectors = new ArrayList(); + + for (int i = 0; i < elemTypes.size(); i++) { + elemInspectors.add(createObjectInspector(elemTypes.get(i), level + 1)); + } + return ObjectInspectorFactory.getStandardStructObjectInspector( + elemNames, elemInspectors); + default: + throw new UnsupportedOperationException("Unsupported type: " + info); + } + } else { + switch (((PrimitiveTypeInfo) info).getPrimitiveCategory()) { + case BOOLEAN: + return PrimitiveObjectInspectorFactory.javaBooleanObjectInspector; + case BYTE: + return PrimitiveObjectInspectorFactory.javaByteObjectInspector; + case SHORT: + return PrimitiveObjectInspectorFactory.javaShortObjectInspector; + case INT: + return PrimitiveObjectInspectorFactory.javaIntObjectInspector; + case LONG: + return PrimitiveObjectInspectorFactory.javaLongObjectInspector; + case FLOAT: + return PrimitiveObjectInspectorFactory.javaFloatObjectInspector; + case DOUBLE: + return PrimitiveObjectInspectorFactory.javaDoubleObjectInspector; + case STRING: + return PrimitiveObjectInspectorFactory.javaStringObjectInspector; + case CHAR: + return new JavaHiveCharObjectInspector((CharTypeInfo) info); + case VARCHAR: + return new JavaHiveVarcharObjectInspector((VarcharTypeInfo) info); + case BINARY: + return PrimitiveObjectInspectorFactory.javaByteArrayObjectInspector; + case DATE: + return PrimitiveObjectInspectorFactory.javaDateObjectInspector; + case TIMESTAMP: + return PrimitiveObjectInspectorFactory.javaTimestampObjectInspector; + case DECIMAL: + return new JavaHiveDecimalObjectInspector((DecimalTypeInfo) info); + default: + throw new UnsupportedOperationException("Unsupported type: " + info); + } } } } diff --git a/java/hive/src/main/java/io/v6d/hive/ql/io/VineyardInputFormat.java b/java/hive/src/main/java/io/v6d/hive/ql/io/VineyardInputFormat.java index 33e83f092..22162d7ca 100644 --- a/java/hive/src/main/java/io/v6d/hive/ql/io/VineyardInputFormat.java +++ b/java/hive/src/main/java/io/v6d/hive/ql/io/VineyardInputFormat.java @@ -20,10 +20,16 @@ import io.v6d.core.client.ds.ObjectFactory; import io.v6d.core.common.util.ObjectID; import io.v6d.core.common.util.VineyardException; +import io.v6d.core.common.util.VineyardException.ObjectNotExists; import io.v6d.modules.basic.arrow.*; +import io.v6d.modules.basic.arrow.util.ObjectResolver; import io.v6d.modules.basic.columnar.ColumnarData; import java.io.IOException; import java.nio.charset.StandardCharsets; +import java.util.ArrayList; +import java.util.LinkedList; +import java.util.List; +import java.util.Queue; import lombok.val; import org.apache.arrow.vector.VectorSchemaRoot; import org.apache.arrow.vector.types.pojo.Schema; @@ -31,6 +37,7 @@ import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hive.common.FileUtils; import org.apache.hadoop.hive.ql.exec.vector.VectorizedInputFormatInterface; import org.apache.hadoop.hive.ql.io.HiveInputFormat; import org.apache.hadoop.io.*; @@ -56,7 +63,8 @@ public InputSplit[] getSplits(JobConf job, int numSplits) throws IOException { Path paths[] = FileInputFormat.getInputPaths(job); val client = Context.getClient(); - val splits = new VineyardSplit[paths.length]; + // split table by paths + List splits = new ArrayList<>(); Arrow.instantiate(); for (int i = 0; i < paths.length; i++) { @@ -66,48 +74,62 @@ public InputSplit[] getSplits(JobConf job, int numSplits) throws IOException { // get object id from vineyard filesystem FileSystem fs = path.getFileSystem(job); - FileStatus[] status = fs.listStatus(path); - if (status.length == 0) { - return new VineyardSplit[0]; + FileStatus[] tableStatus = fs.listStatus(path, FileUtils.HIDDEN_FILES_PATH_FILTER); + if (tableStatus.length == 0) { + continue; } + Queue dirStatus = new LinkedList<>(); + dirStatus.add(tableStatus); // Maybe there exists more than one table file. long numBatches = 0; - for (int j = 0; j < status.length; j++) { - Path tableFilePath = status[j].getPath(); - FSDataInputStream in = fs.open(tableFilePath); - FileStatus fileStatus = fs.getFileStatus(tableFilePath); - byte[] buffer = new byte[(int) fileStatus.getLen()]; - int len = in.read(buffer, 0, (int) fileStatus.getLen()); - // Here must check with the condition of len <= 0, rather than len == -1. - // Because Spark will create an empty file, which will cause the len == 0. - if (len <= 0) { - continue; - } - String[] objectIDs = new String(buffer, StandardCharsets.UTF_8).split("\n"); - for (val objectID : objectIDs) { - try { - ObjectID tableID = ObjectID.fromString(objectID); - Table table = - (Table) - ObjectFactory.getFactory() - .resolve(client.getMetaData(tableID)); - numBatches += table.getBatches().size(); - } catch (Exception e) { - // Skip some invalid file. - Context.println( - "Skipping invalid file: " - + tableFilePath - + ", content: " - + new String(buffer, StandardCharsets.UTF_8)); - break; + + while (!dirStatus.isEmpty()) { + FileStatus[] status = dirStatus.poll(); + for (int j = 0; j < status.length; j++) { + if (status[j].isDirectory()) { + dirStatus.add( + fs.listStatus( + status[j].getPath(), FileUtils.HIDDEN_FILES_PATH_FILTER)); + continue; + } + Path tableFilePath = status[j].getPath(); + FSDataInputStream in = fs.open(tableFilePath); + FileStatus fileStatus = fs.getFileStatus(tableFilePath); + byte[] buffer = new byte[(int) fileStatus.getLen()]; + int len = in.read(buffer, 0, (int) fileStatus.getLen()); + // Here must check with the condition of len <= 0, rather than len == -1. + // Because Spark will create an empty file, which will cause the len == 0. + if (len <= 0) { + continue; + } + String[] objectIDs = new String(buffer, StandardCharsets.UTF_8).split("\n"); + for (val objectID : objectIDs) { + try { + ObjectID tableID = ObjectID.fromString(objectID); + Table table = + (Table) + ObjectFactory.getFactory() + .resolve(client.getMetaData(tableID)); + numBatches += table.getBatches().size(); + } catch (ObjectNotExists | NumberFormatException e) { + // Skip some invalid file. + Context.println( + "Skipping invalid file: " + + tableFilePath + + ", content: " + + new String(buffer, StandardCharsets.UTF_8)); + break; + } } } } // TODO: would generating a split for each record batch be better? - splits[i] = new VineyardSplit(path, 0, numBatches, job); + if (numBatches > 0) { + splits.add(new VineyardSplit(path, 0, numBatches, job)); + } } - return splits; + return splits.toArray(new VineyardSplit[splits.size()]); } } @@ -121,54 +143,71 @@ class VineyardRecordReader implements RecordReader dirStatus = new LinkedList<>(); + dirStatus.add(tableStatus); val client = Context.getClient(); Arrow.instantiate(); - this.batches = new RecordBatch[(int) split.getLength()]; - this.recordBatchIndex = 0; + resolver = new HiveTypeResolver(); - for (int j = 0; j < status.length; j++) { - Path tableFilePath = status[j].getPath(); - FSDataInputStream in = fs.open(tableFilePath); - FileStatus fileStatus = fs.getFileStatus(tableFilePath); - byte[] buffer = new byte[(int) fileStatus.getLen()]; - int len = in.read(buffer, 0, (int) fileStatus.getLen()); - if (len <= 0) { - continue; - } - String[] objectIDs = new String(buffer, StandardCharsets.UTF_8).split("\n"); - for (val objectID : objectIDs) { - try { - ObjectID tableID = ObjectID.fromString(objectID); - Table table = - (Table) ObjectFactory.getFactory().resolve(client.getMetaData(tableID)); - for (val batch : table.getBatches()) { - recordTotal += batch.getRowCount(); - this.batches[this.recordBatchIndex++] = batch; + while (!dirStatus.isEmpty()) { + FileStatus[] status = dirStatus.poll(); + for (int j = 0; j < status.length; j++) { + if (status[j].isDirectory()) { + dirStatus.add( + fs.listStatus(status[j].getPath(), FileUtils.HIDDEN_FILES_PATH_FILTER)); + continue; + } + Path tableFilePath = status[j].getPath(); + FSDataInputStream in = fs.open(tableFilePath); + FileStatus fileStatus = fs.getFileStatus(tableFilePath); + byte[] buffer = new byte[(int) fileStatus.getLen()]; + int len = in.read(buffer, 0, (int) fileStatus.getLen()); + if (len <= 0) { + continue; + } + String[] objectIDs = new String(buffer, StandardCharsets.UTF_8).split("\n"); + for (val objectID : objectIDs) { + try { + ObjectID tableID = ObjectID.fromString(objectID); + Table table = + (Table) + ObjectFactory.getFactory() + .resolve(client.getMetaData(tableID)); + for (val batch : table.getBatches()) { + recordTotal += batch.getRowCount(); + batch.setResolver(resolver); + this.batches[this.recordBatchIndex++] = batch; + } schema = table.getSchema().getSchema(); + break; + } catch (ObjectNotExists | NumberFormatException e) { + // Skip some invalid file. + Context.println( + "Skipping invalid file: " + + tableFilePath + + ", content: " + + new String(buffer, StandardCharsets.UTF_8)); + break; } - } catch (Exception e) { - // Skip some invalid file. - Context.println( - "Skipping invalid file: " - + tableFilePath - + ", content: " - + new String(buffer, StandardCharsets.UTF_8)); - break; } } } @@ -221,7 +260,7 @@ public boolean next(NullWritable key, RecordWrapperWritable value) throws IOExce } // update the value - value.setWritables(columns, recordBatchInnerIndex); + value.setValues(columns, recordBatchInnerIndex); // move cursor to next record recordBatchInnerIndex++; diff --git a/java/hive/src/main/java/io/v6d/hive/ql/io/VineyardOutputFormat.java b/java/hive/src/main/java/io/v6d/hive/ql/io/VineyardOutputFormat.java index d668227e6..08e53a988 100644 --- a/java/hive/src/main/java/io/v6d/hive/ql/io/VineyardOutputFormat.java +++ b/java/hive/src/main/java/io/v6d/hive/ql/io/VineyardOutputFormat.java @@ -20,6 +20,7 @@ import io.v6d.core.client.ds.ObjectMeta; import io.v6d.core.common.util.VineyardException; import io.v6d.modules.basic.arrow.*; +import io.v6d.modules.basic.arrow.util.ObjectTransformer; import io.v6d.modules.basic.columnar.ColumnarDataBuilder; import java.io.IOException; import java.nio.charset.StandardCharsets; @@ -31,6 +32,7 @@ import org.apache.arrow.vector.types.Types; import org.apache.arrow.vector.types.pojo.*; import org.apache.arrow.vector.types.pojo.Schema; +import org.apache.commons.lang.NotImplementedException; import org.apache.hadoop.fs.FSDataOutputStream; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; @@ -40,7 +42,10 @@ import org.apache.hadoop.hive.ql.io.HiveOutputFormat; import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorUtils; import org.apache.hadoop.hive.serde2.typeinfo.DecimalTypeInfo; +import org.apache.hadoop.hive.serde2.typeinfo.ListTypeInfo; +import org.apache.hadoop.hive.serde2.typeinfo.MapTypeInfo; import org.apache.hadoop.hive.serde2.typeinfo.PrimitiveTypeInfo; +import org.apache.hadoop.hive.serde2.typeinfo.StructTypeInfo; import org.apache.hadoop.hive.serde2.typeinfo.TypeInfo; import org.apache.hadoop.io.NullWritable; import org.apache.hadoop.io.Writable; @@ -93,11 +98,13 @@ class SinkRecordWriter implements FileSinkOperator.RecordWriter { private Path finalOutPath; private FileSystem fs; private Progressable progress; + private final ObjectTransformer transformer; // vineyard private FSDataOutputStream output; private static CloseableReentrantLock lock = new CloseableReentrantLock(); private Schema schema; + private TypeInfo[] infos; List chunks = new ArrayList<>(); List current; @@ -139,6 +146,8 @@ public SinkRecordWriter( // initialize the schema this.initializeTableFile(); this.schema = this.initializeTableSchema(tableProperties); + + this.transformer = new HiveTypeTransformer(); Context.println("creating a sink record writer uses: " + watch.stop()); } @@ -151,7 +160,9 @@ public void write(Writable w) throws IOException { writeTimer.start(); if (currentLoc == RECORD_BATCH_SIZE) { - val builder = new RecordBatchBuilder(Context.getClient(), schema, RECORD_BATCH_SIZE); + val builder = + new RecordBatchBuilder( + Context.getClient(), schema, RECORD_BATCH_SIZE, this.transformer); chunks.add(builder); current = builder.getColumnBuilders(); currentLoc = 0; @@ -194,10 +205,8 @@ public void close(boolean abort) throws IOException { } for (int i = 0; i < chunks.size(); i++) { val chunk = chunks.get(i); - Context.println("record batch builder: " + i + ", row size: " + chunk.getNumRows()); tableBuilder.addBatch(chunk); } - Context.println("record batch size:" + tableBuilder.getBatchSize()); ObjectMeta meta = tableBuilder.seal(client); Context.println("Table id in vineyard:" + meta.getId().value()); client.persist(meta.getId()); @@ -218,6 +227,65 @@ private void initializeTableFile() throws IOException { } } + private Field getField(TypeInfo typeInfo) { + Field field = null; + switch (typeInfo.getCategory()) { + case LIST: + List listChildren = new ArrayList<>(); + Field chField = getField(((ListTypeInfo) typeInfo).getListElementTypeInfo()); + listChildren.add(chField); + field = + new Field( + typeInfo.getTypeName(), + FieldType.nullable(toArrowType(typeInfo)), + listChildren); + break; + case STRUCT: + List structChildren = new ArrayList<>(); + for (val child : ((StructTypeInfo) typeInfo).getAllStructFieldTypeInfos()) { + structChildren.add(getField(child)); + } + field = + new Field( + typeInfo.getTypeName(), + FieldType.nullable(toArrowType(typeInfo)), + structChildren); + break; + case MAP: + listChildren = new ArrayList<>(); + structChildren = new ArrayList<>(); + List mapChildren = new ArrayList<>(); + Field keyField = getField(((MapTypeInfo) typeInfo).getMapKeyTypeInfo()); + Field valueField = getField(((MapTypeInfo) typeInfo).getMapValueTypeInfo()); + structChildren.add(keyField); + structChildren.add(valueField); + Field structField = + new Field( + typeInfo.getTypeName(), + FieldType.notNullable(ArrowType.Struct.INSTANCE), + structChildren); + listChildren.add(structField); + Field listField = + new Field( + typeInfo.getTypeName(), + FieldType.notNullable(ArrowType.List.INSTANCE), + listChildren); + mapChildren.add(listField); + field = + new Field( + typeInfo.getTypeName(), + FieldType.nullable(toArrowType(typeInfo)), + mapChildren); + break; + case UNION: + throw new NotImplementedException(); + default: + field = Field.nullable(typeInfo.getTypeName(), toArrowType(typeInfo)); + break; + } + return field; + } + private Schema initializeTableSchema(Properties tableProperties) { val structTypeInfo = TypeContext.computeStructTypeInfo(tableProperties); val targetTypeInfos = @@ -226,9 +294,9 @@ private Schema initializeTableSchema(Properties tableProperties) { List fields = new ArrayList<>(); for (val typeInfo : targetTypeInfos) { - Field field = Field.nullable(typeInfo.getTypeName(), toArrowType(typeInfo)); - fields.add(field); + fields.add(getField(typeInfo)); } + infos = targetTypeInfos; return new Schema(fields); } @@ -257,7 +325,7 @@ private static ArrowType toArrowType(TypeInfo typeInfo) { case DATE: return Types.MinorType.DATEDAY.getType(); case TIMESTAMP: - return new ArrowType.Timestamp(TimeUnit.MICROSECOND, "UTC"); + return new ArrowType.Timestamp(TimeUnit.NANOSECOND, "UTC"); case BINARY: return Types.MinorType.VARBINARY.getType(); case DECIMAL: @@ -269,7 +337,6 @@ private static ArrowType toArrowType(TypeInfo typeInfo) { case INTERVAL_DAY_TIME: return Types.MinorType.INTERVALDAY.getType(); case VOID: - // case TIMESTAMPLOCALTZ: case UNKNOWN: default: throw new IllegalArgumentException(); @@ -279,7 +346,7 @@ private static ArrowType toArrowType(TypeInfo typeInfo) { case STRUCT: return ArrowType.Struct.INSTANCE; case MAP: - return new ArrowType.Map(false); + return new ArrowType.Map(true); case UNION: default: throw new IllegalArgumentException(); diff --git a/java/hive/src/main/java/io/v6d/hive/ql/io/VineyardSerDe.java b/java/hive/src/main/java/io/v6d/hive/ql/io/VineyardSerDe.java index abe9b769b..af3780a31 100644 --- a/java/hive/src/main/java/io/v6d/hive/ql/io/VineyardSerDe.java +++ b/java/hive/src/main/java/io/v6d/hive/ql/io/VineyardSerDe.java @@ -45,7 +45,7 @@ public void initializeTypeInfo(Configuration configuration, Properties tableProp this.rowTypeInfo = TypeContext.computeStructTypeInfo(tableProperties); this.targetTypeInfos = TypeContext.computeTargetTypeInfos( - this.rowTypeInfo, ObjectInspectorUtils.ObjectInspectorCopyOption.WRITABLE); + this.rowTypeInfo, ObjectInspectorUtils.ObjectInspectorCopyOption.JAVA); this.objectInspector = (StructObjectInspector) TypeInfoUtils.getStandardWritableObjectInspectorFromTypeInfo( diff --git a/java/modules/basic/src/main/java/io/v6d/modules/basic/arrow/ArrayBuilder.java b/java/modules/basic/src/main/java/io/v6d/modules/basic/arrow/ArrayBuilder.java index e7386fa8e..980c4e2a1 100644 --- a/java/modules/basic/src/main/java/io/v6d/modules/basic/arrow/ArrayBuilder.java +++ b/java/modules/basic/src/main/java/io/v6d/modules/basic/arrow/ArrayBuilder.java @@ -17,14 +17,19 @@ import io.v6d.core.client.Client; import io.v6d.core.client.ds.ObjectBuilder; import io.v6d.core.common.util.VineyardException; +import io.v6d.modules.basic.arrow.util.ObjectTransformer; import io.v6d.modules.basic.columnar.ColumnarDataBuilder; import org.apache.arrow.vector.FieldVector; public interface ArrayBuilder extends ObjectBuilder { public abstract FieldVector getArray(); + public default ColumnarDataBuilder columnar(ObjectTransformer transformer) { + return new ColumnarDataBuilder(getArray(), transformer); + } + public default ColumnarDataBuilder columnar() { - return new ColumnarDataBuilder(getArray()); + return new ColumnarDataBuilder(getArray(), new ObjectTransformer()); } public abstract void shrink(Client client, long size) throws VineyardException; diff --git a/java/modules/basic/src/main/java/io/v6d/modules/basic/arrow/Arrow.java b/java/modules/basic/src/main/java/io/v6d/modules/basic/arrow/Arrow.java index 514c905d1..caa1fb0b6 100644 --- a/java/modules/basic/src/main/java/io/v6d/modules/basic/arrow/Arrow.java +++ b/java/modules/basic/src/main/java/io/v6d/modules/basic/arrow/Arrow.java @@ -17,7 +17,9 @@ import com.google.common.collect.ImmutableList; import io.v6d.modules.basic.stream.RecordBatchStream; import org.apache.arrow.memory.RootAllocator; +import org.apache.arrow.vector.types.DateUnit; import org.apache.arrow.vector.types.FloatingPointPrecision; +import org.apache.arrow.vector.types.TimeUnit; import org.apache.arrow.vector.types.pojo.ArrowType; import org.apache.arrow.vector.types.pojo.Field; import org.slf4j.Logger; @@ -29,6 +31,8 @@ public final class Arrow { public static class Type { public static final ArrowType Null = new ArrowType.Null(); + public static final ArrowType TinyInt = new ArrowType.Int(8, true); + public static final ArrowType SmallInt = new ArrowType.Int(16, true); public static final ArrowType Int = new ArrowType.Int(32, true); public static final ArrowType UInt = new ArrowType.Int(32, false); public static final ArrowType Int64 = new ArrowType.Int(64, true); @@ -42,6 +46,18 @@ public static class Type { public static final ArrowType LargeVarChar = new ArrowType.LargeUtf8(); public static final ArrowType VarBinary = new ArrowType.Binary(); public static final ArrowType LargeVarBinary = new ArrowType.LargeBinary(); + public static final ArrowType Date = new ArrowType.Date(DateUnit.DAY); + public static final ArrowType TimeStampSec = + new ArrowType.Timestamp(TimeUnit.SECOND, "UTC"); + public static final ArrowType TimeStampMilli = + new ArrowType.Timestamp(TimeUnit.MILLISECOND, "UTC"); + public static final ArrowType TimeStampMicro = + new ArrowType.Timestamp(TimeUnit.MICROSECOND, "UTC"); + public static final ArrowType TimeStampNano = + new ArrowType.Timestamp(TimeUnit.NANOSECOND, "UTC"); + public static final ArrowType List = new ArrowType.List(); + public static final ArrowType Struct = new ArrowType.Struct(); + public static final ArrowType Map = new ArrowType.Map(true); } public static class FieldType { @@ -49,6 +65,10 @@ public static class FieldType { new org.apache.arrow.vector.types.pojo.FieldType(false, Type.Null, null); public static final org.apache.arrow.vector.types.pojo.FieldType Int = new org.apache.arrow.vector.types.pojo.FieldType(false, Arrow.Type.Int, null); + public static final org.apache.arrow.vector.types.pojo.FieldType TinyInt = + new org.apache.arrow.vector.types.pojo.FieldType(false, Type.TinyInt, null); + public static final org.apache.arrow.vector.types.pojo.FieldType SmallInt = + new org.apache.arrow.vector.types.pojo.FieldType(false, Type.SmallInt, null); public static final org.apache.arrow.vector.types.pojo.FieldType UInt = new org.apache.arrow.vector.types.pojo.FieldType(false, Type.UInt, null); public static final org.apache.arrow.vector.types.pojo.FieldType Int64 = @@ -69,6 +89,22 @@ public static class FieldType { new org.apache.arrow.vector.types.pojo.FieldType(false, Type.VarBinary, null); public static final org.apache.arrow.vector.types.pojo.FieldType LargeVarBinary = new org.apache.arrow.vector.types.pojo.FieldType(false, Type.LargeVarBinary, null); + public static final org.apache.arrow.vector.types.pojo.FieldType Date = + new org.apache.arrow.vector.types.pojo.FieldType(false, Type.Date, null); + public static final org.apache.arrow.vector.types.pojo.FieldType TimeStampMilli = + new org.apache.arrow.vector.types.pojo.FieldType(false, Type.TimeStampMilli, null); + public static final org.apache.arrow.vector.types.pojo.FieldType TimeStampMicro = + new org.apache.arrow.vector.types.pojo.FieldType(false, Type.TimeStampMicro, null); + public static final org.apache.arrow.vector.types.pojo.FieldType TimeStampNano = + new org.apache.arrow.vector.types.pojo.FieldType(false, Type.TimeStampNano, null); + public static final org.apache.arrow.vector.types.pojo.FieldType TimeStampSec = + new org.apache.arrow.vector.types.pojo.FieldType(false, Type.TimeStampSec, null); + public static final org.apache.arrow.vector.types.pojo.FieldType List = + new org.apache.arrow.vector.types.pojo.FieldType(false, Type.List, null); + public static final org.apache.arrow.vector.types.pojo.FieldType Struct = + new org.apache.arrow.vector.types.pojo.FieldType(false, Type.Struct, null); + public static final org.apache.arrow.vector.types.pojo.FieldType Map = + new org.apache.arrow.vector.types.pojo.FieldType(false, Type.Map, null); } public static Field makeField( diff --git a/java/modules/basic/src/main/java/io/v6d/modules/basic/arrow/BooleanArray.java b/java/modules/basic/src/main/java/io/v6d/modules/basic/arrow/BooleanArray.java index 5d80c7859..7979a64c9 100644 --- a/java/modules/basic/src/main/java/io/v6d/modules/basic/arrow/BooleanArray.java +++ b/java/modules/basic/src/main/java/io/v6d/modules/basic/arrow/BooleanArray.java @@ -19,7 +19,9 @@ import io.v6d.core.client.ds.ObjectFactory; import io.v6d.core.client.ds.ObjectMeta; import java.util.Arrays; +import java.util.List; import lombok.*; +import org.apache.arrow.memory.ArrowBuf; import org.apache.arrow.vector.BitVector; import org.apache.arrow.vector.FieldVector; import org.apache.arrow.vector.ipc.message.ArrowFieldNode; @@ -32,11 +34,10 @@ public static void instantiate() { ObjectFactory.getFactory().register("vineyard::BooleanArray", new BooleanArrayResolver()); } - public BooleanArray(final ObjectMeta meta, Buffer buffer, long length) { + public BooleanArray(final ObjectMeta meta, List buffers, int length, int nullCount) { super(meta); this.array = new BitVector("", Arrow.default_allocator); - this.array.loadFieldBuffers( - new ArrowFieldNode(length, 0), Arrays.asList(null, buffer.getBuffer())); + this.array.loadFieldBuffers(new ArrowFieldNode(length, nullCount), buffers); } public double get(int index) { @@ -69,7 +70,16 @@ public int hashCode() { class BooleanArrayResolver extends ObjectFactory.Resolver { @Override public Object resolve(final ObjectMeta meta) { - val buffer = (Buffer) ObjectFactory.getFactory().resolve(meta.getMemberMeta("buffer_")); - return new BooleanArray(meta, buffer, meta.getLongValue("length_")); + Buffer dataBuffer = + (Buffer) ObjectFactory.getFactory().resolve(meta.getMemberMeta("buffer_")); + Buffer validityBuffer = + (Buffer) ObjectFactory.getFactory().resolve(meta.getMemberMeta("null_bitmap_")); + int nullCount = meta.getIntValue("null_count_"); + int length = meta.getIntValue("length_"); + return new BooleanArray( + meta, + Arrays.asList(validityBuffer.getBuffer(), dataBuffer.getBuffer()), + length, + nullCount); } } diff --git a/java/modules/basic/src/main/java/io/v6d/modules/basic/arrow/BooleanArrayBuilder.java b/java/modules/basic/src/main/java/io/v6d/modules/basic/arrow/BooleanArrayBuilder.java index 33c93998b..f5f6d0b86 100644 --- a/java/modules/basic/src/main/java/io/v6d/modules/basic/arrow/BooleanArrayBuilder.java +++ b/java/modules/basic/src/main/java/io/v6d/modules/basic/arrow/BooleanArrayBuilder.java @@ -20,23 +20,30 @@ import io.v6d.core.common.util.VineyardException; import java.util.Arrays; import lombok.*; +import org.apache.arrow.memory.ArrowBuf; import org.apache.arrow.vector.BitVector; import org.apache.arrow.vector.FieldVector; import org.apache.arrow.vector.ipc.message.ArrowFieldNode; public class BooleanArrayBuilder implements ArrayBuilder { - private BufferBuilder buffer; + private BufferBuilder dataBufferBuilder; + private BufferBuilder validityBufferBuilder; private BitVector array; public BooleanArrayBuilder(IPCClient client, long length) throws VineyardException { this.array = new BitVector("", Arrow.default_allocator); - this.buffer = new BufferBuilder(client, this.array.getBufferSizeFor((int) length)); + this.dataBufferBuilder = + new BufferBuilder(client, this.array.getBufferSizeFor((int) length)); this.array.loadFieldBuffers( - new ArrowFieldNode(length, 0), Arrays.asList(null, buffer.getBuffer())); + new ArrowFieldNode(length, 0), Arrays.asList(null, dataBufferBuilder.getBuffer())); } @Override - public void build(Client client) throws VineyardException {} + public void build(Client client) throws VineyardException { + ArrowBuf validityBuffer = array.getValidityBuffer(); + validityBufferBuilder = + new BufferBuilder((IPCClient) client, validityBuffer, validityBuffer.capacity()); + } @Override public ObjectMeta seal(Client client) throws VineyardException { @@ -45,10 +52,10 @@ public ObjectMeta seal(Client client) throws VineyardException { meta.setTypename("vineyard::BooleanArray"); meta.setNBytes(array.getBufferSizeFor(array.getValueCount())); meta.setValue("length_", array.getValueCount()); - meta.setValue("null_count_", 0); + meta.setValue("null_count_", array.getNullCount()); meta.setValue("offset_", 0); - meta.addMember("buffer_", buffer.seal(client)); - meta.addMember("null_bitmap_", BufferBuilder.empty(client)); + meta.addMember("buffer_", dataBufferBuilder.seal(client)); + meta.addMember("null_bitmap_", validityBufferBuilder.seal(client)); return client.createMetaData(meta); } @@ -59,7 +66,7 @@ public FieldVector getArray() { @Override public void shrink(Client client, long size) throws VineyardException { - this.buffer.shrink(client, this.array.getBufferSizeFor((int) size)); + this.dataBufferBuilder.shrink(client, this.array.getBufferSizeFor((int) size)); this.array.setValueCount((int) size); } diff --git a/java/modules/basic/src/main/java/io/v6d/modules/basic/arrow/DateArray.java b/java/modules/basic/src/main/java/io/v6d/modules/basic/arrow/DateArray.java new file mode 100644 index 000000000..142260884 --- /dev/null +++ b/java/modules/basic/src/main/java/io/v6d/modules/basic/arrow/DateArray.java @@ -0,0 +1,83 @@ +/** Copyright 2020-2023 Alibaba Group Holding Limited. + * + * Licensed 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 io.v6d.modules.basic.arrow; + +import com.google.common.base.Objects; +import io.v6d.core.client.ds.Object; +import io.v6d.core.client.ds.ObjectFactory; +import io.v6d.core.client.ds.ObjectMeta; +import java.util.Arrays; +import java.util.List; +import org.apache.arrow.memory.ArrowBuf; +import org.apache.arrow.vector.DateMilliVector; +import org.apache.arrow.vector.FieldVector; +import org.apache.arrow.vector.ipc.message.ArrowFieldNode; + +public class DateArray extends Array { + private DateMilliVector array; + + public static void instantiate() { + ObjectFactory.getFactory().register("vineyard::Date", new DateArrayResolver()); + } + + public DateArray(ObjectMeta meta, List buffers, long length, int nullCount) { + super(meta); + this.array = new DateMilliVector("", Arrow.default_allocator); + this.array.loadFieldBuffers(new ArrowFieldNode(length, nullCount), buffers); + } + + public long get(int index) { + return this.array.get(index); + } + + @Override + public FieldVector getArray() { + return this.array; + } + + @Override + public boolean equals(java.lang.Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + DateArray that = (DateArray) o; + return Objects.equal(array, that.array); + } + + @Override + public int hashCode() { + return Objects.hashCode(array); + } +} + +class DateArrayResolver extends ObjectFactory.Resolver { + @Override + public Object resolve(ObjectMeta meta) { + Buffer dataBuffer = + (Buffer) ObjectFactory.getFactory().resolve(meta.getMemberMeta("buffer_")); + Buffer validityBuffer = + (Buffer) ObjectFactory.getFactory().resolve(meta.getMemberMeta("null_bitmap_")); + int nullCount = meta.getIntValue("null_count_"); + int length = meta.getIntValue("length_"); + return new DateArray( + meta, + Arrays.asList(validityBuffer.getBuffer(), dataBuffer.getBuffer()), + length, + nullCount); + } +} diff --git a/java/modules/basic/src/main/java/io/v6d/modules/basic/arrow/DateArrayBuilder.java b/java/modules/basic/src/main/java/io/v6d/modules/basic/arrow/DateArrayBuilder.java new file mode 100644 index 000000000..0a872ddf5 --- /dev/null +++ b/java/modules/basic/src/main/java/io/v6d/modules/basic/arrow/DateArrayBuilder.java @@ -0,0 +1,76 @@ +/** Copyright 2020-2023 Alibaba Group Holding Limited. + * + * Licensed 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 io.v6d.modules.basic.arrow; + +import io.v6d.core.client.Client; +import io.v6d.core.client.IPCClient; +import io.v6d.core.client.ds.ObjectMeta; +import io.v6d.core.common.util.VineyardException; +import java.util.Arrays; +import lombok.val; +import org.apache.arrow.memory.ArrowBuf; +import org.apache.arrow.vector.DateMilliVector; +import org.apache.arrow.vector.FieldVector; +import org.apache.arrow.vector.ipc.message.ArrowFieldNode; + +public class DateArrayBuilder implements ArrayBuilder { + private BufferBuilder dataBufferBuilder; + private BufferBuilder validityBufferBuilder; + private DateMilliVector array; + + public DateArrayBuilder(IPCClient client, long length) throws VineyardException { + this.array = new DateMilliVector("", Arrow.default_allocator); + this.dataBufferBuilder = + new BufferBuilder(client, this.array.getBufferSizeFor((int) length)); + this.array.loadFieldBuffers( + new ArrowFieldNode(length, 0), Arrays.asList(null, dataBufferBuilder.getBuffer())); + } + + @Override + public void build(Client client) throws VineyardException { + ArrowBuf validityBuf = array.getValidityBuffer(); + validityBufferBuilder = + new BufferBuilder((IPCClient) client, validityBuf, validityBuf.capacity()); + } + + @Override + public ObjectMeta seal(Client client) throws VineyardException { + this.build(client); + val meta = ObjectMeta.empty(); + meta.setTypename("vineyard::Date"); + meta.setNBytes(array.getBufferSizeFor(array.getValueCount())); + meta.setValue("length_", array.getValueCount()); + meta.setValue("null_count_", array.getNullCount()); + meta.setValue("offset_", 0); + meta.addMember("buffer_", dataBufferBuilder.seal(client)); + meta.addMember("null_bitmap_", validityBufferBuilder.seal(client)); + return client.createMetaData(meta); + } + + @Override + public FieldVector getArray() { + return this.array; + } + + @Override + public void shrink(Client client, long size) throws VineyardException { + this.dataBufferBuilder.shrink(client, this.array.getBufferSizeFor((int) size)); + this.array.setValueCount((int) size); + } + + void set(int index, long value) { + this.array.set(index, value); + } +} diff --git a/java/modules/basic/src/main/java/io/v6d/modules/basic/arrow/DecimalArray.java b/java/modules/basic/src/main/java/io/v6d/modules/basic/arrow/DecimalArray.java new file mode 100644 index 000000000..2b270d3fe --- /dev/null +++ b/java/modules/basic/src/main/java/io/v6d/modules/basic/arrow/DecimalArray.java @@ -0,0 +1,96 @@ +/** Copyright 2020-2023 Alibaba Group Holding Limited. + * + * Licensed 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 io.v6d.modules.basic.arrow; + +import com.google.common.base.Objects; +import io.v6d.core.client.ds.Object; +import io.v6d.core.client.ds.ObjectFactory; +import io.v6d.core.client.ds.ObjectMeta; +import java.util.Arrays; +import java.util.List; +import org.apache.arrow.memory.ArrowBuf; +import org.apache.arrow.vector.BaseFixedWidthVector; +import org.apache.arrow.vector.Decimal256Vector; +import org.apache.arrow.vector.DecimalVector; +import org.apache.arrow.vector.FieldVector; +import org.apache.arrow.vector.ipc.message.ArrowFieldNode; + +public class DecimalArray extends Array { + private BaseFixedWidthVector array; + + public static void instantiate() { + ObjectFactory.getFactory() + .register("vineyard::DecimalArray<128>", new DecimalArrayResolver()); + ObjectFactory.getFactory() + .register("vineyard::DecimalArray<256>", new DecimalArrayResolver()); + } + + public DecimalArray( + ObjectMeta meta, + List buffers, + int nullCount, + long length, + int maxPrecision, + int maxScale, + int bitWidth) { + super(meta); + if (bitWidth == 128) { + this.array = new DecimalVector("", Arrow.default_allocator, maxPrecision, maxScale); + } else { + this.array = new Decimal256Vector("", Arrow.default_allocator, maxPrecision, maxScale); + } + this.array.loadFieldBuffers(new ArrowFieldNode(length, nullCount), buffers); + } + + @Override + public FieldVector getArray() { + return this.array; + } + + @Override + public boolean equals(java.lang.Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + DecimalArray that = (DecimalArray) o; + return Objects.equal(array, that.array); + } + + @Override + public int hashCode() { + return Objects.hashCode(array); + } +} + +class DecimalArrayResolver extends ObjectFactory.Resolver { + @Override + public Object resolve(ObjectMeta meta) { + Buffer buffer = (Buffer) ObjectFactory.getFactory().resolve(meta.getMemberMeta("buffer_")); + Buffer validityBuffer = + (Buffer) ObjectFactory.getFactory().resolve(meta.getMemberMeta("null_bitmap_")); + int nullCount = meta.getIntValue("null_count_"); + return new DecimalArray( + meta, + Arrays.asList(validityBuffer.getBuffer(), buffer.getBuffer()), + nullCount, + meta.getLongValue("length_"), + meta.getIntValue("max_precision_"), + meta.getIntValue("max_scale_"), + meta.getIntValue("bit_width_")); + } +} diff --git a/java/modules/basic/src/main/java/io/v6d/modules/basic/arrow/DecimalArrayBuilder.java b/java/modules/basic/src/main/java/io/v6d/modules/basic/arrow/DecimalArrayBuilder.java new file mode 100644 index 000000000..e3a974d04 --- /dev/null +++ b/java/modules/basic/src/main/java/io/v6d/modules/basic/arrow/DecimalArrayBuilder.java @@ -0,0 +1,90 @@ +/** Copyright 2020-2023 Alibaba Group Holding Limited. + * + * Licensed 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 io.v6d.modules.basic.arrow; + +import io.v6d.core.client.Client; +import io.v6d.core.client.IPCClient; +import io.v6d.core.client.ds.ObjectMeta; +import io.v6d.core.common.util.VineyardException; +import java.util.Arrays; +import lombok.val; +import org.apache.arrow.memory.ArrowBuf; +import org.apache.arrow.vector.BaseFixedWidthVector; +import org.apache.arrow.vector.Decimal256Vector; +import org.apache.arrow.vector.DecimalVector; +import org.apache.arrow.vector.FieldVector; +import org.apache.arrow.vector.ipc.message.ArrowFieldNode; + +public class DecimalArrayBuilder implements ArrayBuilder { + private BufferBuilder dataBufferBuilder; + private BaseFixedWidthVector array; + private BufferBuilder validityBufferBuilder; + int maxPrecision; + int maxScale; + int bitWidth; + + public DecimalArrayBuilder( + IPCClient client, long length, int maxPrecision, int maxScale, int bitWidth) + throws VineyardException { + this.maxPrecision = maxPrecision; + this.maxScale = maxScale; + this.bitWidth = bitWidth; + if (bitWidth == 128) { + this.array = new DecimalVector("", Arrow.default_allocator, maxPrecision, maxScale); + } else { + this.array = new Decimal256Vector("", Arrow.default_allocator, maxPrecision, maxScale); + } + this.dataBufferBuilder = + new BufferBuilder(client, this.array.getBufferSizeFor((int) length)); + this.array.loadFieldBuffers( + new ArrowFieldNode(length, 0), Arrays.asList(null, dataBufferBuilder.getBuffer())); + } + + @Override + public void build(Client client) throws VineyardException { + ArrowBuf validityBuffer = array.getValidityBuffer(); + + validityBufferBuilder = + new BufferBuilder((IPCClient) client, validityBuffer, validityBuffer.capacity()); + } + + @Override + public ObjectMeta seal(Client client) throws VineyardException { + this.build(client); + val meta = ObjectMeta.empty(); + meta.setTypename("vineyard::DecimalArray<" + String.valueOf(bitWidth) + ">"); + meta.setNBytes(array.getBufferSizeFor(array.getValueCount())); + meta.setValue("length_", array.getValueCount()); + meta.setValue("null_count_", array.getNullCount()); + meta.setValue("offset_", 0); + meta.addMember("buffer_", dataBufferBuilder.seal(client)); + meta.addMember("null_bitmap_", validityBufferBuilder.seal(client)); + meta.setValue("max_precision_", maxPrecision); + meta.setValue("max_scale_", maxScale); + meta.setValue("bit_width_", bitWidth); + return client.createMetaData(meta); + } + + @Override + public FieldVector getArray() { + return this.array; + } + + @Override + public void shrink(Client client, long size) throws VineyardException { + this.dataBufferBuilder.shrink(client, this.array.getBufferSizeFor((int) size)); + this.array.setValueCount((int) size); + } +} diff --git a/java/modules/basic/src/main/java/io/v6d/modules/basic/arrow/DoubleArray.java b/java/modules/basic/src/main/java/io/v6d/modules/basic/arrow/DoubleArray.java index bacae5f62..da1d9bfe2 100644 --- a/java/modules/basic/src/main/java/io/v6d/modules/basic/arrow/DoubleArray.java +++ b/java/modules/basic/src/main/java/io/v6d/modules/basic/arrow/DoubleArray.java @@ -19,12 +19,13 @@ import io.v6d.core.client.ds.ObjectFactory; import io.v6d.core.client.ds.ObjectMeta; import java.util.Arrays; +import java.util.List; import lombok.*; +import org.apache.arrow.memory.ArrowBuf; import org.apache.arrow.vector.FieldVector; import org.apache.arrow.vector.Float8Vector; import org.apache.arrow.vector.ipc.message.ArrowFieldNode; -/** Hello world! */ public class DoubleArray extends Array { private Float8Vector array; @@ -33,11 +34,10 @@ public static void instantiate() { .register("vineyard::NumericArray", new DoubleArrayResolver()); } - public DoubleArray(final ObjectMeta meta, Buffer buffer, long length) { + public DoubleArray(final ObjectMeta meta, List buffers, long length, int nullCount) { super(meta); this.array = new Float8Vector("", Arrow.default_allocator); - this.array.loadFieldBuffers( - new ArrowFieldNode(length, 0), Arrays.asList(null, buffer.getBuffer())); + this.array.loadFieldBuffers(new ArrowFieldNode(length, nullCount), buffers); } public double get(int index) { @@ -70,7 +70,16 @@ public int hashCode() { class DoubleArrayResolver extends ObjectFactory.Resolver { @Override public Object resolve(final ObjectMeta meta) { - val buffer = (Buffer) ObjectFactory.getFactory().resolve(meta.getMemberMeta("buffer_")); - return new DoubleArray(meta, buffer, meta.getLongValue("length_")); + Buffer dataBuffer = + (Buffer) ObjectFactory.getFactory().resolve(meta.getMemberMeta("buffer_")); + Buffer validityBuffer = + (Buffer) ObjectFactory.getFactory().resolve(meta.getMemberMeta("null_bitmap_")); + int nullCount = meta.getIntValue("null_count_"); + int length = meta.getIntValue("length_"); + return new DoubleArray( + meta, + Arrays.asList(validityBuffer.getBuffer(), dataBuffer.getBuffer()), + length, + nullCount); } } diff --git a/java/modules/basic/src/main/java/io/v6d/modules/basic/arrow/DoubleArrayBuilder.java b/java/modules/basic/src/main/java/io/v6d/modules/basic/arrow/DoubleArrayBuilder.java index b26ff419d..10b4dcffa 100644 --- a/java/modules/basic/src/main/java/io/v6d/modules/basic/arrow/DoubleArrayBuilder.java +++ b/java/modules/basic/src/main/java/io/v6d/modules/basic/arrow/DoubleArrayBuilder.java @@ -20,23 +20,29 @@ import io.v6d.core.common.util.VineyardException; import java.util.Arrays; import lombok.*; +import org.apache.arrow.memory.ArrowBuf; import org.apache.arrow.vector.FieldVector; import org.apache.arrow.vector.Float8Vector; import org.apache.arrow.vector.ipc.message.ArrowFieldNode; public class DoubleArrayBuilder implements ArrayBuilder { - private BufferBuilder buffer; + private BufferBuilder dataBufferBuilder; + private BufferBuilder validityBufferBuilder; private Float8Vector array; public DoubleArrayBuilder(IPCClient client, long length) throws VineyardException { this.array = new Float8Vector("", Arrow.default_allocator); - this.buffer = new BufferBuilder(client, this.array.getBufferSizeFor((int) length)); + this.dataBufferBuilder = + new BufferBuilder(client, this.array.getBufferSizeFor((int) length)); this.array.loadFieldBuffers( - new ArrowFieldNode(length, 0), Arrays.asList(null, buffer.getBuffer())); + new ArrowFieldNode(length, 0), Arrays.asList(null, dataBufferBuilder.getBuffer())); } @Override - public void build(Client client) throws VineyardException {} + public void build(Client client) throws VineyardException { + ArrowBuf buf = array.getValidityBuffer(); + validityBufferBuilder = new BufferBuilder((IPCClient) client, buf, buf.capacity()); + } @Override public ObjectMeta seal(Client client) throws VineyardException { @@ -45,10 +51,10 @@ public ObjectMeta seal(Client client) throws VineyardException { meta.setTypename("vineyard::NumericArray"); meta.setNBytes(array.getBufferSizeFor(array.getValueCount())); meta.setValue("length_", array.getValueCount()); - meta.setValue("null_count_", 0); + meta.setValue("null_count_", array.getNullCount()); meta.setValue("offset_", 0); - meta.addMember("buffer_", buffer.seal(client)); - meta.addMember("null_bitmap_", BufferBuilder.empty(client)); + meta.addMember("buffer_", dataBufferBuilder.seal(client)); + meta.addMember("null_bitmap_", validityBufferBuilder.seal(client)); return client.createMetaData(meta); } @@ -59,7 +65,7 @@ public FieldVector getArray() { @Override public void shrink(Client client, long size) throws VineyardException { - this.buffer.shrink(client, this.array.getBufferSizeFor((int) size)); + this.dataBufferBuilder.shrink(client, this.array.getBufferSizeFor((int) size)); this.array.setValueCount((int) size); } diff --git a/java/modules/basic/src/main/java/io/v6d/modules/basic/arrow/FloatArray.java b/java/modules/basic/src/main/java/io/v6d/modules/basic/arrow/FloatArray.java index d0b72802b..d3f7bd1bb 100644 --- a/java/modules/basic/src/main/java/io/v6d/modules/basic/arrow/FloatArray.java +++ b/java/modules/basic/src/main/java/io/v6d/modules/basic/arrow/FloatArray.java @@ -19,12 +19,12 @@ import io.v6d.core.client.ds.ObjectFactory; import io.v6d.core.client.ds.ObjectMeta; import java.util.Arrays; -import lombok.val; +import java.util.List; +import org.apache.arrow.memory.ArrowBuf; import org.apache.arrow.vector.FieldVector; import org.apache.arrow.vector.Float4Vector; import org.apache.arrow.vector.ipc.message.ArrowFieldNode; -/** Hello world! */ public class FloatArray extends Array { private Float4Vector array; @@ -33,11 +33,10 @@ public static void instantiate() { .register("vineyard::NumericArray", new FloatArrayResolver()); } - public FloatArray(final ObjectMeta meta, Buffer buffer, long length) { + public FloatArray(final ObjectMeta meta, List buffers, long length, int nullCount) { super(meta); this.array = new Float4Vector("", Arrow.default_allocator); - this.array.loadFieldBuffers( - new ArrowFieldNode(length, 0), Arrays.asList(null, buffer.getBuffer())); + this.array.loadFieldBuffers(new ArrowFieldNode(length, nullCount), buffers); } public float get(int index) { @@ -70,7 +69,16 @@ public int hashCode() { class FloatArrayResolver extends ObjectFactory.Resolver { @Override public Object resolve(final ObjectMeta meta) { - val buffer = (Buffer) ObjectFactory.getFactory().resolve(meta.getMemberMeta("buffer_")); - return new FloatArray(meta, buffer, meta.getLongValue("length_")); + Buffer dataBuffer = + (Buffer) ObjectFactory.getFactory().resolve(meta.getMemberMeta("buffer_")); + Buffer validityBuffer = + (Buffer) ObjectFactory.getFactory().resolve(meta.getMemberMeta("null_bitmap_")); + int nullCount = meta.getIntValue("null_count_"); + int length = meta.getIntValue("length_"); + return new FloatArray( + meta, + Arrays.asList(validityBuffer.getBuffer(), dataBuffer.getBuffer()), + length, + nullCount); } } diff --git a/java/modules/basic/src/main/java/io/v6d/modules/basic/arrow/FloatArrayBuilder.java b/java/modules/basic/src/main/java/io/v6d/modules/basic/arrow/FloatArrayBuilder.java index 0ae8ad00d..45be457ca 100644 --- a/java/modules/basic/src/main/java/io/v6d/modules/basic/arrow/FloatArrayBuilder.java +++ b/java/modules/basic/src/main/java/io/v6d/modules/basic/arrow/FloatArrayBuilder.java @@ -20,23 +20,29 @@ import io.v6d.core.common.util.VineyardException; import java.util.Arrays; import lombok.val; +import org.apache.arrow.memory.ArrowBuf; import org.apache.arrow.vector.FieldVector; import org.apache.arrow.vector.Float4Vector; import org.apache.arrow.vector.ipc.message.ArrowFieldNode; public class FloatArrayBuilder implements ArrayBuilder { - private BufferBuilder buffer; + private BufferBuilder dataBufferBuilder; + private BufferBuilder validityBufferBuilder; private Float4Vector array; public FloatArrayBuilder(IPCClient client, long length) throws VineyardException { this.array = new Float4Vector("", Arrow.default_allocator); - this.buffer = new BufferBuilder(client, this.array.getBufferSizeFor((int) length)); + this.dataBufferBuilder = + new BufferBuilder(client, this.array.getBufferSizeFor((int) length)); this.array.loadFieldBuffers( - new ArrowFieldNode(length, 0), Arrays.asList(null, buffer.getBuffer())); + new ArrowFieldNode(length, 0), Arrays.asList(null, dataBufferBuilder.getBuffer())); } @Override - public void build(Client client) throws VineyardException {} + public void build(Client client) throws VineyardException { + ArrowBuf buf = array.getValidityBuffer(); + validityBufferBuilder = new BufferBuilder((IPCClient) client, buf, buf.capacity()); + } @Override public ObjectMeta seal(Client client) throws VineyardException { @@ -45,10 +51,10 @@ public ObjectMeta seal(Client client) throws VineyardException { meta.setTypename("vineyard::NumericArray"); meta.setNBytes(array.getBufferSizeFor(array.getValueCount())); meta.setValue("length_", array.getValueCount()); - meta.setValue("null_count_", 0); + meta.setValue("null_count_", array.getNullCount()); meta.setValue("offset_", 0); - meta.addMember("buffer_", buffer.seal(client)); - meta.addMember("null_bitmap_", BufferBuilder.empty(client)); + meta.addMember("buffer_", dataBufferBuilder.seal(client)); + meta.addMember("null_bitmap_", validityBufferBuilder.seal(client)); return client.createMetaData(meta); } @@ -59,7 +65,7 @@ public FieldVector getArray() { @Override public void shrink(Client client, long size) throws VineyardException { - this.buffer.shrink(client, this.array.getBufferSizeFor((int) size)); + this.dataBufferBuilder.shrink(client, this.array.getBufferSizeFor((int) size)); this.array.setValueCount((int) size); } diff --git a/java/modules/basic/src/main/java/io/v6d/modules/basic/arrow/Int16Array.java b/java/modules/basic/src/main/java/io/v6d/modules/basic/arrow/Int16Array.java new file mode 100644 index 000000000..43ea3e2d2 --- /dev/null +++ b/java/modules/basic/src/main/java/io/v6d/modules/basic/arrow/Int16Array.java @@ -0,0 +1,86 @@ +/** Copyright 2020-2023 Alibaba Group Holding Limited. + * + * Licensed 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 io.v6d.modules.basic.arrow; + +import com.google.common.base.Objects; +import io.v6d.core.client.ds.Object; +import io.v6d.core.client.ds.ObjectFactory; +import io.v6d.core.client.ds.ObjectMeta; +import java.util.Arrays; +import java.util.List; +import org.apache.arrow.memory.ArrowBuf; +import org.apache.arrow.vector.FieldVector; +import org.apache.arrow.vector.SmallIntVector; +import org.apache.arrow.vector.ipc.message.ArrowFieldNode; + +public class Int16Array extends Array { + private SmallIntVector array; + + public static void instantiate() { + ObjectFactory.getFactory() + .register("vineyard::NumericArray", new Int16ArrayResolver()); + ObjectFactory.getFactory() + .register("vineyard::NumericArray", new Int16ArrayResolver()); + } + + public Int16Array(ObjectMeta meta, List buffers, long length, int nullCount) { + super(meta); + this.array = new SmallIntVector("", Arrow.default_allocator); + this.array.loadFieldBuffers(new ArrowFieldNode(length, nullCount), buffers); + } + + public short get(int index) { + return this.array.get(index); + } + + @Override + public FieldVector getArray() { + return this.array; + } + + @Override + public boolean equals(java.lang.Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + Int16Array that = (Int16Array) o; + return Objects.equal(array, that.array); + } + + @Override + public int hashCode() { + return Objects.hashCode(array); + } +} + +class Int16ArrayResolver extends ObjectFactory.Resolver { + @Override + public Object resolve(ObjectMeta meta) { + Buffer dataBuffer = + (Buffer) ObjectFactory.getFactory().resolve(meta.getMemberMeta("buffer_")); + Buffer validityBuffer = + (Buffer) ObjectFactory.getFactory().resolve(meta.getMemberMeta("null_bitmap_")); + int nullCount = meta.getIntValue("null_count_"); + int length = meta.getIntValue("length_"); + return new Int16Array( + meta, + Arrays.asList(validityBuffer.getBuffer(), dataBuffer.getBuffer()), + length, + nullCount); + } +} diff --git a/java/modules/basic/src/main/java/io/v6d/modules/basic/arrow/Int16ArrayBuilder.java b/java/modules/basic/src/main/java/io/v6d/modules/basic/arrow/Int16ArrayBuilder.java new file mode 100644 index 000000000..fd6b46ffa --- /dev/null +++ b/java/modules/basic/src/main/java/io/v6d/modules/basic/arrow/Int16ArrayBuilder.java @@ -0,0 +1,75 @@ +/** Copyright 2020-2023 Alibaba Group Holding Limited. + * + * Licensed 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 io.v6d.modules.basic.arrow; + +import io.v6d.core.client.Client; +import io.v6d.core.client.IPCClient; +import io.v6d.core.client.ds.ObjectMeta; +import io.v6d.core.common.util.VineyardException; +import java.util.Arrays; +import lombok.val; +import org.apache.arrow.memory.ArrowBuf; +import org.apache.arrow.vector.FieldVector; +import org.apache.arrow.vector.SmallIntVector; +import org.apache.arrow.vector.ipc.message.ArrowFieldNode; + +public class Int16ArrayBuilder implements ArrayBuilder { + private BufferBuilder dataBufferBuilder; + private BufferBuilder validityBufferBuilder; + private SmallIntVector array; + + public Int16ArrayBuilder(IPCClient client, long length) throws VineyardException { + this.array = new SmallIntVector("", Arrow.default_allocator); + this.dataBufferBuilder = + new BufferBuilder(client, this.array.getBufferSizeFor((int) length)); + this.array.loadFieldBuffers( + new ArrowFieldNode(length, 0), Arrays.asList(null, dataBufferBuilder.getBuffer())); + } + + @Override + public void build(Client client) throws VineyardException { + ArrowBuf buf = array.getValidityBuffer(); + validityBufferBuilder = new BufferBuilder((IPCClient) client, buf, buf.capacity()); + } + + @Override + public ObjectMeta seal(Client client) throws VineyardException { + this.build(client); + val meta = ObjectMeta.empty(); + meta.setTypename("vineyard::NumericArray"); + meta.setNBytes(array.getBufferSizeFor(array.getValueCount())); + meta.setValue("length_", array.getValueCount()); + meta.setValue("null_count_", array.getNullCount()); + meta.setValue("offset_", 0); + meta.addMember("buffer_", dataBufferBuilder.seal(client)); + meta.addMember("null_bitmap_", validityBufferBuilder.seal(client)); + return client.createMetaData(meta); + } + + @Override + public FieldVector getArray() { + return this.array; + } + + @Override + public void shrink(Client client, long size) throws VineyardException { + this.dataBufferBuilder.shrink(client, this.array.getBufferSizeFor((int) size)); + this.array.setValueCount((int) size); + } + + void set(int index, short value) { + this.array.set(index, value); + } +} diff --git a/java/modules/basic/src/main/java/io/v6d/modules/basic/arrow/Int32Array.java b/java/modules/basic/src/main/java/io/v6d/modules/basic/arrow/Int32Array.java index 1aba6f801..f3d81e119 100644 --- a/java/modules/basic/src/main/java/io/v6d/modules/basic/arrow/Int32Array.java +++ b/java/modules/basic/src/main/java/io/v6d/modules/basic/arrow/Int32Array.java @@ -19,12 +19,12 @@ import io.v6d.core.client.ds.ObjectFactory; import io.v6d.core.client.ds.ObjectMeta; import java.util.Arrays; -import lombok.val; +import java.util.List; +import org.apache.arrow.memory.ArrowBuf; import org.apache.arrow.vector.FieldVector; import org.apache.arrow.vector.IntVector; import org.apache.arrow.vector.ipc.message.ArrowFieldNode; -/** Hello world! */ public class Int32Array extends Array { private IntVector array; @@ -35,11 +35,10 @@ public static void instantiate() { .register("vineyard::NumericArray", new Int32ArrayResolver()); } - public Int32Array(ObjectMeta meta, Buffer buffer, long length) { + public Int32Array(ObjectMeta meta, List buffers, long length, int nullCount) { super(meta); this.array = new IntVector("", Arrow.default_allocator); - this.array.loadFieldBuffers( - new ArrowFieldNode(length, 0), Arrays.asList(null, buffer.getBuffer())); + this.array.loadFieldBuffers(new ArrowFieldNode(length, nullCount), buffers); } public double get(int index) { @@ -72,7 +71,15 @@ public int hashCode() { class Int32ArrayResolver extends ObjectFactory.Resolver { @Override public Object resolve(ObjectMeta meta) { - val buffer = (Buffer) ObjectFactory.getFactory().resolve(meta.getMemberMeta("buffer_")); - return new Int32Array(meta, buffer, meta.getLongValue("length_")); + Buffer dataBuffer = + (Buffer) ObjectFactory.getFactory().resolve(meta.getMemberMeta("buffer_")); + Buffer validityBuffer = + (Buffer) ObjectFactory.getFactory().resolve(meta.getMemberMeta("null_bitmap_")); + int nullCount = meta.getIntValue("null_count_"); + return new Int32Array( + meta, + Arrays.asList(validityBuffer.getBuffer(), dataBuffer.getBuffer()), + meta.getLongValue("length_"), + nullCount); } } diff --git a/java/modules/basic/src/main/java/io/v6d/modules/basic/arrow/Int32ArrayBuilder.java b/java/modules/basic/src/main/java/io/v6d/modules/basic/arrow/Int32ArrayBuilder.java index 47e5dc694..dbed7f76e 100644 --- a/java/modules/basic/src/main/java/io/v6d/modules/basic/arrow/Int32ArrayBuilder.java +++ b/java/modules/basic/src/main/java/io/v6d/modules/basic/arrow/Int32ArrayBuilder.java @@ -20,23 +20,29 @@ import io.v6d.core.common.util.VineyardException; import java.util.Arrays; import lombok.val; +import org.apache.arrow.memory.ArrowBuf; import org.apache.arrow.vector.FieldVector; import org.apache.arrow.vector.IntVector; import org.apache.arrow.vector.ipc.message.ArrowFieldNode; public class Int32ArrayBuilder implements ArrayBuilder { - private BufferBuilder buffer; + private BufferBuilder dataBufferBuilder; + private BufferBuilder validityBufferBuilder; private IntVector array; public Int32ArrayBuilder(IPCClient client, long length) throws VineyardException { this.array = new IntVector("", Arrow.default_allocator); - this.buffer = new BufferBuilder(client, this.array.getBufferSizeFor((int) length)); + this.dataBufferBuilder = + new BufferBuilder(client, this.array.getBufferSizeFor((int) length)); this.array.loadFieldBuffers( - new ArrowFieldNode(length, 0), Arrays.asList(null, buffer.getBuffer())); + new ArrowFieldNode(length, 0), Arrays.asList(null, dataBufferBuilder.getBuffer())); } @Override - public void build(Client client) throws VineyardException {} + public void build(Client client) throws VineyardException { + ArrowBuf buf = array.getValidityBuffer(); + validityBufferBuilder = new BufferBuilder((IPCClient) client, buf, buf.capacity()); + } @Override public ObjectMeta seal(Client client) throws VineyardException { @@ -45,10 +51,10 @@ public ObjectMeta seal(Client client) throws VineyardException { meta.setTypename("vineyard::NumericArray"); meta.setNBytes(array.getBufferSizeFor(array.getValueCount())); meta.setValue("length_", array.getValueCount()); - meta.setValue("null_count_", 0); + meta.setValue("null_count_", array.getNullCount()); meta.setValue("offset_", 0); - meta.addMember("buffer_", buffer.seal(client)); - meta.addMember("null_bitmap_", BufferBuilder.empty(client)); + meta.addMember("buffer_", dataBufferBuilder.seal(client)); + meta.addMember("null_bitmap_", validityBufferBuilder.seal(client)); return client.createMetaData(meta); } @@ -59,7 +65,7 @@ public FieldVector getArray() { @Override public void shrink(Client client, long size) throws VineyardException { - this.buffer.shrink(client, this.array.getBufferSizeFor((int) size)); + this.dataBufferBuilder.shrink(client, this.array.getBufferSizeFor((int) size)); this.array.setValueCount((int) size); } diff --git a/java/modules/basic/src/main/java/io/v6d/modules/basic/arrow/Int64Array.java b/java/modules/basic/src/main/java/io/v6d/modules/basic/arrow/Int64Array.java index 187afeecc..fa43bd7e4 100644 --- a/java/modules/basic/src/main/java/io/v6d/modules/basic/arrow/Int64Array.java +++ b/java/modules/basic/src/main/java/io/v6d/modules/basic/arrow/Int64Array.java @@ -19,12 +19,12 @@ import io.v6d.core.client.ds.ObjectFactory; import io.v6d.core.client.ds.ObjectMeta; import java.util.Arrays; -import lombok.val; +import java.util.List; +import org.apache.arrow.memory.ArrowBuf; import org.apache.arrow.vector.BigIntVector; import org.apache.arrow.vector.FieldVector; import org.apache.arrow.vector.ipc.message.ArrowFieldNode; -/** Hello world! */ public class Int64Array extends Array { private BigIntVector array; @@ -35,11 +35,10 @@ public static void instantiate() { .register("vineyard::NumericArray", new Int64ArrayResolver()); } - public Int64Array(final ObjectMeta meta, Buffer buffer, long length) { + public Int64Array(final ObjectMeta meta, List buffers, long length, int nullCount) { super(meta); this.array = new BigIntVector("", Arrow.default_allocator); - this.array.loadFieldBuffers( - new ArrowFieldNode(length, 0), Arrays.asList(null, buffer.getBuffer())); + this.array.loadFieldBuffers(new ArrowFieldNode(length, nullCount), buffers); } public double get(int index) { @@ -72,7 +71,15 @@ public int hashCode() { class Int64ArrayResolver extends ObjectFactory.Resolver { @Override public Object resolve(final ObjectMeta meta) { - val buffer = (Buffer) ObjectFactory.getFactory().resolve(meta.getMemberMeta("buffer_")); - return new Int64Array(meta, buffer, meta.getLongValue("length_")); + Buffer buffer = (Buffer) ObjectFactory.getFactory().resolve(meta.getMemberMeta("buffer_")); + Buffer validityBuffer = + (Buffer) ObjectFactory.getFactory().resolve(meta.getMemberMeta("null_bitmap_")); + int nullCount = meta.getIntValue("null_count_"); + int length = meta.getIntValue("length_"); + return new Int64Array( + meta, + Arrays.asList(validityBuffer.getBuffer(), buffer.getBuffer()), + length, + nullCount); } } diff --git a/java/modules/basic/src/main/java/io/v6d/modules/basic/arrow/Int64ArrayBuilder.java b/java/modules/basic/src/main/java/io/v6d/modules/basic/arrow/Int64ArrayBuilder.java index 06fc2d9b9..25fee053e 100644 --- a/java/modules/basic/src/main/java/io/v6d/modules/basic/arrow/Int64ArrayBuilder.java +++ b/java/modules/basic/src/main/java/io/v6d/modules/basic/arrow/Int64ArrayBuilder.java @@ -20,23 +20,29 @@ import io.v6d.core.common.util.VineyardException; import java.util.Arrays; import lombok.val; +import org.apache.arrow.memory.ArrowBuf; import org.apache.arrow.vector.BigIntVector; import org.apache.arrow.vector.FieldVector; import org.apache.arrow.vector.ipc.message.ArrowFieldNode; public class Int64ArrayBuilder implements ArrayBuilder { - private BufferBuilder buffer; + private BufferBuilder dataBufferBuilder; + private BufferBuilder validityBufferBuilder; private BigIntVector array; public Int64ArrayBuilder(IPCClient client, long length) throws VineyardException { this.array = new BigIntVector("", Arrow.default_allocator); - this.buffer = new BufferBuilder(client, this.array.getBufferSizeFor((int) length)); + this.dataBufferBuilder = + new BufferBuilder(client, this.array.getBufferSizeFor((int) length)); this.array.loadFieldBuffers( - new ArrowFieldNode(length, 0), Arrays.asList(null, buffer.getBuffer())); + new ArrowFieldNode(length, 0), Arrays.asList(null, dataBufferBuilder.getBuffer())); } @Override - public void build(Client client) throws VineyardException {} + public void build(Client client) throws VineyardException { + ArrowBuf buf = array.getValidityBuffer(); + validityBufferBuilder = new BufferBuilder((IPCClient) client, buf, buf.capacity()); + } @Override public ObjectMeta seal(Client client) throws VineyardException { @@ -45,10 +51,10 @@ public ObjectMeta seal(Client client) throws VineyardException { meta.setTypename("vineyard::NumericArray"); meta.setNBytes(array.getBufferSizeFor(array.getValueCount())); meta.setValue("length_", array.getValueCount()); - meta.setValue("null_count_", 0); + meta.setValue("null_count_", array.getNullCount()); meta.setValue("offset_", 0); - meta.addMember("buffer_", buffer.seal(client)); - meta.addMember("null_bitmap_", BufferBuilder.empty(client)); + meta.addMember("buffer_", dataBufferBuilder.seal(client)); + meta.addMember("null_bitmap_", validityBufferBuilder.seal(client)); return client.createMetaData(meta); } @@ -59,7 +65,7 @@ public FieldVector getArray() { @Override public void shrink(Client client, long size) throws VineyardException { - this.buffer.shrink(client, this.array.getBufferSizeFor((int) size)); + this.dataBufferBuilder.shrink(client, this.array.getBufferSizeFor((int) size)); this.array.setValueCount((int) size); } diff --git a/java/modules/basic/src/main/java/io/v6d/modules/basic/arrow/Int8Array.java b/java/modules/basic/src/main/java/io/v6d/modules/basic/arrow/Int8Array.java new file mode 100644 index 000000000..f07a086bf --- /dev/null +++ b/java/modules/basic/src/main/java/io/v6d/modules/basic/arrow/Int8Array.java @@ -0,0 +1,86 @@ +/** Copyright 2020-2023 Alibaba Group Holding Limited. + * + * Licensed 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 io.v6d.modules.basic.arrow; + +import com.google.common.base.Objects; +import io.v6d.core.client.ds.Object; +import io.v6d.core.client.ds.ObjectFactory; +import io.v6d.core.client.ds.ObjectMeta; +import java.util.Arrays; +import java.util.List; +import org.apache.arrow.memory.ArrowBuf; +import org.apache.arrow.vector.FieldVector; +import org.apache.arrow.vector.TinyIntVector; +import org.apache.arrow.vector.ipc.message.ArrowFieldNode; + +public class Int8Array extends Array { + private TinyIntVector array; + + public static void instantiate() { + ObjectFactory.getFactory() + .register("vineyard::NumericArray", new Int8ArrayResolver()); + ObjectFactory.getFactory() + .register("vineyard::NumericArray", new Int8ArrayResolver()); + } + + public Int8Array(ObjectMeta meta, List buffers, long length, int nullCount) { + super(meta); + this.array = new TinyIntVector("", Arrow.default_allocator); + this.array.loadFieldBuffers(new ArrowFieldNode(length, nullCount), buffers); + } + + public byte get(int index) { + return this.array.get(index); + } + + @Override + public FieldVector getArray() { + return this.array; + } + + @Override + public boolean equals(java.lang.Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + Int8Array that = (Int8Array) o; + return Objects.equal(array, that.array); + } + + @Override + public int hashCode() { + return Objects.hashCode(array); + } +} + +class Int8ArrayResolver extends ObjectFactory.Resolver { + @Override + public Object resolve(ObjectMeta meta) { + Buffer dataBuffer = + (Buffer) ObjectFactory.getFactory().resolve(meta.getMemberMeta("buffer_")); + Buffer validityBuffer = + (Buffer) ObjectFactory.getFactory().resolve(meta.getMemberMeta("null_bitmap_")); + int length = meta.getIntValue("length_"); + int nullCount = meta.getIntValue("null_count_"); + return new Int8Array( + meta, + Arrays.asList(validityBuffer.getBuffer(), dataBuffer.getBuffer()), + length, + nullCount); + } +} diff --git a/java/modules/basic/src/main/java/io/v6d/modules/basic/arrow/Int8ArrayBuilder.java b/java/modules/basic/src/main/java/io/v6d/modules/basic/arrow/Int8ArrayBuilder.java new file mode 100644 index 000000000..6e1e6d59e --- /dev/null +++ b/java/modules/basic/src/main/java/io/v6d/modules/basic/arrow/Int8ArrayBuilder.java @@ -0,0 +1,75 @@ +/** Copyright 2020-2023 Alibaba Group Holding Limited. + * + * Licensed 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 io.v6d.modules.basic.arrow; + +import io.v6d.core.client.Client; +import io.v6d.core.client.IPCClient; +import io.v6d.core.client.ds.ObjectMeta; +import io.v6d.core.common.util.VineyardException; +import java.util.Arrays; +import lombok.val; +import org.apache.arrow.memory.ArrowBuf; +import org.apache.arrow.vector.FieldVector; +import org.apache.arrow.vector.TinyIntVector; +import org.apache.arrow.vector.ipc.message.ArrowFieldNode; + +public class Int8ArrayBuilder implements ArrayBuilder { + private BufferBuilder dataBufferBuilder; + private BufferBuilder validityBufferBuilder; + private TinyIntVector array; + + public Int8ArrayBuilder(IPCClient client, long length) throws VineyardException { + this.array = new TinyIntVector("", Arrow.default_allocator); + this.dataBufferBuilder = + new BufferBuilder(client, this.array.getBufferSizeFor((int) length)); + this.array.loadFieldBuffers( + new ArrowFieldNode(length, 0), Arrays.asList(null, dataBufferBuilder.getBuffer())); + } + + @Override + public void build(Client client) throws VineyardException { + ArrowBuf buf = array.getValidityBuffer(); + validityBufferBuilder = new BufferBuilder((IPCClient) client, buf, buf.capacity()); + } + + @Override + public ObjectMeta seal(Client client) throws VineyardException { + this.build(client); + val meta = ObjectMeta.empty(); + meta.setTypename("vineyard::NumericArray"); + meta.setNBytes(array.getBufferSizeFor(array.getValueCount())); + meta.setValue("length_", array.getValueCount()); + meta.setValue("null_count_", array.getNullCount()); + meta.setValue("offset_", 0); + meta.addMember("buffer_", dataBufferBuilder.seal(client)); + meta.addMember("null_bitmap_", validityBufferBuilder.seal(client)); + return client.createMetaData(meta); + } + + @Override + public FieldVector getArray() { + return this.array; + } + + @Override + public void shrink(Client client, long size) throws VineyardException { + this.dataBufferBuilder.shrink(client, this.array.getBufferSizeFor((int) size)); + this.array.setValueCount((int) size); + } + + void set(int index, byte value) { + this.array.set(index, value); + } +} diff --git a/java/modules/basic/src/main/java/io/v6d/modules/basic/arrow/LargeStringArray.java b/java/modules/basic/src/main/java/io/v6d/modules/basic/arrow/LargeStringArray.java index 035828ab7..05860e591 100644 --- a/java/modules/basic/src/main/java/io/v6d/modules/basic/arrow/LargeStringArray.java +++ b/java/modules/basic/src/main/java/io/v6d/modules/basic/arrow/LargeStringArray.java @@ -19,13 +19,14 @@ import io.v6d.core.client.ds.ObjectFactory; import io.v6d.core.client.ds.ObjectMeta; import java.util.Arrays; +import java.util.List; import lombok.*; +import org.apache.arrow.memory.ArrowBuf; import org.apache.arrow.vector.FieldVector; import org.apache.arrow.vector.LargeVarCharVector; import org.apache.arrow.vector.ipc.message.ArrowFieldNode; import org.apache.arrow.vector.util.Text; -/** Hello world! */ public class LargeStringArray extends Array { private LargeVarCharVector array; @@ -38,13 +39,11 @@ public static void instantiate() { .register("vineyard::LargeStringArray", new LargeStringArrayResolver()); } - public LargeStringArray(final ObjectMeta meta, Buffer buffer, Buffer offset, long length) { + public LargeStringArray( + final ObjectMeta meta, List buffers, long length, int nullCount) { super(meta); this.array = new LargeVarCharVector("", Arrow.default_allocator); - this.array.loadFieldBuffers( - new ArrowFieldNode(length, 0), - Arrays.asList(null, offset.getBuffer(), buffer.getBuffer())); - this.array.setValueCount((int) length); + this.array.loadFieldBuffers(new ArrowFieldNode(length, nullCount), buffers); } public byte[] get(int index) { @@ -81,10 +80,21 @@ public int hashCode() { class LargeStringArrayResolver extends ObjectFactory.Resolver { @Override public Object resolve(final ObjectMeta meta) { - val buffer = - (Buffer) ObjectFactory.getFactory().resolve(meta.getMemberMeta("buffer_data_")); - val offsets_buffer = + Buffer data_buffer = + (Buffer) ObjectFactory.getFactory().resolve(meta.getMemberMeta("buffer_")); + Buffer offsets_buffer = (Buffer) ObjectFactory.getFactory().resolve(meta.getMemberMeta("buffer_offsets_")); - return new LargeStringArray(meta, buffer, offsets_buffer, meta.getLongValue("length_")); + Buffer validity_buffer = + (Buffer) ObjectFactory.getFactory().resolve(meta.getMemberMeta("null_bitmap_")); + int null_count = meta.getIntValue("null_count_"); + int length = meta.getIntValue("length_"); + return new LargeStringArray( + meta, + Arrays.asList( + validity_buffer.getBuffer(), + offsets_buffer.getBuffer(), + data_buffer.getBuffer()), + length, + null_count); } } diff --git a/java/modules/basic/src/main/java/io/v6d/modules/basic/arrow/LargeStringArrayBuilder.java b/java/modules/basic/src/main/java/io/v6d/modules/basic/arrow/LargeStringArrayBuilder.java index b37782976..e91f4d550 100644 --- a/java/modules/basic/src/main/java/io/v6d/modules/basic/arrow/LargeStringArrayBuilder.java +++ b/java/modules/basic/src/main/java/io/v6d/modules/basic/arrow/LargeStringArrayBuilder.java @@ -19,6 +19,7 @@ import io.v6d.core.client.ds.ObjectMeta; import io.v6d.core.common.util.VineyardException; import lombok.*; +import org.apache.arrow.memory.ArrowBuf; import org.apache.arrow.vector.FieldVector; import org.apache.arrow.vector.LargeVarCharVector; import org.apache.arrow.vector.util.Text; @@ -26,8 +27,9 @@ public class LargeStringArrayBuilder implements ArrayBuilder { private LargeVarCharVector array; - private BufferBuilder data_buffer_builder; - private BufferBuilder offset_buffer_builder; + private BufferBuilder dataBufferBuilder; + private BufferBuilder offsetBufferBuilder; + private BufferBuilder validityBufferBuilder; public LargeStringArrayBuilder(IPCClient client, final LargeVarCharVector vector) throws VineyardException { @@ -36,7 +38,6 @@ public LargeStringArrayBuilder(IPCClient client, final LargeVarCharVector vector public LargeStringArrayBuilder(IPCClient client, long length) throws VineyardException { this.array = new LargeVarCharVector("", Arrow.default_allocator); - this.array.setValueCount((int) length); } @Override @@ -49,10 +50,15 @@ public void build(Client client) throws VineyardException { ((long) this.array.getValueCount()) * LargeVarCharVector.OFFSET_WIDTH); val data_buffer = this.array.getDataBuffer(); - this.data_buffer_builder = + ArrowBuf validity_buffer = this.array.getValidityBuffer(); + val validity_buffer_size = validity_buffer.capacity(); + + this.dataBufferBuilder = new BufferBuilder((IPCClient) client, data_buffer, data_buffer_size); - this.offset_buffer_builder = + this.offsetBufferBuilder = new BufferBuilder((IPCClient) client, offset_buffer, offset_buffer_size); + this.validityBufferBuilder = + new BufferBuilder((IPCClient) client, validity_buffer, validity_buffer_size); } @Override @@ -62,11 +68,11 @@ public ObjectMeta seal(Client client) throws VineyardException { meta.setTypename("vineyard::BaseBinaryArray"); meta.setNBytes(array.getBufferSizeFor(array.getValueCount())); meta.setValue("length_", array.getValueCount()); - meta.setValue("null_count_", 0); + meta.setValue("null_count_", array.getNullCount()); meta.setValue("offset_", 0); - meta.addMember("buffer_data_", data_buffer_builder.seal(client)); - meta.addMember("buffer_offsets_", offset_buffer_builder.seal(client)); - meta.addMember("null_bitmap_", BufferBuilder.empty(client)); + meta.addMember("buffer_", dataBufferBuilder.seal(client)); + meta.addMember("buffer_offsets_", offsetBufferBuilder.seal(client)); + meta.addMember("null_bitmap_", validityBufferBuilder.seal(client)); return client.createMetaData(meta); } diff --git a/java/modules/basic/src/main/java/io/v6d/modules/basic/arrow/ListArray.java b/java/modules/basic/src/main/java/io/v6d/modules/basic/arrow/ListArray.java new file mode 100644 index 000000000..2660d48cc --- /dev/null +++ b/java/modules/basic/src/main/java/io/v6d/modules/basic/arrow/ListArray.java @@ -0,0 +1,96 @@ +/** Copyright 2020-2023 Alibaba Group Holding Limited. + * + * Licensed 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 io.v6d.modules.basic.arrow; + +import com.google.common.base.Objects; +import io.v6d.core.client.ds.Object; +import io.v6d.core.client.ds.ObjectFactory; +import io.v6d.core.client.ds.ObjectMeta; +import io.v6d.modules.basic.arrow.util.ArrowVectorUtils; +import java.util.LinkedList; +import java.util.List; +import java.util.Queue; +import org.apache.arrow.memory.ArrowBuf; +import org.apache.arrow.vector.FieldVector; +import org.apache.arrow.vector.complex.ListVector; +import org.apache.arrow.vector.types.pojo.Field; + +public class ListArray extends Array { + private ListVector array; + + public static void instantiate() { + ObjectFactory.getFactory().register("vineyard::ListArray", new ListArrayResolver()); + } + + public ListArray( + ObjectMeta meta, + Queue bufs, + Queue valueCountList, + Field listVectorField) { + super(meta); + this.array = ListVector.empty("", Arrow.default_allocator); + + ArrowVectorUtils.buildArrowVector(this.array, bufs, valueCountList, listVectorField); + } + + public List get(int index) { + return this.array.getObject(index); + } + + @Override + public FieldVector getArray() { + return this.array; + } + + @Override + public boolean equals(java.lang.Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + ListArray that = (ListArray) o; + return Objects.equal(array, that.array); + } + + @Override + public int hashCode() { + return Objects.hashCode(array); + } +} + +class ListArrayResolver extends ObjectFactory.Resolver { + @Override + public Object resolve(ObjectMeta meta) { + Queue bufs = new LinkedList<>(); + Queue valueCountQueue = new LinkedList<>(); + + // bufs + int bufsNum = meta.getIntValue("bufs_num_"); + int valueCountNum = meta.getIntValue("value_count_num_"); + for (int i = 0; i < bufsNum; i++) { + ObjectMeta bufMeta = meta.getMemberMeta("buffer_" + String.valueOf(i) + "_"); + bufs.add(((Buffer) ObjectFactory.getFactory().resolve(bufMeta)).getBuffer()); + } + for (int i = 0; i < valueCountNum; i++) { + valueCountQueue.add(meta.getIntValue("value_count_" + String.valueOf(i) + "_")); + } + + Schema schema = (Schema) new SchemaResolver().resolve(meta.getMemberMeta("schema_")); + List fields = schema.getSchema().getFields(); + return new ListArray(meta, bufs, valueCountQueue, fields.get(0)); + } +} diff --git a/java/modules/basic/src/main/java/io/v6d/modules/basic/arrow/ListArrayBuilder.java b/java/modules/basic/src/main/java/io/v6d/modules/basic/arrow/ListArrayBuilder.java new file mode 100644 index 000000000..cc07ff6b5 --- /dev/null +++ b/java/modules/basic/src/main/java/io/v6d/modules/basic/arrow/ListArrayBuilder.java @@ -0,0 +1,109 @@ +/** Copyright 2020-2023 Alibaba Group Holding Limited. + * + * Licensed 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 io.v6d.modules.basic.arrow; + +import io.v6d.core.client.Client; +import io.v6d.core.client.IPCClient; +import io.v6d.core.client.ds.ObjectMeta; +import io.v6d.core.common.util.VineyardException; +import io.v6d.core.common.util.VineyardException.NotImplemented; +import io.v6d.modules.basic.arrow.util.ArrowVectorUtils; +import java.util.ArrayList; +import java.util.List; +import lombok.val; +import org.apache.arrow.memory.ArrowBuf; +import org.apache.arrow.vector.FieldVector; +import org.apache.arrow.vector.ValueVector; +import org.apache.arrow.vector.complex.ListVector; +import org.apache.arrow.vector.types.pojo.ArrowType.ArrowTypeID; +import org.apache.arrow.vector.types.pojo.Field; +import org.apache.arrow.vector.types.pojo.Schema; + +public class ListArrayBuilder implements ArrayBuilder { + private BufferBuilder[] bufferBuilders; + private ListVector array; + + private List valueCountList; + private SchemaBuilder listVectorSchemaBuilder; + + public ListArrayBuilder(IPCClient client, Field field) throws VineyardException { + List childFields = field.getChildren(); + if (childFields.size() != 1) { + throw new NotImplemented("ListArrayBuilder only support one child field"); + } + + this.array = ListVector.empty("", Arrow.default_allocator); + ArrowVectorUtils.buildArrowVector(this.array, field); + + while (childFields.get(0).getType().getTypeID() == ArrowTypeID.List) { + childFields = childFields.get(0).getChildren(); + } + FieldVector vector = array; + while (vector instanceof ListVector) { + vector = ((ListVector) vector).getDataVector(); + } + + List fields = new ArrayList<>(); + fields.add(field); + Schema schema = new Schema(fields); + listVectorSchemaBuilder = SchemaBuilder.fromSchema(schema); + } + + @Override + public void build(Client client) throws VineyardException { + valueCountList = ArrowVectorUtils.getValueCountOfArrowVector(array); + ArrowBuf[] buffers = ArrowVectorUtils.getArrowBuffers(array); + this.bufferBuilders = new BufferBuilder[buffers.length]; + for (int i = 0; i < buffers.length; i++) { + this.bufferBuilders[i] = + new BufferBuilder((IPCClient) client, buffers[i], buffers[i].capacity()); + } + } + + @Override + public ObjectMeta seal(Client client) throws VineyardException { + this.build(client); + val meta = ObjectMeta.empty(); + + meta.setTypename("vineyard::ListArray"); + meta.setValue("bufs_num_", this.bufferBuilders.length); + for (int i = 0; i < this.bufferBuilders.length; i++) { + meta.addMember( + "buffer_" + String.valueOf(i) + "_", this.bufferBuilders[i].seal(client)); + } + + meta.setValue("value_count_num_", valueCountList.size()); + for (int i = 0; i < valueCountList.size(); i++) { + meta.setValue("value_count_" + String.valueOf(i) + "_", valueCountList.get(i)); + } + + meta.addMember("schema_", listVectorSchemaBuilder.seal(client)); + return client.createMetaData(meta); + } + + @Override + public FieldVector getArray() { + return this.array; + } + + @Override + public void shrink(Client client, long size) throws VineyardException { + this.array.setValueCount((int) size); + } + + void set(int index, ValueVector value) { + this.array.copyFromSafe(0, index, value); + } +} diff --git a/java/modules/basic/src/main/java/io/v6d/modules/basic/arrow/MapArray.java b/java/modules/basic/src/main/java/io/v6d/modules/basic/arrow/MapArray.java new file mode 100644 index 000000000..dfd140554 --- /dev/null +++ b/java/modules/basic/src/main/java/io/v6d/modules/basic/arrow/MapArray.java @@ -0,0 +1,94 @@ +/** Copyright 2020-2023 Alibaba Group Holding Limited. + * + * Licensed 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 io.v6d.modules.basic.arrow; + +import com.google.common.base.Objects; +import io.v6d.core.client.ds.Object; +import io.v6d.core.client.ds.ObjectFactory; +import io.v6d.core.client.ds.ObjectMeta; +import io.v6d.modules.basic.arrow.util.ArrowVectorUtils; +import java.util.LinkedList; +import java.util.List; +import java.util.Queue; +import org.apache.arrow.memory.ArrowBuf; +import org.apache.arrow.vector.FieldVector; +import org.apache.arrow.vector.complex.MapVector; +import org.apache.arrow.vector.types.pojo.Field; + +public class MapArray extends Array { + private MapVector array; + + public static void instantiate() { + ObjectFactory.getFactory().register("vineyard::MapArray", new MapArrayResolver()); + } + + public MapArray( + ObjectMeta meta, + Queue bufs, + Queue valueCountList, + Field structVectorField) { + super(meta); + this.array = MapVector.empty("", Arrow.default_allocator, true); + ArrowVectorUtils.buildArrowVector(this.array, bufs, valueCountList, structVectorField); + } + + public List get(int index) { + return this.array.getObject(index); + } + + @Override + public FieldVector getArray() { + return this.array; + } + + @Override + public boolean equals(java.lang.Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + MapArray that = (MapArray) o; + return Objects.equal(array, that.array); + } + + @Override + public int hashCode() { + return Objects.hashCode(array); + } +} + +class MapArrayResolver extends ObjectFactory.Resolver { + @Override + public Object resolve(ObjectMeta meta) { + Queue bufs = new LinkedList<>(); + Queue valueCountQueue = new LinkedList<>(); + + int bufsNum = meta.getIntValue("bufs_num_"); + int valueCountNum = meta.getIntValue("value_count_num_"); + for (int i = 0; i < bufsNum; i++) { + ObjectMeta bufMeta = meta.getMemberMeta("buffer_" + String.valueOf(i) + "_"); + bufs.add(((Buffer) ObjectFactory.getFactory().resolve(bufMeta)).getBuffer()); + } + for (int i = 0; i < valueCountNum; i++) { + valueCountQueue.add(meta.getIntValue("value_count_" + String.valueOf(i) + "_")); + } + + Schema schema = (Schema) new SchemaResolver().resolve(meta.getMemberMeta("schema_")); + List fields = schema.getSchema().getFields(); + return new MapArray(meta, bufs, valueCountQueue, fields.get(0)); + } +} diff --git a/java/modules/basic/src/main/java/io/v6d/modules/basic/arrow/MapArrayBuilder.java b/java/modules/basic/src/main/java/io/v6d/modules/basic/arrow/MapArrayBuilder.java new file mode 100644 index 000000000..35fd15464 --- /dev/null +++ b/java/modules/basic/src/main/java/io/v6d/modules/basic/arrow/MapArrayBuilder.java @@ -0,0 +1,95 @@ +/** Copyright 2020-2023 Alibaba Group Holding Limited. + * + * Licensed 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 io.v6d.modules.basic.arrow; + +import io.v6d.core.client.Client; +import io.v6d.core.client.IPCClient; +import io.v6d.core.client.ds.ObjectMeta; +import io.v6d.core.common.util.VineyardException; +import io.v6d.modules.basic.arrow.util.ArrowVectorUtils; +import java.util.ArrayList; +import java.util.List; +import lombok.val; +import org.apache.arrow.memory.ArrowBuf; +import org.apache.arrow.vector.FieldVector; +import org.apache.arrow.vector.ValueVector; +import org.apache.arrow.vector.complex.MapVector; +import org.apache.arrow.vector.types.pojo.Field; +import org.apache.arrow.vector.types.pojo.Schema; + +public class MapArrayBuilder implements ArrayBuilder { + private BufferBuilder[] bufferBuilders; + private MapVector array; + private List valueCountList; + private SchemaBuilder structVectorSchemaBuilder; + + public MapArrayBuilder(IPCClient client, Field field) throws VineyardException { + this.array = MapVector.empty("", Arrow.default_allocator, true); + + ArrowVectorUtils.buildArrowVector(this.array, field); + + List fields = new ArrayList<>(); + fields.add(field); + Schema schema = new Schema(fields); + structVectorSchemaBuilder = SchemaBuilder.fromSchema(schema); + } + + @Override + public void build(Client client) throws VineyardException { + valueCountList = ArrowVectorUtils.getValueCountOfArrowVector(array); + ArrowBuf[] buffers = ArrowVectorUtils.getArrowBuffers(array); + + this.bufferBuilders = new BufferBuilder[buffers.length]; + for (int i = 0; i < buffers.length; i++) { + this.bufferBuilders[i] = + new BufferBuilder((IPCClient) client, buffers[i], buffers[i].capacity()); + } + } + + @Override + public ObjectMeta seal(Client client) throws VineyardException { + this.build(client); + val meta = ObjectMeta.empty(); + + meta.setTypename("vineyard::MapArray"); + meta.setValue("bufs_num_", this.bufferBuilders.length); + for (int i = 0; i < this.bufferBuilders.length; i++) { + meta.addMember( + "buffer_" + String.valueOf(i) + "_", this.bufferBuilders[i].seal(client)); + } + + meta.setValue("value_count_num_", valueCountList.size()); + for (int i = 0; i < valueCountList.size(); i++) { + meta.setValue("value_count_" + String.valueOf(i) + "_", valueCountList.get(i)); + } + + meta.addMember("schema_", structVectorSchemaBuilder.seal(client)); + return client.createMetaData(meta); + } + + @Override + public FieldVector getArray() { + return this.array; + } + + @Override + public void shrink(Client client, long size) throws VineyardException { + this.array.setValueCount((int) size); + } + + void set(int index, ValueVector value) { + this.array.copyFromSafe(0, index, value); + } +} diff --git a/java/modules/basic/src/main/java/io/v6d/modules/basic/arrow/NullArray.java b/java/modules/basic/src/main/java/io/v6d/modules/basic/arrow/NullArray.java index f9b901897..6e28d5ddd 100644 --- a/java/modules/basic/src/main/java/io/v6d/modules/basic/arrow/NullArray.java +++ b/java/modules/basic/src/main/java/io/v6d/modules/basic/arrow/NullArray.java @@ -23,7 +23,6 @@ import org.apache.arrow.vector.NullVector; import org.apache.arrow.vector.ipc.message.ArrowFieldNode; -/** Hello world! */ public class NullArray extends Array { private NullVector array; diff --git a/java/modules/basic/src/main/java/io/v6d/modules/basic/arrow/RecordBatch.java b/java/modules/basic/src/main/java/io/v6d/modules/basic/arrow/RecordBatch.java index cf520bae3..0f0e8b752 100644 --- a/java/modules/basic/src/main/java/io/v6d/modules/basic/arrow/RecordBatch.java +++ b/java/modules/basic/src/main/java/io/v6d/modules/basic/arrow/RecordBatch.java @@ -18,6 +18,7 @@ import io.v6d.core.client.ds.Object; import io.v6d.core.client.ds.ObjectFactory; import io.v6d.core.client.ds.ObjectMeta; +import io.v6d.modules.basic.arrow.util.ObjectResolver; import io.v6d.modules.basic.columnar.ColumnarData; import java.util.List; import java.util.stream.Collectors; @@ -28,28 +29,38 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -/** Hello world! */ public class RecordBatch extends Object { private static final Logger logger = LoggerFactory.getLogger(RecordBatch.class); private final VectorSchemaRoot batch; + private ObjectResolver resolver; public static void instantiate() { Schema.instantiate(); BooleanArray.instantiate(); + Int8Array.instantiate(); + Int16Array.instantiate(); Int32Array.instantiate(); Int64Array.instantiate(); FloatArray.instantiate(); DoubleArray.instantiate(); StringArray.instantiate(); + VarBinaryArray.instantiate(); LargeStringArray.instantiate(); NullArray.instantiate(); + DateArray.instantiate(); + TimestampArray.instantiate(); + DecimalArray.instantiate(); + ListArray.instantiate(); + StructArray.instantiate(); + MapArray.instantiate(); ObjectFactory.getFactory().register("vineyard::RecordBatch", new RecordBatchResolver()); } public RecordBatch(final ObjectMeta meta, Schema schema, List vectors, int nrow) { super(meta); this.batch = new VectorSchemaRoot(schema.getSchema(), vectors, nrow); + resolver = new ObjectResolver(); } public VectorSchemaRoot getBatch() { @@ -64,8 +75,17 @@ public long getColumnCount() { return batch.getFieldVectors().size(); } + public void setResolver(ObjectResolver resolver) { + this.resolver = resolver; + } + public ColumnarData[] columar() { - return batch.getFieldVectors().stream().map(ColumnarData::new).toArray(ColumnarData[]::new); + List vectors = batch.getFieldVectors(); + ColumnarData[] columnarData = new ColumnarData[vectors.size()]; + for (int i = 0; i < vectors.size(); i++) { + columnarData[i] = new ColumnarData(vectors.get(i), resolver); + } + return columnarData; } @Override diff --git a/java/modules/basic/src/main/java/io/v6d/modules/basic/arrow/RecordBatchBuilder.java b/java/modules/basic/src/main/java/io/v6d/modules/basic/arrow/RecordBatchBuilder.java index 08ea43dfa..000c361ac 100644 --- a/java/modules/basic/src/main/java/io/v6d/modules/basic/arrow/RecordBatchBuilder.java +++ b/java/modules/basic/src/main/java/io/v6d/modules/basic/arrow/RecordBatchBuilder.java @@ -21,10 +21,13 @@ import io.v6d.core.client.ds.ObjectBuilder; import io.v6d.core.client.ds.ObjectMeta; import io.v6d.core.common.util.VineyardException; +import io.v6d.modules.basic.arrow.util.ObjectTransformer; import io.v6d.modules.basic.columnar.ColumnarDataBuilder; import java.util.ArrayList; import java.util.List; import lombok.*; +import org.apache.arrow.vector.types.TimeUnit; +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.slf4j.Logger; @@ -37,6 +40,7 @@ public class RecordBatchBuilder implements ObjectBuilder { private final SchemaBuilder schemaBuilder; private List arrayBuilders; private List columnBuilders; + private ObjectTransformer transformers; private boolean schemaMutable = true; @@ -47,8 +51,15 @@ public RecordBatchBuilder(final IPCClient client, int rows) { public RecordBatchBuilder(final IPCClient client, final Schema schema, int rows) throws VineyardException { + this(client, schema, rows, new ObjectTransformer()); + } + + public RecordBatchBuilder( + final IPCClient client, final Schema schema, int rows, ObjectTransformer transformers) + throws VineyardException { this.rows = rows; schemaBuilder = SchemaBuilder.fromSchema(schema); + this.transformers = transformers; this.finishSchema(client); } @@ -81,7 +92,8 @@ public void finishSchema(IPCClient client) throws VineyardException { for (val field : schemaBuilder.getFields()) { val builder = arrayBuilderFor(client, field); arrayBuilders.add(builder); - columnBuilders.add(builder.columnar()); + ColumnarDataBuilder columnarDataBuilder = builder.columnar(transformers); + columnBuilders.add(columnarDataBuilder); } } @@ -157,11 +169,39 @@ private ArrayBuilder arrayBuilderFor(IPCClient client, Field field) throws Viney } else if (field.getType().equals(Arrow.Type.LargeVarChar)) { return new LargeStringArrayBuilder(client, rows); } else if (field.getType().equals(Arrow.Type.VarBinary)) { - return new StringArrayBuilder(client, rows); + return new VarBinaryArrayBuilder(client, rows); } else if (field.getType().equals(Arrow.Type.LargeVarBinary)) { return new LargeStringArrayBuilder(client, rows); } else if (field.getType().equals(Arrow.Type.Null)) { return new NullArrayBuilder(client, rows); + } else if (field.getType().equals(Arrow.Type.TinyInt)) { + return new Int8ArrayBuilder(client, rows); + } else if (field.getType().equals(Arrow.Type.SmallInt)) { + return new Int16ArrayBuilder(client, rows); + } else if (field.getType().equals(Arrow.Type.Date)) { + return new DateArrayBuilder(client, rows); + } else if (field.getType().equals(Arrow.Type.TimeStampMicro)) { + return new TimestampArrayBuilder(client, rows, TimeUnit.MICROSECOND); + } else if (field.getType().equals(Arrow.Type.TimeStampMilli)) { + return new TimestampArrayBuilder(client, rows, TimeUnit.MILLISECOND); + } else if (field.getType().equals(Arrow.Type.TimeStampNano)) { + return new TimestampArrayBuilder(client, rows, TimeUnit.NANOSECOND); + } else if (field.getType().equals(Arrow.Type.TimeStampSec)) { + return new TimestampArrayBuilder(client, rows, TimeUnit.SECOND); + } else if (field.getType() instanceof ArrowType.Decimal) { + ArrowType.Decimal decimal = (ArrowType.Decimal) field.getType(); + return new DecimalArrayBuilder( + client, + rows, + decimal.getPrecision(), + decimal.getScale(), + decimal.getBitWidth()); + } else if (field.getType().equals(Arrow.Type.List)) { + return new ListArrayBuilder(client, field); + } else if (field.getType().equals(Arrow.Type.Struct)) { + return new StructArrayBuilder(client, field); + } else if (field.getType().equals(Arrow.Type.Map)) { + return new MapArrayBuilder(client, field); } else { throw new VineyardException.NotImplemented( "array builder for type " + field.getType() + " is not supported"); diff --git a/java/modules/basic/src/main/java/io/v6d/modules/basic/arrow/Schema.java b/java/modules/basic/src/main/java/io/v6d/modules/basic/arrow/Schema.java index c4bb812f3..edee1fabc 100644 --- a/java/modules/basic/src/main/java/io/v6d/modules/basic/arrow/Schema.java +++ b/java/modules/basic/src/main/java/io/v6d/modules/basic/arrow/Schema.java @@ -33,7 +33,6 @@ import org.apache.arrow.util.Collections2; import org.apache.arrow.vector.types.pojo.Field; -/** Hello world! */ public class Schema extends Object implements Serializable { private org.apache.arrow.vector.types.pojo.Schema schema; diff --git a/java/modules/basic/src/main/java/io/v6d/modules/basic/arrow/StringArray.java b/java/modules/basic/src/main/java/io/v6d/modules/basic/arrow/StringArray.java index 756a8680c..71b3b6542 100644 --- a/java/modules/basic/src/main/java/io/v6d/modules/basic/arrow/StringArray.java +++ b/java/modules/basic/src/main/java/io/v6d/modules/basic/arrow/StringArray.java @@ -19,13 +19,14 @@ import io.v6d.core.client.ds.ObjectFactory; import io.v6d.core.client.ds.ObjectMeta; import java.util.Arrays; +import java.util.List; import lombok.*; +import org.apache.arrow.memory.ArrowBuf; import org.apache.arrow.vector.FieldVector; import org.apache.arrow.vector.VarCharVector; import org.apache.arrow.vector.ipc.message.ArrowFieldNode; import org.apache.arrow.vector.util.Text; -/** Hello world! */ public class StringArray extends Array { private VarCharVector array; @@ -36,13 +37,10 @@ public static void instantiate() { ObjectFactory.getFactory().register("vineyard::StringArray", new StringArrayResolver()); } - public StringArray(final ObjectMeta meta, Buffer buffer, Buffer offset, long length) { + public StringArray(final ObjectMeta meta, List buffers, long length, int nullCount) { super(meta); this.array = new VarCharVector("", Arrow.default_allocator); - this.array.loadFieldBuffers( - new ArrowFieldNode(length, 0), - Arrays.asList(null, offset.getBuffer(), buffer.getBuffer())); - this.array.setValueCount((int) length); + this.array.loadFieldBuffers(new ArrowFieldNode(length, nullCount), buffers); } public byte[] get(int index) { @@ -79,10 +77,21 @@ public int hashCode() { class StringArrayResolver extends ObjectFactory.Resolver { @Override public Object resolve(final ObjectMeta meta) { - val buffer = - (Buffer) ObjectFactory.getFactory().resolve(meta.getMemberMeta("buffer_data_")); - val offsets_buffer = + Buffer data_buffer = + (Buffer) ObjectFactory.getFactory().resolve(meta.getMemberMeta("buffer_")); + Buffer offsets_buffer = (Buffer) ObjectFactory.getFactory().resolve(meta.getMemberMeta("buffer_offsets_")); - return new StringArray(meta, buffer, offsets_buffer, meta.getLongValue("length_")); + Buffer validity_buffer = + (Buffer) ObjectFactory.getFactory().resolve(meta.getMemberMeta("null_bitmap_")); + int null_count = meta.getIntValue("null_count_"); + int length = meta.getIntValue("length_"); + return new StringArray( + meta, + Arrays.asList( + validity_buffer.getBuffer(), + offsets_buffer.getBuffer(), + data_buffer.getBuffer()), + length, + null_count); } } diff --git a/java/modules/basic/src/main/java/io/v6d/modules/basic/arrow/StringArrayBuilder.java b/java/modules/basic/src/main/java/io/v6d/modules/basic/arrow/StringArrayBuilder.java index 4554b5e75..d8fd1df47 100644 --- a/java/modules/basic/src/main/java/io/v6d/modules/basic/arrow/StringArrayBuilder.java +++ b/java/modules/basic/src/main/java/io/v6d/modules/basic/arrow/StringArrayBuilder.java @@ -19,6 +19,7 @@ import io.v6d.core.client.ds.ObjectMeta; import io.v6d.core.common.util.VineyardException; import lombok.*; +import org.apache.arrow.memory.ArrowBuf; import org.apache.arrow.vector.FieldVector; import org.apache.arrow.vector.VarCharVector; import org.apache.arrow.vector.util.Text; @@ -26,8 +27,9 @@ public class StringArrayBuilder implements ArrayBuilder { private VarCharVector array; - private BufferBuilder data_buffer_builder; - private BufferBuilder offset_buffer_builder; + private BufferBuilder dataBufferBuilder; + private BufferBuilder offsetBufferBuilder; + private BufferBuilder validityBufferBuilder; public StringArrayBuilder(IPCClient client, final VarCharVector vector) throws VineyardException { @@ -49,10 +51,15 @@ public void build(Client client) throws VineyardException { ((long) this.array.getValueCount()) * VarCharVector.OFFSET_WIDTH); val data_buffer = this.array.getDataBuffer(); - this.data_buffer_builder = + ArrowBuf validity_buffer = this.array.getValidityBuffer(); + val validity_buffer_size = validity_buffer.capacity(); + + this.dataBufferBuilder = new BufferBuilder((IPCClient) client, data_buffer, data_buffer_size); - this.offset_buffer_builder = + this.offsetBufferBuilder = new BufferBuilder((IPCClient) client, offset_buffer, offset_buffer_size); + this.validityBufferBuilder = + new BufferBuilder((IPCClient) client, validity_buffer, validity_buffer_size); } @Override @@ -62,11 +69,11 @@ public ObjectMeta seal(Client client) throws VineyardException { meta.setTypename("vineyard::BaseBinaryArray"); meta.setNBytes(array.getBufferSizeFor(array.getValueCount())); meta.setValue("length_", array.getValueCount()); - meta.setValue("null_count_", 0); + meta.setValue("null_count_", array.getNullCount()); meta.setValue("offset_", 0); - meta.addMember("buffer_data_", data_buffer_builder.seal(client)); - meta.addMember("buffer_offsets_", offset_buffer_builder.seal(client)); - meta.addMember("null_bitmap_", BufferBuilder.empty(client)); + meta.addMember("buffer_", dataBufferBuilder.seal(client)); + meta.addMember("buffer_offsets_", offsetBufferBuilder.seal(client)); + meta.addMember("null_bitmap_", validityBufferBuilder.seal(client)); return client.createMetaData(meta); } diff --git a/java/modules/basic/src/main/java/io/v6d/modules/basic/arrow/StructArray.java b/java/modules/basic/src/main/java/io/v6d/modules/basic/arrow/StructArray.java new file mode 100644 index 000000000..0c19dd6dd --- /dev/null +++ b/java/modules/basic/src/main/java/io/v6d/modules/basic/arrow/StructArray.java @@ -0,0 +1,97 @@ +/** Copyright 2020-2023 Alibaba Group Holding Limited. + * + * Licensed 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 io.v6d.modules.basic.arrow; + +import com.google.common.base.Objects; +import io.v6d.core.client.ds.Object; +import io.v6d.core.client.ds.ObjectFactory; +import io.v6d.core.client.ds.ObjectMeta; +import io.v6d.core.common.util.VineyardException; +import io.v6d.modules.basic.arrow.util.ArrowVectorUtils; +import java.util.LinkedList; +import java.util.List; +import java.util.Queue; +import org.apache.arrow.memory.ArrowBuf; +import org.apache.arrow.vector.FieldVector; +import org.apache.arrow.vector.complex.StructVector; +import org.apache.arrow.vector.types.pojo.Field; + +public class StructArray extends Array { + private StructVector array; + + public static void instantiate() { + ObjectFactory.getFactory().register("vineyard::StructArray", new StructArrayResolver()); + } + + public StructArray( + ObjectMeta meta, + Queue bufs, + Queue valueCountList, + Field structVectorField) { + super(meta); + this.array = StructVector.empty("", Arrow.default_allocator); + + ArrowVectorUtils.buildArrowVector(this.array, bufs, valueCountList, structVectorField); + } + + public List get(int index) throws VineyardException { + throw new UnsupportedOperationException(); + } + + @Override + public FieldVector getArray() { + return this.array; + } + + @Override + public boolean equals(java.lang.Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + StructArray that = (StructArray) o; + return Objects.equal(array, that.array); + } + + @Override + public int hashCode() { + return Objects.hashCode(array); + } +} + +class StructArrayResolver extends ObjectFactory.Resolver { + @Override + public Object resolve(ObjectMeta meta) { + Queue bufs = new LinkedList<>(); + Queue valueCountQueue = new LinkedList<>(); + + // bufs + int bufsNum = meta.getIntValue("bufs_num_"); + int valueCountNum = meta.getIntValue("value_count_num_"); + for (int i = 0; i < bufsNum; i++) { + ObjectMeta bufMeta = meta.getMemberMeta("buffer_" + String.valueOf(i) + "_"); + bufs.add(((Buffer) ObjectFactory.getFactory().resolve(bufMeta)).getBuffer()); + } + for (int i = 0; i < valueCountNum; i++) { + valueCountQueue.add(meta.getIntValue("value_count_" + String.valueOf(i) + "_")); + } + + Schema schema = (Schema) new SchemaResolver().resolve(meta.getMemberMeta("schema_")); + List fields = schema.getSchema().getFields(); + return new StructArray(meta, bufs, valueCountQueue, fields.get(0)); + } +} diff --git a/java/modules/basic/src/main/java/io/v6d/modules/basic/arrow/StructArrayBuilder.java b/java/modules/basic/src/main/java/io/v6d/modules/basic/arrow/StructArrayBuilder.java new file mode 100644 index 000000000..e618ac0aa --- /dev/null +++ b/java/modules/basic/src/main/java/io/v6d/modules/basic/arrow/StructArrayBuilder.java @@ -0,0 +1,94 @@ +/** Copyright 2020-2023 Alibaba Group Holding Limited. + * + * Licensed 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 io.v6d.modules.basic.arrow; + +import io.v6d.core.client.Client; +import io.v6d.core.client.IPCClient; +import io.v6d.core.client.ds.ObjectMeta; +import io.v6d.core.common.util.VineyardException; +import io.v6d.modules.basic.arrow.util.ArrowVectorUtils; +import java.util.ArrayList; +import java.util.List; +import lombok.val; +import org.apache.arrow.memory.ArrowBuf; +import org.apache.arrow.vector.FieldVector; +import org.apache.arrow.vector.ValueVector; +import org.apache.arrow.vector.complex.StructVector; +import org.apache.arrow.vector.types.pojo.Field; +import org.apache.arrow.vector.types.pojo.Schema; + +public class StructArrayBuilder implements ArrayBuilder { + private BufferBuilder[] bufferBuilders; + private StructVector array; + private List valueCountList; + private SchemaBuilder structVectorSchemaBuilder; + + public StructArrayBuilder(IPCClient client, Field field) throws VineyardException { + this.array = StructVector.empty("", Arrow.default_allocator); + ArrowVectorUtils.buildArrowVector(this.array, field); + + List fields = new ArrayList<>(); + fields.add(field); + Schema schema = new Schema(fields); + structVectorSchemaBuilder = SchemaBuilder.fromSchema(schema); + } + + @Override + public void build(Client client) throws VineyardException { + valueCountList = ArrowVectorUtils.getValueCountOfArrowVector(array); + ArrowBuf[] buffers = ArrowVectorUtils.getArrowBuffers(array); + + this.bufferBuilders = new BufferBuilder[buffers.length]; + for (int i = 0; i < buffers.length; i++) { + this.bufferBuilders[i] = + new BufferBuilder((IPCClient) client, buffers[i], buffers[i].capacity()); + } + } + + @Override + public ObjectMeta seal(Client client) throws VineyardException { + this.build(client); + val meta = ObjectMeta.empty(); + + meta.setTypename("vineyard::StructArray"); + meta.setValue("bufs_num_", this.bufferBuilders.length); + for (int i = 0; i < this.bufferBuilders.length; i++) { + meta.addMember( + "buffer_" + String.valueOf(i) + "_", this.bufferBuilders[i].seal(client)); + } + + meta.setValue("value_count_num_", valueCountList.size()); + for (int i = 0; i < valueCountList.size(); i++) { + meta.setValue("value_count_" + String.valueOf(i) + "_", valueCountList.get(i)); + } + + meta.addMember("schema_", structVectorSchemaBuilder.seal(client)); + return client.createMetaData(meta); + } + + @Override + public FieldVector getArray() { + return this.array; + } + + @Override + public void shrink(Client client, long size) throws VineyardException { + this.array.setValueCount((int) size); + } + + void set(int index, ValueVector value) { + this.array.copyFromSafe(0, index, value); + } +} diff --git a/java/modules/basic/src/main/java/io/v6d/modules/basic/arrow/Table.java b/java/modules/basic/src/main/java/io/v6d/modules/basic/arrow/Table.java index 7b61c8fd2..7c4919af7 100644 --- a/java/modules/basic/src/main/java/io/v6d/modules/basic/arrow/Table.java +++ b/java/modules/basic/src/main/java/io/v6d/modules/basic/arrow/Table.java @@ -24,7 +24,6 @@ import lombok.val; import org.apache.arrow.vector.VectorSchemaRoot; -/** Hello world! */ public class Table extends Object { private final int rows; private final int columns; diff --git a/java/modules/basic/src/main/java/io/v6d/modules/basic/arrow/TimestampArray.java b/java/modules/basic/src/main/java/io/v6d/modules/basic/arrow/TimestampArray.java new file mode 100644 index 000000000..32543d500 --- /dev/null +++ b/java/modules/basic/src/main/java/io/v6d/modules/basic/arrow/TimestampArray.java @@ -0,0 +1,110 @@ +/** Copyright 2020-2023 Alibaba Group Holding Limited. + * + * Licensed 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 io.v6d.modules.basic.arrow; + +import com.google.common.base.Objects; +import io.v6d.core.client.ds.Object; +import io.v6d.core.client.ds.ObjectFactory; +import io.v6d.core.client.ds.ObjectMeta; +import java.util.Arrays; +import java.util.List; +import org.apache.arrow.memory.ArrowBuf; +import org.apache.arrow.vector.FieldVector; +import org.apache.arrow.vector.TimeStampMicroVector; +import org.apache.arrow.vector.TimeStampMilliVector; +import org.apache.arrow.vector.TimeStampNanoVector; +import org.apache.arrow.vector.TimeStampSecVector; +import org.apache.arrow.vector.TimeStampVector; +import org.apache.arrow.vector.ipc.message.ArrowFieldNode; +import org.apache.arrow.vector.types.TimeUnit; + +public class TimestampArray extends Array { + private TimeStampVector array; + private TimeUnit timeUnit; + + public static void instantiate() { + for (TimeUnit timeUnit : TimeUnit.values()) { + ObjectFactory.getFactory() + .register( + "vineyard::Timestamp<" + timeUnit.toString() + ">", + new TimestampArrayResolver()); + } + } + + public TimestampArray( + ObjectMeta meta, List buffers, long length, int nullCount, short timeUnitID) { + super(meta); + switch (TimeUnit.values()[timeUnitID]) { + case MICROSECOND: + this.array = new TimeStampMicroVector("", Arrow.default_allocator); + break; + case NANOSECOND: + this.array = new TimeStampNanoVector("", Arrow.default_allocator); + break; + case SECOND: + this.array = new TimeStampSecVector("", Arrow.default_allocator); + break; + case MILLISECOND: + this.array = new TimeStampMilliVector("", Arrow.default_allocator); + break; + } + this.array.loadFieldBuffers(new ArrowFieldNode(length, nullCount), buffers); + this.timeUnit = TimeUnit.values()[timeUnitID]; + } + + public long get(int index) { + return this.array.get(index); + } + + @Override + public FieldVector getArray() { + return this.array; + } + + @Override + public boolean equals(java.lang.Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + TimestampArray that = (TimestampArray) o; + return Objects.equal(array, that.array); + } + + @Override + public int hashCode() { + return Objects.hashCode(array); + } +} + +class TimestampArrayResolver extends ObjectFactory.Resolver { + @Override + public Object resolve(ObjectMeta meta) { + Buffer data_buffer = + (Buffer) ObjectFactory.getFactory().resolve(meta.getMemberMeta("buffer_")); + Buffer validity_buffer = + (Buffer) ObjectFactory.getFactory().resolve(meta.getMemberMeta("null_bitmap_")); + int null_count = meta.getIntValue("null_count_"); + int length = meta.getIntValue("length_"); + return new TimestampArray( + meta, + Arrays.asList(validity_buffer.getBuffer(), data_buffer.getBuffer()), + length, + null_count, + (short) meta.getLongValue("time_unit_id_")); + } +} diff --git a/java/modules/basic/src/main/java/io/v6d/modules/basic/arrow/TimestampArrayBuilder.java b/java/modules/basic/src/main/java/io/v6d/modules/basic/arrow/TimestampArrayBuilder.java new file mode 100644 index 000000000..f7f34d67c --- /dev/null +++ b/java/modules/basic/src/main/java/io/v6d/modules/basic/arrow/TimestampArrayBuilder.java @@ -0,0 +1,101 @@ +/** Copyright 2020-2023 Alibaba Group Holding Limited. + * + * Licensed 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 io.v6d.modules.basic.arrow; + +import io.v6d.core.client.Client; +import io.v6d.core.client.IPCClient; +import io.v6d.core.client.ds.ObjectMeta; +import io.v6d.core.common.util.VineyardException; +import io.v6d.core.common.util.VineyardException.NotImplemented; +import java.util.Arrays; +import lombok.val; +import org.apache.arrow.memory.ArrowBuf; +import org.apache.arrow.vector.FieldVector; +import org.apache.arrow.vector.TimeStampMicroVector; +import org.apache.arrow.vector.TimeStampMilliVector; +import org.apache.arrow.vector.TimeStampNanoVector; +import org.apache.arrow.vector.TimeStampSecVector; +import org.apache.arrow.vector.TimeStampVector; +import org.apache.arrow.vector.ipc.message.ArrowFieldNode; +import org.apache.arrow.vector.types.TimeUnit; + +public class TimestampArrayBuilder implements ArrayBuilder { + private BufferBuilder dataBufferBuilder; + private BufferBuilder validityBufferBuilder; + private TimeStampVector array; + private TimeUnit timeUnit; + + public TimestampArrayBuilder(IPCClient client, long length, TimeUnit timeUnit) + throws VineyardException { + this.timeUnit = timeUnit; + switch (timeUnit) { + case SECOND: + this.array = new TimeStampSecVector("", Arrow.default_allocator); + break; + case MILLISECOND: + this.array = new TimeStampMilliVector("", Arrow.default_allocator); + break; + case MICROSECOND: + this.array = new TimeStampMicroVector("", Arrow.default_allocator); + break; + case NANOSECOND: + this.array = new TimeStampNanoVector("", Arrow.default_allocator); + break; + default: + throw new NotImplemented("Unsupported time unit: " + timeUnit); + } + this.dataBufferBuilder = + new BufferBuilder(client, this.array.getBufferSizeFor((int) length)); + this.array.loadFieldBuffers( + new ArrowFieldNode(length, 0), Arrays.asList(null, dataBufferBuilder.getBuffer())); + } + + @Override + public void build(Client client) throws VineyardException { + ArrowBuf validityBuffer = this.array.getValidityBuffer(); + validityBufferBuilder = + new BufferBuilder((IPCClient) client, validityBuffer, validityBuffer.capacity()); + } + + @Override + public ObjectMeta seal(Client client) throws VineyardException { + this.build(client); + val meta = ObjectMeta.empty(); + meta.setTypename("vineyard::Timestamp<" + timeUnit.toString() + ">"); + meta.setNBytes(array.getBufferSizeFor(array.getValueCount())); + meta.setValue("length_", array.getValueCount()); + meta.setValue("null_count_", array.getNullCount()); + meta.setValue("offset_", 0); + meta.addMember("buffer_", dataBufferBuilder.seal(client)); + meta.addMember("null_bitmap_", validityBufferBuilder.seal(client)); + meta.setValue("time_unit_id_", timeUnit.getFlatbufID()); + return client.createMetaData(meta); + } + + @Override + public FieldVector getArray() { + return this.array; + } + + @Override + public void shrink(Client client, long size) throws VineyardException { + this.dataBufferBuilder.shrink(client, this.array.getBufferSizeFor((int) size)); + this.array.setValueCount((int) size); + } + + void set(int index, long value) { + this.array.set(index, value); + } +} diff --git a/java/modules/basic/src/main/java/io/v6d/modules/basic/arrow/VarBinaryArray.java b/java/modules/basic/src/main/java/io/v6d/modules/basic/arrow/VarBinaryArray.java new file mode 100644 index 000000000..5cf74e331 --- /dev/null +++ b/java/modules/basic/src/main/java/io/v6d/modules/basic/arrow/VarBinaryArray.java @@ -0,0 +1,88 @@ +/** Copyright 2020-2023 Alibaba Group Holding Limited. + * + * Licensed 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 io.v6d.modules.basic.arrow; + +import com.google.common.base.Objects; +import io.v6d.core.client.ds.Object; +import io.v6d.core.client.ds.ObjectFactory; +import io.v6d.core.client.ds.ObjectMeta; +import java.util.Arrays; +import java.util.List; +import lombok.*; +import org.apache.arrow.memory.ArrowBuf; +import org.apache.arrow.vector.FieldVector; +import org.apache.arrow.vector.VarBinaryVector; +import org.apache.arrow.vector.ipc.message.ArrowFieldNode; + +public class VarBinaryArray extends Array { + private VarBinaryVector array; + + public static void instantiate() { + ObjectFactory.getFactory() + .register("vineyard::VarBinaryArray", new VarBinaryArrayResolver()); + } + + public VarBinaryArray( + final ObjectMeta meta, List buffers, int length, int nullCount) { + super(meta); + this.array = new VarBinaryVector("", Arrow.default_allocator); + this.array.loadFieldBuffers(new ArrowFieldNode(length, nullCount), buffers); + this.array.setValueCount((int) length); + } + + @Override + public FieldVector getArray() { + return this.array; + } + + @Override + public boolean equals(java.lang.Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + VarBinaryArray that = (VarBinaryArray) o; + return Objects.equal(array, that.array); + } + + @Override + public int hashCode() { + return Objects.hashCode(array); + } +} + +class VarBinaryArrayResolver extends ObjectFactory.Resolver { + @Override + public Object resolve(final ObjectMeta meta) { + Buffer data_buffer = + (Buffer) ObjectFactory.getFactory().resolve(meta.getMemberMeta("buffer_")); + Buffer offsets_buffer = + (Buffer) ObjectFactory.getFactory().resolve(meta.getMemberMeta("buffer_offsets_")); + Buffer validityBuffer = + (Buffer) ObjectFactory.getFactory().resolve(meta.getMemberMeta("null_bitmap_")); + int nullCount = meta.getIntValue("null_count_"); + int length = meta.getIntValue("length_"); + return new VarBinaryArray( + meta, + Arrays.asList( + validityBuffer.getBuffer(), + offsets_buffer.getBuffer(), + data_buffer.getBuffer()), + length, + nullCount); + } +} diff --git a/java/modules/basic/src/main/java/io/v6d/modules/basic/arrow/VarBinaryArrayBuilder.java b/java/modules/basic/src/main/java/io/v6d/modules/basic/arrow/VarBinaryArrayBuilder.java new file mode 100644 index 000000000..1342afda6 --- /dev/null +++ b/java/modules/basic/src/main/java/io/v6d/modules/basic/arrow/VarBinaryArrayBuilder.java @@ -0,0 +1,84 @@ +/** Copyright 2020-2023 Alibaba Group Holding Limited. + * + * Licensed 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 io.v6d.modules.basic.arrow; + +import io.v6d.core.client.Client; +import io.v6d.core.client.IPCClient; +import io.v6d.core.client.ds.ObjectMeta; +import io.v6d.core.common.util.VineyardException; +import lombok.*; +import org.apache.arrow.memory.ArrowBuf; +import org.apache.arrow.vector.FieldVector; +import org.apache.arrow.vector.VarBinaryVector; + +public class VarBinaryArrayBuilder implements ArrayBuilder { + private VarBinaryVector array; + + private BufferBuilder dataBufferBuilder; + private BufferBuilder offsetBufferBuilder; + private BufferBuilder validityBufferBuilder; + + public VarBinaryArrayBuilder(IPCClient client, long length) throws VineyardException { + this.array = new VarBinaryVector("", Arrow.default_allocator); + this.array.setValueCount((int) length); + } + + @Override + public void build(Client client) throws VineyardException { + val offset_buffer_size = (this.array.getValueCount() + 1) * VarBinaryVector.OFFSET_WIDTH; + val offset_buffer = this.array.getOffsetBuffer(); + + val data_buffer_size = + offset_buffer.getLong( + ((long) this.array.getValueCount()) * VarBinaryVector.OFFSET_WIDTH); + val data_buffer = this.array.getDataBuffer(); + + ArrowBuf validityBuffer = array.getValidityBuffer(); + validityBufferBuilder = + new BufferBuilder((IPCClient) client, validityBuffer, validityBuffer.capacity()); + + this.dataBufferBuilder = + new BufferBuilder((IPCClient) client, data_buffer, data_buffer_size); + this.offsetBufferBuilder = + new BufferBuilder((IPCClient) client, offset_buffer, offset_buffer_size); + this.validityBufferBuilder = + new BufferBuilder((IPCClient) client, validityBuffer, validityBuffer.capacity()); + } + + @Override + public ObjectMeta seal(Client client) throws VineyardException { + this.build(client); + val meta = ObjectMeta.empty(); + meta.setTypename("vineyard::VarBinaryArray"); + meta.setNBytes(array.getBufferSizeFor(array.getValueCount())); + meta.setValue("length_", array.getValueCount()); + meta.setValue("null_count_", array.getNullCount()); + meta.setValue("offset_", 0); + meta.addMember("buffer_", dataBufferBuilder.seal(client)); + meta.addMember("buffer_offsets_", offsetBufferBuilder.seal(client)); + meta.addMember("null_bitmap_", validityBufferBuilder.seal(client)); + return client.createMetaData(meta); + } + + @Override + public FieldVector getArray() { + return this.array; + } + + @Override + public void shrink(Client client, long size) throws VineyardException { + this.array.setValueCount((int) size); + } +} diff --git a/java/modules/basic/src/main/java/io/v6d/modules/basic/arrow/util/ArrowVectorUtils.java b/java/modules/basic/src/main/java/io/v6d/modules/basic/arrow/util/ArrowVectorUtils.java new file mode 100644 index 000000000..01f3583ea --- /dev/null +++ b/java/modules/basic/src/main/java/io/v6d/modules/basic/arrow/util/ArrowVectorUtils.java @@ -0,0 +1,208 @@ +/** Copyright 2020-2023 Alibaba Group Holding Limited. + * + * Licensed 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 io.v6d.modules.basic.arrow.util; + +import io.v6d.core.client.Context; +import io.v6d.core.common.util.VineyardException; +import io.v6d.core.common.util.VineyardException.NotImplemented; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Queue; +import org.apache.arrow.memory.ArrowBuf; +import org.apache.arrow.vector.FieldVector; +import org.apache.arrow.vector.complex.ListVector; +import org.apache.arrow.vector.complex.MapVector; +import org.apache.arrow.vector.complex.StructVector; +import org.apache.arrow.vector.ipc.message.ArrowFieldNode; +import org.apache.arrow.vector.types.pojo.ArrowType; +import org.apache.arrow.vector.types.pojo.ArrowType.ArrowTypeID; +import org.apache.arrow.vector.types.pojo.Field; + +public class ArrowVectorUtils { + private static Map defaultTransformers; + private static Map defaultResolvers; + + public static ArrowBuf[] getArrowBuffers(FieldVector vector) throws VineyardException { + List result = new ArrayList<>(); + if (vector instanceof StructVector) { + result.add(vector.getValidityBuffer()); + for (FieldVector child : ((StructVector) vector).getChildrenFromFields()) { + result.addAll(Arrays.asList(getArrowBuffers(child))); + } + } else if (vector instanceof ListVector) { + result.add(vector.getValidityBuffer()); + result.add(vector.getOffsetBuffer()); + result.addAll(Arrays.asList(getArrowBuffers(((ListVector) vector).getDataVector()))); + } else { + result.addAll(Arrays.asList(vector.getBuffers(false))); + } + return result.toArray(new ArrowBuf[result.size()]); + } + + public static List getValueCountOfArrowVector(FieldVector vector) + throws VineyardException { + List result = new ArrayList<>(); + result.add(vector.getValueCount()); + if (vector instanceof StructVector) { + for (FieldVector child : ((StructVector) vector).getChildrenFromFields()) { + result.addAll(getValueCountOfArrowVector(child)); + } + } else if (vector instanceof ListVector) { + result.addAll(getValueCountOfArrowVector(((ListVector) vector).getDataVector())); + } + return result; + } + + public static void buildArrowVector(FieldVector vector, Field field) throws VineyardException { + List childFields = field.getChildren(); + if (vector instanceof ListVector) { + if (vector instanceof MapVector) { + childFields = childFields.get(0).getChildren(); + } + if (childFields.size() != 1) { + throw new NotImplemented("ListArrayBuilder only support one child field"); + } + ((ListVector) vector).addOrGetVector(childFields.get(0).getFieldType()); + buildArrowVector(((ListVector) vector).getDataVector(), childFields.get(0)); + } else if (vector instanceof StructVector) { + for (int i = 0; i < childFields.size(); i++) { + FieldVector childVector = + ((StructVector) vector) + .addOrGet( + String.valueOf(i), + childFields.get(i).getFieldType(), + FieldVector.class); + buildArrowVector(childVector, childFields.get(i)); + } + } else { + Context.println("Primitive type. Nothing to do."); + } + } + + public static void buildArrowVector( + FieldVector vector, Queue bufs, Queue valueCountQueue, Field field) { + int valueCount = valueCountQueue.poll(); + List childFields = field.getChildren(); + List currentBufs = new ArrayList<>(); + + switch (field.getType().getTypeID()) { + case Struct: + // prepare and load buf + currentBufs.add(bufs.poll()); + vector.loadFieldBuffers(new ArrowFieldNode(valueCount, 0), currentBufs); + + // process child vector + for (int i = 0; i < childFields.size(); i++) { + FieldVector childFieldVector = + ((StructVector) vector) + .addOrGet( + String.valueOf(i), + childFields.get(i).getFieldType(), + FieldVector.class); + buildArrowVector(childFieldVector, bufs, valueCountQueue, childFields.get(i)); + } + break; + case Map: + // Map type is map->list->struct + childFields = childFields.get(0).getChildren(); + case List: + assert childFields.size() == 1 : "ListArrayBuilder only support one child field"; + currentBufs.add(bufs.poll()); + currentBufs.add(bufs.poll()); + vector.loadFieldBuffers(new ArrowFieldNode(valueCount, 0), currentBufs); + ((ListVector) vector).addOrGetVector(childFields.get(0).getFieldType()); + FieldVector childFieldVector = ((ListVector) vector).getDataVector(); + buildArrowVector(childFieldVector, bufs, valueCountQueue, childFields.get(0)); + break; + default: + switch (field.getType().getTypeID()) { + case Int: + case FloatingPoint: + case Bool: + case Binary: + case Date: + case Decimal: + case Timestamp: + currentBufs.add(bufs.poll()); + currentBufs.add(bufs.poll()); + vector.loadFieldBuffers(new ArrowFieldNode(valueCount, 0), currentBufs); + break; + case Utf8: + case LargeUtf8: + currentBufs.add(bufs.poll()); + currentBufs.add(bufs.poll()); + currentBufs.add(bufs.poll()); + vector.loadFieldBuffers(new ArrowFieldNode(valueCount, 0), currentBufs); + break; + default: + assert false : "Unsupported type: " + field.getType().getTypeID().name(); + } + } + } + + public static void printFields(Field field, int level) { + Context.println("--------------------------".substring(level * 3)); + Context.println("Field type:" + field.getType().getTypeID().name()); + Context.println("Field name:" + field.getName()); + switch (field.getType().getTypeID()) { + case Int: + Context.println("bitWidth:" + ((ArrowType.Int) field.getType()).getBitWidth()); + break; + case FloatingPoint: + Context.println( + "precision:" + + ((ArrowType.FloatingPoint) field.getType()) + .getPrecision() + .name()); + break; + case Timestamp: + Context.println( + "timeUnit:" + ((ArrowType.Timestamp) field.getType()).getUnit().name()); + break; + case Struct: + for (int i = 0; i < field.getChildren().size(); i++) { + printFields(field.getChildren().get(i), level + 1); + } + default: + break; + } + Context.println("--------------------------"); + } + + public static Map getDefaultTransformers() { + if (defaultTransformers == null) { + defaultTransformers = new HashMap<>(); + ArrowTypeID[] arrowTypeIDs = ArrowTypeID.values(); + for (ArrowTypeID arrowTypeID : arrowTypeIDs) { + defaultTransformers.put(arrowTypeID, new ObjectTransformer()); + } + } + return defaultTransformers; + } + + public static Map getDefaultResolver() { + if (defaultResolvers == null) { + defaultResolvers = new HashMap<>(); + ArrowTypeID[] arrowTypeIDs = ArrowTypeID.values(); + for (ArrowTypeID arrowTypeID : arrowTypeIDs) { + defaultResolvers.put(arrowTypeID, new ObjectResolver()); + } + } + return defaultResolvers; + } +} diff --git a/java/modules/basic/src/main/java/io/v6d/modules/basic/arrow/util/ObjectResolver.java b/java/modules/basic/src/main/java/io/v6d/modules/basic/arrow/util/ObjectResolver.java new file mode 100644 index 000000000..c6537fcba --- /dev/null +++ b/java/modules/basic/src/main/java/io/v6d/modules/basic/arrow/util/ObjectResolver.java @@ -0,0 +1,83 @@ +/** Copyright 2020-2023 Alibaba Group Holding Limited. + * + * Licensed 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 io.v6d.modules.basic.arrow.util; + +import java.math.BigDecimal; +import java.sql.Date; +import java.sql.Timestamp; + +public class ObjectResolver { + public ObjectResolver() {} + + public Object resolve(Object object) { + return object; + } + + public Object resolveInt(int object) { + return object; + } + + public Object resolveLong(long object) { + return object; + } + + public Object resolveShort(short object) { + return object; + } + + public byte resolveByte(byte object) { + return object; + } + + public float resolveFloat(float object) { + return object; + } + + public double resolveDouble(double object) { + return object; + } + + public Object resolveBoolean(int object) { + return object == 1; + } + + public Object resolveUtf8(String object) { + return object; + } + + public Object resolveLargeUtf8(String object) { + return object; + } + + public Object resolveBinary(byte[] object) { + return object; + } + + public Object resolveLargeBinary(byte[] object) { + return object; + } + + public Object resolveDecimal(BigDecimal object, int precision, int scale, int bitWidth) { + return object; + } + + public Object resolveTimestamp(Timestamp object) { + return object; + } + + public Object resolveDate(Date object) { + return object; + } +} diff --git a/java/modules/basic/src/main/java/io/v6d/modules/basic/arrow/util/ObjectTransformer.java b/java/modules/basic/src/main/java/io/v6d/modules/basic/arrow/util/ObjectTransformer.java new file mode 100644 index 000000000..11cc641f5 --- /dev/null +++ b/java/modules/basic/src/main/java/io/v6d/modules/basic/arrow/util/ObjectTransformer.java @@ -0,0 +1,84 @@ +/** Copyright 2020-2023 Alibaba Group Holding Limited. + * + * Licensed 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 io.v6d.modules.basic.arrow.util; + +import java.math.BigDecimal; +import java.nio.charset.StandardCharsets; +import java.sql.Date; +import java.sql.Timestamp; + +public class ObjectTransformer { + public ObjectTransformer() {} + + public Object transform(Object object) { + return object; + } + + public int transformInt(Object object) { + return (int) object; + } + + public long transformLong(Object object) { + return (long) object; + } + + public short transformShort(Object object) { + return (short) object; + } + + public byte transformByte(Object object) { + return (byte) object; + } + + public float transformFloat(Object object) { + return (float) object; + } + + public double transformDouble(Object object) { + return (double) object; + } + + public int transformBoolean(Object object) { + return (boolean) object == true ? 1 : 0; + } + + public byte[] transformUtf8(Object object) { + return object.toString().getBytes(StandardCharsets.UTF_8); + } + + public byte[] transformLargeUtf8(Object object) { + return object.toString().getBytes(StandardCharsets.UTF_8); + } + + public byte[] transformBinary(Object object) { + return (byte[]) object; + } + + public byte[] transformLargeBinary(Object object) { + return (byte[]) object; + } + + public BigDecimal transformDecimal(Object object, int precision, int scale, int bitWidth) { + return (BigDecimal) object; + } + + public Timestamp transformTimestamp(Object object) { + return (Timestamp) object; + } + + public Date transformDate(Object object) { + return (Date) object; + } +} diff --git a/java/modules/basic/src/main/java/io/v6d/modules/basic/columnar/ColumnarData.java b/java/modules/basic/src/main/java/io/v6d/modules/basic/columnar/ColumnarData.java index 3c14df0da..28fb3e276 100644 --- a/java/modules/basic/src/main/java/io/v6d/modules/basic/columnar/ColumnarData.java +++ b/java/modules/basic/src/main/java/io/v6d/modules/basic/columnar/ColumnarData.java @@ -31,11 +31,18 @@ * or implied. * See the License for the specific language governing permissions and * limitations * under the License. */ +import io.v6d.modules.basic.arrow.util.ObjectResolver; import java.math.BigDecimal; +import java.sql.Date; +import java.sql.Timestamp; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; import org.apache.arrow.vector.BigIntVector; import org.apache.arrow.vector.BitVector; -import org.apache.arrow.vector.DateDayVector; +import org.apache.arrow.vector.DateMilliVector; import org.apache.arrow.vector.DecimalVector; +import org.apache.arrow.vector.FieldVector; import org.apache.arrow.vector.Float4Vector; import org.apache.arrow.vector.Float8Vector; import org.apache.arrow.vector.IntVector; @@ -47,6 +54,13 @@ import org.apache.arrow.vector.SmallIntVector; import org.apache.arrow.vector.TimeStampMicroTZVector; import org.apache.arrow.vector.TimeStampMicroVector; +import org.apache.arrow.vector.TimeStampMilliTZVector; +import org.apache.arrow.vector.TimeStampMilliVector; +import org.apache.arrow.vector.TimeStampNanoTZVector; +import org.apache.arrow.vector.TimeStampNanoVector; +import org.apache.arrow.vector.TimeStampSecTZVector; +import org.apache.arrow.vector.TimeStampSecVector; +import org.apache.arrow.vector.TimeStampVector; import org.apache.arrow.vector.TinyIntVector; import org.apache.arrow.vector.UInt1Vector; import org.apache.arrow.vector.UInt2Vector; @@ -55,6 +69,9 @@ import org.apache.arrow.vector.ValueVector; import org.apache.arrow.vector.VarBinaryVector; import org.apache.arrow.vector.VarCharVector; +import org.apache.arrow.vector.complex.ListVector; +import org.apache.arrow.vector.complex.MapVector; +import org.apache.arrow.vector.complex.StructVector; import org.apache.arrow.vector.holders.NullableIntervalDayHolder; import org.apache.arrow.vector.holders.NullableVarCharHolder; import org.apache.arrow.vector.util.Text; @@ -64,50 +81,60 @@ public class ColumnarData { private final ArrowVectorAccessor accessor; public ColumnarData(ValueVector vector) { + this(vector, new ObjectResolver()); + } + + public ColumnarData(ValueVector vector, ObjectResolver resolver) { if (vector instanceof BitVector) { - accessor = new BooleanAccessor((BitVector) vector); + accessor = new BooleanAccessor((BitVector) vector, resolver); } else if (vector instanceof TinyIntVector) { - accessor = new ByteAccessor((TinyIntVector) vector); + accessor = new ByteAccessor((TinyIntVector) vector, resolver); } else if (vector instanceof UInt1Vector) { - accessor = new UByteAccessor((UInt1Vector) vector); + accessor = new UByteAccessor((UInt1Vector) vector, resolver); } else if (vector instanceof SmallIntVector) { - accessor = new ShortAccessor((SmallIntVector) vector); + accessor = new ShortAccessor((SmallIntVector) vector, resolver); } else if (vector instanceof UInt2Vector) { - accessor = new UShortAccessor((UInt2Vector) vector); + accessor = new UShortAccessor((UInt2Vector) vector, resolver); } else if (vector instanceof IntVector) { - accessor = new IntAccessor((IntVector) vector); + accessor = new IntAccessor((IntVector) vector, resolver); } else if (vector instanceof UInt4Vector) { - accessor = new UIntAccessor((UInt4Vector) vector); + accessor = new UIntAccessor((UInt4Vector) vector, resolver); } else if (vector instanceof BigIntVector) { - accessor = new LongAccessor((BigIntVector) vector); + accessor = new LongAccessor((BigIntVector) vector, resolver); } else if (vector instanceof UInt8Vector) { - accessor = new ULongAccessor((UInt8Vector) vector); + accessor = new ULongAccessor((UInt8Vector) vector, resolver); } else if (vector instanceof Float4Vector) { - accessor = new FloatAccessor((Float4Vector) vector); + accessor = new FloatAccessor((Float4Vector) vector, resolver); } else if (vector instanceof Float8Vector) { - accessor = new DoubleAccessor((Float8Vector) vector); + accessor = new DoubleAccessor((Float8Vector) vector, resolver); } else if (vector instanceof DecimalVector) { - accessor = new DecimalAccessor((DecimalVector) vector); + accessor = new DecimalAccessor((DecimalVector) vector, resolver); } else if (vector instanceof VarCharVector) { - accessor = new StringAccessor((VarCharVector) vector); + accessor = new StringAccessor((VarCharVector) vector, resolver); } else if (vector instanceof LargeVarCharVector) { - accessor = new LargeStringAccessor((LargeVarCharVector) vector); + accessor = new LargeStringAccessor((LargeVarCharVector) vector, resolver); } else if (vector instanceof VarBinaryVector) { - accessor = new BinaryAccessor((VarBinaryVector) vector); + accessor = new BinaryAccessor((VarBinaryVector) vector, resolver); } else if (vector instanceof LargeVarBinaryVector) { - accessor = new LargeBinaryAccessor((LargeVarBinaryVector) vector); - } else if (vector instanceof DateDayVector) { - accessor = new DateAccessor((DateDayVector) vector); - } else if (vector instanceof TimeStampMicroTZVector) { - accessor = new TimestampAccessor((TimeStampMicroTZVector) vector); - } else if (vector instanceof TimeStampMicroVector) { - accessor = new TimestampNTZAccessor((TimeStampMicroVector) vector); + accessor = new LargeBinaryAccessor((LargeVarBinaryVector) vector, resolver); + } else if (vector instanceof DateMilliVector) { + accessor = new DateAccessor((DateMilliVector) vector, resolver); + } else if (vector instanceof TimeStampNanoTZVector) { + accessor = new TimestampNanoAccessor((TimeStampNanoTZVector) vector, resolver); + } else if (vector instanceof TimeStampNanoVector) { + accessor = new TimestampNanoNTZAccessor((TimeStampNanoVector) vector, resolver); } else if (vector instanceof NullVector) { - accessor = new NullAccessor((NullVector) vector); + accessor = new NullAccessor((NullVector) vector, resolver); } else if (vector instanceof IntervalYearVector) { - accessor = new IntervalYearAccessor((IntervalYearVector) vector); + accessor = new IntervalYearAccessor((IntervalYearVector) vector, resolver); } else if (vector instanceof IntervalDayVector) { - accessor = new IntervalDayAccessor((IntervalDayVector) vector); + accessor = new IntervalDayAccessor((IntervalDayVector) vector, resolver); + } else if (vector instanceof ListVector) { + accessor = new NestedVectorAccessor((FieldVector) vector, resolver); + } else if (vector instanceof StructVector) { + accessor = new NestedVectorAccessor((FieldVector) vector, resolver); + } else if (vector instanceof MapVector) { + accessor = new NestedVectorAccessor((FieldVector) vector, resolver); } else { throw new UnsupportedOperationException( "array type is not supported yet: " + vector.getClass()); @@ -271,15 +298,20 @@ byte[] getBinary(int rowId) { private static class BooleanAccessor extends ArrowVectorAccessor { private final BitVector accessor; + private final ObjectResolver resolver; - BooleanAccessor(BitVector vector) { + BooleanAccessor(BitVector vector, ObjectResolver resolver) { super(vector); this.accessor = vector; + this.resolver = resolver; } @Override Object getObject(int rowId) { - return getBoolean(rowId); + if (accessor.isNull(rowId)) { + return null; + } + return resolver.resolveBoolean(accessor.get(rowId)); } @Override @@ -291,15 +323,20 @@ final boolean getBoolean(int rowId) { private static class ByteAccessor extends ArrowVectorAccessor { private final TinyIntVector accessor; + private final ObjectResolver resolver; - ByteAccessor(TinyIntVector vector) { + ByteAccessor(TinyIntVector vector, ObjectResolver resolver) { super(vector); this.accessor = vector; + this.resolver = resolver; } @Override Object getObject(int rowId) { - return getByte(rowId); + if (accessor.isNull(rowId)) { + return null; + } + return resolver.resolveByte(accessor.getObject(rowId)); } @Override @@ -311,15 +348,17 @@ final byte getByte(int rowId) { private static class UByteAccessor extends ArrowVectorAccessor { private final UInt1Vector accessor; + private final ObjectResolver resolver; - UByteAccessor(UInt1Vector vector) { + UByteAccessor(UInt1Vector vector, ObjectResolver resolver) { super(vector); this.accessor = vector; + this.resolver = resolver; } @Override Object getObject(int rowId) { - return getByte(rowId); + return resolver.resolveByte(getByte(rowId)); } @Override @@ -331,15 +370,20 @@ final byte getByte(int rowId) { private static class ShortAccessor extends ArrowVectorAccessor { private final SmallIntVector accessor; + private final ObjectResolver resolver; - ShortAccessor(SmallIntVector vector) { + ShortAccessor(SmallIntVector vector, ObjectResolver resolver) { super(vector); this.accessor = vector; + this.resolver = resolver; } @Override Object getObject(int rowId) { - return getShort(rowId); + if (accessor.isNull(rowId)) { + return null; + } + return resolver.resolveShort(accessor.getObject(rowId)); } @Override @@ -351,15 +395,17 @@ final short getShort(int rowId) { private static class UShortAccessor extends ArrowVectorAccessor { private final UInt2Vector accessor; + private final ObjectResolver resolver; - UShortAccessor(UInt2Vector vector) { + UShortAccessor(UInt2Vector vector, ObjectResolver resolver) { super(vector); this.accessor = vector; + this.resolver = resolver; } @Override Object getObject(int rowId) { - return getShort(rowId); + return resolver.resolveShort(getShort(rowId)); } @Override @@ -371,35 +417,37 @@ final short getShort(int rowId) { private static class IntAccessor extends ArrowVectorAccessor { private final IntVector accessor; + private final ObjectResolver resolver; - IntAccessor(IntVector vector) { + IntAccessor(IntVector vector, ObjectResolver resolver) { super(vector); this.accessor = vector; + this.resolver = resolver; } @Override Object getObject(int rowId) { - return getInt(rowId); - } - - @Override - final int getInt(int rowId) { - return accessor.get(rowId); + if (accessor.isNull(rowId)) { + return null; + } + return resolver.resolveInt(accessor.get(rowId)); } } private static class UIntAccessor extends ArrowVectorAccessor { private final UInt4Vector accessor; + private final ObjectResolver resolver; - UIntAccessor(UInt4Vector vector) { + UIntAccessor(UInt4Vector vector, ObjectResolver resolver) { super(vector); this.accessor = vector; + this.resolver = resolver; } @Override Object getObject(int rowId) { - return getInt(rowId); + return resolver.resolveInt(getInt(rowId)); } @Override @@ -411,15 +459,20 @@ final int getInt(int rowId) { private static class LongAccessor extends ArrowVectorAccessor { private final BigIntVector accessor; + private final ObjectResolver resolver; - LongAccessor(BigIntVector vector) { + LongAccessor(BigIntVector vector, ObjectResolver resolver) { super(vector); this.accessor = vector; + this.resolver = resolver; } @Override Object getObject(int rowId) { - return getLong(rowId); + if (accessor.isNull(rowId)) { + return null; + } + return resolver.resolveLong(accessor.getObject(rowId)); } @Override @@ -431,15 +484,17 @@ final long getLong(int rowId) { private static class ULongAccessor extends ArrowVectorAccessor { private final UInt8Vector accessor; + private final ObjectResolver resolver; - ULongAccessor(UInt8Vector vector) { + ULongAccessor(UInt8Vector vector, ObjectResolver resolver) { super(vector); this.accessor = vector; + this.resolver = resolver; } @Override Object getObject(int rowId) { - return getLong(rowId); + return resolver.resolveLong(getLong(rowId)); } @Override @@ -451,60 +506,61 @@ final long getLong(int rowId) { private static class FloatAccessor extends ArrowVectorAccessor { private final Float4Vector accessor; + private final ObjectResolver resolver; - FloatAccessor(Float4Vector vector) { + FloatAccessor(Float4Vector vector, ObjectResolver resolver) { super(vector); this.accessor = vector; + this.resolver = resolver; } @Override Object getObject(int rowId) { - return getFloat(rowId); - } - - @Override - final float getFloat(int rowId) { - return accessor.get(rowId); + if (accessor.isNull(rowId)) { + return null; + } + return resolver.resolveFloat(accessor.getObject(rowId)); } } private static class DoubleAccessor extends ArrowVectorAccessor { private final Float8Vector accessor; + private final ObjectResolver resolver; - DoubleAccessor(Float8Vector vector) { + DoubleAccessor(Float8Vector vector, ObjectResolver resolver) { super(vector); this.accessor = vector; + this.resolver = resolver; } @Override Object getObject(int rowId) { - return getDouble(rowId); - } - - @Override - final double getDouble(int rowId) { - return accessor.get(rowId); + if (accessor.isNull(rowId)) { + return null; + } + return resolver.resolveDouble(accessor.getObject(rowId)); } } private static class DecimalAccessor extends ArrowVectorAccessor { private final DecimalVector accessor; + private ObjectResolver resolver; - DecimalAccessor(DecimalVector vector) { + DecimalAccessor(DecimalVector vector, ObjectResolver resolver) { super(vector); this.accessor = vector; + this.resolver = resolver; } @Override Object getObject(int rowId) { - return getDecimal(rowId, 24, 8); - } - - @Override - final BigDecimal getDecimal(int rowId, int precision, int scale) { - return accessor.getObject(rowId); + if (accessor.isNull(rowId)) { + return null; + } + return resolver.resolveDecimal( + accessor.getObject(rowId), accessor.getPrecision(), accessor.getScale(), 128); } } @@ -512,20 +568,20 @@ private static class StringAccessor extends ArrowVectorAccessor { private final VarCharVector accessor; private final NullableVarCharHolder stringResult = new NullableVarCharHolder(); + private final ObjectResolver resolver; - StringAccessor(VarCharVector vector) { + StringAccessor(VarCharVector vector, ObjectResolver resolver) { super(vector); this.accessor = vector; + this.resolver = resolver; } @Override Object getObject(int rowId) { - return getUTF8String(rowId); - } - - @Override - final Text getUTF8String(int rowId) { - return accessor.getObject(rowId); + if (accessor.isNull(rowId)) { + return null; + } + return resolver.resolveUtf8(accessor.getObject(rowId).toString()); } } @@ -533,15 +589,20 @@ private static class LargeStringAccessor extends ArrowVectorAccessor { private final LargeVarCharVector accessor; private final NullableVarCharHolder stringResult = new NullableVarCharHolder(); + private final ObjectResolver resolver; - LargeStringAccessor(LargeVarCharVector vector) { + LargeStringAccessor(LargeVarCharVector vector, ObjectResolver resolver) { super(vector); this.accessor = vector; + this.resolver = resolver; } @Override Object getObject(int rowId) { - return getUTF8String(rowId); + if (accessor.isNull(rowId)) { + return null; + } + return resolver.resolveLargeUtf8(accessor.getObject(rowId).toString()); } @Override @@ -553,15 +614,20 @@ final Text getUTF8String(int rowId) { private static class BinaryAccessor extends ArrowVectorAccessor { private final VarBinaryVector accessor; + private final ObjectResolver resolver; - BinaryAccessor(VarBinaryVector vector) { + BinaryAccessor(VarBinaryVector vector, ObjectResolver resolver) { super(vector); this.accessor = vector; + this.resolver = resolver; } @Override Object getObject(int rowId) { - return getBinary(rowId); + if (accessor.isNull(rowId)) { + return null; + } + return resolver.resolveBinary(getBinary(rowId)); } @Override @@ -573,15 +639,17 @@ final byte[] getBinary(int rowId) { private static class LargeBinaryAccessor extends ArrowVectorAccessor { private final LargeVarBinaryVector accessor; + private final ObjectResolver resolver; - LargeBinaryAccessor(LargeVarBinaryVector vector) { + LargeBinaryAccessor(LargeVarBinaryVector vector, ObjectResolver resolver) { super(vector); this.accessor = vector; + this.resolver = resolver; } @Override Object getObject(int rowId) { - return getBinary(rowId); + return resolver.resolveLargeBinary(getBinary(rowId)); } @Override @@ -592,29 +660,36 @@ final byte[] getBinary(int rowId) { private static class DateAccessor extends ArrowVectorAccessor { - private final DateDayVector accessor; + private final DateMilliVector accessor; + private final ObjectResolver resolver; - DateAccessor(DateDayVector vector) { + DateAccessor(DateMilliVector vector, ObjectResolver resolver) { super(vector); this.accessor = vector; + this.resolver = resolver; } @Override Object getObject(int rowId) { - return getInt(rowId); + if (accessor.isNull(rowId)) { + return null; + } + long millis = accessor.get(rowId); + Date date = new Date(millis); + return resolver.resolveDate(date); } @Override - final int getInt(int rowId) { + final long getLong(int rowId) { return accessor.get(rowId); } } - private static class TimestampAccessor extends ArrowVectorAccessor { + private static class TimestampMicroAccessor extends ArrowVectorAccessor { private final TimeStampMicroTZVector accessor; - TimestampAccessor(TimeStampMicroTZVector vector) { + TimestampMicroAccessor(TimeStampMicroTZVector vector) { super(vector); this.accessor = vector; } @@ -630,11 +705,77 @@ final long getLong(int rowId) { } } - private static class TimestampNTZAccessor extends ArrowVectorAccessor { + private static class TimestampMicroNTZAccessor extends ArrowVectorAccessor { private final TimeStampMicroVector accessor; - TimestampNTZAccessor(TimeStampMicroVector vector) { + TimestampMicroNTZAccessor(TimeStampMicroVector vector) { + super(vector); + this.accessor = vector; + } + + @Override + Object getObject(int rowId) { + return getLong(rowId); + } + + @Override + final long getLong(int rowId) { + return accessor.get(rowId); + } + } + + private static class TimestampMilliAccessor extends ArrowVectorAccessor { + + private final TimeStampMilliTZVector accessor; + + TimestampMilliAccessor(TimeStampMilliTZVector vector) { + super(vector); + this.accessor = vector; + } + + @Override + Object getObject(int rowId) { + return getLong(rowId); + } + + @Override + final long getLong(int rowId) { + return accessor.get(rowId); + } + } + + private static class TimestampMilliNTZAccessor extends ArrowVectorAccessor { + + private final TimeStampMilliVector accessor; + + TimestampMilliNTZAccessor(TimeStampMilliVector vector) { + super(vector); + this.accessor = vector; + } + + @Override + Object getObject(int rowId) { + if (accessor.isNull(rowId)) { + return null; + } + // mills + long value = getLong(rowId); + Timestamp t = new Timestamp(value); + return t; + } + + @Override + final long getLong(int rowId) { + return accessor.get(rowId); + } + } + + private static class TimestampSecAccessor extends ArrowVectorAccessor { + + private final TimeStampSecTZVector accessor; + + TimestampSecAccessor(TimeStampSecTZVector vector) { super(vector); this.accessor = vector; } @@ -650,9 +791,85 @@ final long getLong(int rowId) { } } + private static class TimestampSecNTZAccessor extends ArrowVectorAccessor { + + private final TimeStampSecVector accessor; + + TimestampSecNTZAccessor(TimeStampSecVector vector) { + super(vector); + this.accessor = vector; + } + + @Override + Object getObject(int rowId) { + return getLong(rowId); + } + + @Override + final long getLong(int rowId) { + return accessor.get(rowId); + } + } + + private static class TimestampNanoAccessor extends ArrowVectorAccessor { + + private final TimeStampNanoTZVector accessor; + private final ObjectResolver resolver; + + TimestampNanoAccessor(TimeStampNanoTZVector vector, ObjectResolver resolver) { + super(vector); + this.accessor = vector; + this.resolver = resolver; + } + + @Override + Object getObject(int rowId) { + return resolver.resolve(getLong(rowId)); + } + + @Override + final long getLong(int rowId) { + return accessor.get(rowId); + } + } + + private static class TimestampNanoNTZAccessor extends ArrowVectorAccessor { + + private final TimeStampNanoVector accessor; + private final ObjectResolver resolver; + + TimestampNanoNTZAccessor(TimeStampNanoVector vector, ObjectResolver resolver) { + super(vector); + this.accessor = vector; + this.resolver = resolver; + } + + @Override + Object getObject(int rowId) { + if (accessor.isNull(rowId)) { + return null; + } + long value = getLong(rowId); + long nano = value % DateTimeConstants.NANOS_PER_SECOND; + long second = value / DateTimeConstants.NANOS_PER_SECOND; + if (nano < 0) { + nano += DateTimeConstants.NANOS_PER_SECOND; + second -= 1; + } + Timestamp t = new Timestamp(second * DateTimeConstants.MILLIS_PER_SECOND); + t.setNanos((int) nano); + return resolver.resolveTimestamp(t); + } + + @Override + final long getLong(int rowId) { + return accessor.get(rowId); + } + } + private static class NullAccessor extends ArrowVectorAccessor { - NullAccessor(NullVector vector) { + NullAccessor(NullVector vector, ObjectResolver resolver) { super(vector); } } @@ -660,15 +877,17 @@ private static class NullAccessor extends ArrowVectorAccessor { private static class IntervalYearAccessor extends ArrowVectorAccessor { private final IntervalYearVector accessor; + private final ObjectResolver resolver; - IntervalYearAccessor(IntervalYearVector vector) { + IntervalYearAccessor(IntervalYearVector vector, ObjectResolver resolver) { super(vector); this.accessor = vector; + this.resolver = resolver; } @Override Object getObject(int rowId) { - return getInt(rowId); + return resolver.resolve(getInt(rowId)); } @Override @@ -681,15 +900,17 @@ private static class IntervalDayAccessor extends ArrowVectorAccessor { private final IntervalDayVector accessor; private final NullableIntervalDayHolder intervalDayHolder = new NullableIntervalDayHolder(); + private final ObjectResolver resolver; - IntervalDayAccessor(IntervalDayVector vector) { + IntervalDayAccessor(IntervalDayVector vector, ObjectResolver resolver) { super(vector); this.accessor = vector; + this.resolver = resolver; } @Override Object getObject(int rowId) { - return getLong(rowId); + return resolver.resolve(getLong(rowId)); } @Override @@ -701,6 +922,241 @@ long getLong(int rowId) { } } + private static class NestedVectorAccessor extends ArrowVectorAccessor { + private final FieldVector vector; + private final ObjectResolver resolver; + + NestedVectorAccessor(FieldVector vector, ObjectResolver resolver) { + super(vector); + this.vector = vector; + this.resolver = resolver; + } + + @Override + Object getObject(int rowId) { + Object value = getObject(vector, rowId, 1); + if (value == null) { + return null; + } + return ((Object[]) value)[0]; + } + + private Object getObject(FieldVector vector, int rowId, int rows) { + List result = new ArrayList<>(); + if (vector.getValueCount() == 0) { + return null; + } + + if (vector instanceof ListVector) { + /* + * TODO: + * This code refer to apache arrow. Call ListVector.getObject will trigger Exception. + * So we need to get the value by ourself. It may be fixed in the future. + */ + if (vector instanceof MapVector) { + HashMap map = new HashMap<>(); + FieldVector fv = ((ListVector) vector).getDataVector(); + + for (int i = rowId; i < rowId + rows; i++) { + int start = ((ListVector) vector).getOffsetBuffer().getInt((long) (i * 4)); + int end = + ((ListVector) vector) + .getOffsetBuffer() + .getInt((long) ((i + 1) * 4)); + + // struct, struct ..... + Object value = getObject(fv, start, end - start); + for (int j = 0; j < ((Object[]) value).length; j++) { + // value[j] is struct + List kvList = (List) (((Object[]) value)[j]); + map.put(kvList.get(0), kvList.get(1)); + } + } + result.add(map); + } else { + FieldVector fv = ((ListVector) vector).getDataVector(); + + for (int i = rowId; i < rowId + rows; i++) { + if (vector.isNull(i)) { + result.add(null); + } else { + List vals = new ArrayList<>(); + int start = + ((ListVector) vector).getOffsetBuffer().getInt((long) (i * 4)); + int end = + ((ListVector) vector) + .getOffsetBuffer() + .getInt((long) ((i + 1) * 4)); + + Object value = getObject(fv, start, end - start); + for (int j = 0; j < ((Object[]) value).length; j++) { + vals.add(((Object[]) value)[j]); + } + result.add(vals); + } + } + } + } else if (vector instanceof StructVector) { + List childFieldVectors = + ((StructVector) vector).getChildrenFromFields(); + for (int i = rowId; i < rowId + rows; i++) { + List vals = new ArrayList<>(); + for (int j = 0; j < childFieldVectors.size(); j++) { + Object value = getObject(childFieldVectors.get(j), i, 1); + for (int k = 0; k < ((Object[]) value).length; k++) { + vals.add(((Object[]) value)[k]); + } + } + result.add(vals); + } + } else { + // primitive type + if (vector instanceof DateMilliVector) { + for (int i = 0; i < rows; i++) { + if (vector.isNull(i)) { + result.add(null); + } else { + Date date = new Date(((DateMilliVector) vector).get(rowId)); + result.add(resolver.resolveDate(date)); + } + } + } else if (vector instanceof TimeStampNanoVector) { + for (int i = 0; i < rows; i++) { + if (vector.isNull(i)) { + result.add(null); + } else { + long value = ((TimeStampVector) vector).get(rowId + i); + long nano = value % DateTimeConstants.NANOS_PER_SECOND; + long second = value / DateTimeConstants.NANOS_PER_SECOND; + if (nano < 0) { + nano += DateTimeConstants.NANOS_PER_SECOND; + second -= 1; + } + Timestamp t = + new Timestamp(second * DateTimeConstants.MILLIS_PER_SECOND); + t.setNanos((int) nano); + result.add(resolver.resolveTimestamp(t)); + } + } + } else if (vector instanceof DecimalVector) { + for (int i = 0; i < rows; i++) { + if (vector.isNull(i)) { + result.add(null); + } else { + DecimalVector decimalVector = (DecimalVector) vector; + BigDecimal bigDecimal = (BigDecimal) vector.getObject(rowId + i); + result.add( + resolver.resolveDecimal( + bigDecimal, + decimalVector.getPrecision(), + decimalVector.getScale(), + 128)); + } + } + } else if (vector instanceof VarCharVector) { + for (int i = 0; i < rows; i++) { + if (vector.isNull(i)) { + result.add(null); + } else { + result.add( + resolver.resolveUtf8(vector.getObject(rowId + i).toString())); + } + } + } else if (vector instanceof LargeVarCharVector) { + for (int i = 0; i < rows; i++) { + if (vector.isNull(i)) { + result.add(null); + } else { + result.add( + resolver.resolveLargeUtf8( + vector.getObject(rowId + i).toString())); + } + } + } else if (vector instanceof VarBinaryVector) { + for (int i = 0; i < rows; i++) { + if (vector.isNull(i)) { + result.add(null); + } else { + result.add( + resolver.resolveBinary((byte[]) vector.getObject(rowId + i))); + } + } + } else if (vector instanceof LargeVarBinaryVector) { + for (int i = 0; i < rows; i++) { + if (vector.isNull(i)) { + result.add(null); + } else { + result.add( + resolver.resolveLargeBinary( + (byte[]) vector.getObject(rowId + i))); + } + } + } else if (vector instanceof IntVector) { + for (int i = 0; i < rows; i++) { + if (vector.isNull(i)) { + result.add(null); + } else { + result.add(resolver.resolveInt((int) vector.getObject(rowId + i))); + } + } + } else if (vector instanceof BigIntVector) { + for (int i = 0; i < rows; i++) { + if (vector.isNull(i)) { + result.add(null); + } else { + result.add(resolver.resolveLong((long) vector.getObject(rowId + i))); + } + } + } else if (vector instanceof SmallIntVector) { + for (int i = 0; i < rows; i++) { + if (vector.isNull(i)) { + result.add(null); + } else { + result.add(resolver.resolveShort((short) vector.getObject(rowId + i))); + } + } + } else if (vector instanceof TinyIntVector) { + for (int i = 0; i < rows; i++) { + if (vector.isNull(i)) { + result.add(null); + } else { + result.add(resolver.resolveByte((byte) vector.getObject(rowId + i))); + } + } + } else if (vector instanceof Float4Vector) { + for (int i = 0; i < rows; i++) { + if (vector.isNull(i)) { + result.add(null); + } else { + result.add(resolver.resolveFloat((float) vector.getObject(rowId + i))); + } + } + } else if (vector instanceof Float8Vector) { + for (int i = 0; i < rows; i++) { + if (vector.isNull(i)) { + result.add(null); + } else { + result.add( + resolver.resolveDouble((double) vector.getObject(rowId + i))); + } + } + } else if (vector instanceof BitVector) { + for (int i = 0; i < rows; i++) { + if (vector.isNull(i)) { + result.add(null); + } else { + result.add(resolver.resolveBoolean((int) vector.getObject(rowId + i))); + } + } + } else { + throw new UnsupportedOperationException( + "array type is not supported yet: " + vector.getClass()); + } + } + return result.toArray(); + } + } + // refer from spark: // https://github.com/apache/spark/blob/master/common/unsafe/src/main/java/org/apache/spark/sql/catalyst/util/DateTimeConstants.java private class DateTimeConstants { diff --git a/java/modules/basic/src/main/java/io/v6d/modules/basic/columnar/ColumnarDataBuilder.java b/java/modules/basic/src/main/java/io/v6d/modules/basic/columnar/ColumnarDataBuilder.java index 492f18108..d1f70965a 100644 --- a/java/modules/basic/src/main/java/io/v6d/modules/basic/columnar/ColumnarDataBuilder.java +++ b/java/modules/basic/src/main/java/io/v6d/modules/basic/columnar/ColumnarDataBuilder.java @@ -14,28 +14,17 @@ */ package io.v6d.modules.basic.columnar; -/** - * The implementation is heavily referred from spark, see also - * - *

https://github.com/apache/spark/blob/master/sql/catalyst/src/main/java/org/apache/spark/sql/vectorized/ArrowColumnVector.java - * - *

The original file has the following copyright header: - * - *

* 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. - */ +import io.v6d.modules.basic.arrow.util.ObjectTransformer; import java.math.BigDecimal; +import java.sql.Date; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; import org.apache.arrow.vector.BigIntVector; import org.apache.arrow.vector.BitVector; -import org.apache.arrow.vector.DateDayVector; +import org.apache.arrow.vector.DateMilliVector; import org.apache.arrow.vector.DecimalVector; +import org.apache.arrow.vector.FieldVector; import org.apache.arrow.vector.Float4Vector; import org.apache.arrow.vector.Float8Vector; import org.apache.arrow.vector.IntVector; @@ -47,6 +36,12 @@ import org.apache.arrow.vector.SmallIntVector; import org.apache.arrow.vector.TimeStampMicroTZVector; import org.apache.arrow.vector.TimeStampMicroVector; +import org.apache.arrow.vector.TimeStampMilliTZVector; +import org.apache.arrow.vector.TimeStampMilliVector; +import org.apache.arrow.vector.TimeStampNanoTZVector; +import org.apache.arrow.vector.TimeStampNanoVector; +import org.apache.arrow.vector.TimeStampSecTZVector; +import org.apache.arrow.vector.TimeStampSecVector; import org.apache.arrow.vector.TinyIntVector; import org.apache.arrow.vector.UInt1Vector; import org.apache.arrow.vector.UInt2Vector; @@ -55,6 +50,9 @@ import org.apache.arrow.vector.ValueVector; import org.apache.arrow.vector.VarBinaryVector; import org.apache.arrow.vector.VarCharVector; +import org.apache.arrow.vector.complex.ListVector; +import org.apache.arrow.vector.complex.MapVector; +import org.apache.arrow.vector.complex.StructVector; import org.apache.arrow.vector.holders.NullableIntervalDayHolder; import org.apache.arrow.vector.holders.NullableVarCharHolder; import org.apache.arrow.vector.util.Text; @@ -63,51 +61,55 @@ public class ColumnarDataBuilder { private final ArrowVectorAccessor accessor; - public ColumnarDataBuilder(ValueVector vector) { + public ColumnarDataBuilder(ValueVector vector, ObjectTransformer transformer) { if (vector instanceof BitVector) { - accessor = new BooleanAccessor((BitVector) vector); + accessor = new BooleanAccessor((BitVector) vector, transformer); } else if (vector instanceof TinyIntVector) { - accessor = new ByteAccessor((TinyIntVector) vector); + accessor = new ByteAccessor((TinyIntVector) vector, transformer); } else if (vector instanceof UInt1Vector) { - accessor = new UByteAccessor((UInt1Vector) vector); + accessor = new UByteAccessor((UInt1Vector) vector, transformer); } else if (vector instanceof SmallIntVector) { - accessor = new ShortAccessor((SmallIntVector) vector); + accessor = new ShortAccessor((SmallIntVector) vector, transformer); } else if (vector instanceof UInt2Vector) { - accessor = new UShortAccessor((UInt2Vector) vector); + accessor = new UShortAccessor((UInt2Vector) vector, transformer); } else if (vector instanceof IntVector) { - accessor = new IntAccessor((IntVector) vector); + accessor = new IntAccessor((IntVector) vector, transformer); } else if (vector instanceof UInt4Vector) { - accessor = new UIntAccessor((UInt4Vector) vector); + accessor = new UIntAccessor((UInt4Vector) vector, transformer); } else if (vector instanceof BigIntVector) { - accessor = new LongAccessor((BigIntVector) vector); + accessor = new LongAccessor((BigIntVector) vector, transformer); } else if (vector instanceof UInt8Vector) { - accessor = new ULongAccessor((UInt8Vector) vector); + accessor = new ULongAccessor((UInt8Vector) vector, transformer); } else if (vector instanceof Float4Vector) { - accessor = new FloatAccessor((Float4Vector) vector); + accessor = new FloatAccessor((Float4Vector) vector, transformer); } else if (vector instanceof Float8Vector) { - accessor = new DoubleAccessor((Float8Vector) vector); + accessor = new DoubleAccessor((Float8Vector) vector, transformer); } else if (vector instanceof DecimalVector) { - accessor = new DecimalAccessor((DecimalVector) vector); + accessor = new DecimalAccessor((DecimalVector) vector, transformer); } else if (vector instanceof VarCharVector) { - accessor = new StringAccessor((VarCharVector) vector); + accessor = new StringAccessor((VarCharVector) vector, transformer); } else if (vector instanceof LargeVarCharVector) { - accessor = new LargeStringAccessor((LargeVarCharVector) vector); + accessor = new LargeStringAccessor((LargeVarCharVector) vector, transformer); } else if (vector instanceof VarBinaryVector) { - accessor = new BinaryAccessor((VarBinaryVector) vector); + accessor = new BinaryAccessor((VarBinaryVector) vector, transformer); } else if (vector instanceof LargeVarBinaryVector) { - accessor = new LargeBinaryAccessor((LargeVarBinaryVector) vector); - } else if (vector instanceof DateDayVector) { - accessor = new DateAccessor((DateDayVector) vector); - } else if (vector instanceof TimeStampMicroTZVector) { - accessor = new TimestampAccessor((TimeStampMicroTZVector) vector); - } else if (vector instanceof TimeStampMicroVector) { - accessor = new TimestampNTZAccessor((TimeStampMicroVector) vector); + accessor = new LargeBinaryAccessor((LargeVarBinaryVector) vector, transformer); + } else if (vector instanceof DateMilliVector) { + accessor = new DateAccessor((DateMilliVector) vector, transformer); + } else if (vector instanceof TimeStampNanoTZVector) { + accessor = new TimestampNanoAccessor((TimeStampNanoTZVector) vector, transformer); + } else if (vector instanceof TimeStampNanoVector) { + accessor = new TimestampNanoNTZAccessor((TimeStampNanoVector) vector, transformer); } else if (vector instanceof NullVector) { - accessor = new NullAccessor((NullVector) vector); + accessor = new NullAccessor((NullVector) vector, null); } else if (vector instanceof IntervalYearVector) { - accessor = new IntervalYearAccessor((IntervalYearVector) vector); + accessor = new IntervalYearAccessor((IntervalYearVector) vector, transformer); } else if (vector instanceof IntervalDayVector) { - accessor = new IntervalDayAccessor((IntervalDayVector) vector); + accessor = new IntervalDayAccessor((IntervalDayVector) vector, transformer); + } else if (vector instanceof ListVector) { + accessor = new NestedVectorAccessor((ListVector) vector, transformer); + } else if (vector instanceof StructVector) { + accessor = new NestedVectorAccessor((StructVector) vector, transformer); } else { throw new UnsupportedOperationException( "array type is not supported yet: " + vector.getClass()); @@ -359,10 +361,12 @@ void setBinary(int rowId, byte[] value) { private static class BooleanAccessor extends ArrowVectorAccessor { private final BitVector accessor; + private final ObjectTransformer transformer; - BooleanAccessor(BitVector vector) { + BooleanAccessor(BitVector vector, ObjectTransformer transformer) { super(vector); this.accessor = vector; + this.transformer = transformer; } @Override @@ -372,7 +376,11 @@ Object getObject(int rowId) { @Override void setObject(int rowId, Object value) { - this.setBoolean(rowId, (Boolean) value); + if (value == null) { + this.accessor.setNull(rowId); + } else { + this.accessor.setSafe(rowId, transformer.transformBoolean(value)); + } } @Override @@ -389,10 +397,12 @@ void setBoolean(int rowId, boolean value) { private static class ByteAccessor extends ArrowVectorAccessor { private final TinyIntVector accessor; + private final ObjectTransformer transformer; - ByteAccessor(TinyIntVector vector) { + ByteAccessor(TinyIntVector vector, ObjectTransformer transformer) { super(vector); this.accessor = vector; + this.transformer = transformer; } @Override @@ -400,6 +410,15 @@ Object getObject(int rowId) { return getByte(rowId); } + @Override + void setObject(int rowId, Object value) { + if (value == null) { + accessor.setNull(rowId); + } else { + this.setByte(rowId, transformer.transformByte(value)); + } + } + @Override final byte getByte(int rowId) { return accessor.get(rowId); @@ -414,10 +433,12 @@ final void setByte(int rowId, byte value) { private static class UByteAccessor extends ArrowVectorAccessor { private final UInt1Vector accessor; + private final ObjectTransformer transformer; - UByteAccessor(UInt1Vector vector) { + UByteAccessor(UInt1Vector vector, ObjectTransformer transformer) { super(vector); this.accessor = vector; + this.transformer = transformer; } @Override @@ -427,7 +448,7 @@ Object getObject(int rowId) { @Override void setObject(int rowId, Object value) { - this.setByte(rowId, (Byte) value); + this.setByte(rowId, transformer.transformByte(value)); } @Override @@ -444,10 +465,12 @@ final void setByte(int rowId, byte value) { private static class ShortAccessor extends ArrowVectorAccessor { private final SmallIntVector accessor; + private final ObjectTransformer transformer; - ShortAccessor(SmallIntVector vector) { + ShortAccessor(SmallIntVector vector, ObjectTransformer transformer) { super(vector); this.accessor = vector; + this.transformer = transformer; } @Override @@ -457,7 +480,11 @@ Object getObject(int rowId) { @Override void setObject(int rowId, Object value) { - this.setShort(rowId, (Short) value); + if (value == null) { + accessor.setNull(rowId); + } else { + this.setShort(rowId, transformer.transformShort(value)); + } } @Override @@ -474,10 +501,12 @@ final void setShort(int rowId, short value) { private static class UShortAccessor extends ArrowVectorAccessor { private final UInt2Vector accessor; + private final ObjectTransformer transformer; - UShortAccessor(UInt2Vector vector) { + UShortAccessor(UInt2Vector vector, ObjectTransformer transformer) { super(vector); this.accessor = vector; + this.transformer = transformer; } @Override @@ -487,7 +516,7 @@ Object getObject(int rowId) { @Override void setObject(int rowId, Object value) { - this.setShort(rowId, (Short) value); + this.setShort(rowId, transformer.transformShort(value)); } @Override @@ -504,10 +533,12 @@ final void setShort(int rowId, short value) { private static class IntAccessor extends ArrowVectorAccessor { private final IntVector accessor; + private final ObjectTransformer transformer; - IntAccessor(IntVector vector) { + IntAccessor(IntVector vector, ObjectTransformer transformer) { super(vector); this.accessor = vector; + this.transformer = transformer; } @Override @@ -517,7 +548,11 @@ Object getObject(int rowId) { @Override void setObject(int rowId, Object value) { - this.setInt(rowId, (Integer) value); + if (value != null) { + this.setInt(rowId, transformer.transformInt(value)); + } else { + accessor.setNull(rowId); + } } @Override @@ -534,10 +569,12 @@ final void setInt(int rowId, int value) { private static class UIntAccessor extends ArrowVectorAccessor { private final UInt4Vector accessor; + private final ObjectTransformer transformer; - UIntAccessor(UInt4Vector vector) { + UIntAccessor(UInt4Vector vector, ObjectTransformer transformer) { super(vector); this.accessor = vector; + this.transformer = transformer; } @Override @@ -547,7 +584,7 @@ Object getObject(int rowId) { @Override void setObject(int rowId, Object value) { - this.setInt(rowId, (Integer) value); + this.setInt(rowId, transformer.transformInt(value)); } @Override @@ -564,10 +601,12 @@ final void setInt(int rowId, int value) { private static class LongAccessor extends ArrowVectorAccessor { private final BigIntVector accessor; + private final ObjectTransformer transformer; - LongAccessor(BigIntVector vector) { + LongAccessor(BigIntVector vector, ObjectTransformer transformer) { super(vector); this.accessor = vector; + this.transformer = transformer; } @Override @@ -577,7 +616,11 @@ Object getObject(int rowId) { @Override void setObject(int rowId, Object value) { - this.setLong(rowId, (Long) value); + if (value == null) { + accessor.setNull(rowId); + } else { + accessor.setSafe(rowId, transformer.transformLong(value)); + } } @Override @@ -594,10 +637,12 @@ final void setLong(int rowId, long value) { private static class ULongAccessor extends ArrowVectorAccessor { private final UInt8Vector accessor; + private final ObjectTransformer transformer; - ULongAccessor(UInt8Vector vector) { + ULongAccessor(UInt8Vector vector, ObjectTransformer transformer) { super(vector); this.accessor = vector; + this.transformer = transformer; } @Override @@ -607,7 +652,7 @@ Object getObject(int rowId) { @Override void setObject(int rowId, Object value) { - this.setLong(rowId, (Long) value); + this.setLong(rowId, transformer.transformLong(value)); } @Override @@ -624,10 +669,12 @@ final void setLong(int rowId, long value) { private static class FloatAccessor extends ArrowVectorAccessor { private final Float4Vector accessor; + private final ObjectTransformer transformer; - FloatAccessor(Float4Vector vector) { + FloatAccessor(Float4Vector vector, ObjectTransformer transformer) { super(vector); this.accessor = vector; + this.transformer = transformer; } @Override @@ -637,7 +684,11 @@ Object getObject(int rowId) { @Override void setObject(int rowId, Object value) { - this.setFloat(rowId, (Float) value); + if (value == null) { + this.accessor.setNull(rowId); + } else { + this.accessor.setSafe(rowId, transformer.transformFloat(value)); + } } @Override @@ -654,10 +705,12 @@ final void setFloat(int rowId, float value) { private static class DoubleAccessor extends ArrowVectorAccessor { private final Float8Vector accessor; + private final ObjectTransformer transformer; - DoubleAccessor(Float8Vector vector) { + DoubleAccessor(Float8Vector vector, ObjectTransformer transformer) { super(vector); this.accessor = vector; + this.transformer = transformer; } @Override @@ -667,7 +720,11 @@ Object getObject(int rowId) { @Override void setObject(int rowId, Object value) { - this.setDouble(rowId, (Double) value); + if (value == null) { + accessor.setNull(rowId); + } else { + this.setDouble(rowId, transformer.transformDouble(value)); + } } @Override @@ -684,10 +741,12 @@ final void setDouble(int rowId, double value) { private static class DecimalAccessor extends ArrowVectorAccessor { private final DecimalVector accessor; + private final ObjectTransformer transformer; - DecimalAccessor(DecimalVector vector) { + DecimalAccessor(DecimalVector vector, ObjectTransformer transformer) { super(vector); this.accessor = vector; + this.transformer = transformer; } @Override @@ -697,63 +756,59 @@ Object getObject(int rowId) { @Override void setObject(int rowId, Object value) { - this.setDecimal(rowId, (BigDecimal) value); + if (value == null) { + accessor.setNull(rowId); + } else { + this.accessor.setSafe( + rowId, + transformer.transformDecimal( + value, accessor.getPrecision(), accessor.getScale(), 128)); + } } @Override final BigDecimal getDecimal(int rowId, int precision, int scale) { return accessor.getObject(rowId); } - - @Override - final void setDecimal(int rowId, BigDecimal value) { - accessor.set(rowId, value); - } } private static class StringAccessor extends ArrowVectorAccessor { private final VarCharVector accessor; private final NullableVarCharHolder stringResult = new NullableVarCharHolder(); + private final ObjectTransformer transformer; - StringAccessor(VarCharVector vector) { + StringAccessor(VarCharVector vector, ObjectTransformer transformer) { super(vector); this.accessor = vector; + this.transformer = transformer; } @Override Object getObject(int rowId) { - return getUTF8String(rowId); + return accessor.getObject(rowId).toString(); } @Override void setObject(int rowId, Object value) { - if (value instanceof String) { - this.setUTF8String(rowId, new Text((String) value)); + if (value == null) { + accessor.setNull(rowId); } else { - this.setUTF8String(rowId, (Text) value); + accessor.setSafe(rowId, transformer.transformUtf8(value)); } } - - @Override - final Text getUTF8String(int rowId) { - return accessor.getObject(rowId); - } - - @Override - final void setUTF8String(int rowId, Text value) { - accessor.setSafe(rowId, value); - } } private static class LargeStringAccessor extends ArrowVectorAccessor { private final LargeVarCharVector accessor; private final NullableVarCharHolder stringResult = new NullableVarCharHolder(); + private final ObjectTransformer transformer; - LargeStringAccessor(LargeVarCharVector vector) { + LargeStringAccessor(LargeVarCharVector vector, ObjectTransformer transformer) { super(vector); this.accessor = vector; + this.transformer = transformer; } @Override @@ -763,31 +818,23 @@ Object getObject(int rowId) { @Override void setObject(int rowId, Object value) { - if (value instanceof String) { - this.setUTF8String(rowId, new Text((String) value)); + if (value == null) { + accessor.setNull(rowId); } else { - this.setUTF8String(rowId, (Text) value); + this.accessor.setSafe(rowId, transformer.transformLargeUtf8(value)); } } - - @Override - final Text getUTF8String(int rowId) { - return accessor.getObject(rowId); - } - - @Override - final void setUTF8String(int rowId, Text value) { - accessor.setSafe(rowId, value); - } } private static class BinaryAccessor extends ArrowVectorAccessor { private final VarBinaryVector accessor; + private final ObjectTransformer transformer; - BinaryAccessor(VarBinaryVector vector) { + BinaryAccessor(VarBinaryVector vector, ObjectTransformer transformer) { super(vector); this.accessor = vector; + this.transformer = transformer; } @Override @@ -797,7 +844,11 @@ Object getObject(int rowId) { @Override void setObject(int rowId, Object value) { - this.setBinary(rowId, (byte[]) value); + if (value == null) { + accessor.setNull(rowId); + } else { + this.setBinary(rowId, transformer.transformBinary(value)); + } } @Override @@ -807,17 +858,19 @@ final byte[] getBinary(int rowId) { @Override final void setBinary(int rowId, byte[] value) { - accessor.set(rowId, value); + accessor.setSafe(rowId, value); } } private static class LargeBinaryAccessor extends ArrowVectorAccessor { private final LargeVarBinaryVector accessor; + private final ObjectTransformer transformer; - LargeBinaryAccessor(LargeVarBinaryVector vector) { + LargeBinaryAccessor(LargeVarBinaryVector vector, ObjectTransformer transformer) { super(vector); this.accessor = vector; + this.transformer = transformer; } @Override @@ -827,7 +880,7 @@ Object getObject(int rowId) { @Override void setObject(int rowId, Object value) { - this.setBinary(rowId, (byte[]) value); + this.setBinary(rowId, transformer.transformLargeBinary(value)); } @Override @@ -843,41 +896,50 @@ final void setBinary(int rowId, byte[] value) { private static class DateAccessor extends ArrowVectorAccessor { - private final DateDayVector accessor; + private final DateMilliVector accessor; + private final ObjectTransformer transformer; - DateAccessor(DateDayVector vector) { + DateAccessor(DateMilliVector vector, ObjectTransformer transformer) { super(vector); this.accessor = vector; + this.transformer = transformer; } @Override Object getObject(int rowId) { - return getInt(rowId); + return getLong(rowId); } @Override void setObject(int rowId, Object value) { - this.setInt(rowId, (Integer) value); + if (value == null) { + accessor.setNull(rowId); + } else { + long millis = ((Date) transformer.transformDate(value)).getTime(); + accessor.set(rowId, millis); + } } @Override - final int getInt(int rowId) { + final long getLong(int rowId) { return accessor.get(rowId); } @Override - final void setInt(int rowId, int value) { + final void setLong(int rowId, long value) { accessor.set(rowId, value); } } - private static class TimestampAccessor extends ArrowVectorAccessor { + private static class TimestampMicroAccessor extends ArrowVectorAccessor { private final TimeStampMicroTZVector accessor; + private final ObjectTransformer transformer; - TimestampAccessor(TimeStampMicroTZVector vector) { + TimestampMicroAccessor(TimeStampMicroTZVector vector, ObjectTransformer transformer) { super(vector); this.accessor = vector; + this.transformer = transformer; } @Override @@ -901,11 +963,112 @@ final void setLong(int rowId, long value) { } } - private static class TimestampNTZAccessor extends ArrowVectorAccessor { + private static class TimestampMicroNTZAccessor extends ArrowVectorAccessor { private final TimeStampMicroVector accessor; - TimestampNTZAccessor(TimeStampMicroVector vector) { + TimestampMicroNTZAccessor(TimeStampMicroVector vector, ObjectTransformer transformer) { + super(vector); + this.accessor = vector; + } + + @Override + Object getObject(int rowId) { + return getLong(rowId); + } + + @Override + void setObject(int rowId, Object value) { + if (value instanceof Long) { + this.setLong(rowId, (Long) value); + } else { + accessor.set( + rowId, + (long) (((java.sql.Timestamp) (value)).getTime()) * 1000 + + (((java.sql.Timestamp) (value)).getNanos() % 1000000) / 1000); + } + } + + @Override + final long getLong(int rowId) { + return accessor.get(rowId); + } + + @Override + final void setLong(int rowId, long value) { + accessor.set(rowId, value); + } + } + + private static class TimestampMilliAccessor extends ArrowVectorAccessor { + + private final TimeStampMilliTZVector accessor; + + TimestampMilliAccessor(TimeStampMilliTZVector vector) { + super(vector); + this.accessor = vector; + } + + @Override + Object getObject(int rowId) { + return getLong(rowId); + } + + @Override + void setObject(int rowId, Object value) { + this.setLong(rowId, (Long) value); + } + + @Override + final long getLong(int rowId) { + return accessor.get(rowId); + } + + @Override + final void setLong(int rowId, long value) { + accessor.set(rowId, value); + } + } + + private static class TimestampMilliNTZAccessor extends ArrowVectorAccessor { + + private final TimeStampMilliVector accessor; + + TimestampMilliNTZAccessor(TimeStampMilliVector vector) { + super(vector); + this.accessor = vector; + } + + @Override + Object getObject(int rowId) { + return getLong(rowId); + } + + @Override + void setObject(int rowId, Object value) { + if (value == null) { + accessor.setNull(rowId); + } else { + accessor.set(rowId, (long) (((java.sql.Timestamp) value).getTime())); + } + } + + @Override + final long getLong(int rowId) { + return accessor.get(rowId); + } + + @Override + final void setLong(int rowId, long value) { + accessor.set(rowId, value); + } + } + + private static class TimestampSecAccessor extends ArrowVectorAccessor { + + private final TimeStampSecTZVector accessor; + + TimestampSecAccessor(TimeStampSecTZVector vector) { super(vector); this.accessor = vector; } @@ -931,8 +1094,120 @@ final void setLong(int rowId, long value) { } } + private static class TimestampSecNTZAccessor extends ArrowVectorAccessor { + + private final TimeStampSecVector accessor; + + TimestampSecNTZAccessor(TimeStampSecVector vector) { + super(vector); + this.accessor = vector; + } + + @Override + Object getObject(int rowId) { + return getLong(rowId); + } + + @Override + void setObject(int rowId, Object value) { + if (value instanceof Long) { + this.setLong(rowId, (Long) value); + } else { + accessor.set(rowId, (long) (((java.sql.Timestamp) value).getTime()) / 1000); + } + } + + @Override + final long getLong(int rowId) { + return accessor.get(rowId); + } + + @Override + final void setLong(int rowId, long value) { + accessor.set(rowId, value); + } + } + + private static class TimestampNanoAccessor extends ArrowVectorAccessor { + + private final TimeStampNanoTZVector accessor; + private final ObjectTransformer transformer; + + TimestampNanoAccessor(TimeStampNanoTZVector vector, ObjectTransformer transformer) { + super(vector); + this.accessor = vector; + this.transformer = transformer; + } + + @Override + Object getObject(int rowId) { + return getLong(rowId); + } + + @Override + void setObject(int rowId, Object value) { + this.setLong(rowId, (Long) transformer.transformTimestamp(value).getTime()); + } + + @Override + final long getLong(int rowId) { + return accessor.get(rowId); + } + + @Override + final void setLong(int rowId, long value) { + accessor.set(rowId, value); + } + } + + private static class TimestampNanoNTZAccessor extends ArrowVectorAccessor { + + private final TimeStampNanoVector accessor; + private final ObjectTransformer transformer; + + TimestampNanoNTZAccessor(TimeStampNanoVector vector, ObjectTransformer transformer) { + super(vector); + this.accessor = vector; + this.transformer = transformer; + } + + @Override + Object getObject(int rowId) { + return getLong(rowId); + } + + @Override + void setObject(int rowId, Object value) { + if (value == null) { + accessor.setNull(rowId); + } else { + accessor.setSafe( + rowId, + (long) + (((java.sql.Timestamp) + transformer.transformTimestamp( + value)) + .getTime()) + * DateTimeConstants.NANOS_PER_MILLIS + + (((java.sql.Timestamp) transformer.transformTimestamp(value)) + .getNanos() + % DateTimeConstants.NANOS_PER_MILLIS)); + } + } + + @Override + final long getLong(int rowId) { + return accessor.get(rowId); + } + + @Override + final void setLong(int rowId, long value) { + accessor.set(rowId, value); + } + } + private static class NullAccessor extends ArrowVectorAccessor { - NullAccessor(NullVector vector) { + NullAccessor(NullVector vector, ObjectTransformer transformer) { super(vector); } } @@ -940,10 +1215,12 @@ private static class NullAccessor extends ArrowVectorAccessor { private static class IntervalYearAccessor extends ArrowVectorAccessor { private final IntervalYearVector accessor; + private final ObjectTransformer transformer; - IntervalYearAccessor(IntervalYearVector vector) { + IntervalYearAccessor(IntervalYearVector vector, ObjectTransformer transformer) { super(vector); this.accessor = vector; + this.transformer = transformer; } @Override @@ -953,7 +1230,7 @@ Object getObject(int rowId) { @Override void setObject(int rowId, Object value) { - this.setInt(rowId, (Integer) value); + this.setInt(rowId, (Integer) transformer.transform(value)); } @Override @@ -971,10 +1248,12 @@ private static class IntervalDayAccessor extends ArrowVectorAccessor { private final IntervalDayVector accessor; private final NullableIntervalDayHolder intervalDayHolder = new NullableIntervalDayHolder(); + private final ObjectTransformer transformer; - IntervalDayAccessor(IntervalDayVector vector) { + IntervalDayAccessor(IntervalDayVector vector, ObjectTransformer transformer) { super(vector); this.accessor = vector; + this.transformer = transformer; } @Override @@ -984,7 +1263,7 @@ Object getObject(int rowId) { @Override void setObject(int rowId, Object value) { - this.setLong(rowId, (Long) value); + this.setLong(rowId, (Long) transformer.transform(value)); } @Override @@ -1004,6 +1283,146 @@ final void setLong(int rowId, long value) { } } + private static class NestedVectorAccessor extends ArrowVectorAccessor { + + private final FieldVector accessor; + private final ObjectTransformer transformer; + + NestedVectorAccessor(FieldVector vector, ObjectTransformer transformer) { + super(vector); + this.accessor = vector; + this.transformer = transformer; + } + + @Override + void setObject(int rowId, Object value) { + setObject(accessor, rowId, value); + } + + private void setObject(FieldVector vector, int rowId, Object value) { + if (vector instanceof StructVector) { + ArrayList valueList = (ArrayList) value; + StructVector structVector = (StructVector) vector; + List childVectors = structVector.getChildrenFromFields(); + for (int i = 0; i < valueList.size(); i++) { + setObject(childVectors.get(i), rowId, valueList.get(i)); + } + structVector.setValueCount(structVector.getValueCount() + 1); + } else if (vector instanceof MapVector) { + MapVector mapVector = (MapVector) vector; + if (value == null) { + mapVector.setNull(rowId); + } else { + HashMap valueMap = (HashMap) value; + List objects = new ArrayList<>(); + + for (Object key : valueMap.keySet()) { + List temp = new ArrayList<>(); + temp.add(key); + temp.add(valueMap.get(key)); + objects.add(temp); + } + + mapVector.startNewValue(rowId); + int childRowId = mapVector.getDataVector().getValueCount(); + for (int i = 0; i < objects.size(); i++) { + setObject(mapVector.getDataVector(), childRowId + i, objects.get(i)); + } + mapVector.endValue(rowId, objects.size()); + vector.setValueCount(vector.getValueCount() + objects.size()); + } + } else if (vector instanceof ListVector) { + ListVector listVector = (ListVector) vector; + if (value == null) { + listVector.setNull(rowId); + } else { + ArrayList valueList = (ArrayList) value; + listVector.startNewValue(rowId); + int childRowId = listVector.getDataVector().getValueCount(); + for (int i = 0; i < valueList.size(); i++) { + setObject(listVector.getDataVector(), childRowId + i, valueList.get(i)); + } + listVector.endValue(rowId, valueList.size()); + vector.setValueCount(vector.getValueCount() + valueList.size()); + } + } else { + if (value == null) { + vector.setNull(rowId); + vector.setValueCount(vector.getValueCount() + 1); + } else if (vector instanceof IntVector) { + IntVector intVector = (IntVector) vector; + intVector.setSafe(rowId, transformer.transformInt(value)); + intVector.setValueCount(intVector.getValueCount() + 1); + } else if (vector instanceof BigIntVector) { + BigIntVector bigIntVector = (BigIntVector) vector; + bigIntVector.setSafe(rowId, transformer.transformLong(value)); + bigIntVector.setValueCount(bigIntVector.getValueCount() + 1); + } else if (vector instanceof SmallIntVector) { + SmallIntVector smallIntVector = (SmallIntVector) vector; + smallIntVector.setSafe(rowId, transformer.transformShort(value)); + smallIntVector.setValueCount(smallIntVector.getValueCount() + 1); + } else if (vector instanceof TinyIntVector) { + TinyIntVector tinyIntVector = (TinyIntVector) vector; + tinyIntVector.setSafe(rowId, transformer.transformByte(value)); + tinyIntVector.setValueCount(tinyIntVector.getValueCount() + 1); + } else if (vector instanceof Float8Vector) { + Float8Vector doubleVector = (Float8Vector) vector; + doubleVector.setSafe(rowId, transformer.transformDouble(value)); + doubleVector.setValueCount(doubleVector.getValueCount() + 1); + } else if (vector instanceof Float4Vector) { + Float4Vector floatVector = (Float4Vector) vector; + floatVector.setSafe(rowId, transformer.transformFloat(value)); + floatVector.setValueCount(floatVector.getValueCount() + 1); + } else if (vector instanceof BitVector) { + BitVector bitVector = (BitVector) vector; + bitVector.setSafe(rowId, transformer.transformBoolean(value)); + bitVector.setValueCount(bitVector.getValueCount() + 1); + } else if (vector instanceof DateMilliVector) { + DateMilliVector dateMilliVector = (DateMilliVector) vector; + dateMilliVector.setSafe(rowId, (transformer.transformDate(value)).getTime()); + dateMilliVector.setValueCount(dateMilliVector.getValueCount() + 1); + } else if (vector instanceof TimeStampNanoVector) { + TimeStampNanoVector timeStampNanoVector = (TimeStampNanoVector) vector; + timeStampNanoVector.setSafe( + rowId, + (long) ((transformer.transformTimestamp(value)).getTime()) + * DateTimeConstants.NANOS_PER_MILLIS + + ((transformer.transformTimestamp(value)).getNanos() + % DateTimeConstants.NANOS_PER_MILLIS)); + timeStampNanoVector.setValueCount(timeStampNanoVector.getValueCount() + 1); + } else if (vector instanceof DecimalVector) { + DecimalVector decimalVector = (DecimalVector) vector; + BigDecimal bigDecimal = + transformer.transformDecimal( + value, + decimalVector.getPrecision(), + decimalVector.getScale(), + 128); + decimalVector.setSafe(rowId, bigDecimal); + decimalVector.setValueCount(decimalVector.getValueCount() + 1); + } else if (vector instanceof LargeVarCharVector) { + LargeVarCharVector largeVarCharVector = (LargeVarCharVector) vector; + largeVarCharVector.setSafe(rowId, transformer.transformLargeUtf8(value)); + largeVarCharVector.setValueCount(largeVarCharVector.getValueCount() + 1); + } else if (vector instanceof VarCharVector) { + VarCharVector varCharVector = (VarCharVector) vector; + varCharVector.setSafe(rowId, transformer.transformUtf8(value)); + varCharVector.setValueCount(varCharVector.getValueCount() + 1); + } else if (vector instanceof VarBinaryVector) { + VarBinaryVector varBinaryVector = (VarBinaryVector) vector; + varBinaryVector.setSafe(rowId, transformer.transformBinary(value)); + varBinaryVector.setValueCount(varBinaryVector.getValueCount() + 1); + } else if (vector instanceof LargeVarBinaryVector) { + LargeVarBinaryVector largeVarBinaryVector = (LargeVarBinaryVector) vector; + largeVarBinaryVector.setSafe(rowId, transformer.transformLargeBinary(value)); + largeVarBinaryVector.setValueCount(largeVarBinaryVector.getValueCount() + 1); + } else { + assert false : "Unsupported vector type:" + vector.getClass().getName(); + } + } + } + } + // refer from spark: // https://github.com/apache/spark/blob/master/common/unsafe/src/main/java/org/apache/spark/sql/catalyst/util/DateTimeConstants.java private class DateTimeConstants {