From 62ece3e7b60600c74ed637aa60941554536dcc25 Mon Sep 17 00:00:00 2001 From: linlong Date: Mon, 11 Dec 2023 15:17:40 +0800 Subject: [PATCH 01/17] =?UTF-8?q?=E3=80=90Spark=20Extension=E3=80=91orc=20?= =?UTF-8?q?support=20chinese?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- .../cpp/src/CMakeLists.txt | 2 + .../cpp/src/io/orcfile/OmniOrcFile.cc | 35 +++++ .../cpp/src/io/orcfile/OmniOrcFile.hh | 46 ++++++ .../cpp/src/io/orcfile/OmniOrcHdfsFile.cc | 148 ++++++++++++++++++ .../cpp/src/jni/OrcColumnarBatchJniReader.cpp | 17 +- .../spark/jni/OrcColumnarBatchJniReader.java | 14 +- .../orc/OmniOrcColumnarBatchReader.java | 2 +- .../jni/OrcColumnarBatchJniReaderTest.java | 7 +- 8 files changed, 260 insertions(+), 11 deletions(-) create mode 100644 omnioperator/omniop-spark-extension/cpp/src/io/orcfile/OmniOrcFile.cc create mode 100644 omnioperator/omniop-spark-extension/cpp/src/io/orcfile/OmniOrcFile.hh create mode 100644 omnioperator/omniop-spark-extension/cpp/src/io/orcfile/OmniOrcHdfsFile.cc diff --git a/omnioperator/omniop-spark-extension/cpp/src/CMakeLists.txt b/omnioperator/omniop-spark-extension/cpp/src/CMakeLists.txt index 7256a02cb..f8b4f2587 100644 --- a/omnioperator/omniop-spark-extension/cpp/src/CMakeLists.txt +++ b/omnioperator/omniop-spark-extension/cpp/src/CMakeLists.txt @@ -10,6 +10,8 @@ set (SOURCE_FILES io/OutputStream.cc io/SparkFile.cc io/WriterOptions.cc + io/orcfile/OmniOrcFile.cc + io/orcfile/OmniOrcHdfsFile.cc shuffle/splitter.cpp common/common.cpp jni/SparkJniWrapper.cpp diff --git a/omnioperator/omniop-spark-extension/cpp/src/io/orcfile/OmniOrcFile.cc b/omnioperator/omniop-spark-extension/cpp/src/io/orcfile/OmniOrcFile.cc new file mode 100644 index 000000000..c484a434e --- /dev/null +++ b/omnioperator/omniop-spark-extension/cpp/src/io/orcfile/OmniOrcFile.cc @@ -0,0 +1,35 @@ +/** + * 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. + */ + +#include "OmniOrcFile.hh" +#include "orc/Exceptions.hh" + +#ifdef _MSC_VER +#else +#define O_BINARY 0 +#endif + +namespace orc { + std::unique_ptr readOmniFile(const std::string& path, const std::string& schema) { + if (schema == "hdfs") { + return orc::readOmniHdfsFile(std::string(path)); + } else { + return orc::readLocalFile(std::string(path)); + } + } +} diff --git a/omnioperator/omniop-spark-extension/cpp/src/io/orcfile/OmniOrcFile.hh b/omnioperator/omniop-spark-extension/cpp/src/io/orcfile/OmniOrcFile.hh new file mode 100644 index 000000000..a849c86a1 --- /dev/null +++ b/omnioperator/omniop-spark-extension/cpp/src/io/orcfile/OmniOrcFile.hh @@ -0,0 +1,46 @@ +/** + * 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. + */ + +#ifndef ORC_FILE_REWRITE_HH +#define ORC_FILE_REWRITE_HH + +#include + +#include "hdfspp/options.h" +#include "orc/OrcFile.hh" + +/** /file orc/OrcFile.hh + @brief The top level interface to ORC. +*/ + +namespace orc { + + /** + * Create a stream to a local file or HDFS file if path begins with "hdfs://" + * @param path the name of the file in the local file system or HDFS + */ + ORC_UNIQUE_PTR readOmniFile(const std::string& path, const std::string& schema); + + /** + * Create a stream to an HDFS file. + * @param path the uri of the file in HDFS + */ + ORC_UNIQUE_PTR readOmniHdfsFile(const std::string& path); +} + +#endif diff --git a/omnioperator/omniop-spark-extension/cpp/src/io/orcfile/OmniOrcHdfsFile.cc b/omnioperator/omniop-spark-extension/cpp/src/io/orcfile/OmniOrcHdfsFile.cc new file mode 100644 index 000000000..e4b08ea13 --- /dev/null +++ b/omnioperator/omniop-spark-extension/cpp/src/io/orcfile/OmniOrcHdfsFile.cc @@ -0,0 +1,148 @@ +/** + * 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. + */ + +#include "OmniOrcFile.hh" + +#include "orc/Exceptions.hh" + +#include +#include + +#include "hdfspp/hdfspp.h" + +namespace orc { + + class OmniHdfsFileInputStream : public InputStream { + private: + std::string filename; + std::unique_ptr file; + std::unique_ptr file_system; + uint64_t totalLength; + const uint64_t READ_SIZE = 1024 * 1024; //1 MB + + public: + OmniHdfsFileInputStream(std::string _filename) { + filename = _filename ; + + hdfs::ConfigParser parser; + if(!parser.LoadDefaultResources()){ + throw ParseError("Could not load default resources. "); + } + auto stats = parser.ValidateResources(); + //validating core-site.xml + if(!stats[0].second.ok()){ + throw ParseError(stats[0].first + " is invalid: " + stats[0].second.ToString()); + } + //validating hdfs-site.xml + if(!stats[1].second.ok()){ + throw ParseError(stats[1].first + " is invalid: " + stats[1].second.ToString()); + } + hdfs::Options options; + if(!parser.get_options(options)){ + throw ParseError("Could not load Options object. "); + } + hdfs::IoService * io_service = hdfs::IoService::New(); + //Wrapping file_system into a unique pointer to guarantee deletion + file_system = std::unique_ptr( + hdfs::FileSystem::New(io_service, "", options)); + if (file_system.get() == nullptr) { + throw ParseError("Can't create FileSystem object. "); + } + hdfs::Status status; + + status = file_system->ConnectToDefaultFs(); + if (!status.ok()) { + if(!options.defaultFS.get_host().empty()){ + throw ParseError("Error connecting to " + + options.defaultFS.str() + ". " + status.ToString()); + } else { + throw ParseError( + "Error connecting to the cluster: defaultFS is empty. " + + status.ToString()); + } + } + + if (file_system.get() == nullptr) { + throw ParseError("Can't connect the file system. "); + } + + hdfs::FileHandle *file_raw = nullptr; + status = file_system->Open(filename, &file_raw); + if (!status.ok()) { + throw ParseError("Can't open " + + filename + ". " + status.ToString()); + } + //Wrapping file_raw into a unique pointer to guarantee deletion + file.reset(file_raw); + + hdfs::StatInfo stat_info; + status = file_system->GetFileInfo(filename, stat_info); + if (!status.ok()) { + throw ParseError("Can't stat " + + filename + ". " + status.ToString()); + } + totalLength = stat_info.length; + } + + uint64_t getLength() const override { + return totalLength; + } + + uint64_t getNaturalReadSize() const override { + return READ_SIZE; + } + + void read(void* buf, + uint64_t length, + uint64_t offset) override { + + if (!buf) { + throw ParseError("Buffer is null"); + } + + char* buf_ptr = reinterpret_cast(buf); + hdfs::Status status; + size_t total_bytes_read = 0; + size_t last_bytes_read = 0; + + do { + status = file->PositionRead(buf_ptr, + static_cast(length) - total_bytes_read, + static_cast(offset + total_bytes_read), &last_bytes_read); + if(!status.ok()) { + throw ParseError("Error reading the file: " + status.ToString()); + } + total_bytes_read += last_bytes_read; + buf_ptr += last_bytes_read; + } while (total_bytes_read < length); + } + + const std::string& getName() const override { + return filename; + } + + ~OmniHdfsFileInputStream() override; + }; + + OmniHdfsFileInputStream::~OmniHdfsFileInputStream() { + } + + std::unique_ptr readOmniHdfsFile(const std::string& path) { + return std::unique_ptr(new OmniHdfsFileInputStream(path)); + } +} diff --git a/omnioperator/omniop-spark-extension/cpp/src/jni/OrcColumnarBatchJniReader.cpp b/omnioperator/omniop-spark-extension/cpp/src/jni/OrcColumnarBatchJniReader.cpp index c0f4c1ae1..1e4d36ff1 100644 --- a/omnioperator/omniop-spark-extension/cpp/src/jni/OrcColumnarBatchJniReader.cpp +++ b/omnioperator/omniop-spark-extension/cpp/src/jni/OrcColumnarBatchJniReader.cpp @@ -39,8 +39,6 @@ JNIEXPORT jlong JNICALL Java_com_huawei_boostkit_spark_jni_OrcColumnarBatchJniRe jlong tailLocation = env->CallLongMethod(jsonObj, jsonMethodLong, env->NewStringUTF("tailLocation")); jstring serTailJstr = (jstring)env->CallObjectMethod(jsonObj, jsonMethodString, env->NewStringUTF("serializedTail")); - const char *pathPtr = env->GetStringUTFChars(path, nullptr); - std::string filePath(pathPtr); orc::MemoryPool *pool = orc::getDefaultPool(); orc::ReaderOptions readerOptions; readerOptions.setMemoryPool(*pool); @@ -52,8 +50,19 @@ JNIEXPORT jlong JNICALL Java_com_huawei_boostkit_spark_jni_OrcColumnarBatchJniRe env->ReleaseStringUTFChars(serTailJstr, ptr); } - std::unique_ptr reader = createReader(orc::readFile(filePath), readerOptions); - env->ReleaseStringUTFChars(path, pathPtr); + jstring schemaJstr = (jstring)env->CallObjectMethod(jsonObj, jsonMethodString, env->NewStringUTF("scheme")); + const char *schemaPtr = env->GetStringUTFChars(schemaJstr, nullptr); + std::string schemaStr(schemaPtr); + env->ReleaseStringUTFChars(schemaJstr, schemaPtr); + + jstring fileJstr = (jstring)env->CallObjectMethod(jsonObj, jsonMethodString, env->NewStringUTF("path")); + const char *filePtr = env->GetStringUTFChars(fileJstr, nullptr); + std::string fileStr(filePtr); + env->ReleaseStringUTFChars(fileJstr, filePtr); + + std::unique_ptr reader; + reader = createReader(orc::readOmniFile(fileStr, schemaStr), readerOptions); + orc::Reader *readerNew = reader.release(); return (jlong)(readerNew); JNI_FUNC_END(runtimeExceptionClass) diff --git a/omnioperator/omniop-spark-extension/java/src/main/java/com/huawei/boostkit/spark/jni/OrcColumnarBatchJniReader.java b/omnioperator/omniop-spark-extension/java/src/main/java/com/huawei/boostkit/spark/jni/OrcColumnarBatchJniReader.java index d80a23653..c4120036a 100644 --- a/omnioperator/omniop-spark-extension/java/src/main/java/com/huawei/boostkit/spark/jni/OrcColumnarBatchJniReader.java +++ b/omnioperator/omniop-spark-extension/java/src/main/java/com/huawei/boostkit/spark/jni/OrcColumnarBatchJniReader.java @@ -35,6 +35,7 @@ import java.sql.Date; import java.util.ArrayList; import java.util.Arrays; import java.util.List; +import java.net.URI; public class OrcColumnarBatchJniReader { @@ -133,10 +134,10 @@ public class OrcColumnarBatchJniReader { /** * Init Orc reader. * - * @param path split file path + * @param uri split file path * @param options split file options */ - public long initializeReaderJava(String path, ReaderOptions options) { + public long initializeReaderJava(URI uri, ReaderOptions options) { JSONObject job = new JSONObject(); if (options.getOrcTail() == null) { job.put("serializedTail", ""); @@ -144,10 +145,15 @@ public class OrcColumnarBatchJniReader { job.put("serializedTail", options.getOrcTail().getSerializedTail().toString()); } job.put("tailLocation", 9223372036854775807L); - reader = initializeReader(path, job); + + job.put("scheme", uri.getScheme()); + job.put("host", uri.getHost()); + job.put("port", uri.getPort()); + job.put("path", uri.getPath()); + + reader = initializeReader(uri.getPath(), job); return reader; } - /** * Init Orc RecordReader. * diff --git a/omnioperator/omniop-spark-extension/java/src/main/java/org/apache/spark/sql/execution/datasources/orc/OmniOrcColumnarBatchReader.java b/omnioperator/omniop-spark-extension/java/src/main/java/org/apache/spark/sql/execution/datasources/orc/OmniOrcColumnarBatchReader.java index c170b04e4..bb6b4e827 100644 --- a/omnioperator/omniop-spark-extension/java/src/main/java/org/apache/spark/sql/execution/datasources/orc/OmniOrcColumnarBatchReader.java +++ b/omnioperator/omniop-spark-extension/java/src/main/java/org/apache/spark/sql/execution/datasources/orc/OmniOrcColumnarBatchReader.java @@ -134,7 +134,7 @@ public class OmniOrcColumnarBatchReader extends RecordReader Date: Mon, 11 Dec 2023 15:47:32 +0800 Subject: [PATCH 02/17] =?UTF-8?q?=E3=80=90Spark=20Extension=E3=80=91orc=20?= =?UTF-8?q?support=20chinese?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- .../cpp/src/jni/OrcColumnarBatchJniReader.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/omnioperator/omniop-spark-extension/cpp/src/jni/OrcColumnarBatchJniReader.h b/omnioperator/omniop-spark-extension/cpp/src/jni/OrcColumnarBatchJniReader.h index 714d97ee6..413cc77d2 100644 --- a/omnioperator/omniop-spark-extension/cpp/src/jni/OrcColumnarBatchJniReader.h +++ b/omnioperator/omniop-spark-extension/cpp/src/jni/OrcColumnarBatchJniReader.h @@ -33,7 +33,7 @@ #include #include #include -#include +#include #include #include #include -- Gitee From 34a9154375866e3c3e7a3c1bd662697f31105b7e Mon Sep 17 00:00:00 2001 From: d00807371 Date: Tue, 12 Dec 2023 17:50:43 +0800 Subject: [PATCH 03/17] spark extension 311 supports chinese partition --- .../omniop-spark-extension/cpp/CMakeLists.txt | 12 +++--- .../cpp/src/io/orcfile/OmniOrcFile.cc | 5 ++- .../cpp/src/io/orcfile/OmniOrcFile.hh | 6 ++- .../cpp/src/io/orcfile/OmniOrcHdfsFile.cc | 38 +++++++++++++------ .../cpp/src/jni/OrcColumnarBatchJniReader.cpp | 14 ++++++- .../cpp/src/jni/OrcColumnarBatchJniReader.h | 2 +- .../spark/jni/OrcColumnarBatchJniReader.java | 12 +++--- ...OrcColumnarBatchJniReaderDataTypeTest.java | 18 ++++++--- ...ColumnarBatchJniReaderNotPushDownTest.java | 18 ++++++--- ...OrcColumnarBatchJniReaderPushDownTest.java | 18 ++++++--- ...BatchJniReaderSparkORCNotPushDownTest.java | 18 ++++++--- ...narBatchJniReaderSparkORCPushDownTest.java | 18 ++++++--- .../jni/OrcColumnarBatchJniReaderTest.java | 9 ++++- 13 files changed, 131 insertions(+), 57 deletions(-) diff --git a/omnioperator/omniop-spark-extension/cpp/CMakeLists.txt b/omnioperator/omniop-spark-extension/cpp/CMakeLists.txt index 491cfb708..a3564f831 100644 --- a/omnioperator/omniop-spark-extension/cpp/CMakeLists.txt +++ b/omnioperator/omniop-spark-extension/cpp/CMakeLists.txt @@ -33,12 +33,12 @@ add_subdirectory(src) message(STATUS "Build by ${CMAKE_BUILD_TYPE}") -option(BUILD_CPP_TESTS "test" ON) -message(STATUS "Option BUILD_CPP_TESTS: ${BUILD_CPP_TESTS}") -if(${BUILD_CPP_TESTS}) - enable_testing() - add_subdirectory(test) -endif () +#option(BUILD_CPP_TESTS "test" ON) +#message(STATUS "Option BUILD_CPP_TESTS: ${BUILD_CPP_TESTS}") +#if(${BUILD_CPP_TESTS}) +# enable_testing() +# add_subdirectory(test) +#endif () # options option(DEBUG_RUNTIME "Debug" OFF) diff --git a/omnioperator/omniop-spark-extension/cpp/src/io/orcfile/OmniOrcFile.cc b/omnioperator/omniop-spark-extension/cpp/src/io/orcfile/OmniOrcFile.cc index c484a434e..58e7461d1 100644 --- a/omnioperator/omniop-spark-extension/cpp/src/io/orcfile/OmniOrcFile.cc +++ b/omnioperator/omniop-spark-extension/cpp/src/io/orcfile/OmniOrcFile.cc @@ -25,9 +25,10 @@ #endif namespace orc { - std::unique_ptr readOmniFile(const std::string& path, const std::string& schema) { + std::unique_ptr readOmniFile(const std::string& path, const std::string& schema, + const std::string& uri_host, const std::string& uri_port) { if (schema == "hdfs") { - return orc::readOmniHdfsFile(std::string(path)); + return orc::readOmniHdfsFile(std::string(path), std::string(uri_host), std::string(uri_port)); } else { return orc::readLocalFile(std::string(path)); } diff --git a/omnioperator/omniop-spark-extension/cpp/src/io/orcfile/OmniOrcFile.hh b/omnioperator/omniop-spark-extension/cpp/src/io/orcfile/OmniOrcFile.hh index a849c86a1..fa6748c63 100644 --- a/omnioperator/omniop-spark-extension/cpp/src/io/orcfile/OmniOrcFile.hh +++ b/omnioperator/omniop-spark-extension/cpp/src/io/orcfile/OmniOrcFile.hh @@ -34,13 +34,15 @@ namespace orc { * Create a stream to a local file or HDFS file if path begins with "hdfs://" * @param path the name of the file in the local file system or HDFS */ - ORC_UNIQUE_PTR readOmniFile(const std::string& path, const std::string& schema); + ORC_UNIQUE_PTR readOmniFile(const std::string& path, const std::string& schema, + const std::string& uri_host, const std::string& uri_port); /** * Create a stream to an HDFS file. * @param path the uri of the file in HDFS */ - ORC_UNIQUE_PTR readOmniHdfsFile(const std::string& path); + ORC_UNIQUE_PTR readOmniHdfsFile(const std::string& path, const std::string& uri_host, + const std::string& uri_port); } #endif diff --git a/omnioperator/omniop-spark-extension/cpp/src/io/orcfile/OmniOrcHdfsFile.cc b/omnioperator/omniop-spark-extension/cpp/src/io/orcfile/OmniOrcHdfsFile.cc index e4b08ea13..7514c9c0e 100644 --- a/omnioperator/omniop-spark-extension/cpp/src/io/orcfile/OmniOrcHdfsFile.cc +++ b/omnioperator/omniop-spark-extension/cpp/src/io/orcfile/OmniOrcHdfsFile.cc @@ -30,14 +30,18 @@ namespace orc { class OmniHdfsFileInputStream : public InputStream { private: std::string filename; + std::string uri_host; + std::string uri_port; std::unique_ptr file; std::unique_ptr file_system; uint64_t totalLength; const uint64_t READ_SIZE = 1024 * 1024; //1 MB public: - OmniHdfsFileInputStream(std::string _filename) { - filename = _filename ; + OmniHdfsFileInputStream(std::string _filename, std::string _uri_host, std::string _uri_port) { + filename = _filename; + uri_host = _uri_host; + uri_port = _uri_port; hdfs::ConfigParser parser; if(!parser.LoadDefaultResources()){ @@ -65,16 +69,27 @@ namespace orc { } hdfs::Status status; - status = file_system->ConnectToDefaultFs(); - if (!status.ok()) { + //Checking if the user supplied the host + if(!uri_host.empty()){ + std::string port = !uri_port.empty() ? + uri_port : ""; + status = file_system->Connect(uri_host, port); + if (!status.ok()) { + throw ParseError("Can't connect to " + uri_host + + ":" + port + ". " + status.ToString()); + } + } else { + status = file_system->ConnectToDefaultFs(); + if (!status.ok()) { if(!options.defaultFS.get_host().empty()){ - throw ParseError("Error connecting to " + - options.defaultFS.str() + ". " + status.ToString()); + throw ParseError("Error connecting to " + + options.defaultFS.str() + ". " + status.ToString()); } else { - throw ParseError( - "Error connecting to the cluster: defaultFS is empty. " - + status.ToString()); + throw ParseError( + "Error connecting to the cluster: defaultFS is empty. " + + status.ToString()); } + } } if (file_system.get() == nullptr) { @@ -142,7 +157,8 @@ namespace orc { OmniHdfsFileInputStream::~OmniHdfsFileInputStream() { } - std::unique_ptr readOmniHdfsFile(const std::string& path) { - return std::unique_ptr(new OmniHdfsFileInputStream(path)); + std::unique_ptr readOmniHdfsFile(const std::string& path, const std::string& uri_host, + const std::string& uri_port) { + return std::unique_ptr(new OmniHdfsFileInputStream(path, uri_host, uri_port)); } } diff --git a/omnioperator/omniop-spark-extension/cpp/src/jni/OrcColumnarBatchJniReader.cpp b/omnioperator/omniop-spark-extension/cpp/src/jni/OrcColumnarBatchJniReader.cpp index 1e4d36ff1..0fd65da56 100644 --- a/omnioperator/omniop-spark-extension/cpp/src/jni/OrcColumnarBatchJniReader.cpp +++ b/omnioperator/omniop-spark-extension/cpp/src/jni/OrcColumnarBatchJniReader.cpp @@ -29,7 +29,7 @@ using namespace orc; static constexpr int32_t MAX_DECIMAL64_DIGITS = 18; JNIEXPORT jlong JNICALL Java_com_huawei_boostkit_spark_jni_OrcColumnarBatchJniReader_initializeReader(JNIEnv *env, - jobject jObj, jstring path, jobject jsonObj) + jobject jObj, jobject jsonObj) { JNI_FUNC_START @@ -60,8 +60,18 @@ JNIEXPORT jlong JNICALL Java_com_huawei_boostkit_spark_jni_OrcColumnarBatchJniRe std::string fileStr(filePtr); env->ReleaseStringUTFChars(fileJstr, filePtr); + jstring hostJstr = (jstring)env->CallObjectMethod(jsonObj, jsonMethodString, env->NewStringUTF("host")); + const char *hostPtr = env->GetStringUTFChars(hostJstr, nullptr); + std::string hostStr(hostPtr); + env->ReleaseStringUTFChars(hostJstr, hostPtr); + + jstring portJstr = (jstring)env->CallObjectMethod(jsonObj, jsonMethodString, env->NewStringUTF("port")); + const char *portPtr = env->GetStringUTFChars(portJstr, nullptr); + std::string portStr(portPtr); + env->ReleaseStringUTFChars(portJstr, portPtr); + std::unique_ptr reader; - reader = createReader(orc::readOmniFile(fileStr, schemaStr), readerOptions); + reader = createReader(orc::readOmniFile(fileStr, schemaStr, hostStr, portStr), readerOptions); orc::Reader *readerNew = reader.release(); return (jlong)(readerNew); diff --git a/omnioperator/omniop-spark-extension/cpp/src/jni/OrcColumnarBatchJniReader.h b/omnioperator/omniop-spark-extension/cpp/src/jni/OrcColumnarBatchJniReader.h index 413cc77d2..6c7560a7f 100644 --- a/omnioperator/omniop-spark-extension/cpp/src/jni/OrcColumnarBatchJniReader.h +++ b/omnioperator/omniop-spark-extension/cpp/src/jni/OrcColumnarBatchJniReader.h @@ -69,7 +69,7 @@ enum class PredicateOperatorType { * Signature: (Ljava/lang/String;Lorg/json/simple/JSONObject;)J */ JNIEXPORT jlong JNICALL Java_com_huawei_boostkit_spark_jni_OrcColumnarBatchJniReader_initializeReader - (JNIEnv* env, jobject jObj, jstring path, jobject job); + (JNIEnv* env, jobject jObj, jobject job); /* * Class: com_huawei_boostkit_spark_jni_OrcColumnarBatchJniReader diff --git a/omnioperator/omniop-spark-extension/java/src/main/java/com/huawei/boostkit/spark/jni/OrcColumnarBatchJniReader.java b/omnioperator/omniop-spark-extension/java/src/main/java/com/huawei/boostkit/spark/jni/OrcColumnarBatchJniReader.java index c4120036a..ae0c0f6fd 100644 --- a/omnioperator/omniop-spark-extension/java/src/main/java/com/huawei/boostkit/spark/jni/OrcColumnarBatchJniReader.java +++ b/omnioperator/omniop-spark-extension/java/src/main/java/com/huawei/boostkit/spark/jni/OrcColumnarBatchJniReader.java @@ -146,12 +146,12 @@ public class OrcColumnarBatchJniReader { } job.put("tailLocation", 9223372036854775807L); - job.put("scheme", uri.getScheme()); - job.put("host", uri.getHost()); - job.put("port", uri.getPort()); - job.put("path", uri.getPath()); + job.put("scheme", uri.getScheme() == null ? "" : uri.getScheme()); + job.put("host", uri.getHost() == null ? "" : uri.getHost()); + job.put("port", uri.getPort() == -1 ? "" : String.valueOf(uri.getPort())); + job.put("path", uri.getPath() == null ? "" : uri.getPath()); - reader = initializeReader(uri.getPath(), job); + reader = initializeReader(job); return reader; } /** @@ -291,7 +291,7 @@ public class OrcColumnarBatchJniReader { return (int)rtn; } - public native long initializeReader(String path, JSONObject job); + public native long initializeReader(JSONObject job); public native long initializeRecordReader(long reader, JSONObject job); diff --git a/omnioperator/omniop-spark-extension/java/src/test/java/com/huawei/boostkit/spark/jni/OrcColumnarBatchJniReaderDataTypeTest.java b/omnioperator/omniop-spark-extension/java/src/test/java/com/huawei/boostkit/spark/jni/OrcColumnarBatchJniReaderDataTypeTest.java index 73db9a981..d2740bb98 100644 --- a/omnioperator/omniop-spark-extension/java/src/test/java/com/huawei/boostkit/spark/jni/OrcColumnarBatchJniReaderDataTypeTest.java +++ b/omnioperator/omniop-spark-extension/java/src/test/java/com/huawei/boostkit/spark/jni/OrcColumnarBatchJniReaderDataTypeTest.java @@ -30,8 +30,12 @@ import org.junit.Before; import org.junit.FixMethodOrder; import org.junit.Test; import org.junit.runners.MethodSorters; +import org.apache.hadoop.conf.Configuration; +import org.apache.orc.OrcFile; import java.io.File; +import java.net.URI; +import java.net.URISyntaxException; import java.util.ArrayList; import static org.junit.Assert.*; @@ -54,12 +58,16 @@ public class OrcColumnarBatchJniReaderDataTypeTest extends TestCase { } public void initReaderJava() { - JSONObject job = new JSONObject(); - job.put("serializedTail",""); - job.put("tailLocation",9223372036854775807L); File directory = new File("src/test/java/com/huawei/boostkit/spark/jni/orcsrc/000000_0"); - System.out.println(directory.getAbsolutePath()); - orcColumnarBatchJniReader.reader = orcColumnarBatchJniReader.initializeReader(directory.getAbsolutePath(), job); + String absolutePath = directory.getAbsolutePath(); + System.out.println(absolutePath); + URI uri = null; + try { + uri = new URI(absolutePath); + } catch (URISyntaxException ignore) { + } + assertTrue(uri != null); + orcColumnarBatchJniReader.reader = orcColumnarBatchJniReader.initializeReaderJava(uri, OrcFile.readerOptions(new Configuration())); assertTrue(orcColumnarBatchJniReader.reader != 0); } diff --git a/omnioperator/omniop-spark-extension/java/src/test/java/com/huawei/boostkit/spark/jni/OrcColumnarBatchJniReaderNotPushDownTest.java b/omnioperator/omniop-spark-extension/java/src/test/java/com/huawei/boostkit/spark/jni/OrcColumnarBatchJniReaderNotPushDownTest.java index d9fe13683..7a220470b 100644 --- a/omnioperator/omniop-spark-extension/java/src/test/java/com/huawei/boostkit/spark/jni/OrcColumnarBatchJniReaderNotPushDownTest.java +++ b/omnioperator/omniop-spark-extension/java/src/test/java/com/huawei/boostkit/spark/jni/OrcColumnarBatchJniReaderNotPushDownTest.java @@ -30,8 +30,12 @@ import org.junit.Before; import org.junit.FixMethodOrder; import org.junit.Test; import org.junit.runners.MethodSorters; +import org.apache.hadoop.conf.Configuration; +import org.apache.orc.OrcFile; import java.io.File; +import java.net.URI; +import java.net.URISyntaxException; import java.util.ArrayList; import static org.junit.Assert.*; @@ -54,12 +58,16 @@ public class OrcColumnarBatchJniReaderNotPushDownTest extends TestCase { } public void initReaderJava() { - JSONObject job = new JSONObject(); - job.put("serializedTail",""); - job.put("tailLocation",9223372036854775807L); File directory = new File("src/test/java/com/huawei/boostkit/spark/jni/orcsrc/000000_0"); - System.out.println(directory.getAbsolutePath()); - orcColumnarBatchJniReader.reader = orcColumnarBatchJniReader.initializeReader(directory.getAbsolutePath(), job); + String absolutePath = directory.getAbsolutePath(); + System.out.println(absolutePath); + URI uri = null; + try { + uri = new URI(absolutePath); + } catch (URISyntaxException ignore) { + } + assertTrue(uri != null); + orcColumnarBatchJniReader.reader = orcColumnarBatchJniReader.initializeReaderJava(uri, OrcFile.readerOptions(new Configuration())); assertTrue(orcColumnarBatchJniReader.reader != 0); } diff --git a/omnioperator/omniop-spark-extension/java/src/test/java/com/huawei/boostkit/spark/jni/OrcColumnarBatchJniReaderPushDownTest.java b/omnioperator/omniop-spark-extension/java/src/test/java/com/huawei/boostkit/spark/jni/OrcColumnarBatchJniReaderPushDownTest.java index 87f0cc1d2..3930bd67c 100644 --- a/omnioperator/omniop-spark-extension/java/src/test/java/com/huawei/boostkit/spark/jni/OrcColumnarBatchJniReaderPushDownTest.java +++ b/omnioperator/omniop-spark-extension/java/src/test/java/com/huawei/boostkit/spark/jni/OrcColumnarBatchJniReaderPushDownTest.java @@ -23,6 +23,8 @@ import junit.framework.TestCase; import org.apache.hadoop.mapred.join.ArrayListBackedIterator; import org.apache.orc.OrcFile.ReaderOptions; import org.apache.orc.Reader.Options; +import org.apache.hadoop.conf.Configuration; +import org.apache.orc.OrcFile; import org.hamcrest.Condition; import org.json.JSONObject; import org.junit.After; @@ -38,6 +40,8 @@ import nova.hetu.omniruntime.vector.Vec; import java.io.File; import java.lang.reflect.Array; +import java.net.URI; +import java.net.URISyntaxException; import java.util.ArrayList; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -60,12 +64,16 @@ public class OrcColumnarBatchJniReaderPushDownTest extends TestCase { } public void initReaderJava() { - JSONObject job = new JSONObject(); - job.put("serializedTail",""); - job.put("tailLocation",9223372036854775807L); File directory = new File("src/test/java/com/huawei/boostkit/spark/jni/orcsrc/000000_0"); - System.out.println(directory.getAbsolutePath()); - orcColumnarBatchJniReader.reader = orcColumnarBatchJniReader.initializeReader(directory.getAbsolutePath(), job); + String absolutePath = directory.getAbsolutePath(); + System.out.println(absolutePath); + URI uri = null; + try { + uri = new URI(absolutePath); + } catch (URISyntaxException ignore) { + } + assertTrue(uri != null); + orcColumnarBatchJniReader.reader = orcColumnarBatchJniReader.initializeReaderJava(uri, OrcFile.readerOptions(new Configuration())); assertTrue(orcColumnarBatchJniReader.reader != 0); } diff --git a/omnioperator/omniop-spark-extension/java/src/test/java/com/huawei/boostkit/spark/jni/OrcColumnarBatchJniReaderSparkORCNotPushDownTest.java b/omnioperator/omniop-spark-extension/java/src/test/java/com/huawei/boostkit/spark/jni/OrcColumnarBatchJniReaderSparkORCNotPushDownTest.java index 484365c53..6262bb6b2 100644 --- a/omnioperator/omniop-spark-extension/java/src/test/java/com/huawei/boostkit/spark/jni/OrcColumnarBatchJniReaderSparkORCNotPushDownTest.java +++ b/omnioperator/omniop-spark-extension/java/src/test/java/com/huawei/boostkit/spark/jni/OrcColumnarBatchJniReaderSparkORCNotPushDownTest.java @@ -22,6 +22,8 @@ import junit.framework.TestCase; import nova.hetu.omniruntime.vector.IntVec; import nova.hetu.omniruntime.vector.LongVec; import nova.hetu.omniruntime.vector.VarcharVec; +import org.apache.hadoop.conf.Configuration; +import org.apache.orc.OrcFile; import org.json.JSONObject; import org.junit.After; import org.junit.Before; @@ -30,6 +32,8 @@ import org.junit.Test; import org.junit.runners.MethodSorters; import java.io.File; +import java.net.URI; +import java.net.URISyntaxException; import java.util.ArrayList; import static org.junit.Assert.*; @@ -52,12 +56,16 @@ public class OrcColumnarBatchJniReaderSparkORCNotPushDownTest extends TestCase { } public void initReaderJava() { - JSONObject job = new JSONObject(); - job.put("serializedTail",""); - job.put("tailLocation",9223372036854775807L); File directory = new File("src/test/java/com/huawei/boostkit/spark/jni/orcsrc/part-00000-2d6ca713-08b0-4b40-828c-f7ee0c81bb9a-c000.snappy.orc"); - System.out.println(directory.getAbsolutePath()); - orcColumnarBatchJniReader.reader = orcColumnarBatchJniReader.initializeReader(directory.getAbsolutePath(), job); + String absolutePath = directory.getAbsolutePath(); + System.out.println(absolutePath); + URI uri = null; + try { + uri = new URI(absolutePath); + } catch (URISyntaxException ignore) { + } + assertTrue(uri != null); + orcColumnarBatchJniReader.reader = orcColumnarBatchJniReader.initializeReaderJava(uri, OrcFile.readerOptions(new Configuration())); assertTrue(orcColumnarBatchJniReader.reader != 0); } diff --git a/omnioperator/omniop-spark-extension/java/src/test/java/com/huawei/boostkit/spark/jni/OrcColumnarBatchJniReaderSparkORCPushDownTest.java b/omnioperator/omniop-spark-extension/java/src/test/java/com/huawei/boostkit/spark/jni/OrcColumnarBatchJniReaderSparkORCPushDownTest.java index b03d60aac..e7c1d334e 100644 --- a/omnioperator/omniop-spark-extension/java/src/test/java/com/huawei/boostkit/spark/jni/OrcColumnarBatchJniReaderSparkORCPushDownTest.java +++ b/omnioperator/omniop-spark-extension/java/src/test/java/com/huawei/boostkit/spark/jni/OrcColumnarBatchJniReaderSparkORCPushDownTest.java @@ -24,6 +24,8 @@ import nova.hetu.omniruntime.vector.IntVec; import nova.hetu.omniruntime.vector.LongVec; import nova.hetu.omniruntime.vector.VarcharVec; import nova.hetu.omniruntime.vector.Vec; +import org.apache.hadoop.conf.Configuration; +import org.apache.orc.OrcFile; import org.json.JSONObject; import org.junit.After; import org.junit.Before; @@ -32,6 +34,8 @@ import org.junit.Test; import org.junit.runners.MethodSorters; import java.io.File; +import java.net.URI; +import java.net.URISyntaxException; import java.util.ArrayList; import static org.junit.Assert.*; @@ -54,12 +58,16 @@ public class OrcColumnarBatchJniReaderSparkORCPushDownTest extends TestCase { } public void initReaderJava() { - JSONObject job = new JSONObject(); - job.put("serializedTail",""); - job.put("tailLocation",9223372036854775807L); File directory = new File("src/test/java/com/huawei/boostkit/spark/jni/orcsrc/part-00000-2d6ca713-08b0-4b40-828c-f7ee0c81bb9a-c000.snappy.orc"); - System.out.println(directory.getAbsolutePath()); - orcColumnarBatchJniReader.reader = orcColumnarBatchJniReader.initializeReader(directory.getAbsolutePath(), job); + String absolutePath = directory.getAbsolutePath(); + System.out.println(absolutePath); + URI uri = null; + try { + uri = new URI(absolutePath); + } catch (URISyntaxException ignore) { + } + assertTrue(uri != null); + orcColumnarBatchJniReader.reader = orcColumnarBatchJniReader.initializeReaderJava(uri, OrcFile.readerOptions(new Configuration())); assertTrue(orcColumnarBatchJniReader.reader != 0); } diff --git a/omnioperator/omniop-spark-extension/java/src/test/java/com/huawei/boostkit/spark/jni/OrcColumnarBatchJniReaderTest.java b/omnioperator/omniop-spark-extension/java/src/test/java/com/huawei/boostkit/spark/jni/OrcColumnarBatchJniReaderTest.java index 58ca85529..d4bcd0a58 100644 --- a/omnioperator/omniop-spark-extension/java/src/test/java/com/huawei/boostkit/spark/jni/OrcColumnarBatchJniReaderTest.java +++ b/omnioperator/omniop-spark-extension/java/src/test/java/com/huawei/boostkit/spark/jni/OrcColumnarBatchJniReaderTest.java @@ -90,11 +90,16 @@ public class OrcColumnarBatchJniReaderTest extends TestCase { System.out.println("orcColumnarBatchJniReader test finished"); } - public void initReaderJava() throws URISyntaxException { + public void initReaderJava() { OrcFile.ReaderOptions readerOptions = OrcFile.readerOptions(conf); File directory = new File("src/test/java/com/huawei/boostkit/spark/jni/orcsrc/000000_0"); String path = directory.getAbsolutePath(); - URI uri = new URI(path); + URI uri = null; + try { + uri = new URI(path); + } catch (URISyntaxException ignore) { + } + assertTrue(uri != null); orcColumnarBatchJniReader.reader = orcColumnarBatchJniReader.initializeReaderJava(uri, readerOptions); assertTrue(orcColumnarBatchJniReader.reader != 0); } -- Gitee From 842ed47cce16d002c366b8c3705c7d50d28807ef Mon Sep 17 00:00:00 2001 From: zc_deng2023 Date: Thu, 14 Dec 2023 08:52:36 +0000 Subject: [PATCH 04/17] update omnioperator/omniop-spark-extension/cpp/CMakeLists.txt. Signed-off-by: zc_deng2023 --- .../omniop-spark-extension/cpp/CMakeLists.txt | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) diff --git a/omnioperator/omniop-spark-extension/cpp/CMakeLists.txt b/omnioperator/omniop-spark-extension/cpp/CMakeLists.txt index a3564f831..491cfb708 100644 --- a/omnioperator/omniop-spark-extension/cpp/CMakeLists.txt +++ b/omnioperator/omniop-spark-extension/cpp/CMakeLists.txt @@ -33,12 +33,12 @@ add_subdirectory(src) message(STATUS "Build by ${CMAKE_BUILD_TYPE}") -#option(BUILD_CPP_TESTS "test" ON) -#message(STATUS "Option BUILD_CPP_TESTS: ${BUILD_CPP_TESTS}") -#if(${BUILD_CPP_TESTS}) -# enable_testing() -# add_subdirectory(test) -#endif () +option(BUILD_CPP_TESTS "test" ON) +message(STATUS "Option BUILD_CPP_TESTS: ${BUILD_CPP_TESTS}") +if(${BUILD_CPP_TESTS}) + enable_testing() + add_subdirectory(test) +endif () # options option(DEBUG_RUNTIME "Debug" OFF) -- Gitee From 14ac539147fb1d2acf79de0bbcd0b45fcab7a5ed Mon Sep 17 00:00:00 2001 From: d00807371 Date: Thu, 21 Dec 2023 11:19:25 +0800 Subject: [PATCH 05/17] omni spark 311 support parquet Chinese character --- .../cpp/src/CMakeLists.txt | 7 +- .../arrowadapter/filesystem/OmniFileSystem.cc | 115 +++++++++++++++++ .../arrowadapter/filesystem/OmniFileSystem.h | 84 ++++++++++++ .../io/arrowadapter/filesystem/OmniHdfs.cc | 119 +++++++++++++++++ .../src/io/arrowadapter/filesystem/OmniHdfs.h | 41 ++++++ .../io/arrowadapter/filesystem/OmniLocalfs.cc | 67 ++++++++++ .../io/arrowadapter/filesystem/OmniLocalfs.h | 39 ++++++ .../arrowadapter/filesystem/UtilInternal.cc | 41 ++++++ .../io/arrowadapter/filesystem/UtilInternal.h | 43 +++++++ .../cpp/src/io/arrowadapter/util/SimpleUri.cc | 120 ++++++++++++++++++ .../cpp/src/io/arrowadapter/util/SimpleUri.h | 105 +++++++++++++++ .../src/jni/ParquetColumnarBatchJniReader.cpp | 32 ++++- .../cpp/src/tablescan/ParquetReader.cpp | 19 +-- .../cpp/src/tablescan/ParquetReader.h | 10 +- .../jni/ParquetColumnarBatchJniReader.java | 20 ++- .../OmniParquetColumnarBatchReader.java | 16 +-- .../ParquetColumnarBatchJniReaderTest.java | 5 +- 17 files changed, 844 insertions(+), 39 deletions(-) create mode 100644 omnioperator/omniop-spark-extension/cpp/src/io/arrowadapter/filesystem/OmniFileSystem.cc create mode 100644 omnioperator/omniop-spark-extension/cpp/src/io/arrowadapter/filesystem/OmniFileSystem.h create mode 100644 omnioperator/omniop-spark-extension/cpp/src/io/arrowadapter/filesystem/OmniHdfs.cc create mode 100644 omnioperator/omniop-spark-extension/cpp/src/io/arrowadapter/filesystem/OmniHdfs.h create mode 100644 omnioperator/omniop-spark-extension/cpp/src/io/arrowadapter/filesystem/OmniLocalfs.cc create mode 100644 omnioperator/omniop-spark-extension/cpp/src/io/arrowadapter/filesystem/OmniLocalfs.h create mode 100644 omnioperator/omniop-spark-extension/cpp/src/io/arrowadapter/filesystem/UtilInternal.cc create mode 100644 omnioperator/omniop-spark-extension/cpp/src/io/arrowadapter/filesystem/UtilInternal.h create mode 100644 omnioperator/omniop-spark-extension/cpp/src/io/arrowadapter/util/SimpleUri.cc create mode 100644 omnioperator/omniop-spark-extension/cpp/src/io/arrowadapter/util/SimpleUri.h diff --git a/omnioperator/omniop-spark-extension/cpp/src/CMakeLists.txt b/omnioperator/omniop-spark-extension/cpp/src/CMakeLists.txt index f8b4f2587..9b809e96b 100644 --- a/omnioperator/omniop-spark-extension/cpp/src/CMakeLists.txt +++ b/omnioperator/omniop-spark-extension/cpp/src/CMakeLists.txt @@ -19,7 +19,12 @@ set (SOURCE_FILES jni/jni_common.cpp jni/ParquetColumnarBatchJniReader.cpp tablescan/ParquetReader.cpp - ) + io/arrowadapter/filesystem/OmniFileSystem.cc + io/arrowadapter/filesystem/UtilInternal.cc + io/arrowadapter/filesystem/OmniHdfs.cc + io/arrowadapter/filesystem/OmniLocalfs.cc + io/arrowadapter/util/SimpleUri.cc +) #Find required protobuf package find_package(Protobuf REQUIRED) diff --git a/omnioperator/omniop-spark-extension/cpp/src/io/arrowadapter/filesystem/OmniFileSystem.cc b/omnioperator/omniop-spark-extension/cpp/src/io/arrowadapter/filesystem/OmniFileSystem.cc new file mode 100644 index 000000000..86f2bbded --- /dev/null +++ b/omnioperator/omniop-spark-extension/cpp/src/io/arrowadapter/filesystem/OmniFileSystem.cc @@ -0,0 +1,115 @@ +/** + * 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. + */ + +#include +#include + +#include "OmniFileSystem.h" +#include "arrow/filesystem/hdfs.h" +#include "io/arrowadapter/filesystem/OmniHdfs.h" +#include "io/arrowadapter/filesystem/OmniLocalfs.h" +#include "arrow/filesystem/localfs.h" +#include "arrow/filesystem/mockfs.h" +#include "arrow/filesystem/path_util.h" +#include "io/arrowadapter/filesystem/UtilInternal.h" +#include "arrow/io/slow.h" +#include "arrow/result.h" +#include "arrow/status.h" +#include "arrow/util/checked_cast.h" +#include "arrow/util/macros.h" +#include "arrow/util/parallel.h" + +namespace arrow { + +using internal::checked_pointer_cast; +using internal::TaskHints; +using internal::Uri; + +namespace fs { + +using internal::ConcatAbstractPath; +using internal::EnsureTrailingSlash; +using internal::GetAbstractPathParent; +using internal::kSep; +using internal::RemoveLeadingSlash; +using internal::RemoveTrailingSlash; +using internal::ToSlashes; + +namespace { + +Result> +FileSystemFromUriReal(const SimpleUri &uri, const io::IOContext &io_context, std::string *out_path) { + const auto scheme = uri.scheme(); + + if (scheme == "file") { + std::string path; + ARROW_ASSIGN_OR_RAISE(auto options, buildLocalfsOptionsFromUri(uri, out_path)); + if (out_path != nullptr) { + *out_path = path; + } + return std::make_shared(options, io_context); + } + + if (scheme == "hdfs" || scheme == "viewfs") { + ARROW_ASSIGN_OR_RAISE(auto options, buildHdfsOptionsFromUri(uri)); + if (out_path != nullptr) { + *out_path = uri.path(); + } + ARROW_ASSIGN_OR_RAISE(auto hdfs, HadoopFileSystem::Make(options, io_context)); + return hdfs; + } + + if (scheme == "mock") { + // MockFileSystem does not have an absolute / relative path distinction, + // normalize path by removing leading slash. + if (out_path != nullptr) { + *out_path = std::string(RemoveLeadingSlash(uri.path())); + } + return std::make_shared(internal::CurrentTimePoint(), + io_context); + } + + return FileSystemFromUri(uri.ToString(), io_context, out_path); +} + +} // namespace + + +Result> FileSystemFromUriOrPath(const SimpleUri &uri, + std::string *out_path) { + return FileSystemFromUriOrPath(uri, io::default_io_context(), out_path); +} + +Result> FileSystemFromUriOrPath( + const SimpleUri &uri, const io::IOContext &io_context, + std::string *out_path) { + const auto& uri_string = uri.ToString(); + if (internal::DetectAbsolutePath(uri_string)) { + // Normalize path separators + if (out_path != nullptr) { + *out_path = ToSlashes(uri_string); + } + return std::make_shared(); + } + return FileSystemFromUriReal(uri, io_context, out_path); +} + +} +// namespace fs +} +// namespace arrow \ No newline at end of file diff --git a/omnioperator/omniop-spark-extension/cpp/src/io/arrowadapter/filesystem/OmniFileSystem.h b/omnioperator/omniop-spark-extension/cpp/src/io/arrowadapter/filesystem/OmniFileSystem.h new file mode 100644 index 000000000..aa21c9f72 --- /dev/null +++ b/omnioperator/omniop-spark-extension/cpp/src/io/arrowadapter/filesystem/OmniFileSystem.h @@ -0,0 +1,84 @@ +/** + * 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. + */ + +#pragma once + +#include +#include +#include +#include +#include +#include +#include +#include + +#include "arrow/filesystem/type_fwd.h" +#include "arrow/io/interfaces.h" +#include "arrow/type_fwd.h" +#include "arrow/util/compare.h" +#include "arrow/util/macros.h" +#include "arrow/util/type_fwd.h" +#include "arrow/util/visibility.h" +#include "arrow/util/windows_fixup.h" +#include "io/arrowadapter/util/SimpleUri.h" + +namespace arrow { + +using internal::SimpleUri; + +namespace fs { + +/// \defgroup filesystem-factories Functions for creating FileSystem instances + +/// @{ + +/// \brief Create a new FileSystem by URI +/// +/// Same as FileSystemFromUriOrPath, but it use uri that constructed by client +ARROW_EXPORT +Result> FileSystemFromUriOrPath(const SimpleUri &uri, + std::string* out_path = NULLPTR); + + +/// \brief Create a new FileSystem by URI with a custom IO context +/// +/// Recognized schemes are "file", "mock", "hdfs", "viewfs", "s3", +/// "gs" and "gcs". +/// +/// \param[in] uri a URI-based path, ex: file:///some/local/path +/// \param[in] io_context an IOContext which will be associated with the filesystem +/// \param[out] out_path (optional) Path inside the filesystem. +/// \return out_fs FileSystem instance. + + +/// \brief Create a new FileSystem by URI with a custom IO context +/// +/// Same as FileSystemFromUri, but in addition also recognize non-URIs +/// and treat them as local filesystem paths. Only absolute local filesystem +/// paths are allowed. +ARROW_EXPORT +Result> FileSystemFromUriOrPath( + const SimpleUri &uri, const io::IOContext &io_context, + std::string *out_path = NULLPTR); + +/// @} + +} +// namespace fs +} +// namespace arrow \ No newline at end of file diff --git a/omnioperator/omniop-spark-extension/cpp/src/io/arrowadapter/filesystem/OmniHdfs.cc b/omnioperator/omniop-spark-extension/cpp/src/io/arrowadapter/filesystem/OmniHdfs.cc new file mode 100644 index 000000000..3b44985bf --- /dev/null +++ b/omnioperator/omniop-spark-extension/cpp/src/io/arrowadapter/filesystem/OmniHdfs.cc @@ -0,0 +1,119 @@ +/** + * 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. + */ + +#include +#include +#include + +#include "arrow/filesystem/hdfs.h" +#include "arrow/filesystem/path_util.h" +#include "arrow/util/value_parsing.h" +#include "io/arrowadapter/filesystem/OmniHdfs.h" + +namespace arrow { + +using internal::ParseValue; + +namespace fs { + +Result buildHdfsOptionsFromUri(const SimpleUri &uri){ + HdfsOptions options; + + std::unordered_map options_map; + ARROW_ASSIGN_OR_RAISE(const auto options_items, uri.query_items()); + for (const auto& kv : options_items) { + options_map.emplace(kv.first, kv.second); + } + + std::string host; + host = uri.scheme() + "://" + uri.host(); + + // configure endpoint + const auto port = uri.port(); + if (port == -1) { + // default port will be determined by hdfs FileSystem impl + options.ConfigureEndPoint(host, 0); + } else { + options.ConfigureEndPoint(host, port); + } + + // configure replication + auto it = options_map.find("replication"); + if (it != options_map.end()) { + const auto& v = it->second; + int16_t replication; + if (!ParseValue(v.data(), v.size(), &replication)) { + return Status::Invalid("Invalid value for option 'replication': '", v, "'"); + } + options.ConfigureReplication(replication); + options_map.erase(it); + } + + // configure buffer_size + it = options_map.find("buffer_size"); + if (it != options_map.end()) { + const auto& v = it->second; + int32_t buffer_size; + if (!ParseValue(v.data(), v.size(), &buffer_size)) { + return Status::Invalid("Invalid value for option 'buffer_size': '", v, "'"); + } + options.ConfigureBufferSize(buffer_size); + options_map.erase(it); + } + + // configure default_block_size + it = options_map.find("default_block_size"); + if (it != options_map.end()) { + const auto& v = it->second; + int64_t default_block_size; + if (!ParseValue(v.data(), v.size(), &default_block_size)) { + return Status::Invalid("Invalid value for option 'default_block_size': '", v, "'"); + } + options.ConfigureBlockSize(default_block_size); + options_map.erase(it); + } + + // configure user + it = options_map.find("user"); + if (it != options_map.end()) { + const auto& user = it->second; + options.ConfigureUser(user); + options_map.erase(it); + } + + // configure kerberos + it = options_map.find("kerb_ticket"); + if (it != options_map.end()) { + const auto& ticket = it->second; + options.ConfigureKerberosTicketCachePath(ticket); + options_map.erase(it); + } + + // configure other options + for (const auto& it : options_map) { + options.ConfigureExtraConf(it.first, it.second); + } + + return options; +} + + +} +// namespace fs +} +// namespace arrow \ No newline at end of file diff --git a/omnioperator/omniop-spark-extension/cpp/src/io/arrowadapter/filesystem/OmniHdfs.h b/omnioperator/omniop-spark-extension/cpp/src/io/arrowadapter/filesystem/OmniHdfs.h new file mode 100644 index 000000000..8bc5cc8b1 --- /dev/null +++ b/omnioperator/omniop-spark-extension/cpp/src/io/arrowadapter/filesystem/OmniHdfs.h @@ -0,0 +1,41 @@ +/** + * 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. + */ + +#pragma once + +#include +#include +#include + +#include "arrow/filesystem/filesystem.h" +#include "arrow/filesystem/hdfs.h" +#include "io/arrowadapter/util/SimpleUri.h" + +namespace arrow { + +using internal::SimpleUri; + +namespace fs { + +ARROW_EXPORT +Result buildHdfsOptionsFromUri(const SimpleUri &uri); + +} +// namespace fs +} +// namespace arrow \ No newline at end of file diff --git a/omnioperator/omniop-spark-extension/cpp/src/io/arrowadapter/filesystem/OmniLocalfs.cc b/omnioperator/omniop-spark-extension/cpp/src/io/arrowadapter/filesystem/OmniLocalfs.cc new file mode 100644 index 000000000..65be81cd2 --- /dev/null +++ b/omnioperator/omniop-spark-extension/cpp/src/io/arrowadapter/filesystem/OmniLocalfs.cc @@ -0,0 +1,67 @@ +/** + * 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. + */ + +#include +#include +#include + +#ifdef _WIN32 +#include "arrow/util/windows_compatibility.h" +#endif + +#include "arrow/filesystem/localfs.h" +#include "arrow/util/io_util.h" +#include "io/arrowadapter/filesystem/OmniLocalfs.h" + +namespace arrow { + namespace fs { + +using ::arrow::internal::IOErrorFromErrno; +#ifdef _WIN32 +using ::arrow::internal::IOErrorFromWinError; +#endif +using ::arrow::internal::NativePathString; +using ::arrow::internal::PlatformFilename; + +Result buildLocalfsOptionsFromUri(const ::arrow::internal::SimpleUri &uri, std::string* out_path){ + if (!uri.username().empty() || !uri.password().empty()) { + return Status::Invalid("Unsupported username or password in local URI: '", + uri.ToString(), "'"); + } + std::string path; + const auto host = uri.host(); + if (!host.empty()) { +#ifdef _WIN32 + std::stringstream ss; +ss << "//" << host << "/" << internal::RemoveLeadingSlash(uri.path()); +*out_path = ss.str(); +#else + return Status::Invalid("Unsupported hostname in non-Windows local URI: '", + uri.ToString(), "'"); +#endif + } else { + *out_path = uri.path(); + } + + return LocalFileSystemOptions(); +} + + } +// namespace fs +} +// namespace arrow \ No newline at end of file diff --git a/omnioperator/omniop-spark-extension/cpp/src/io/arrowadapter/filesystem/OmniLocalfs.h b/omnioperator/omniop-spark-extension/cpp/src/io/arrowadapter/filesystem/OmniLocalfs.h new file mode 100644 index 000000000..f359d565c --- /dev/null +++ b/omnioperator/omniop-spark-extension/cpp/src/io/arrowadapter/filesystem/OmniLocalfs.h @@ -0,0 +1,39 @@ +/** + * 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. + */ + +#pragma once + +#include +#include +#include + +#include "arrow/filesystem/filesystem.h" +#include "arrow/filesystem/localfs.h" +#include "io/arrowadapter/util/SimpleUri.h" + +namespace arrow { + +namespace fs { + +ARROW_EXPORT +Result buildLocalfsOptionsFromUri(const ::arrow::internal::SimpleUri &uri, std::string* out_path); + +} +// namespace fs +} +// namespace arrow \ No newline at end of file diff --git a/omnioperator/omniop-spark-extension/cpp/src/io/arrowadapter/filesystem/UtilInternal.cc b/omnioperator/omniop-spark-extension/cpp/src/io/arrowadapter/filesystem/UtilInternal.cc new file mode 100644 index 000000000..8f2f3dc0d --- /dev/null +++ b/omnioperator/omniop-spark-extension/cpp/src/io/arrowadapter/filesystem/UtilInternal.cc @@ -0,0 +1,41 @@ +/** + * 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. + */ + +#include "io/arrowadapter/filesystem/UtilInternal.h" + +#include "arrow/util/io_util.h" + +namespace arrow { + +using internal::StatusDetailFromErrno; + +namespace fs { +namespace internal { + +TimePoint CurrentTimePoint() { + auto now = std::chrono::system_clock::now(); + return TimePoint( + std::chrono::duration_cast(now.time_since_epoch())); +} + +} +// namespace internal +} +// namespace fs +} +// namespace arrow \ No newline at end of file diff --git a/omnioperator/omniop-spark-extension/cpp/src/io/arrowadapter/filesystem/UtilInternal.h b/omnioperator/omniop-spark-extension/cpp/src/io/arrowadapter/filesystem/UtilInternal.h new file mode 100644 index 000000000..1bcd92625 --- /dev/null +++ b/omnioperator/omniop-spark-extension/cpp/src/io/arrowadapter/filesystem/UtilInternal.h @@ -0,0 +1,43 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +#pragma once + +#include +#include +#include + +#include "arrow/filesystem/filesystem.h" +#include "arrow/io/interfaces.h" +#include "arrow/status.h" +#include "arrow/util/visibility.h" + +namespace arrow { +namespace fs { +namespace internal { + +ARROW_EXPORT + +TimePoint CurrentTimePoint(); + +} +// namespace internal +} +// namespace fs +} +// namespace arrow \ No newline at end of file diff --git a/omnioperator/omniop-spark-extension/cpp/src/io/arrowadapter/util/SimpleUri.cc b/omnioperator/omniop-spark-extension/cpp/src/io/arrowadapter/util/SimpleUri.cc new file mode 100644 index 000000000..da1562569 --- /dev/null +++ b/omnioperator/omniop-spark-extension/cpp/src/io/arrowadapter/util/SimpleUri.cc @@ -0,0 +1,120 @@ +/** + * 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. + */ + +#include +#include +#include + +#include "arrow/result.h" +#include "SimpleUri.h" + +namespace arrow { + namespace internal { + +SimpleUri::SimpleUri(std::string _uri, std::string _scheme, std::string _path, std::string _host, + int32_t _port) { + uri_string = std::move(_uri); + scheme_string = std::move(_scheme); + path_string = std::move(_path); + host_string = std::move(_host); + port_number = _port; +} + +SimpleUri::SimpleUri(std::string _uri, std::string _scheme, std::string _path, std::string _host, int32_t _port, + std::vector> _query_list, std::string _username, std::string _password) + : SimpleUri(std::move(_uri), + std::move(_scheme), + std::move(_path), + std::move(_host), + _port, + std::move(_username), + std::move(_password)) { + query_list = std::move(_query_list); +} + +SimpleUri::SimpleUri(std::string _uri, std::string _scheme, std::string _path, std::string _host, int32_t _port, + std::vector> _query_list) + : SimpleUri(std::move(_uri), + std::move(_scheme), + std::move(_path), + std::move(_host), + _port +) { + query_list = std::move(_query_list); +} + +SimpleUri::SimpleUri(std::string _uri, std::string _scheme, std::string _path, std::string _host, int32_t _port, + std::string _username, std::string _password) : SimpleUri(std::move(_uri), + std::move(_scheme), + std::move(_path), + std::move(_host), + _port) { + username_string = std::move(_username); + password_string = std::move(_password); +} + +SimpleUri::~SimpleUri() {} + +std::string SimpleUri::scheme() const { + return scheme_string; +} + +bool SimpleUri::is_file_scheme() const { + return scheme_string == "file"; +} + +bool SimpleUri::has_host() const { + return !host_string.empty(); +} + +std::string SimpleUri::host() const { + return host_string; +} + +std::string SimpleUri::port_text() const { + return std::to_string(port_number); +} + +int32_t SimpleUri::port() const { + return port_number; +} + +std::string SimpleUri::username() const { + return username_string; +} + +std::string SimpleUri::password() const { + return password_string; +} + +std::string SimpleUri::path() const { + return path_string; +} + +Result>> SimpleUri::query_items() const { + return query_list; +} + +const std::string &SimpleUri::ToString() const { + return uri_string; +} + + } + // namespace internal +} +// namespace arrow \ No newline at end of file diff --git a/omnioperator/omniop-spark-extension/cpp/src/io/arrowadapter/util/SimpleUri.h b/omnioperator/omniop-spark-extension/cpp/src/io/arrowadapter/util/SimpleUri.h new file mode 100644 index 000000000..1dc1776f1 --- /dev/null +++ b/omnioperator/omniop-spark-extension/cpp/src/io/arrowadapter/util/SimpleUri.h @@ -0,0 +1,105 @@ +/** + * 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. + */ + +#pragma once + +#include +#include +#include +#include +#include +#include + +#include "arrow/result.h" +#include "arrow/type_fwd.h" +#include "arrow/util/visibility.h" + +namespace arrow { + namespace internal { + +/// \brief A parsed URI +class ARROW_EXPORT SimpleUri { +public: + SimpleUri(std::string _uri, std::string _scheme, std::string _path, std::string _host, int32_t _port); + + SimpleUri(std::string _uri, std::string _scheme, std::string _path, std::string _host, int32_t _port, + std::string _username, std::string _password); + + SimpleUri(std::string _uri, std::string _scheme, std::string _path, std::string _host, int32_t _port, + std::vector>, std::string _username, std::string _password); + + SimpleUri(std::string _uri, std::string _scheme, std::string _path, std::string _host, int32_t _port, + std::vector>); + + ~SimpleUri(); + + std::string scheme() const; + +/// Convenience function that returns true if the scheme() is "file" + virtual bool is_file_scheme() const; + +/// Whether the URI has an explicit host name. This may return true if +/// the URI has an empty host (e.g. "file:///tmp/foo"), while it returns +/// false is the URI has not host component at all (e.g. "file:/tmp/foo"). + virtual bool has_host() const; + +/// The URI host name, such as "localhost", "127.0.0.1" or "::1", or the empty +/// string is the URI does not have a host component. + virtual std::string host() const; + +/// The URI port number, as a string such as "80", or the empty string is the URI +/// does not have a port number component. + virtual std::string port_text() const; + +/// The URI port parsed as an integer, or -1 if the URI does not have a port +/// number component. + virtual int32_t port() const; + +/// The username specified in the URI. + virtual std::string username() const; + +/// The password specified in the URI. + virtual std::string password() const; + +/// The URI path component. + virtual std::string path() const; + +/// The URI query items +/// +/// Note this API doesn't allow differentiating between an empty value +/// and a missing value, such in "a&b=1" vs. "a=&b=1". + virtual Result>> query_items() const; + +/// Get the string representation of this URI. + virtual const std::string &ToString() const; + +private: + std::string host_string; + std::string scheme_string; + int32_t port_number; + std::vector> query_list; + std::string path_string; + std::string uri_string; + std::string username_string; + std::string password_string; +}; + + } +// namespace internal +} +// namespace arrow \ No newline at end of file diff --git a/omnioperator/omniop-spark-extension/cpp/src/jni/ParquetColumnarBatchJniReader.cpp b/omnioperator/omniop-spark-extension/cpp/src/jni/ParquetColumnarBatchJniReader.cpp index fda647658..6f8c84e96 100644 --- a/omnioperator/omniop-spark-extension/cpp/src/jni/ParquetColumnarBatchJniReader.cpp +++ b/omnioperator/omniop-spark-extension/cpp/src/jni/ParquetColumnarBatchJniReader.cpp @@ -20,6 +20,7 @@ #include "ParquetColumnarBatchJniReader.h" #include "jni_common.h" #include "tablescan/ParquetReader.h" +#include "io/arrowadapter/util/SimpleUri.h" using namespace omniruntime::vec; using namespace omniruntime::type; @@ -45,17 +46,36 @@ JNIEXPORT jlong JNICALL Java_com_huawei_boostkit_spark_jni_ParquetColumnarBatchJ jobject jObj, jobject jsonObj) { JNI_FUNC_START - // Get filePath - jstring path = (jstring)env->CallObjectMethod(jsonObj, jsonMethodString, env->NewStringUTF("filePath")); - const char *filePath = env->GetStringUTFChars(path, JNI_FALSE); - std::string file(filePath); - env->ReleaseStringUTFChars(path, filePath); + // Get uriStr + jstring uri = (jstring)env->CallObjectMethod(jsonObj, jsonMethodString, env->NewStringUTF("uri")); + const char *uriStr = env->GetStringUTFChars(uri, JNI_FALSE); + std::string uriString(uriStr); + env->ReleaseStringUTFChars(uri, uriStr); jstring ugiTemp = (jstring)env->CallObjectMethod(jsonObj, jsonMethodString, env->NewStringUTF("ugi")); const char *ugi = env->GetStringUTFChars(ugiTemp, JNI_FALSE); std::string ugiString(ugi); env->ReleaseStringUTFChars(ugiTemp, ugi); + jstring schemeTmp = (jstring)env->CallObjectMethod(jsonObj, jsonMethodString, env->NewStringUTF("scheme")); + const char *scheme = env->GetStringUTFChars(schemeTmp, JNI_FALSE); + std::string schemeString(scheme); + env->ReleaseStringUTFChars(schemeTmp, scheme); + + jstring hostTmp = (jstring)env->CallObjectMethod(jsonObj, jsonMethodString, env->NewStringUTF("host")); + const char *host = env->GetStringUTFChars(hostTmp, JNI_FALSE); + std::string hostString(host); + env->ReleaseStringUTFChars(hostTmp, host); + + jstring pathTmp = (jstring)env->CallObjectMethod(jsonObj, jsonMethodString, env->NewStringUTF("path")); + const char *path = env->GetStringUTFChars(pathTmp, JNI_FALSE); + std::string pathString(ugi); + env->ReleaseStringUTFChars(pathTmp, path); + + jint port = (jint)env->CallIntMethod(jsonObj, jsonMethodInt, env->NewStringUTF("port")); + + arrow::internal::SimpleUri simpleUri(uriString, schemeString, pathString, hostString, port); + // Get capacity for each record batch int64_t capacity = (int64_t)env->CallLongMethod(jsonObj, jsonMethodLong, env->NewStringUTF("capacity")); @@ -64,7 +84,7 @@ JNIEXPORT jlong JNICALL Java_com_huawei_boostkit_spark_jni_ParquetColumnarBatchJ auto column_indices = GetIndices(env, jsonObj, "columnIndices"); ParquetReader *pReader = new ParquetReader(); - auto state = pReader->InitRecordReader(file, capacity, row_group_indices, column_indices, ugiString); + auto state = pReader->InitRecordReader(simpleUri, capacity, row_group_indices, column_indices, ugiString); if (state != Status::OK()) { env->ThrowNew(runtimeExceptionClass, state.ToString().c_str()); return 0; diff --git a/omnioperator/omniop-spark-extension/cpp/src/tablescan/ParquetReader.cpp b/omnioperator/omniop-spark-extension/cpp/src/tablescan/ParquetReader.cpp index 4f917e22c..9f185efb5 100644 --- a/omnioperator/omniop-spark-extension/cpp/src/tablescan/ParquetReader.cpp +++ b/omnioperator/omniop-spark-extension/cpp/src/tablescan/ParquetReader.cpp @@ -19,8 +19,9 @@ #include #include -#include +#include "io/arrowadapter/filesystem/OmniFileSystem.h" #include +#include "io/arrowadapter/util/SimpleUri.h" #include "jni/jni_common.h" #include "ParquetReader.h" @@ -30,6 +31,7 @@ using namespace arrow; using namespace parquet::arrow; using namespace arrow::compute; using namespace spark::reader; +using namespace arrow::internal; static std::mutex mutex_; static std::map restore_filesysptr; @@ -69,15 +71,15 @@ std::string spark::reader::GetFileSystemKey(std::string& path, std::string& ugi) return result; } -Filesystem* spark::reader::GetFileSystemPtr(std::string& path, std::string& ugi, arrow::Status &status) +Filesystem* spark::reader::GetFileSystemPtr(SimpleUri &uri, std::string& ugi, arrow::Status &status) { - auto key = GetFileSystemKey(path, ugi); + auto key = GetFileSystemKey(const_cast(uri.ToString()), ugi); // if not find key, create the filesystem ptr auto iter = restore_filesysptr.find(key); if (iter == restore_filesysptr.end()) { Filesystem* fs = new Filesystem(); - auto result = fs::FileSystemFromUriOrPath(path); + auto result = fs::FileSystemFromUriOrPath(uri); status = result.status(); if (!status.ok()) { return nullptr; @@ -89,8 +91,9 @@ Filesystem* spark::reader::GetFileSystemPtr(std::string& path, std::string& ugi, return restore_filesysptr[key]; } -Status ParquetReader::InitRecordReader(std::string& filePath, int64_t capacity, - const std::vector& row_group_indices, const std::vector& column_indices, std::string& ugi) +Status ParquetReader::InitRecordReader(SimpleUri &uri, int64_t capacity, + const std::vector& row_group_indices, const std::vector& column_indices, + std::string& ugi) { arrow::MemoryPool* pool = default_memory_pool(); @@ -104,12 +107,12 @@ Status ParquetReader::InitRecordReader(std::string& filePath, int64_t capacity, // Get the file from filesystem Status result; mutex_.lock(); - Filesystem* fs = GetFileSystemPtr(filePath, ugi, result); + Filesystem* fs = GetFileSystemPtr(uri, ugi, result); mutex_.unlock(); if (fs == nullptr || fs->filesys_ptr == nullptr) { return Status::IOError(result); } - ARROW_ASSIGN_OR_RAISE(auto file, fs->filesys_ptr->OpenInputFile(filePath)); + ARROW_ASSIGN_OR_RAISE(auto file, fs->filesys_ptr->OpenInputFile(uri.ToString())); FileReaderBuilder reader_builder; ARROW_RETURN_NOT_OK(reader_builder.Open(file, reader_properties)); diff --git a/omnioperator/omniop-spark-extension/cpp/src/tablescan/ParquetReader.h b/omnioperator/omniop-spark-extension/cpp/src/tablescan/ParquetReader.h index 8fef9d495..3143f47db 100644 --- a/omnioperator/omniop-spark-extension/cpp/src/tablescan/ParquetReader.h +++ b/omnioperator/omniop-spark-extension/cpp/src/tablescan/ParquetReader.h @@ -34,14 +34,18 @@ #include #include #include +#include "io/arrowadapter/util/SimpleUri.h" + +using namespace arrow::internal; namespace spark::reader { class ParquetReader { public: ParquetReader() {} - arrow::Status InitRecordReader(std::string& path, int64_t capacity, - const std::vector& row_group_indices, const std::vector& column_indices, std::string& ugi); + arrow::Status InitRecordReader(SimpleUri &uri, int64_t capacity, + const std::vector& row_group_indices, + const std::vector& column_indices, std::string& ugi); arrow::Status ReadNextBatch(std::shared_ptr *batch); @@ -62,7 +66,7 @@ namespace spark::reader { std::string GetFileSystemKey(std::string& path, std::string& ugi); - Filesystem* GetFileSystemPtr(std::string& path, std::string& ugi, arrow::Status &status); + Filesystem* GetFileSystemPtr(SimpleUri &uri, std::string& ugi, arrow::Status &status); int CopyToOmniVec(std::shared_ptr vcType, int &omniTypeId, uint64_t &omniVecId, std::shared_ptr array); diff --git a/omnioperator/omniop-spark-extension/java/src/main/java/com/huawei/boostkit/spark/jni/ParquetColumnarBatchJniReader.java b/omnioperator/omniop-spark-extension/java/src/main/java/com/huawei/boostkit/spark/jni/ParquetColumnarBatchJniReader.java index 3a5cffb09..5f4b35a50 100644 --- a/omnioperator/omniop-spark-extension/java/src/main/java/com/huawei/boostkit/spark/jni/ParquetColumnarBatchJniReader.java +++ b/omnioperator/omniop-spark-extension/java/src/main/java/com/huawei/boostkit/spark/jni/ParquetColumnarBatchJniReader.java @@ -20,13 +20,13 @@ package com.huawei.boostkit.spark.jni; import nova.hetu.omniruntime.type.DataType; import nova.hetu.omniruntime.vector.*; - -import org.apache.spark.sql.catalyst.util.RebaseDateTime; - +import org.apache.hadoop.fs.Path; import org.json.JSONObject; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import java.io.UnsupportedEncodingException; +import java.net.URI; import java.util.List; public class ParquetColumnarBatchJniReader { @@ -38,14 +38,22 @@ public class ParquetColumnarBatchJniReader { NativeLoader.getInstance(); } - public long initializeReaderJava(String path, int capacity, - List rowgroupIndices, List columnIndices, String ugi) { + public long initializeReaderJava(Path path, int capacity, + List rowgroupIndices, List columnIndices, String ugi) throws UnsupportedEncodingException { JSONObject job = new JSONObject(); + URI uri = path.toUri(); + + job.put("uri", path.toString()); job.put("filePath", path); job.put("capacity", capacity); job.put("rowGroupIndices", rowgroupIndices.stream().mapToInt(Integer::intValue).toArray()); job.put("columnIndices", columnIndices.stream().mapToInt(Integer::intValue).toArray()); job.put("ugi", ugi); + + job.put("scheme", uri.getScheme() == null ? "" : uri.getScheme()); + job.put("host", uri.getHost() == null ? "" : uri.getHost()); + job.put("port", uri.getPort()); + job.put("path", uri.getPath() == null ? "" : uri.getPath()); parquetReader = initializeReader(job); return parquetReader; } @@ -101,7 +109,7 @@ public class ParquetColumnarBatchJniReader { } nativeGetId++; } - return (int)rtn; + return (int) rtn; } public void close() { diff --git a/omnioperator/omniop-spark-extension/java/src/main/java/org/apache/spark/sql/execution/datasources/parquet/OmniParquetColumnarBatchReader.java b/omnioperator/omniop-spark-extension/java/src/main/java/org/apache/spark/sql/execution/datasources/parquet/OmniParquetColumnarBatchReader.java index 3aa70dfee..819ef2f7c 100644 --- a/omnioperator/omniop-spark-extension/java/src/main/java/org/apache/spark/sql/execution/datasources/parquet/OmniParquetColumnarBatchReader.java +++ b/omnioperator/omniop-spark-extension/java/src/main/java/org/apache/spark/sql/execution/datasources/parquet/OmniParquetColumnarBatchReader.java @@ -18,26 +18,16 @@ package org.apache.spark.sql.execution.datasources.parquet; -import static org.apache.parquet.filter2.compat.RowGroupFilter.filterRowGroups; -import static org.apache.parquet.format.converter.ParquetMetadataConverter.range; -import static org.apache.parquet.hadoop.ParquetFileReader.readFooter; -import static org.apache.parquet.hadoop.ParquetInputFormat.getFilter; - import com.huawei.boostkit.spark.jni.ParquetColumnarBatchJniReader; import nova.hetu.omniruntime.vector.Vec; import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.fs.Path; import org.apache.hadoop.mapreduce.InputSplit; import org.apache.hadoop.mapreduce.RecordReader; import org.apache.hadoop.mapreduce.TaskAttemptContext; import org.apache.hadoop.security.UserGroupInformation; import org.apache.parquet.column.ColumnDescriptor; import org.apache.parquet.filter2.compat.FilterCompat; -import org.apache.parquet.filter2.compat.RowGroupFilter; -import org.apache.parquet.filter2.predicate.FilterPredicate; import org.apache.parquet.filter2.statisticslevel.StatisticsFilter; -import org.apache.parquet.format.ColumnChunk; -import org.apache.parquet.format.RowGroup; import org.apache.parquet.hadoop.BadConfigurationException; import org.apache.parquet.hadoop.ParquetInputFormat; import org.apache.parquet.hadoop.ParquetInputSplit; @@ -53,17 +43,17 @@ import org.apache.parquet.schema.Type; import org.apache.spark.sql.catalyst.InternalRow; import org.apache.spark.sql.execution.vectorized.ColumnVectorUtils; import org.apache.spark.sql.execution.vectorized.OmniColumnVector; -import org.apache.spark.sql.execution.vectorized.OnHeapColumnVector; import org.apache.spark.sql.types.StructField; import org.apache.spark.sql.types.StructType; import org.apache.spark.sql.types.StructType$; import org.apache.spark.sql.vectorized.ColumnarBatch; -import org.json.JSONObject; import java.io.IOException; import java.lang.reflect.InvocationTargetException; import java.util.*; +import static org.apache.parquet.hadoop.ParquetInputFormat.getFilter; + /** * To support parquet file format in native, OmniParquetColumnarBatchReader uses ParquetColumnarBatchJniReader to * read data and return batch to next operator. @@ -155,7 +145,7 @@ public class OmniParquetColumnarBatchReader extends RecordReader rowgroupIndices = getFilteredBlocks(split.getStart(), split.getEnd()); List columnIndices = getColumnIndices(requestedSchema.getColumns(), fileSchema.getColumns()); String ugi = UserGroupInformation.getCurrentUser().toString(); - reader.initializeReaderJava(split.getPath().toString(), capacity, rowgroupIndices, columnIndices, ugi); + reader.initializeReaderJava(split.getPath(), capacity, rowgroupIndices, columnIndices, ugi); // Add missing Cols flags. initializeInternal(); } diff --git a/omnioperator/omniop-spark-extension/java/src/test/java/com/huawei/boostkit/spark/jni/ParquetColumnarBatchJniReaderTest.java b/omnioperator/omniop-spark-extension/java/src/test/java/com/huawei/boostkit/spark/jni/ParquetColumnarBatchJniReaderTest.java index 599641355..eca898981 100644 --- a/omnioperator/omniop-spark-extension/java/src/test/java/com/huawei/boostkit/spark/jni/ParquetColumnarBatchJniReaderTest.java +++ b/omnioperator/omniop-spark-extension/java/src/test/java/com/huawei/boostkit/spark/jni/ParquetColumnarBatchJniReaderTest.java @@ -20,6 +20,7 @@ package com.huawei.boostkit.spark.jni; import junit.framework.TestCase; import nova.hetu.omniruntime.vector.*; +import org.apache.hadoop.fs.Path; import org.junit.After; import org.junit.Before; import org.junit.FixMethodOrder; @@ -46,8 +47,8 @@ public class ParquetColumnarBatchJniReaderTest extends TestCase { List columnIndices = new ArrayList<>(); Collections.addAll(columnIndices, 0, 1, 3, 6, 7, 8, 9, 10, 12); File file = new File("../cpp/test/tablescan/resources/parquet_data_all_type"); - String path = file.getAbsolutePath(); - parquetColumnarBatchJniReader.initializeReaderJava(path, 100000, rowGroupIndices, columnIndices, "root@sample"); + parquetColumnarBatchJniReader.initializeReaderJava(new Path(file.getAbsolutePath()), 100000, + rowGroupIndices, columnIndices, "root@sample"); vecs = new Vec[9]; } -- Gitee From ce0b091b3053fbe62449419a236c9692dcfe8e1d Mon Sep 17 00:00:00 2001 From: d00807371 Date: Thu, 21 Dec 2023 17:30:08 +0800 Subject: [PATCH 06/17] revise the unit test --- .../cpp/test/tablescan/parquet_scan_test.cpp | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/omnioperator/omniop-spark-extension/cpp/test/tablescan/parquet_scan_test.cpp b/omnioperator/omniop-spark-extension/cpp/test/tablescan/parquet_scan_test.cpp index a7da7f0ff..fbe65e862 100644 --- a/omnioperator/omniop-spark-extension/cpp/test/tablescan/parquet_scan_test.cpp +++ b/omnioperator/omniop-spark-extension/cpp/test/tablescan/parquet_scan_test.cpp @@ -44,7 +44,8 @@ TEST(read, test_parquet_reader) ParquetReader *reader = new ParquetReader(); std::string ugi = "root@sample"; - auto state1 = reader->InitRecordReader(filename, 1024, row_group_indices, column_indices, ugi); + SimpleUri simpleUri(filename, "", "", "", -1); + auto state1 = reader->InitRecordReader(simpleUri, 1024, row_group_indices, column_indices, ugi); ASSERT_EQ(state1, Status::OK()); std::shared_ptr batch; -- Gitee From 65c70a797a957d023cb82c46c5187c2f459c6368 Mon Sep 17 00:00:00 2001 From: d00807371 Date: Tue, 26 Dec 2023 10:16:12 +0800 Subject: [PATCH 07/17] =?UTF-8?q?=E4=BF=AE=E6=94=B9=E6=A3=80=E8=A7=86?= =?UTF-8?q?=E6=84=8F=E8=A7=81?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- .../cpp/src/CMakeLists.txt | 10 +- .../cpp/src/io/UriInfo.cc | 76 +++++++++++ .../util/SimpleUri.h => UriInfo.h} | 88 +++++-------- .../{filesystem => }/OmniFileSystem.cc | 47 ++++--- .../{filesystem => }/OmniFileSystem.h | 12 +- .../arrowadapter/{filesystem => }/OmniHdfs.cc | 79 ++---------- .../arrowadapter/{filesystem => }/OmniHdfs.h | 10 +- .../{filesystem => }/OmniLocalfs.cc | 34 ++--- .../{filesystem => }/OmniLocalfs.h | 11 +- .../{filesystem => }/UtilInternal.cc | 9 +- .../{filesystem => }/UtilInternal.h | 7 +- .../cpp/src/io/arrowadapter/util/SimpleUri.cc | 120 ------------------ .../cpp/src/io/orcfile/OmniOrcFile.cc | 9 +- .../cpp/src/io/orcfile/OmniOrcFile.hh | 9 +- .../cpp/src/io/orcfile/OmniOrcHdfsFile.cc | 45 +++---- .../cpp/src/jni/OrcColumnarBatchJniReader.cpp | 10 +- .../src/jni/ParquetColumnarBatchJniReader.cpp | 7 +- .../cpp/src/tablescan/ParquetReader.cpp | 11 +- .../cpp/src/tablescan/ParquetReader.h | 7 +- .../cpp/test/tablescan/parquet_scan_test.cpp | 5 +- .../spark/jni/OrcColumnarBatchJniReader.java | 2 +- 21 files changed, 235 insertions(+), 373 deletions(-) create mode 100644 omnioperator/omniop-spark-extension/cpp/src/io/UriInfo.cc rename omnioperator/omniop-spark-extension/cpp/src/io/{arrowadapter/util/SimpleUri.h => UriInfo.h} (31%) rename omnioperator/omniop-spark-extension/cpp/src/io/arrowadapter/{filesystem => }/OmniFileSystem.cc (71%) rename omnioperator/omniop-spark-extension/cpp/src/io/arrowadapter/{filesystem => }/OmniFileSystem.h (92%) rename omnioperator/omniop-spark-extension/cpp/src/io/arrowadapter/{filesystem => }/OmniHdfs.cc (35%) rename omnioperator/omniop-spark-extension/cpp/src/io/arrowadapter/{filesystem => }/OmniHdfs.h (83%) rename omnioperator/omniop-spark-extension/cpp/src/io/arrowadapter/{filesystem => }/OmniLocalfs.cc (67%) rename omnioperator/omniop-spark-extension/cpp/src/io/arrowadapter/{filesystem => }/OmniLocalfs.h (84%) rename omnioperator/omniop-spark-extension/cpp/src/io/arrowadapter/{filesystem => }/UtilInternal.cc (88%) rename omnioperator/omniop-spark-extension/cpp/src/io/arrowadapter/{filesystem => }/UtilInternal.h (95%) delete mode 100644 omnioperator/omniop-spark-extension/cpp/src/io/arrowadapter/util/SimpleUri.cc diff --git a/omnioperator/omniop-spark-extension/cpp/src/CMakeLists.txt b/omnioperator/omniop-spark-extension/cpp/src/CMakeLists.txt index 9b809e96b..bf96ffb38 100644 --- a/omnioperator/omniop-spark-extension/cpp/src/CMakeLists.txt +++ b/omnioperator/omniop-spark-extension/cpp/src/CMakeLists.txt @@ -19,11 +19,11 @@ set (SOURCE_FILES jni/jni_common.cpp jni/ParquetColumnarBatchJniReader.cpp tablescan/ParquetReader.cpp - io/arrowadapter/filesystem/OmniFileSystem.cc - io/arrowadapter/filesystem/UtilInternal.cc - io/arrowadapter/filesystem/OmniHdfs.cc - io/arrowadapter/filesystem/OmniLocalfs.cc - io/arrowadapter/util/SimpleUri.cc + io/arrowadapter/OmniFileSystem.cc + io/arrowadapter/UtilInternal.cc + io/arrowadapter/OmniHdfs.cc + io/arrowadapter/OmniLocalfs.cc + io/UriInfo.cc ) #Find required protobuf package diff --git a/omnioperator/omniop-spark-extension/cpp/src/io/UriInfo.cc b/omnioperator/omniop-spark-extension/cpp/src/io/UriInfo.cc new file mode 100644 index 000000000..5eb3cdc75 --- /dev/null +++ b/omnioperator/omniop-spark-extension/cpp/src/io/UriInfo.cc @@ -0,0 +1,76 @@ +/** + * 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. + */ + +#include +#include + +#include "UriInfo.h" + +namespace uri { + namespace internal { + + UriInfo::UriInfo(std::string _uri, std::string _scheme, std::string _path, std::string _host, + int32_t _port): host_string(std::move(_host)), + scheme_string(std::move(_scheme)), + port_number(_port), + port_string(std::to_string(port_number)), + path_string(std::move(_path)), + uri_string(std::move(_uri)) + { + } + + UriInfo::UriInfo(std::string _scheme, std::string _path, std::string _host, + int32_t _port): host_string(std::move(_host)), + scheme_string(std::move(_scheme)), + port_number(_port), + port_string(std::to_string(port_number)), + path_string(std::move(_path)), + uri_string("Not initialize origin uri!") + { + } + + UriInfo::~UriInfo() {} + + const std::string& UriInfo::scheme() const { + return scheme_string; + } + + const std::string& UriInfo::host() const { + return host_string; + } + + const std::string& UriInfo::port_text() const { + return port_string; + } + + int32_t UriInfo::port() const { + return port_number; + } + + const std::string& UriInfo::path() const { + return path_string; + } + + const std::string &UriInfo::ToString() const { + return uri_string; + } + + } + // namespace internal +} +// namespace arrow_adapter \ No newline at end of file diff --git a/omnioperator/omniop-spark-extension/cpp/src/io/arrowadapter/util/SimpleUri.h b/omnioperator/omniop-spark-extension/cpp/src/io/UriInfo.h similarity index 31% rename from omnioperator/omniop-spark-extension/cpp/src/io/arrowadapter/util/SimpleUri.h rename to omnioperator/omniop-spark-extension/cpp/src/io/UriInfo.h index 1dc1776f1..6977e9267 100644 --- a/omnioperator/omniop-spark-extension/cpp/src/io/arrowadapter/util/SimpleUri.h +++ b/omnioperator/omniop-spark-extension/cpp/src/io/UriInfo.h @@ -29,77 +29,49 @@ #include "arrow/type_fwd.h" #include "arrow/util/visibility.h" -namespace arrow { +namespace uri{ + namespace internal { /// \brief A parsed URI -class ARROW_EXPORT SimpleUri { -public: - SimpleUri(std::string _uri, std::string _scheme, std::string _path, std::string _host, int32_t _port); - - SimpleUri(std::string _uri, std::string _scheme, std::string _path, std::string _host, int32_t _port, - std::string _username, std::string _password); - - SimpleUri(std::string _uri, std::string _scheme, std::string _path, std::string _host, int32_t _port, - std::vector>, std::string _username, std::string _password); - - SimpleUri(std::string _uri, std::string _scheme, std::string _path, std::string _host, int32_t _port, - std::vector>); - - ~SimpleUri(); - - std::string scheme() const; - -/// Convenience function that returns true if the scheme() is "file" - virtual bool is_file_scheme() const; - -/// Whether the URI has an explicit host name. This may return true if -/// the URI has an empty host (e.g. "file:///tmp/foo"), while it returns -/// false is the URI has not host component at all (e.g. "file:/tmp/foo"). - virtual bool has_host() const; + class ARROW_EXPORT UriInfo { + public: + UriInfo(std::string _uri, std::string _scheme, std::string _path, std::string _host, int32_t _port); -/// The URI host name, such as "localhost", "127.0.0.1" or "::1", or the empty -/// string is the URI does not have a host component. - virtual std::string host() const; + UriInfo(std::string _scheme, std::string _path, std::string _host, int32_t _port); -/// The URI port number, as a string such as "80", or the empty string is the URI -/// does not have a port number component. - virtual std::string port_text() const; + ~UriInfo(); -/// The URI port parsed as an integer, or -1 if the URI does not have a port -/// number component. - virtual int32_t port() const; + const std::string& scheme() const ; -/// The username specified in the URI. - virtual std::string username() const; + /// The URI host name, such as "localhost", "127.0.0.1" or "::1", or the empty + /// string is the URI does not have a host component. + const std::string& host() const ; -/// The password specified in the URI. - virtual std::string password() const; + /// The URI port parsed as an integer, or -1 if the URI does not have a port + /// number component. + int32_t port() const ; -/// The URI path component. - virtual std::string path() const; + /// The URI path component. + const std::string& path() const ; -/// The URI query items -/// -/// Note this API doesn't allow differentiating between an empty value -/// and a missing value, such in "a&b=1" vs. "a=&b=1". - virtual Result>> query_items() const; + /// The URI port number, as a string such as "80", or the empty string is the URI + /// does not have a port number component. + const std::string& port_text() const ; -/// Get the string representation of this URI. - virtual const std::string &ToString() const; + /// Get the string representation of this URI. + const std::string &ToString() const ; -private: - std::string host_string; - std::string scheme_string; - int32_t port_number; - std::vector> query_list; - std::string path_string; - std::string uri_string; - std::string username_string; - std::string password_string; -}; + private: + std::string host_string; + std::string scheme_string; + int32_t port_number; + std::string port_string; + std::string path_string; + std::string uri_string; + }; } // namespace internal } -// namespace arrow \ No newline at end of file +// namespace arrow_adapter \ No newline at end of file diff --git a/omnioperator/omniop-spark-extension/cpp/src/io/arrowadapter/filesystem/OmniFileSystem.cc b/omnioperator/omniop-spark-extension/cpp/src/io/arrowadapter/OmniFileSystem.cc similarity index 71% rename from omnioperator/omniop-spark-extension/cpp/src/io/arrowadapter/filesystem/OmniFileSystem.cc rename to omnioperator/omniop-spark-extension/cpp/src/io/arrowadapter/OmniFileSystem.cc index 86f2bbded..ea56dfa52 100644 --- a/omnioperator/omniop-spark-extension/cpp/src/io/arrowadapter/filesystem/OmniFileSystem.cc +++ b/omnioperator/omniop-spark-extension/cpp/src/io/arrowadapter/OmniFileSystem.cc @@ -21,12 +21,12 @@ #include "OmniFileSystem.h" #include "arrow/filesystem/hdfs.h" -#include "io/arrowadapter/filesystem/OmniHdfs.h" -#include "io/arrowadapter/filesystem/OmniLocalfs.h" +#include "OmniHdfs.h" +#include "OmniLocalfs.h" #include "arrow/filesystem/localfs.h" #include "arrow/filesystem/mockfs.h" #include "arrow/filesystem/path_util.h" -#include "io/arrowadapter/filesystem/UtilInternal.h" +#include "UtilInternal.h" #include "arrow/io/slow.h" #include "arrow/result.h" #include "arrow/status.h" @@ -34,26 +34,31 @@ #include "arrow/util/macros.h" #include "arrow/util/parallel.h" -namespace arrow { +namespace arrow_adapter { -using internal::checked_pointer_cast; -using internal::TaskHints; -using internal::Uri; +using arrow::internal::checked_pointer_cast; +using arrow::internal::TaskHints; +using arrow::internal::Uri; namespace fs { -using internal::ConcatAbstractPath; -using internal::EnsureTrailingSlash; -using internal::GetAbstractPathParent; -using internal::kSep; -using internal::RemoveLeadingSlash; -using internal::RemoveTrailingSlash; -using internal::ToSlashes; +using arrow::fs::internal::ConcatAbstractPath; +using arrow::fs::internal::EnsureTrailingSlash; +using arrow::fs::internal::GetAbstractPathParent; +using arrow::fs::internal::RemoveLeadingSlash; +using arrow::fs::internal::RemoveTrailingSlash; +using arrow::fs::internal::ToSlashes; +using arrow::fs::FileSystem; +using arrow::fs::HadoopFileSystem; +using arrow::fs::LocalFileSystem; +using arrow::fs::internal::MockFileSystem; +using uri::internal::UriInfo; +using arrow::Result; namespace { Result> -FileSystemFromUriReal(const SimpleUri &uri, const io::IOContext &io_context, std::string *out_path) { +FileSystemFromUriReal(const UriInfo &uri, const arrow::io::IOContext &io_context, std::string *out_path) { const auto scheme = uri.scheme(); if (scheme == "file") { @@ -80,26 +85,26 @@ FileSystemFromUriReal(const SimpleUri &uri, const io::IOContext &io_context, std if (out_path != nullptr) { *out_path = std::string(RemoveLeadingSlash(uri.path())); } - return std::make_shared(internal::CurrentTimePoint(), + return std::make_shared(internal::CurrentTimePoint(), io_context); } - return FileSystemFromUri(uri.ToString(), io_context, out_path); + return arrow::fs::FileSystemFromUri(uri.ToString(), io_context, out_path); } } // namespace -Result> FileSystemFromUriOrPath(const SimpleUri &uri, +Result> FileSystemFromUriOrPath(const UriInfo &uri, std::string *out_path) { - return FileSystemFromUriOrPath(uri, io::default_io_context(), out_path); + return FileSystemFromUriOrPath(uri, arrow::io::IOContext(), out_path); } Result> FileSystemFromUriOrPath( - const SimpleUri &uri, const io::IOContext &io_context, + const UriInfo &uri, const arrow::io::IOContext &io_context, std::string *out_path) { const auto& uri_string = uri.ToString(); - if (internal::DetectAbsolutePath(uri_string)) { + if (arrow::fs::internal::DetectAbsolutePath(uri_string)) { // Normalize path separators if (out_path != nullptr) { *out_path = ToSlashes(uri_string); diff --git a/omnioperator/omniop-spark-extension/cpp/src/io/arrowadapter/filesystem/OmniFileSystem.h b/omnioperator/omniop-spark-extension/cpp/src/io/arrowadapter/OmniFileSystem.h similarity index 92% rename from omnioperator/omniop-spark-extension/cpp/src/io/arrowadapter/filesystem/OmniFileSystem.h rename to omnioperator/omniop-spark-extension/cpp/src/io/arrowadapter/OmniFileSystem.h index aa21c9f72..f23abf1b4 100644 --- a/omnioperator/omniop-spark-extension/cpp/src/io/arrowadapter/filesystem/OmniFileSystem.h +++ b/omnioperator/omniop-spark-extension/cpp/src/io/arrowadapter/OmniFileSystem.h @@ -35,11 +35,13 @@ #include "arrow/util/type_fwd.h" #include "arrow/util/visibility.h" #include "arrow/util/windows_fixup.h" -#include "io/arrowadapter/util/SimpleUri.h" +#include "io/UriInfo.h" -namespace arrow { +namespace arrow_adapter { -using internal::SimpleUri; +using arrow::Result; +using uri::internal::UriInfo; +using arrow::fs::FileSystem; namespace fs { @@ -51,7 +53,7 @@ namespace fs { /// /// Same as FileSystemFromUriOrPath, but it use uri that constructed by client ARROW_EXPORT -Result> FileSystemFromUriOrPath(const SimpleUri &uri, +Result> FileSystemFromUriOrPath(const UriInfo &uri, std::string* out_path = NULLPTR); @@ -73,7 +75,7 @@ Result> FileSystemFromUriOrPath(const SimpleUri &uri /// paths are allowed. ARROW_EXPORT Result> FileSystemFromUriOrPath( - const SimpleUri &uri, const io::IOContext &io_context, + const UriInfo &uri, const arrow::io::IOContext &io_context, std::string *out_path = NULLPTR); /// @} diff --git a/omnioperator/omniop-spark-extension/cpp/src/io/arrowadapter/filesystem/OmniHdfs.cc b/omnioperator/omniop-spark-extension/cpp/src/io/arrowadapter/OmniHdfs.cc similarity index 35% rename from omnioperator/omniop-spark-extension/cpp/src/io/arrowadapter/filesystem/OmniHdfs.cc rename to omnioperator/omniop-spark-extension/cpp/src/io/arrowadapter/OmniHdfs.cc index 3b44985bf..64540361a 100644 --- a/omnioperator/omniop-spark-extension/cpp/src/io/arrowadapter/filesystem/OmniHdfs.cc +++ b/omnioperator/omniop-spark-extension/cpp/src/io/arrowadapter/OmniHdfs.cc @@ -17,29 +17,24 @@ */ #include -#include -#include +#include #include "arrow/filesystem/hdfs.h" -#include "arrow/filesystem/path_util.h" #include "arrow/util/value_parsing.h" -#include "io/arrowadapter/filesystem/OmniHdfs.h" +#include "OmniHdfs.h" -namespace arrow { +namespace arrow_adapter { -using internal::ParseValue; +using arrow::internal::ParseValue; +using uri::internal::UriInfo; +using arrow::Result; +using arrow::fs::HdfsOptions; namespace fs { -Result buildHdfsOptionsFromUri(const SimpleUri &uri){ +Result buildHdfsOptionsFromUri(const UriInfo &uri){ HdfsOptions options; - std::unordered_map options_map; - ARROW_ASSIGN_OR_RAISE(const auto options_items, uri.query_items()); - for (const auto& kv : options_items) { - options_map.emplace(kv.first, kv.second); - } - std::string host; host = uri.scheme() + "://" + uri.host(); @@ -52,67 +47,9 @@ Result buildHdfsOptionsFromUri(const SimpleUri &uri){ options.ConfigureEndPoint(host, port); } - // configure replication - auto it = options_map.find("replication"); - if (it != options_map.end()) { - const auto& v = it->second; - int16_t replication; - if (!ParseValue(v.data(), v.size(), &replication)) { - return Status::Invalid("Invalid value for option 'replication': '", v, "'"); - } - options.ConfigureReplication(replication); - options_map.erase(it); - } - - // configure buffer_size - it = options_map.find("buffer_size"); - if (it != options_map.end()) { - const auto& v = it->second; - int32_t buffer_size; - if (!ParseValue(v.data(), v.size(), &buffer_size)) { - return Status::Invalid("Invalid value for option 'buffer_size': '", v, "'"); - } - options.ConfigureBufferSize(buffer_size); - options_map.erase(it); - } - - // configure default_block_size - it = options_map.find("default_block_size"); - if (it != options_map.end()) { - const auto& v = it->second; - int64_t default_block_size; - if (!ParseValue(v.data(), v.size(), &default_block_size)) { - return Status::Invalid("Invalid value for option 'default_block_size': '", v, "'"); - } - options.ConfigureBlockSize(default_block_size); - options_map.erase(it); - } - - // configure user - it = options_map.find("user"); - if (it != options_map.end()) { - const auto& user = it->second; - options.ConfigureUser(user); - options_map.erase(it); - } - - // configure kerberos - it = options_map.find("kerb_ticket"); - if (it != options_map.end()) { - const auto& ticket = it->second; - options.ConfigureKerberosTicketCachePath(ticket); - options_map.erase(it); - } - - // configure other options - for (const auto& it : options_map) { - options.ConfigureExtraConf(it.first, it.second); - } - return options; } - } // namespace fs } diff --git a/omnioperator/omniop-spark-extension/cpp/src/io/arrowadapter/filesystem/OmniHdfs.h b/omnioperator/omniop-spark-extension/cpp/src/io/arrowadapter/OmniHdfs.h similarity index 83% rename from omnioperator/omniop-spark-extension/cpp/src/io/arrowadapter/filesystem/OmniHdfs.h rename to omnioperator/omniop-spark-extension/cpp/src/io/arrowadapter/OmniHdfs.h index 8bc5cc8b1..55db17739 100644 --- a/omnioperator/omniop-spark-extension/cpp/src/io/arrowadapter/filesystem/OmniHdfs.h +++ b/omnioperator/omniop-spark-extension/cpp/src/io/arrowadapter/OmniHdfs.h @@ -24,16 +24,18 @@ #include "arrow/filesystem/filesystem.h" #include "arrow/filesystem/hdfs.h" -#include "io/arrowadapter/util/SimpleUri.h" +#include "io/UriInfo.h" -namespace arrow { +namespace arrow_adapter { -using internal::SimpleUri; +using uri::internal::UriInfo; +using arrow::Result; +using arrow::fs::HdfsOptions; namespace fs { ARROW_EXPORT -Result buildHdfsOptionsFromUri(const SimpleUri &uri); +Result buildHdfsOptionsFromUri(const UriInfo &uri); } // namespace fs diff --git a/omnioperator/omniop-spark-extension/cpp/src/io/arrowadapter/filesystem/OmniLocalfs.cc b/omnioperator/omniop-spark-extension/cpp/src/io/arrowadapter/OmniLocalfs.cc similarity index 67% rename from omnioperator/omniop-spark-extension/cpp/src/io/arrowadapter/filesystem/OmniLocalfs.cc rename to omnioperator/omniop-spark-extension/cpp/src/io/arrowadapter/OmniLocalfs.cc index 65be81cd2..da71af71d 100644 --- a/omnioperator/omniop-spark-extension/cpp/src/io/arrowadapter/filesystem/OmniLocalfs.cc +++ b/omnioperator/omniop-spark-extension/cpp/src/io/arrowadapter/OmniLocalfs.cc @@ -18,42 +18,28 @@ #include #include -#include - -#ifdef _WIN32 -#include "arrow/util/windows_compatibility.h" -#endif - +#include #include "arrow/filesystem/localfs.h" #include "arrow/util/io_util.h" -#include "io/arrowadapter/filesystem/OmniLocalfs.h" +#include "OmniLocalfs.h" -namespace arrow { - namespace fs { +namespace arrow_adapter { + +namespace fs { using ::arrow::internal::IOErrorFromErrno; -#ifdef _WIN32 -using ::arrow::internal::IOErrorFromWinError; -#endif using ::arrow::internal::NativePathString; using ::arrow::internal::PlatformFilename; +using arrow::Result; +using arrow::fs::LocalFileSystemOptions; +using arrow::Status; -Result buildLocalfsOptionsFromUri(const ::arrow::internal::SimpleUri &uri, std::string* out_path){ - if (!uri.username().empty() || !uri.password().empty()) { - return Status::Invalid("Unsupported username or password in local URI: '", - uri.ToString(), "'"); - } +Result buildLocalfsOptionsFromUri(const UriInfo &uri, std::string* out_path){ std::string path; const auto host = uri.host(); if (!host.empty()) { -#ifdef _WIN32 - std::stringstream ss; -ss << "//" << host << "/" << internal::RemoveLeadingSlash(uri.path()); -*out_path = ss.str(); -#else return Status::Invalid("Unsupported hostname in non-Windows local URI: '", uri.ToString(), "'"); -#endif } else { *out_path = uri.path(); } @@ -61,7 +47,7 @@ ss << "//" << host << "/" << internal::RemoveLeadingSlash(uri.path()); return LocalFileSystemOptions(); } - } +} // namespace fs } // namespace arrow \ No newline at end of file diff --git a/omnioperator/omniop-spark-extension/cpp/src/io/arrowadapter/filesystem/OmniLocalfs.h b/omnioperator/omniop-spark-extension/cpp/src/io/arrowadapter/OmniLocalfs.h similarity index 84% rename from omnioperator/omniop-spark-extension/cpp/src/io/arrowadapter/filesystem/OmniLocalfs.h rename to omnioperator/omniop-spark-extension/cpp/src/io/arrowadapter/OmniLocalfs.h index f359d565c..71e00d474 100644 --- a/omnioperator/omniop-spark-extension/cpp/src/io/arrowadapter/filesystem/OmniLocalfs.h +++ b/omnioperator/omniop-spark-extension/cpp/src/io/arrowadapter/OmniLocalfs.h @@ -24,14 +24,19 @@ #include "arrow/filesystem/filesystem.h" #include "arrow/filesystem/localfs.h" -#include "io/arrowadapter/util/SimpleUri.h" +#include "io/UriInfo.h" -namespace arrow { +namespace arrow_adapter { namespace fs { +using uri::internal::UriInfo; +using arrow::Result; +using arrow::fs::LocalFileSystemOptions; +using arrow::Status; + ARROW_EXPORT -Result buildLocalfsOptionsFromUri(const ::arrow::internal::SimpleUri &uri, std::string* out_path); +Result buildLocalfsOptionsFromUri(const UriInfo &uri, std::string* out_path); } // namespace fs diff --git a/omnioperator/omniop-spark-extension/cpp/src/io/arrowadapter/filesystem/UtilInternal.cc b/omnioperator/omniop-spark-extension/cpp/src/io/arrowadapter/UtilInternal.cc similarity index 88% rename from omnioperator/omniop-spark-extension/cpp/src/io/arrowadapter/filesystem/UtilInternal.cc rename to omnioperator/omniop-spark-extension/cpp/src/io/arrowadapter/UtilInternal.cc index 8f2f3dc0d..572388ad5 100644 --- a/omnioperator/omniop-spark-extension/cpp/src/io/arrowadapter/filesystem/UtilInternal.cc +++ b/omnioperator/omniop-spark-extension/cpp/src/io/arrowadapter/UtilInternal.cc @@ -16,15 +16,14 @@ * limitations under the License. */ -#include "io/arrowadapter/filesystem/UtilInternal.h" +#include "UtilInternal.h" -#include "arrow/util/io_util.h" +namespace arrow_adapter { -namespace arrow { - -using internal::StatusDetailFromErrno; +using arrow::fs::TimePoint; namespace fs { + namespace internal { TimePoint CurrentTimePoint() { diff --git a/omnioperator/omniop-spark-extension/cpp/src/io/arrowadapter/filesystem/UtilInternal.h b/omnioperator/omniop-spark-extension/cpp/src/io/arrowadapter/UtilInternal.h similarity index 95% rename from omnioperator/omniop-spark-extension/cpp/src/io/arrowadapter/filesystem/UtilInternal.h rename to omnioperator/omniop-spark-extension/cpp/src/io/arrowadapter/UtilInternal.h index 1bcd92625..a6b9c8641 100644 --- a/omnioperator/omniop-spark-extension/cpp/src/io/arrowadapter/filesystem/UtilInternal.h +++ b/omnioperator/omniop-spark-extension/cpp/src/io/arrowadapter/UtilInternal.h @@ -27,12 +27,15 @@ #include "arrow/status.h" #include "arrow/util/visibility.h" -namespace arrow { +namespace arrow_adapter { + +using arrow::fs::TimePoint; + namespace fs { + namespace internal { ARROW_EXPORT - TimePoint CurrentTimePoint(); } diff --git a/omnioperator/omniop-spark-extension/cpp/src/io/arrowadapter/util/SimpleUri.cc b/omnioperator/omniop-spark-extension/cpp/src/io/arrowadapter/util/SimpleUri.cc deleted file mode 100644 index da1562569..000000000 --- a/omnioperator/omniop-spark-extension/cpp/src/io/arrowadapter/util/SimpleUri.cc +++ /dev/null @@ -1,120 +0,0 @@ -/** - * 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. - */ - -#include -#include -#include - -#include "arrow/result.h" -#include "SimpleUri.h" - -namespace arrow { - namespace internal { - -SimpleUri::SimpleUri(std::string _uri, std::string _scheme, std::string _path, std::string _host, - int32_t _port) { - uri_string = std::move(_uri); - scheme_string = std::move(_scheme); - path_string = std::move(_path); - host_string = std::move(_host); - port_number = _port; -} - -SimpleUri::SimpleUri(std::string _uri, std::string _scheme, std::string _path, std::string _host, int32_t _port, - std::vector> _query_list, std::string _username, std::string _password) - : SimpleUri(std::move(_uri), - std::move(_scheme), - std::move(_path), - std::move(_host), - _port, - std::move(_username), - std::move(_password)) { - query_list = std::move(_query_list); -} - -SimpleUri::SimpleUri(std::string _uri, std::string _scheme, std::string _path, std::string _host, int32_t _port, - std::vector> _query_list) - : SimpleUri(std::move(_uri), - std::move(_scheme), - std::move(_path), - std::move(_host), - _port -) { - query_list = std::move(_query_list); -} - -SimpleUri::SimpleUri(std::string _uri, std::string _scheme, std::string _path, std::string _host, int32_t _port, - std::string _username, std::string _password) : SimpleUri(std::move(_uri), - std::move(_scheme), - std::move(_path), - std::move(_host), - _port) { - username_string = std::move(_username); - password_string = std::move(_password); -} - -SimpleUri::~SimpleUri() {} - -std::string SimpleUri::scheme() const { - return scheme_string; -} - -bool SimpleUri::is_file_scheme() const { - return scheme_string == "file"; -} - -bool SimpleUri::has_host() const { - return !host_string.empty(); -} - -std::string SimpleUri::host() const { - return host_string; -} - -std::string SimpleUri::port_text() const { - return std::to_string(port_number); -} - -int32_t SimpleUri::port() const { - return port_number; -} - -std::string SimpleUri::username() const { - return username_string; -} - -std::string SimpleUri::password() const { - return password_string; -} - -std::string SimpleUri::path() const { - return path_string; -} - -Result>> SimpleUri::query_items() const { - return query_list; -} - -const std::string &SimpleUri::ToString() const { - return uri_string; -} - - } - // namespace internal -} -// namespace arrow \ No newline at end of file diff --git a/omnioperator/omniop-spark-extension/cpp/src/io/orcfile/OmniOrcFile.cc b/omnioperator/omniop-spark-extension/cpp/src/io/orcfile/OmniOrcFile.cc index 58e7461d1..af2077da8 100644 --- a/omnioperator/omniop-spark-extension/cpp/src/io/orcfile/OmniOrcFile.cc +++ b/omnioperator/omniop-spark-extension/cpp/src/io/orcfile/OmniOrcFile.cc @@ -25,12 +25,11 @@ #endif namespace orc { - std::unique_ptr readOmniFile(const std::string& path, const std::string& schema, - const std::string& uri_host, const std::string& uri_port) { - if (schema == "hdfs") { - return orc::readOmniHdfsFile(std::string(path), std::string(uri_host), std::string(uri_port)); + std::unique_ptr readOmniFile(const UriInfo &uri) { + if (uri.scheme() == "hdfs") { + return orc::readOmniHdfsFile(uri); } else { - return orc::readLocalFile(std::string(path)); + return orc::readLocalFile(uri.path()); } } } diff --git a/omnioperator/omniop-spark-extension/cpp/src/io/orcfile/OmniOrcFile.hh b/omnioperator/omniop-spark-extension/cpp/src/io/orcfile/OmniOrcFile.hh index fa6748c63..ac8c1fbe5 100644 --- a/omnioperator/omniop-spark-extension/cpp/src/io/orcfile/OmniOrcFile.hh +++ b/omnioperator/omniop-spark-extension/cpp/src/io/orcfile/OmniOrcFile.hh @@ -23,6 +23,7 @@ #include "hdfspp/options.h" #include "orc/OrcFile.hh" +#include "io/UriInfo.h" /** /file orc/OrcFile.hh @brief The top level interface to ORC. @@ -30,19 +31,19 @@ namespace orc { + using uri::internal::UriInfo; + /** * Create a stream to a local file or HDFS file if path begins with "hdfs://" * @param path the name of the file in the local file system or HDFS */ - ORC_UNIQUE_PTR readOmniFile(const std::string& path, const std::string& schema, - const std::string& uri_host, const std::string& uri_port); + std::unique_ptr readOmniFile(const UriInfo &uri); /** * Create a stream to an HDFS file. * @param path the uri of the file in HDFS */ - ORC_UNIQUE_PTR readOmniHdfsFile(const std::string& path, const std::string& uri_host, - const std::string& uri_port); + std::unique_ptr readOmniHdfsFile(const UriInfo &uri); } #endif diff --git a/omnioperator/omniop-spark-extension/cpp/src/io/orcfile/OmniOrcHdfsFile.cc b/omnioperator/omniop-spark-extension/cpp/src/io/orcfile/OmniOrcHdfsFile.cc index 7514c9c0e..7eec8bed0 100644 --- a/omnioperator/omniop-spark-extension/cpp/src/io/orcfile/OmniOrcHdfsFile.cc +++ b/omnioperator/omniop-spark-extension/cpp/src/io/orcfile/OmniOrcHdfsFile.cc @@ -17,31 +17,23 @@ */ #include "OmniOrcFile.hh" - #include "orc/Exceptions.hh" - -#include -#include - #include "hdfspp/hdfspp.h" namespace orc { + using uri::internal::UriInfo; + class OmniHdfsFileInputStream : public InputStream { private: - std::string filename; - std::string uri_host; - std::string uri_port; + UriInfo uri; std::unique_ptr file; std::unique_ptr file_system; uint64_t totalLength; const uint64_t READ_SIZE = 1024 * 1024; //1 MB public: - OmniHdfsFileInputStream(std::string _filename, std::string _uri_host, std::string _uri_port) { - filename = _filename; - uri_host = _uri_host; - uri_port = _uri_port; + explicit OmniHdfsFileInputStream(const UriInfo& _uri): uri(_uri) { hdfs::ConfigParser parser; if(!parser.LoadDefaultResources()){ @@ -64,18 +56,18 @@ namespace orc { //Wrapping file_system into a unique pointer to guarantee deletion file_system = std::unique_ptr( hdfs::FileSystem::New(io_service, "", options)); - if (file_system.get() == nullptr) { + if (file_system == nullptr) { throw ParseError("Can't create FileSystem object. "); } hdfs::Status status; //Checking if the user supplied the host - if(!uri_host.empty()){ - std::string port = !uri_port.empty() ? - uri_port : ""; - status = file_system->Connect(uri_host, port); + if(!uri.host().empty()){ + std::string port = !uri.port_text().empty() ? + uri.port_text() : ""; + status = file_system->Connect(uri.host(), port); if (!status.ok()) { - throw ParseError("Can't connect to " + uri_host + throw ParseError("Can't connect to " + uri.host() + ":" + port + ". " + status.ToString()); } } else { @@ -92,24 +84,24 @@ namespace orc { } } - if (file_system.get() == nullptr) { + if (file_system == nullptr) { throw ParseError("Can't connect the file system. "); } hdfs::FileHandle *file_raw = nullptr; - status = file_system->Open(filename, &file_raw); + status = file_system->Open(uri.path(), &file_raw); if (!status.ok()) { throw ParseError("Can't open " - + filename + ". " + status.ToString()); + + uri.path() + ". " + status.ToString()); } //Wrapping file_raw into a unique pointer to guarantee deletion file.reset(file_raw); hdfs::StatInfo stat_info; - status = file_system->GetFileInfo(filename, stat_info); + status = file_system->GetFileInfo(uri.path(), stat_info); if (!status.ok()) { throw ParseError("Can't stat " - + filename + ". " + status.ToString()); + + uri.path() + ". " + status.ToString()); } totalLength = stat_info.length; } @@ -148,7 +140,7 @@ namespace orc { } const std::string& getName() const override { - return filename; + return uri.path(); } ~OmniHdfsFileInputStream() override; @@ -157,8 +149,7 @@ namespace orc { OmniHdfsFileInputStream::~OmniHdfsFileInputStream() { } - std::unique_ptr readOmniHdfsFile(const std::string& path, const std::string& uri_host, - const std::string& uri_port) { - return std::unique_ptr(new OmniHdfsFileInputStream(path, uri_host, uri_port)); + std::unique_ptr readOmniHdfsFile(const UriInfo &uri) { + return std::unique_ptr(new OmniHdfsFileInputStream(uri)); } } diff --git a/omnioperator/omniop-spark-extension/cpp/src/jni/OrcColumnarBatchJniReader.cpp b/omnioperator/omniop-spark-extension/cpp/src/jni/OrcColumnarBatchJniReader.cpp index 0fd65da56..4647a2583 100644 --- a/omnioperator/omniop-spark-extension/cpp/src/jni/OrcColumnarBatchJniReader.cpp +++ b/omnioperator/omniop-spark-extension/cpp/src/jni/OrcColumnarBatchJniReader.cpp @@ -20,11 +20,13 @@ #include "OrcColumnarBatchJniReader.h" #include #include "jni_common.h" +#include "io/UriInfo.h" using namespace omniruntime::vec; using namespace omniruntime::type; using namespace std; using namespace orc; +using uri::internal::UriInfo; static constexpr int32_t MAX_DECIMAL64_DIGITS = 18; @@ -65,13 +67,11 @@ JNIEXPORT jlong JNICALL Java_com_huawei_boostkit_spark_jni_OrcColumnarBatchJniRe std::string hostStr(hostPtr); env->ReleaseStringUTFChars(hostJstr, hostPtr); - jstring portJstr = (jstring)env->CallObjectMethod(jsonObj, jsonMethodString, env->NewStringUTF("port")); - const char *portPtr = env->GetStringUTFChars(portJstr, nullptr); - std::string portStr(portPtr); - env->ReleaseStringUTFChars(portJstr, portPtr); + jint port = (jint)env->CallIntMethod(jsonObj, jsonMethodInt, env->NewStringUTF("port")); + UriInfo uri(schemaStr, fileStr, hostStr, port); std::unique_ptr reader; - reader = createReader(orc::readOmniFile(fileStr, schemaStr, hostStr, portStr), readerOptions); + reader = createReader(orc::readOmniFile(uri), readerOptions); orc::Reader *readerNew = reader.release(); return (jlong)(readerNew); diff --git a/omnioperator/omniop-spark-extension/cpp/src/jni/ParquetColumnarBatchJniReader.cpp b/omnioperator/omniop-spark-extension/cpp/src/jni/ParquetColumnarBatchJniReader.cpp index 6f8c84e96..84606b032 100644 --- a/omnioperator/omniop-spark-extension/cpp/src/jni/ParquetColumnarBatchJniReader.cpp +++ b/omnioperator/omniop-spark-extension/cpp/src/jni/ParquetColumnarBatchJniReader.cpp @@ -20,7 +20,7 @@ #include "ParquetColumnarBatchJniReader.h" #include "jni_common.h" #include "tablescan/ParquetReader.h" -#include "io/arrowadapter/util/SimpleUri.h" +#include "io/UriInfo.h" using namespace omniruntime::vec; using namespace omniruntime::type; @@ -28,6 +28,7 @@ using namespace std; using namespace arrow; using namespace parquet::arrow; using namespace spark::reader; +using uri::internal::UriInfo; std::vector GetIndices(JNIEnv *env, jobject jsonObj, const char* name) { @@ -74,7 +75,7 @@ JNIEXPORT jlong JNICALL Java_com_huawei_boostkit_spark_jni_ParquetColumnarBatchJ jint port = (jint)env->CallIntMethod(jsonObj, jsonMethodInt, env->NewStringUTF("port")); - arrow::internal::SimpleUri simpleUri(uriString, schemeString, pathString, hostString, port); + UriInfo uriInfo(uriString, schemeString, pathString, hostString, port); // Get capacity for each record batch int64_t capacity = (int64_t)env->CallLongMethod(jsonObj, jsonMethodLong, env->NewStringUTF("capacity")); @@ -84,7 +85,7 @@ JNIEXPORT jlong JNICALL Java_com_huawei_boostkit_spark_jni_ParquetColumnarBatchJ auto column_indices = GetIndices(env, jsonObj, "columnIndices"); ParquetReader *pReader = new ParquetReader(); - auto state = pReader->InitRecordReader(simpleUri, capacity, row_group_indices, column_indices, ugiString); + auto state = pReader->InitRecordReader(uriInfo, capacity, row_group_indices, column_indices, ugiString); if (state != Status::OK()) { env->ThrowNew(runtimeExceptionClass, state.ToString().c_str()); return 0; diff --git a/omnioperator/omniop-spark-extension/cpp/src/tablescan/ParquetReader.cpp b/omnioperator/omniop-spark-extension/cpp/src/tablescan/ParquetReader.cpp index 9f185efb5..ef3247e76 100644 --- a/omnioperator/omniop-spark-extension/cpp/src/tablescan/ParquetReader.cpp +++ b/omnioperator/omniop-spark-extension/cpp/src/tablescan/ParquetReader.cpp @@ -19,9 +19,9 @@ #include #include -#include "io/arrowadapter/filesystem/OmniFileSystem.h" +#include "io/arrowadapter/OmniFileSystem.h" #include -#include "io/arrowadapter/util/SimpleUri.h" +#include "io/UriInfo.h" #include "jni/jni_common.h" #include "ParquetReader.h" @@ -32,6 +32,7 @@ using namespace parquet::arrow; using namespace arrow::compute; using namespace spark::reader; using namespace arrow::internal; +using uri::internal::UriInfo; static std::mutex mutex_; static std::map restore_filesysptr; @@ -71,7 +72,7 @@ std::string spark::reader::GetFileSystemKey(std::string& path, std::string& ugi) return result; } -Filesystem* spark::reader::GetFileSystemPtr(SimpleUri &uri, std::string& ugi, arrow::Status &status) +Filesystem* spark::reader::GetFileSystemPtr(UriInfo &uri, std::string& ugi, arrow::Status &status) { auto key = GetFileSystemKey(const_cast(uri.ToString()), ugi); @@ -79,7 +80,7 @@ Filesystem* spark::reader::GetFileSystemPtr(SimpleUri &uri, std::string& ugi, ar auto iter = restore_filesysptr.find(key); if (iter == restore_filesysptr.end()) { Filesystem* fs = new Filesystem(); - auto result = fs::FileSystemFromUriOrPath(uri); + auto result = arrow_adapter::fs::FileSystemFromUriOrPath(uri); status = result.status(); if (!status.ok()) { return nullptr; @@ -91,7 +92,7 @@ Filesystem* spark::reader::GetFileSystemPtr(SimpleUri &uri, std::string& ugi, ar return restore_filesysptr[key]; } -Status ParquetReader::InitRecordReader(SimpleUri &uri, int64_t capacity, +Status ParquetReader::InitRecordReader(UriInfo &uri, int64_t capacity, const std::vector& row_group_indices, const std::vector& column_indices, std::string& ugi) { diff --git a/omnioperator/omniop-spark-extension/cpp/src/tablescan/ParquetReader.h b/omnioperator/omniop-spark-extension/cpp/src/tablescan/ParquetReader.h index 3143f47db..5c4641cb2 100644 --- a/omnioperator/omniop-spark-extension/cpp/src/tablescan/ParquetReader.h +++ b/omnioperator/omniop-spark-extension/cpp/src/tablescan/ParquetReader.h @@ -34,16 +34,17 @@ #include #include #include -#include "io/arrowadapter/util/SimpleUri.h" +#include "io/UriInfo.h" using namespace arrow::internal; +using uri::internal::UriInfo; namespace spark::reader { class ParquetReader { public: ParquetReader() {} - arrow::Status InitRecordReader(SimpleUri &uri, int64_t capacity, + arrow::Status InitRecordReader(UriInfo &uri, int64_t capacity, const std::vector& row_group_indices, const std::vector& column_indices, std::string& ugi); @@ -66,7 +67,7 @@ namespace spark::reader { std::string GetFileSystemKey(std::string& path, std::string& ugi); - Filesystem* GetFileSystemPtr(SimpleUri &uri, std::string& ugi, arrow::Status &status); + Filesystem* GetFileSystemPtr(UriInfo &uri, std::string& ugi, arrow::Status &status); int CopyToOmniVec(std::shared_ptr vcType, int &omniTypeId, uint64_t &omniVecId, std::shared_ptr array); diff --git a/omnioperator/omniop-spark-extension/cpp/test/tablescan/parquet_scan_test.cpp b/omnioperator/omniop-spark-extension/cpp/test/tablescan/parquet_scan_test.cpp index fbe65e862..10b3bc792 100644 --- a/omnioperator/omniop-spark-extension/cpp/test/tablescan/parquet_scan_test.cpp +++ b/omnioperator/omniop-spark-extension/cpp/test/tablescan/parquet_scan_test.cpp @@ -26,6 +26,7 @@ using namespace spark::reader; using namespace arrow; using namespace omniruntime::vec; +using uri::internal::UriInfo; /* * CREATE TABLE `parquet_test` ( `c1` int, `c2` varChar(60), `c3` string, `c4` bigint, @@ -44,8 +45,8 @@ TEST(read, test_parquet_reader) ParquetReader *reader = new ParquetReader(); std::string ugi = "root@sample"; - SimpleUri simpleUri(filename, "", "", "", -1); - auto state1 = reader->InitRecordReader(simpleUri, 1024, row_group_indices, column_indices, ugi); + UriInfo uri(filename, "", "", "", -1); + auto state1 = reader->InitRecordReader(uri, 1024, row_group_indices, column_indices, ugi); ASSERT_EQ(state1, Status::OK()); std::shared_ptr batch; diff --git a/omnioperator/omniop-spark-extension/java/src/main/java/com/huawei/boostkit/spark/jni/OrcColumnarBatchJniReader.java b/omnioperator/omniop-spark-extension/java/src/main/java/com/huawei/boostkit/spark/jni/OrcColumnarBatchJniReader.java index ae0c0f6fd..94b7cd5ee 100644 --- a/omnioperator/omniop-spark-extension/java/src/main/java/com/huawei/boostkit/spark/jni/OrcColumnarBatchJniReader.java +++ b/omnioperator/omniop-spark-extension/java/src/main/java/com/huawei/boostkit/spark/jni/OrcColumnarBatchJniReader.java @@ -148,8 +148,8 @@ public class OrcColumnarBatchJniReader { job.put("scheme", uri.getScheme() == null ? "" : uri.getScheme()); job.put("host", uri.getHost() == null ? "" : uri.getHost()); - job.put("port", uri.getPort() == -1 ? "" : String.valueOf(uri.getPort())); job.put("path", uri.getPath() == null ? "" : uri.getPath()); + job.put("port", uri.getPort()); reader = initializeReader(job); return reader; -- Gitee From 23811c4f06753f61f8005f1e7cee4ad2637ea0f2 Mon Sep 17 00:00:00 2001 From: d00807371 Date: Tue, 26 Dec 2023 11:27:07 +0800 Subject: [PATCH 08/17] =?UTF-8?q?UriInfo=E5=88=A0=E9=99=A4arrow=E4=BE=9D?= =?UTF-8?q?=E8=B5=96?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- .../cpp/src/io/UriInfo.cc | 80 +++++++++---------- .../cpp/src/io/UriInfo.h | 74 +++++++---------- .../cpp/src/io/arrowadapter/OmniFileSystem.cc | 2 +- .../cpp/src/io/arrowadapter/OmniFileSystem.h | 2 +- .../cpp/src/io/arrowadapter/OmniHdfs.cc | 2 +- .../cpp/src/io/arrowadapter/OmniHdfs.h | 2 +- .../cpp/src/io/arrowadapter/OmniLocalfs.h | 2 +- .../cpp/src/io/orcfile/OmniOrcFile.hh | 2 +- .../cpp/src/io/orcfile/OmniOrcHdfsFile.cc | 2 +- .../cpp/src/jni/OrcColumnarBatchJniReader.cpp | 2 +- .../src/jni/ParquetColumnarBatchJniReader.cpp | 2 +- .../cpp/src/tablescan/ParquetReader.cpp | 2 +- .../cpp/src/tablescan/ParquetReader.h | 2 +- .../cpp/test/tablescan/parquet_scan_test.cpp | 2 +- 14 files changed, 75 insertions(+), 103 deletions(-) diff --git a/omnioperator/omniop-spark-extension/cpp/src/io/UriInfo.cc b/omnioperator/omniop-spark-extension/cpp/src/io/UriInfo.cc index 5eb3cdc75..b2724cb08 100644 --- a/omnioperator/omniop-spark-extension/cpp/src/io/UriInfo.cc +++ b/omnioperator/omniop-spark-extension/cpp/src/io/UriInfo.cc @@ -21,56 +21,48 @@ #include "UriInfo.h" -namespace uri { - namespace internal { - - UriInfo::UriInfo(std::string _uri, std::string _scheme, std::string _path, std::string _host, - int32_t _port): host_string(std::move(_host)), - scheme_string(std::move(_scheme)), - port_number(_port), - port_string(std::to_string(port_number)), - path_string(std::move(_path)), - uri_string(std::move(_uri)) - { - } - - UriInfo::UriInfo(std::string _scheme, std::string _path, std::string _host, - int32_t _port): host_string(std::move(_host)), - scheme_string(std::move(_scheme)), - port_number(_port), - port_string(std::to_string(port_number)), - path_string(std::move(_path)), - uri_string("Not initialize origin uri!") - { - } +UriInfo::UriInfo(std::string _uri, std::string _scheme, std::string _path, std::string _host, + int32_t _port): host_string(std::move(_host)), + scheme_string(std::move(_scheme)), + port_number(_port), + port_string(std::to_string(port_number)), + path_string(std::move(_path)), + uri_string(std::move(_uri)) +{ +} - UriInfo::~UriInfo() {} +UriInfo::UriInfo(std::string _scheme, std::string _path, std::string _host, + int32_t _port): host_string(std::move(_host)), + scheme_string(std::move(_scheme)), + port_number(_port), + port_string(std::to_string(port_number)), + path_string(std::move(_path)), + uri_string("Not initialize origin uri!") +{ +} - const std::string& UriInfo::scheme() const { - return scheme_string; - } +UriInfo::~UriInfo() {} - const std::string& UriInfo::host() const { - return host_string; - } +const std::string& UriInfo::scheme() const { + return scheme_string; +} - const std::string& UriInfo::port_text() const { - return port_string; - } +const std::string& UriInfo::host() const { + return host_string; +} - int32_t UriInfo::port() const { - return port_number; - } +const std::string& UriInfo::port_text() const { + return port_string; +} - const std::string& UriInfo::path() const { - return path_string; - } +int32_t UriInfo::port() const { + return port_number; +} - const std::string &UriInfo::ToString() const { - return uri_string; - } +const std::string& UriInfo::path() const { + return path_string; +} - } - // namespace internal +const std::string &UriInfo::ToString() const { + return uri_string; } -// namespace arrow_adapter \ No newline at end of file diff --git a/omnioperator/omniop-spark-extension/cpp/src/io/UriInfo.h b/omnioperator/omniop-spark-extension/cpp/src/io/UriInfo.h index 6977e9267..ca9c39749 100644 --- a/omnioperator/omniop-spark-extension/cpp/src/io/UriInfo.h +++ b/omnioperator/omniop-spark-extension/cpp/src/io/UriInfo.h @@ -18,60 +18,40 @@ #pragma once -#include -#include -#include -#include -#include -#include - -#include "arrow/result.h" -#include "arrow/type_fwd.h" -#include "arrow/util/visibility.h" - -namespace uri{ - - namespace internal { - /// \brief A parsed URI - class ARROW_EXPORT UriInfo { - public: - UriInfo(std::string _uri, std::string _scheme, std::string _path, std::string _host, int32_t _port); - - UriInfo(std::string _scheme, std::string _path, std::string _host, int32_t _port); +class UriInfo { +public: + UriInfo(std::string _uri, std::string _scheme, std::string _path, std::string _host, int32_t _port); - ~UriInfo(); + UriInfo(std::string _scheme, std::string _path, std::string _host, int32_t _port); - const std::string& scheme() const ; + ~UriInfo(); - /// The URI host name, such as "localhost", "127.0.0.1" or "::1", or the empty - /// string is the URI does not have a host component. - const std::string& host() const ; + const std::string& scheme() const ; - /// The URI port parsed as an integer, or -1 if the URI does not have a port - /// number component. - int32_t port() const ; + /// The URI host name, such as "localhost", "127.0.0.1" or "::1", or the empty + /// string is the URI does not have a host component. + const std::string& host() const ; - /// The URI path component. - const std::string& path() const ; + /// The URI port parsed as an integer, or -1 if the URI does not have a port + /// number component. + int32_t port() const ; - /// The URI port number, as a string such as "80", or the empty string is the URI - /// does not have a port number component. - const std::string& port_text() const ; + /// The URI path component. + const std::string& path() const ; - /// Get the string representation of this URI. - const std::string &ToString() const ; + /// The URI port number, as a string such as "80", or the empty string is the URI + /// does not have a port number component. + const std::string& port_text() const ; - private: - std::string host_string; - std::string scheme_string; - int32_t port_number; - std::string port_string; - std::string path_string; - std::string uri_string; - }; + /// Get the string representation of this URI. + const std::string &ToString() const ; - } -// namespace internal -} -// namespace arrow_adapter \ No newline at end of file +private: + std::string host_string; + std::string scheme_string; + int32_t port_number; + std::string port_string; + std::string path_string; + std::string uri_string; +}; \ No newline at end of file diff --git a/omnioperator/omniop-spark-extension/cpp/src/io/arrowadapter/OmniFileSystem.cc b/omnioperator/omniop-spark-extension/cpp/src/io/arrowadapter/OmniFileSystem.cc index ea56dfa52..01b4c2e3e 100644 --- a/omnioperator/omniop-spark-extension/cpp/src/io/arrowadapter/OmniFileSystem.cc +++ b/omnioperator/omniop-spark-extension/cpp/src/io/arrowadapter/OmniFileSystem.cc @@ -52,7 +52,7 @@ using arrow::fs::FileSystem; using arrow::fs::HadoopFileSystem; using arrow::fs::LocalFileSystem; using arrow::fs::internal::MockFileSystem; -using uri::internal::UriInfo; + using arrow::Result; namespace { diff --git a/omnioperator/omniop-spark-extension/cpp/src/io/arrowadapter/OmniFileSystem.h b/omnioperator/omniop-spark-extension/cpp/src/io/arrowadapter/OmniFileSystem.h index f23abf1b4..d585424e2 100644 --- a/omnioperator/omniop-spark-extension/cpp/src/io/arrowadapter/OmniFileSystem.h +++ b/omnioperator/omniop-spark-extension/cpp/src/io/arrowadapter/OmniFileSystem.h @@ -40,7 +40,7 @@ namespace arrow_adapter { using arrow::Result; -using uri::internal::UriInfo; + using arrow::fs::FileSystem; namespace fs { diff --git a/omnioperator/omniop-spark-extension/cpp/src/io/arrowadapter/OmniHdfs.cc b/omnioperator/omniop-spark-extension/cpp/src/io/arrowadapter/OmniHdfs.cc index 64540361a..702a17f92 100644 --- a/omnioperator/omniop-spark-extension/cpp/src/io/arrowadapter/OmniHdfs.cc +++ b/omnioperator/omniop-spark-extension/cpp/src/io/arrowadapter/OmniHdfs.cc @@ -26,7 +26,7 @@ namespace arrow_adapter { using arrow::internal::ParseValue; -using uri::internal::UriInfo; + using arrow::Result; using arrow::fs::HdfsOptions; diff --git a/omnioperator/omniop-spark-extension/cpp/src/io/arrowadapter/OmniHdfs.h b/omnioperator/omniop-spark-extension/cpp/src/io/arrowadapter/OmniHdfs.h index 55db17739..3ee21f32d 100644 --- a/omnioperator/omniop-spark-extension/cpp/src/io/arrowadapter/OmniHdfs.h +++ b/omnioperator/omniop-spark-extension/cpp/src/io/arrowadapter/OmniHdfs.h @@ -28,7 +28,7 @@ namespace arrow_adapter { -using uri::internal::UriInfo; + using arrow::Result; using arrow::fs::HdfsOptions; diff --git a/omnioperator/omniop-spark-extension/cpp/src/io/arrowadapter/OmniLocalfs.h b/omnioperator/omniop-spark-extension/cpp/src/io/arrowadapter/OmniLocalfs.h index 71e00d474..a546a39dc 100644 --- a/omnioperator/omniop-spark-extension/cpp/src/io/arrowadapter/OmniLocalfs.h +++ b/omnioperator/omniop-spark-extension/cpp/src/io/arrowadapter/OmniLocalfs.h @@ -30,7 +30,7 @@ namespace arrow_adapter { namespace fs { -using uri::internal::UriInfo; + using arrow::Result; using arrow::fs::LocalFileSystemOptions; using arrow::Status; diff --git a/omnioperator/omniop-spark-extension/cpp/src/io/orcfile/OmniOrcFile.hh b/omnioperator/omniop-spark-extension/cpp/src/io/orcfile/OmniOrcFile.hh index ac8c1fbe5..302b3c858 100644 --- a/omnioperator/omniop-spark-extension/cpp/src/io/orcfile/OmniOrcFile.hh +++ b/omnioperator/omniop-spark-extension/cpp/src/io/orcfile/OmniOrcFile.hh @@ -31,7 +31,7 @@ namespace orc { - using uri::internal::UriInfo; + /** * Create a stream to a local file or HDFS file if path begins with "hdfs://" diff --git a/omnioperator/omniop-spark-extension/cpp/src/io/orcfile/OmniOrcHdfsFile.cc b/omnioperator/omniop-spark-extension/cpp/src/io/orcfile/OmniOrcHdfsFile.cc index 7eec8bed0..4fe2d75a2 100644 --- a/omnioperator/omniop-spark-extension/cpp/src/io/orcfile/OmniOrcHdfsFile.cc +++ b/omnioperator/omniop-spark-extension/cpp/src/io/orcfile/OmniOrcHdfsFile.cc @@ -22,7 +22,7 @@ namespace orc { - using uri::internal::UriInfo; + class OmniHdfsFileInputStream : public InputStream { private: diff --git a/omnioperator/omniop-spark-extension/cpp/src/jni/OrcColumnarBatchJniReader.cpp b/omnioperator/omniop-spark-extension/cpp/src/jni/OrcColumnarBatchJniReader.cpp index 4647a2583..221a255ac 100644 --- a/omnioperator/omniop-spark-extension/cpp/src/jni/OrcColumnarBatchJniReader.cpp +++ b/omnioperator/omniop-spark-extension/cpp/src/jni/OrcColumnarBatchJniReader.cpp @@ -26,7 +26,7 @@ using namespace omniruntime::vec; using namespace omniruntime::type; using namespace std; using namespace orc; -using uri::internal::UriInfo; + static constexpr int32_t MAX_DECIMAL64_DIGITS = 18; diff --git a/omnioperator/omniop-spark-extension/cpp/src/jni/ParquetColumnarBatchJniReader.cpp b/omnioperator/omniop-spark-extension/cpp/src/jni/ParquetColumnarBatchJniReader.cpp index 84606b032..b02254630 100644 --- a/omnioperator/omniop-spark-extension/cpp/src/jni/ParquetColumnarBatchJniReader.cpp +++ b/omnioperator/omniop-spark-extension/cpp/src/jni/ParquetColumnarBatchJniReader.cpp @@ -28,7 +28,7 @@ using namespace std; using namespace arrow; using namespace parquet::arrow; using namespace spark::reader; -using uri::internal::UriInfo; + std::vector GetIndices(JNIEnv *env, jobject jsonObj, const char* name) { diff --git a/omnioperator/omniop-spark-extension/cpp/src/tablescan/ParquetReader.cpp b/omnioperator/omniop-spark-extension/cpp/src/tablescan/ParquetReader.cpp index ef3247e76..a9efc91fd 100644 --- a/omnioperator/omniop-spark-extension/cpp/src/tablescan/ParquetReader.cpp +++ b/omnioperator/omniop-spark-extension/cpp/src/tablescan/ParquetReader.cpp @@ -32,7 +32,7 @@ using namespace parquet::arrow; using namespace arrow::compute; using namespace spark::reader; using namespace arrow::internal; -using uri::internal::UriInfo; + static std::mutex mutex_; static std::map restore_filesysptr; diff --git a/omnioperator/omniop-spark-extension/cpp/src/tablescan/ParquetReader.h b/omnioperator/omniop-spark-extension/cpp/src/tablescan/ParquetReader.h index 5c4641cb2..15b89869b 100644 --- a/omnioperator/omniop-spark-extension/cpp/src/tablescan/ParquetReader.h +++ b/omnioperator/omniop-spark-extension/cpp/src/tablescan/ParquetReader.h @@ -37,7 +37,7 @@ #include "io/UriInfo.h" using namespace arrow::internal; -using uri::internal::UriInfo; + namespace spark::reader { class ParquetReader { diff --git a/omnioperator/omniop-spark-extension/cpp/test/tablescan/parquet_scan_test.cpp b/omnioperator/omniop-spark-extension/cpp/test/tablescan/parquet_scan_test.cpp index 10b3bc792..cf1e045be 100644 --- a/omnioperator/omniop-spark-extension/cpp/test/tablescan/parquet_scan_test.cpp +++ b/omnioperator/omniop-spark-extension/cpp/test/tablescan/parquet_scan_test.cpp @@ -26,7 +26,7 @@ using namespace spark::reader; using namespace arrow; using namespace omniruntime::vec; -using uri::internal::UriInfo; + /* * CREATE TABLE `parquet_test` ( `c1` int, `c2` varChar(60), `c3` string, `c4` bigint, -- Gitee From fcf44c6ee67c321843a242c59687c5ef377b1adb Mon Sep 17 00:00:00 2001 From: d00807371 Date: Tue, 26 Dec 2023 11:33:57 +0800 Subject: [PATCH 09/17] =?UTF-8?q?UriInfo=E5=88=A0=E9=99=A4arrow=E4=BE=9D?= =?UTF-8?q?=E8=B5=96?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- .../cpp/src/io/arrowadapter/OmniLocalfs.cc | 1 + 1 file changed, 1 insertion(+) diff --git a/omnioperator/omniop-spark-extension/cpp/src/io/arrowadapter/OmniLocalfs.cc b/omnioperator/omniop-spark-extension/cpp/src/io/arrowadapter/OmniLocalfs.cc index da71af71d..1582470b1 100644 --- a/omnioperator/omniop-spark-extension/cpp/src/io/arrowadapter/OmniLocalfs.cc +++ b/omnioperator/omniop-spark-extension/cpp/src/io/arrowadapter/OmniLocalfs.cc @@ -22,6 +22,7 @@ #include "arrow/filesystem/localfs.h" #include "arrow/util/io_util.h" #include "OmniLocalfs.h" +#include "arrow/result.h" namespace arrow_adapter { -- Gitee From e3e5d3ecd578981217a0a9f5e77b7d1e55f89055 Mon Sep 17 00:00:00 2001 From: d00807371 Date: Thu, 28 Dec 2023 10:18:44 +0800 Subject: [PATCH 10/17] =?UTF-8?q?=E7=B2=BE=E7=AE=80namespace?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- .../cpp/src/io/arrowadapter/OmniFileSystem.cc | 14 +------------- .../cpp/src/io/arrowadapter/OmniFileSystem.h | 3 --- .../cpp/src/io/arrowadapter/OmniHdfs.cc | 4 ---- .../cpp/src/io/arrowadapter/OmniHdfs.h | 5 ----- .../cpp/src/io/arrowadapter/OmniLocalfs.cc | 4 ---- .../cpp/src/io/arrowadapter/OmniLocalfs.h | 5 ----- .../cpp/src/io/arrowadapter/UtilInternal.cc | 8 -------- .../cpp/src/io/arrowadapter/UtilInternal.h | 8 -------- .../cpp/src/tablescan/ParquetReader.cpp | 2 +- 9 files changed, 2 insertions(+), 51 deletions(-) diff --git a/omnioperator/omniop-spark-extension/cpp/src/io/arrowadapter/OmniFileSystem.cc b/omnioperator/omniop-spark-extension/cpp/src/io/arrowadapter/OmniFileSystem.cc index 01b4c2e3e..f3a5e22e4 100644 --- a/omnioperator/omniop-spark-extension/cpp/src/io/arrowadapter/OmniFileSystem.cc +++ b/omnioperator/omniop-spark-extension/cpp/src/io/arrowadapter/OmniFileSystem.cc @@ -36,23 +36,13 @@ namespace arrow_adapter { -using arrow::internal::checked_pointer_cast; -using arrow::internal::TaskHints; using arrow::internal::Uri; - -namespace fs { - -using arrow::fs::internal::ConcatAbstractPath; -using arrow::fs::internal::EnsureTrailingSlash; -using arrow::fs::internal::GetAbstractPathParent; using arrow::fs::internal::RemoveLeadingSlash; -using arrow::fs::internal::RemoveTrailingSlash; using arrow::fs::internal::ToSlashes; using arrow::fs::FileSystem; using arrow::fs::HadoopFileSystem; using arrow::fs::LocalFileSystem; using arrow::fs::internal::MockFileSystem; - using arrow::Result; namespace { @@ -85,7 +75,7 @@ FileSystemFromUriReal(const UriInfo &uri, const arrow::io::IOContext &io_context if (out_path != nullptr) { *out_path = std::string(RemoveLeadingSlash(uri.path())); } - return std::make_shared(internal::CurrentTimePoint(), + return std::make_shared(CurrentTimePoint(), io_context); } @@ -114,7 +104,5 @@ Result> FileSystemFromUriOrPath( return FileSystemFromUriReal(uri, io_context, out_path); } -} -// namespace fs } // namespace arrow \ No newline at end of file diff --git a/omnioperator/omniop-spark-extension/cpp/src/io/arrowadapter/OmniFileSystem.h b/omnioperator/omniop-spark-extension/cpp/src/io/arrowadapter/OmniFileSystem.h index d585424e2..cbb10e6fa 100644 --- a/omnioperator/omniop-spark-extension/cpp/src/io/arrowadapter/OmniFileSystem.h +++ b/omnioperator/omniop-spark-extension/cpp/src/io/arrowadapter/OmniFileSystem.h @@ -43,8 +43,6 @@ using arrow::Result; using arrow::fs::FileSystem; -namespace fs { - /// \defgroup filesystem-factories Functions for creating FileSystem instances /// @{ @@ -80,7 +78,6 @@ Result> FileSystemFromUriOrPath( /// @} -} // namespace fs } // namespace arrow \ No newline at end of file diff --git a/omnioperator/omniop-spark-extension/cpp/src/io/arrowadapter/OmniHdfs.cc b/omnioperator/omniop-spark-extension/cpp/src/io/arrowadapter/OmniHdfs.cc index 702a17f92..dfc18f2f8 100644 --- a/omnioperator/omniop-spark-extension/cpp/src/io/arrowadapter/OmniHdfs.cc +++ b/omnioperator/omniop-spark-extension/cpp/src/io/arrowadapter/OmniHdfs.cc @@ -30,8 +30,6 @@ using arrow::internal::ParseValue; using arrow::Result; using arrow::fs::HdfsOptions; -namespace fs { - Result buildHdfsOptionsFromUri(const UriInfo &uri){ HdfsOptions options; @@ -50,7 +48,5 @@ Result buildHdfsOptionsFromUri(const UriInfo &uri){ return options; } -} -// namespace fs } // namespace arrow \ No newline at end of file diff --git a/omnioperator/omniop-spark-extension/cpp/src/io/arrowadapter/OmniHdfs.h b/omnioperator/omniop-spark-extension/cpp/src/io/arrowadapter/OmniHdfs.h index 3ee21f32d..445083638 100644 --- a/omnioperator/omniop-spark-extension/cpp/src/io/arrowadapter/OmniHdfs.h +++ b/omnioperator/omniop-spark-extension/cpp/src/io/arrowadapter/OmniHdfs.h @@ -28,16 +28,11 @@ namespace arrow_adapter { - using arrow::Result; using arrow::fs::HdfsOptions; -namespace fs { - ARROW_EXPORT Result buildHdfsOptionsFromUri(const UriInfo &uri); -} -// namespace fs } // namespace arrow \ No newline at end of file diff --git a/omnioperator/omniop-spark-extension/cpp/src/io/arrowadapter/OmniLocalfs.cc b/omnioperator/omniop-spark-extension/cpp/src/io/arrowadapter/OmniLocalfs.cc index 1582470b1..71d8c6ab0 100644 --- a/omnioperator/omniop-spark-extension/cpp/src/io/arrowadapter/OmniLocalfs.cc +++ b/omnioperator/omniop-spark-extension/cpp/src/io/arrowadapter/OmniLocalfs.cc @@ -26,8 +26,6 @@ namespace arrow_adapter { -namespace fs { - using ::arrow::internal::IOErrorFromErrno; using ::arrow::internal::NativePathString; using ::arrow::internal::PlatformFilename; @@ -48,7 +46,5 @@ Result buildLocalfsOptionsFromUri(const UriInfo &uri, st return LocalFileSystemOptions(); } -} -// namespace fs } // namespace arrow \ No newline at end of file diff --git a/omnioperator/omniop-spark-extension/cpp/src/io/arrowadapter/OmniLocalfs.h b/omnioperator/omniop-spark-extension/cpp/src/io/arrowadapter/OmniLocalfs.h index a546a39dc..47f256706 100644 --- a/omnioperator/omniop-spark-extension/cpp/src/io/arrowadapter/OmniLocalfs.h +++ b/omnioperator/omniop-spark-extension/cpp/src/io/arrowadapter/OmniLocalfs.h @@ -28,9 +28,6 @@ namespace arrow_adapter { -namespace fs { - - using arrow::Result; using arrow::fs::LocalFileSystemOptions; using arrow::Status; @@ -38,7 +35,5 @@ using arrow::Status; ARROW_EXPORT Result buildLocalfsOptionsFromUri(const UriInfo &uri, std::string* out_path); -} -// namespace fs } // namespace arrow \ No newline at end of file diff --git a/omnioperator/omniop-spark-extension/cpp/src/io/arrowadapter/UtilInternal.cc b/omnioperator/omniop-spark-extension/cpp/src/io/arrowadapter/UtilInternal.cc index 572388ad5..d82676df4 100644 --- a/omnioperator/omniop-spark-extension/cpp/src/io/arrowadapter/UtilInternal.cc +++ b/omnioperator/omniop-spark-extension/cpp/src/io/arrowadapter/UtilInternal.cc @@ -22,19 +22,11 @@ namespace arrow_adapter { using arrow::fs::TimePoint; -namespace fs { - -namespace internal { - TimePoint CurrentTimePoint() { auto now = std::chrono::system_clock::now(); return TimePoint( std::chrono::duration_cast(now.time_since_epoch())); } -} -// namespace internal -} -// namespace fs } // namespace arrow \ No newline at end of file diff --git a/omnioperator/omniop-spark-extension/cpp/src/io/arrowadapter/UtilInternal.h b/omnioperator/omniop-spark-extension/cpp/src/io/arrowadapter/UtilInternal.h index a6b9c8641..67d51eb46 100644 --- a/omnioperator/omniop-spark-extension/cpp/src/io/arrowadapter/UtilInternal.h +++ b/omnioperator/omniop-spark-extension/cpp/src/io/arrowadapter/UtilInternal.h @@ -31,16 +31,8 @@ namespace arrow_adapter { using arrow::fs::TimePoint; -namespace fs { - -namespace internal { - ARROW_EXPORT TimePoint CurrentTimePoint(); -} -// namespace internal -} -// namespace fs } // namespace arrow \ No newline at end of file diff --git a/omnioperator/omniop-spark-extension/cpp/src/tablescan/ParquetReader.cpp b/omnioperator/omniop-spark-extension/cpp/src/tablescan/ParquetReader.cpp index a9efc91fd..359345c7e 100644 --- a/omnioperator/omniop-spark-extension/cpp/src/tablescan/ParquetReader.cpp +++ b/omnioperator/omniop-spark-extension/cpp/src/tablescan/ParquetReader.cpp @@ -80,7 +80,7 @@ Filesystem* spark::reader::GetFileSystemPtr(UriInfo &uri, std::string& ugi, arro auto iter = restore_filesysptr.find(key); if (iter == restore_filesysptr.end()) { Filesystem* fs = new Filesystem(); - auto result = arrow_adapter::fs::FileSystemFromUriOrPath(uri); + auto result = arrow_adapter::FileSystemFromUriOrPath(uri); status = result.status(); if (!status.ok()) { return nullptr; -- Gitee From 06837fa70935f030c66fc3e917a474c4a1932dd9 Mon Sep 17 00:00:00 2001 From: d00807371 Date: Tue, 2 Jan 2024 15:20:28 +0800 Subject: [PATCH 11/17] add unit tests --- .../cpp/test/CMakeLists.txt | 4 + .../cpp/test/io/arrowadapter/CMakeLists.txt | 7 + .../io/arrowadapter/OmniFileSystemTest.cc | 200 ++++++++++++++++++ .../cpp/test/io/orcfile/CMakeLists.txt | 11 + .../test/io/orcfile/OmniOrcHdfsFileTest.cc | 46 ++++ .../cpp/test/io/orcfile/orcfile_test.h.in | 1 + .../cpp/test/utils/test_utils.cpp | 18 +- .../cpp/test/utils/test_utils.h | 14 ++ 8 files changed, 300 insertions(+), 1 deletion(-) create mode 100644 omnioperator/omniop-spark-extension/cpp/test/io/arrowadapter/CMakeLists.txt create mode 100644 omnioperator/omniop-spark-extension/cpp/test/io/arrowadapter/OmniFileSystemTest.cc create mode 100644 omnioperator/omniop-spark-extension/cpp/test/io/orcfile/CMakeLists.txt create mode 100644 omnioperator/omniop-spark-extension/cpp/test/io/orcfile/OmniOrcHdfsFileTest.cc create mode 100644 omnioperator/omniop-spark-extension/cpp/test/io/orcfile/orcfile_test.h.in diff --git a/omnioperator/omniop-spark-extension/cpp/test/CMakeLists.txt b/omnioperator/omniop-spark-extension/cpp/test/CMakeLists.txt index ba1ad3a77..1ce718c70 100644 --- a/omnioperator/omniop-spark-extension/cpp/test/CMakeLists.txt +++ b/omnioperator/omniop-spark-extension/cpp/test/CMakeLists.txt @@ -1,5 +1,7 @@ aux_source_directory(${CMAKE_CURRENT_LIST_DIR} TEST_ROOT_SRCS) +add_subdirectory(io/arrowadapter) +add_subdirectory(io/orcfile) add_subdirectory(shuffle) add_subdirectory(utils) add_subdirectory(tablescan) @@ -10,6 +12,8 @@ set(MY_LINK shuffletest utilstest tablescantest + arrowadaptertest + orcfiletest ) # find gtest package diff --git a/omnioperator/omniop-spark-extension/cpp/test/io/arrowadapter/CMakeLists.txt b/omnioperator/omniop-spark-extension/cpp/test/io/arrowadapter/CMakeLists.txt new file mode 100644 index 000000000..aec5bbc40 --- /dev/null +++ b/omnioperator/omniop-spark-extension/cpp/test/io/arrowadapter/CMakeLists.txt @@ -0,0 +1,7 @@ +aux_source_directory(${CMAKE_CURRENT_LIST_DIR} ARROW_ADAPTER_TESTS_LIST) +set(ARROW_ADAPTER_TARGET arrowadaptertest) +add_library(${ARROW_ADAPTER_TARGET} STATIC ${ARROW_ADAPTER_TESTS_LIST}) +target_compile_options(${ARROW_ADAPTER_TARGET} PUBLIC ) +target_include_directories(${ARROW_ADAPTER_TARGET} PUBLIC ${CMAKE_BINARY_DIR}/src) +target_include_directories(${ARROW_ADAPTER_TARGET} PUBLIC $ENV{JAVA_HOME}/include) +target_include_directories(${ARROW_ADAPTER_TARGET} PUBLIC $ENV{JAVA_HOME}/include/linux) diff --git a/omnioperator/omniop-spark-extension/cpp/test/io/arrowadapter/OmniFileSystemTest.cc b/omnioperator/omniop-spark-extension/cpp/test/io/arrowadapter/OmniFileSystemTest.cc new file mode 100644 index 000000000..cf10316b7 --- /dev/null +++ b/omnioperator/omniop-spark-extension/cpp/test/io/arrowadapter/OmniFileSystemTest.cc @@ -0,0 +1,200 @@ +/** + * Copyright (C) 2020-2022. Huawei Technologies Co., Ltd. All rights reserved. + * 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. + */ + + +#include +#include "gtest/gtest.h" +#include "io/arrowadapter/OmniFileSystem.h" +#include "arrow/filesystem/filesystem.h" +#include "arrow/filesystem/mockfs.h" +#include "arrow/util/checked_cast.h" +#include "arrow/result.h" +#include "arrow/testing/gtest_util.h" +#include "arrow/util/io_util.h" +#include "arrow/filesystem/path_util.h" +#include "arrow/filesystem/localfs.h" +#include "../../utils/test_utils.h" +#include "arrow/util/uri.h" + +using namespace arrow::fs::internal; +using arrow::fs::TimePoint; +using arrow::fs::FileSystem; +using arrow_adapter::FileSystemFromUriOrPath; +using arrow::internal::TemporaryDir; +using arrow::fs::LocalFileSystem; +using arrow::fs::LocalFileSystemOptions; +using arrow::internal::PlatformFilename; +using arrow::internal::FileDescriptor; +using arrow::Result; +using arrow::fs::HadoopFileSystem; +using arrow::fs::HdfsOptions; + +class TestMockFS : public ::testing::Test { +public: + void SetUp() override { + time_ = TimePoint(TimePoint::duration(42)); + fs_ = std::make_shared(time_); + } + + std::vector AllDirs() { + return arrow::internal::checked_pointer_cast(fs_)->AllDirs(); + } + + void CheckDirs(const std::vector& expected) { + ASSERT_EQ(AllDirs(), expected); + } + +protected: + TimePoint time_; + std::shared_ptr fs_; +}; + +TEST_F(TestMockFS, FileSystemFromUriOrPath) { + std::string path; + UriInfo uri1("mock", "", "", -1); + ASSERT_OK_AND_ASSIGN(fs_, FileSystemFromUriOrPath(uri1, &path)); + ASSERT_EQ(path, ""); + CheckDirs({}); // Ensures it's a MockFileSystem + + UriInfo uri2("mock", "foo/bar", "", -1); + ASSERT_OK_AND_ASSIGN(fs_, FileSystemFromUriOrPath(uri2, &path)); + ASSERT_EQ(path, "foo/bar"); + CheckDirs({}); + + UriInfo ur3("mock", "/foo/bar", "", -1); + ASSERT_OK_AND_ASSIGN(fs_, FileSystemFromUriOrPath(ur3, &path)); + ASSERT_EQ(path, "foo/bar"); + CheckDirs({}); +} + +struct CommonPathFormatter { + std::string operator()(std::string fn) { return fn; } + bool supports_uri() { return true; } +}; + +using PathFormatters = ::testing::Types; + +// Non-overloaded version of FileSystemFromUri, for template resolution +Result> FSFromUriOrPath(const UriInfo& uri, + std::string* out_path = NULLPTR) { + return arrow_adapter::FileSystemFromUriOrPath(uri, out_path); +} + + +template +class TestLocalFs : public ::testing::Test { +public: + void SetUp() override { + ASSERT_OK_AND_ASSIGN(temp_dir_, TemporaryDir::Make("test-localfs-")); + local_path_ = EnsureTrailingSlash(path_formatter_(temp_dir_->path().ToString())); + MakeFileSystem(); + } + + void MakeFileSystem() { + local_fs_ = std::make_shared(options_); + } + + template + void CheckFileSystemFromUriFunc(const UriInfo& uri, + FileSystemFromUriFunc&& fs_from_uri) { + if (!path_formatter_.supports_uri()) { + return; // skip + } + std::string path; + ASSERT_OK_AND_ASSIGN(fs_, fs_from_uri(uri, &path)); + ASSERT_EQ(path, local_path_); + + // Test that the right location on disk is accessed + CreateFile(fs_.get(), local_path_ + "abc", "some data"); + CheckConcreteFile(this->temp_dir_->path().ToString() + "abc", 9); + } + + void TestFileSystemFromUri(const UriInfo& uri) { + CheckFileSystemFromUriFunc(uri, FSFromUriOrPath); + } + + void CheckConcreteFile(const std::string& path, int64_t expected_size) { + ASSERT_OK_AND_ASSIGN(auto fn, PlatformFilename::FromString(path)); + ASSERT_OK_AND_ASSIGN(FileDescriptor fd, ::arrow::internal::FileOpenReadable(fn)); + auto result = ::arrow::internal::FileGetSize(fd.fd()); + ASSERT_OK_AND_ASSIGN(int64_t size, result); + ASSERT_EQ(size, expected_size); + } + + void TestLocalUri(const UriInfo& uri, const std::string& expected_path) { + CheckLocalUri(uri, expected_path, FSFromUriOrPath); + } + + template + void CheckLocalUri(const UriInfo& uri, const std::string& expected_path, + FileSystemFromUriFunc&& fs_from_uri) { + if (!path_formatter_.supports_uri()) { + return; // skip + } + std::string path; + ASSERT_OK_AND_ASSIGN(fs_, fs_from_uri(uri, &path)); + ASSERT_EQ(fs_->type_name(), "local"); + ASSERT_EQ(path, expected_path); + } + + void TestInvalidUri(const UriInfo& uri) { + if (!path_formatter_.supports_uri()) { + return; // skip + } + ASSERT_RAISES(Invalid, FSFromUriOrPath(uri)); + } + +protected: + std::unique_ptr temp_dir_; + std::shared_ptr fs_; + std::string local_path_; + PathFormatter path_formatter_; + std::shared_ptr local_fs_; + LocalFileSystemOptions options_ = LocalFileSystemOptions::Defaults(); +}; + +TYPED_TEST_SUITE(TestLocalFs, PathFormatters); + +TYPED_TEST(TestLocalFs, FileSystemFromUriFile){ + std::string path; + ASSERT_OK_AND_ASSIGN(auto uri_string, arrow::internal::UriFromAbsolutePath(this->local_path_)); + UriInfo uri1(uri_string, "", uri_string, "", -1); + this->TestFileSystemFromUri(uri1); + + path = "/foo/bar"; + UriInfo uri2("file", path, "", -1); + this->TestLocalUri(uri2, path); + + path = "/some path/%percent"; + UriInfo uri3("file", path, "", -1); + this->TestLocalUri(uri3, path); + + path = "/some path/%中文魑魅魍魉"; + UriInfo uri4("file", path, "", -1); + this->TestLocalUri(uri4, path); +} + +TYPED_TEST(TestLocalFs, FileSystemFromUriNoScheme){ + + UriInfo uri1(this->local_path_, "", "", "", -1); + this->TestFileSystemFromUri(uri1); + + UriInfo uri2("foo/bar", "", "", "", -1); + this->TestInvalidUri(uri2); +} diff --git a/omnioperator/omniop-spark-extension/cpp/test/io/orcfile/CMakeLists.txt b/omnioperator/omniop-spark-extension/cpp/test/io/orcfile/CMakeLists.txt new file mode 100644 index 000000000..cdb765aa3 --- /dev/null +++ b/omnioperator/omniop-spark-extension/cpp/test/io/orcfile/CMakeLists.txt @@ -0,0 +1,11 @@ +aux_source_directory(${CMAKE_CURRENT_LIST_DIR} ORC_FILE_TESTS_LIST) +set(MAIN_PATH ${CMAKE_CURRENT_SOURCE_DIR}) + +configure_file(orcfile_test.h.in ${CMAKE_CURRENT_SOURCE_DIR}/orcfile_test.h) +set(ORC_FILE_TARGET orcfiletest) + +add_library(${ORC_FILE_TARGET} STATIC ${ORC_FILE_TESTS_LIST}) +target_compile_options(${ORC_FILE_TARGET} PUBLIC ) +target_include_directories(${ORC_FILE_TARGET} PUBLIC ${CMAKE_BINARY_DIR}/src) +target_include_directories(${ORC_FILE_TARGET} PUBLIC $ENV{JAVA_HOME}/include) +target_include_directories(${ORC_FILE_TARGET} PUBLIC $ENV{JAVA_HOME}/include/linux) diff --git a/omnioperator/omniop-spark-extension/cpp/test/io/orcfile/OmniOrcHdfsFileTest.cc b/omnioperator/omniop-spark-extension/cpp/test/io/orcfile/OmniOrcHdfsFileTest.cc new file mode 100644 index 000000000..72c3a0dd2 --- /dev/null +++ b/omnioperator/omniop-spark-extension/cpp/test/io/orcfile/OmniOrcHdfsFileTest.cc @@ -0,0 +1,46 @@ +/** + * Copyright (C) 2020-2022. Huawei Technologies Co., Ltd. All rights reserved. + * 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. + */ + +#include "gtest/gtest.h" +#include "io/orcfile/OmniOrcFile.hh" +#include "orcfile_test.h" + +TEST(OrcReader, createLocalFileReader) { + std::string filename = "/resources/TestOrcFile.testSeek.orc"; + filename = PROJECT_PATH + filename; + + std::unique_ptr reader; + std::unique_ptr rowReader; + std::unique_ptr batch; + orc::ReaderOptions readerOpts; + orc::RowReaderOptions rowReaderOpts; + std::list cols; + + cols.push_back(1); + rowReaderOpts.include(cols); + UriInfo uriInfo("file", filename, "", -1); + reader = orc::createReader(orc::readOmniFile(uriInfo), readerOpts); + rowReader = reader->createRowReader(rowReaderOpts); + EXPECT_EQ(483517, reader->getMemoryUseByFieldId(cols)); + batch = rowReader->createRowBatch(1); + EXPECT_EQ(10, batch->getMemoryUsage()); + batch = rowReader->createRowBatch(1000); + EXPECT_EQ(10000, batch->getMemoryUsage()); + EXPECT_FALSE(batch->hasVariableLength()); +} diff --git a/omnioperator/omniop-spark-extension/cpp/test/io/orcfile/orcfile_test.h.in b/omnioperator/omniop-spark-extension/cpp/test/io/orcfile/orcfile_test.h.in new file mode 100644 index 000000000..5ca616ec4 --- /dev/null +++ b/omnioperator/omniop-spark-extension/cpp/test/io/orcfile/orcfile_test.h.in @@ -0,0 +1 @@ +#define PROJECT_PATH "@MAIN_PATH@" \ No newline at end of file diff --git a/omnioperator/omniop-spark-extension/cpp/test/utils/test_utils.cpp b/omnioperator/omniop-spark-extension/cpp/test/utils/test_utils.cpp index 9010cf150..7451eece7 100644 --- a/omnioperator/omniop-spark-extension/cpp/test/utils/test_utils.cpp +++ b/omnioperator/omniop-spark-extension/cpp/test/utils/test_utils.cpp @@ -481,4 +481,20 @@ void DeletePathAll(const char* path) { rmdir(path); } } -} \ No newline at end of file +} + +void CreateFile(FileSystem* fs, const std::string& path, const std::string& data) { + ASSERT_OK_AND_ASSIGN(auto stream, fs->OpenOutputStream(path)); + ASSERT_OK(stream->Write(data)); + ASSERT_OK(stream->Close()); +} + +void AssertFileInfo(FileSystem* fs, const std::string& path, FileType type) { + ASSERT_OK_AND_ASSIGN(FileInfo info, fs->GetFileInfo(path)); + AssertFileInfo(info, path, type); +} + +void AssertFileInfo(const FileInfo& info, const std::string& path, FileType type) { + ASSERT_EQ(info.path(), path); + ASSERT_EQ(info.type(), type) << "For path '" << info.path() << "'"; +} diff --git a/omnioperator/omniop-spark-extension/cpp/test/utils/test_utils.h b/omnioperator/omniop-spark-extension/cpp/test/utils/test_utils.h index b7380254a..a6d0eff86 100644 --- a/omnioperator/omniop-spark-extension/cpp/test/utils/test_utils.h +++ b/omnioperator/omniop-spark-extension/cpp/test/utils/test_utils.h @@ -27,6 +27,14 @@ #include #include "shuffle/splitter.h" #include "jni/concurrent_map.h" +#include "arrow/filesystem/filesystem.h" +#include "arrow/result.h" +#include "arrow/testing/gtest_util.h" +#include "arrow/filesystem/type_fwd.h" + +using arrow::fs::FileSystem; +using arrow::fs::FileInfo; +using arrow::fs::FileType; static ConcurrentMap> testShuffleSplitterHolder; @@ -131,4 +139,10 @@ void GetFilePath(const char *path, const char *filename, char *filepath); void DeletePathAll(const char* path); +void CreateFile(FileSystem* fs, const std::string& path, const std::string& data); + +void AssertFileInfo(const FileInfo& info, const std::string& path, FileType type); + +void AssertFileInfo(FileSystem* fs, const std::string& path, FileType type); + #endif //SPARK_THESTRAL_PLUGIN_TEST_UTILS_H \ No newline at end of file -- Gitee From 7b064962cb3b64e3e76f7798bb346f5f9d6fccbe Mon Sep 17 00:00:00 2001 From: d00807371 Date: Tue, 2 Jan 2024 15:31:07 +0800 Subject: [PATCH 12/17] modify unit tests --- .../cpp/test/io/orcfile/OmniOrcHdfsFileTest.cc | 10 ++-------- .../test/io/orcfile/resources/orc_data_all_type | Bin 0 -> 4470 bytes 2 files changed, 2 insertions(+), 8 deletions(-) create mode 100644 omnioperator/omniop-spark-extension/cpp/test/io/orcfile/resources/orc_data_all_type diff --git a/omnioperator/omniop-spark-extension/cpp/test/io/orcfile/OmniOrcHdfsFileTest.cc b/omnioperator/omniop-spark-extension/cpp/test/io/orcfile/OmniOrcHdfsFileTest.cc index 72c3a0dd2..f8771f848 100644 --- a/omnioperator/omniop-spark-extension/cpp/test/io/orcfile/OmniOrcHdfsFileTest.cc +++ b/omnioperator/omniop-spark-extension/cpp/test/io/orcfile/OmniOrcHdfsFileTest.cc @@ -22,7 +22,7 @@ #include "orcfile_test.h" TEST(OrcReader, createLocalFileReader) { - std::string filename = "/resources/TestOrcFile.testSeek.orc"; + std::string filename = "/resources/orc_data_all_type"; filename = PROJECT_PATH + filename; std::unique_ptr reader; @@ -36,11 +36,5 @@ TEST(OrcReader, createLocalFileReader) { rowReaderOpts.include(cols); UriInfo uriInfo("file", filename, "", -1); reader = orc::createReader(orc::readOmniFile(uriInfo), readerOpts); - rowReader = reader->createRowReader(rowReaderOpts); - EXPECT_EQ(483517, reader->getMemoryUseByFieldId(cols)); - batch = rowReader->createRowBatch(1); - EXPECT_EQ(10, batch->getMemoryUsage()); - batch = rowReader->createRowBatch(1000); - EXPECT_EQ(10000, batch->getMemoryUsage()); - EXPECT_FALSE(batch->hasVariableLength()); + EXPECT_NE(nullptr, reader); } diff --git a/omnioperator/omniop-spark-extension/cpp/test/io/orcfile/resources/orc_data_all_type b/omnioperator/omniop-spark-extension/cpp/test/io/orcfile/resources/orc_data_all_type new file mode 100644 index 0000000000000000000000000000000000000000..9cc57fa78ccdae728d2d902f587c30c337b0e4a5 GIT binary patch literal 4470 zcmbu>3p~^79|!Q?FlNpAnWd31l9kfTwZdE{bz-4h{^TAyU2rtF(!$8C+&cf`l0p}i z$Tc~Z+)KIDIpGu)bLm7fGX5F=-(1SSUL!QnLQHO@WmAiWqvz}CpPbM~o{ss-R$_uvw_yIL&SR%{ikdqUL>2bpL zys;rh?gP#$(6yjBc+2eW<1LekLsPC2mE-2ufQRax^}K+A!BBwq{7zh)Kns>ra$>)m zlT!fbnb|LkaNSjng9hk+H6T{Y-R}zVP*#|1EMC}EfCvB^p%Iwg9ydA+Z-y3>KeUMw z22kcF?w$LIwrGmHk1~ZHk2Z8Q)G?He4=v)u0gBMsh$01`vjH%|2-s$RBy71869>g{ z;?()y+3Vs8@Z&|%lx@0Et4)GwxiQvsDzwLK&{S*Yj~R`C?LZ)|+0$3>!wfP>a*_-i?G;_8)3_E(a(2&1_UL2G(rDy1 zrF503?itI$y}=GY8S1umTw<^318)`~Pd9SM@Apqko_y)Jl2HOkz!6q5D!*t{miOdY zE*^9MuYnd+h=5M^8;@Fd!c70dqx@fZyi=yvqz~4|hpRwrzV>KORw(K9d154_k2A#7 zBb@%RNBMu|5eMRMxaB;yKYwq(HXq1@_V|s*Ymi5%EqEj>@kpBYxTxtMkgoTcpT$ys zG7ME7zx-tPv%O3Bb^KZN4L{rhKYGMcej5oez~TIl$Z8@OKISRC*D@YS!cJ#O*C`Eh z^LuC{xk>qxDV;;_uk@vwqxb9_gppqBQ_>#^TazBOlqJCZ3wsQj*=gfhfeu_HTJ~L? z6x3k7lXh{@si=_fY%xivE+@z4hsO)prg- z#+?QyXnZYhwjqbDWd6q8yXPB2oxq~O;h{9a5<9p5$d|GS)v*S;PnIv+1yw;oca@b>&o zWsw9Wagxip6d9Wx{;ta$$R*SkTxu_IsXFiStDKzgZJpD)JkvZAbsken=@{3_;Y&*X zFVp!@m0V!T$Xm*kAPc2r^ByFh3MkNOMw(J2B6h31HgE_4^DlEMqmS-Aj$Qj^-&?u2 zKkv4KGf(*eAIsY4I~}~Qw7nP0Q%i#@+WBNQ`QGISHv73)UYtqFX`nL82j~~J6Juu9 z1y<^L*0Qwi$oOYZxW=50PYkZ-jxHrt>)^Y#`JELng845node}$PKxTxv?N^Wv_8Hn zf1p=nC8AW2ileSXRP65&6o%~=O-Blp!bvUT zadk_}u-*4^@+RaFY6~8zOFXJA>v6ufbrk>{17KS8#T3;+YUkJcr^DET&VX zA`_c+Hy9ag%&fjw^lOB55Qf=hJ6JTK_E9{r;Yr0~>tr1(k8SDQZkk#Ovu|$%*0y~5 z2w#by8|a2}TZy3M-y>-GU4k~Pk2fw7{7zb0u3W@zh?W*zpNqH_L=w9He!g|837#p7 zLWdF~HpGfT#8$XbE+^B{^1CvZyHQGm(m3g5WQwfoA369vnYoZms4d8JTOw0^S()>_ z3o;`ve-6w2OKW=56D?j?s`6e3zl7z7KkJrgoxf5_FK9hKvsCLA4DU(_g{3~q=#wgU zAqE)Q|^$=8ElgB+cV$5oZq`_e_qr13WU6Umi z$KB+(51qN=)`#ma%}0JbcD|*73rE8NW-{5+QeUcJBvIpFUg^Puh7}x1mz>L0`l%ct zBa)wmcTngq!J~w0IWfQ42xyuJ?K~TbFyP-UTyj|PM!8>WT~(W{kN19~bnIz=j`BTK zVG*?S^yI0ZrrBkjsgRMjPhLAH-m!z&DzgZdZN`ABRZAiuUPL=?xuekL1F2GjwU5y4M-E7~0{IH!H_p+3Waw@IJ#)ds~G=r;8&8TfkS!2L=&ILM-e)qox%&=?OmOy5 zKKK|&`&*!PXAF5X`d+KvTtRv$Q8^HMNy`wCYRtY9J>5);*7wQLWzRfhnfx*&KkK<2 zzb*V0Lw{z9dG%kH;DKB1x6WR0~@DSn>!-eg%YgxGt)r^A-~@jQ$NsK zqR$GGo^qPJh#1-_y1_pb!@5%D`?hkbU@SX(I==e%>JPoOF(Z}D=|bdU?~+MHa{LEF z=9SR5D&~BxAxCa=#Y1uqO}4?vJECOFhtj|zoBP(6WSm>hdzZ;%4x4zM%03pd&Xp>K zq(%88m7`uLk~c+Il-WH@>PN5(UG^wXkz+bgV48R>Dbx}dWUBcrCU#uhHeqekcu3&v z?kmBxqdj((RHrkQduM;ZJ9ebi5pc?{f?HC5I<@EhtKs7!GiRe4jL@x`Eqa}|z$CjT z%4d4bl9kSM#v}-sCP*0JkGhGc7@7$buQf4_ib=#O98J|r6Za0p>=!O*{oVa+G@kY> z;_#pmF(_zXn<^;g6K_{r49|5PlqBX0h=dpq{jxrL=T!wXm{(N#a^&_F&REAM#S1pI zwOQy<;@bo7b0_j&mHnKb_o_Vl#%wM|Y5h}&ry_SMNGL~8_hoS%ciaAF_Q-aVZ+AX7WDKU+q>lPM34uT{oJ*^j5CP_jG<(P32g!-%MP+ zotq*4j(wp2af{|y=T%J!v*?ckIAM`p;q~n%bcINJOdu`2{@HGL>w#e}^=vF!n_>G- z>xA%n_lEWzkFMA9lggi)I(gjcX=teWwafnfF!SZBrUG*QcB06K!!KAnU(~JkbT@Cl gq%i{^+WA2kj!p;V_y80k8cq}p9p~paw=k6WFP%g)9{>OV literal 0 HcmV?d00001 -- Gitee From c80790f7997bf77d7ac4b3d675347edb42cb8d6d Mon Sep 17 00:00:00 2001 From: d00807371 Date: Tue, 2 Jan 2024 20:59:27 +0800 Subject: [PATCH 13/17] add unit tests --- .../cpp/src/io/arrowadapter/OmniFileSystem.cc | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/omnioperator/omniop-spark-extension/cpp/src/io/arrowadapter/OmniFileSystem.cc b/omnioperator/omniop-spark-extension/cpp/src/io/arrowadapter/OmniFileSystem.cc index f3a5e22e4..5753ff31e 100644 --- a/omnioperator/omniop-spark-extension/cpp/src/io/arrowadapter/OmniFileSystem.cc +++ b/omnioperator/omniop-spark-extension/cpp/src/io/arrowadapter/OmniFileSystem.cc @@ -53,7 +53,7 @@ FileSystemFromUriReal(const UriInfo &uri, const arrow::io::IOContext &io_context if (scheme == "file") { std::string path; - ARROW_ASSIGN_OR_RAISE(auto options, buildLocalfsOptionsFromUri(uri, out_path)); + ARROW_ASSIGN_OR_RAISE(auto options, buildLocalfsOptionsFromUri(uri, &path)); if (out_path != nullptr) { *out_path = path; } -- Gitee From 877bf43671fa8b429398d3a43cc04a5b0318c66d Mon Sep 17 00:00:00 2001 From: d00807371 Date: Mon, 8 Jan 2024 14:58:53 +0800 Subject: [PATCH 14/17] =?UTF-8?q?fix=20bug=EF=BC=9A=20"in"=20expression=20?= =?UTF-8?q?supports=20null?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- .../boostkit/spark/jni/OrcColumnarBatchJniReader.java | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/omnioperator/omniop-spark-extension/java/src/main/java/com/huawei/boostkit/spark/jni/OrcColumnarBatchJniReader.java b/omnioperator/omniop-spark-extension/java/src/main/java/com/huawei/boostkit/spark/jni/OrcColumnarBatchJniReader.java index 94b7cd5ee..18f998b0f 100644 --- a/omnioperator/omniop-spark-extension/java/src/main/java/com/huawei/boostkit/spark/jni/OrcColumnarBatchJniReader.java +++ b/omnioperator/omniop-spark-extension/java/src/main/java/com/huawei/boostkit/spark/jni/OrcColumnarBatchJniReader.java @@ -104,8 +104,12 @@ public class OrcColumnarBatchJniReader { jsonObject.put("literal", ""); } if ((pl.getLiteralList() != null) && (pl.getLiteralList().size() != 0)){ - List lst = new ArrayList(); + List lst = new ArrayList<>(); for (Object ob : pl.getLiteralList()) { + if(ob == null){ + lst.add(""); + continue; + } if (pl.getType() == PredicateLeaf.Type.DECIMAL) { int decimalP = schema.findSubtype(pl.getColumnName()).getPrecision(); int decimalS = schema.findSubtype(pl.getColumnName()).getScale(); -- Gitee From 6b906aa1e59a88a5599ba37ee8b3669300235323 Mon Sep 17 00:00:00 2001 From: d00807371 Date: Mon, 8 Jan 2024 17:24:13 +0800 Subject: [PATCH 15/17] update uriInfo interface --- .../cpp/src/io/UriInfo.cc | 38 ++++++++----------- .../cpp/src/io/UriInfo.h | 25 ++++++------ .../cpp/src/io/arrowadapter/OmniHdfs.cc | 4 +- .../cpp/src/io/orcfile/OmniOrcHdfsFile.cc | 6 +-- .../cpp/src/jni/OrcColumnarBatchJniReader.cpp | 2 +- .../src/jni/ParquetColumnarBatchJniReader.cpp | 2 +- .../cpp/src/tablescan/ParquetReader.cpp | 3 +- .../io/arrowadapter/OmniFileSystemTest.cc | 18 ++++----- .../test/io/orcfile/OmniOrcHdfsFileTest.cc | 2 +- .../cpp/test/tablescan/parquet_scan_test.cpp | 2 +- 10 files changed, 46 insertions(+), 56 deletions(-) diff --git a/omnioperator/omniop-spark-extension/cpp/src/io/UriInfo.cc b/omnioperator/omniop-spark-extension/cpp/src/io/UriInfo.cc index b2724cb08..2c635ac64 100644 --- a/omnioperator/omniop-spark-extension/cpp/src/io/UriInfo.cc +++ b/omnioperator/omniop-spark-extension/cpp/src/io/UriInfo.cc @@ -22,47 +22,41 @@ #include "UriInfo.h" UriInfo::UriInfo(std::string _uri, std::string _scheme, std::string _path, std::string _host, - int32_t _port): host_string(std::move(_host)), - scheme_string(std::move(_scheme)), - port_number(_port), - port_string(std::to_string(port_number)), - path_string(std::move(_path)), - uri_string(std::move(_uri)) + std::string _port): host_string(std::move(_host)), + scheme_string(std::move(_scheme)), + port_string(std::move(_port)), + path_string(std::move(_path)), + uri_string(std::move(_uri)) { } UriInfo::UriInfo(std::string _scheme, std::string _path, std::string _host, - int32_t _port): host_string(std::move(_host)), - scheme_string(std::move(_scheme)), - port_number(_port), - port_string(std::to_string(port_number)), - path_string(std::move(_path)), - uri_string("Not initialize origin uri!") + std::string _port): host_string(std::move(_host)), + scheme_string(std::move(_scheme)), + port_string(std::move(_port)), + path_string(std::move(_path)), + uri_string("Not initialize origin uri!") { } UriInfo::~UriInfo() {} -const std::string& UriInfo::scheme() const { +const std::string UriInfo::scheme() const { return scheme_string; } -const std::string& UriInfo::host() const { +const std::string UriInfo::host() const { return host_string; } -const std::string& UriInfo::port_text() const { +const std::string UriInfo::port() const { return port_string; } -int32_t UriInfo::port() const { - return port_number; -} - -const std::string& UriInfo::path() const { +const std::string UriInfo::path() const { return path_string; } -const std::string &UriInfo::ToString() const { +const std::string UriInfo::ToString() const { return uri_string; -} +} \ No newline at end of file diff --git a/omnioperator/omniop-spark-extension/cpp/src/io/UriInfo.h b/omnioperator/omniop-spark-extension/cpp/src/io/UriInfo.h index ca9c39749..30ca0a456 100644 --- a/omnioperator/omniop-spark-extension/cpp/src/io/UriInfo.h +++ b/omnioperator/omniop-spark-extension/cpp/src/io/UriInfo.h @@ -16,42 +16,39 @@ * limitations under the License. */ -#pragma once +#ifndef URI_INFO_H +#define URI_INFO_H /// \brief A parsed URI class UriInfo { public: - UriInfo(std::string _uri, std::string _scheme, std::string _path, std::string _host, int32_t _port); + UriInfo(std::string _uri, std::string _scheme, std::string _path, std::string _host, std::string _port); - UriInfo(std::string _scheme, std::string _path, std::string _host, int32_t _port); + UriInfo(std::string _scheme, std::string _path, std::string _host, std::string _port); ~UriInfo(); - const std::string& scheme() const ; + const std::string scheme() const ; /// The URI host name, such as "localhost", "127.0.0.1" or "::1", or the empty /// string is the URI does not have a host component. - const std::string& host() const ; - - /// The URI port parsed as an integer, or -1 if the URI does not have a port - /// number component. - int32_t port() const ; + const std::string host() const ; /// The URI path component. - const std::string& path() const ; + const std::string path() const ; /// The URI port number, as a string such as "80", or the empty string is the URI /// does not have a port number component. - const std::string& port_text() const ; + const std::string port() const ; /// Get the string representation of this URI. - const std::string &ToString() const ; + const std::string ToString() const ; private: std::string host_string; std::string scheme_string; - int32_t port_number; std::string port_string; std::string path_string; std::string uri_string; -}; \ No newline at end of file +}; +#endif \ No newline at end of file diff --git a/omnioperator/omniop-spark-extension/cpp/src/io/arrowadapter/OmniHdfs.cc b/omnioperator/omniop-spark-extension/cpp/src/io/arrowadapter/OmniHdfs.cc index dfc18f2f8..851ff441b 100644 --- a/omnioperator/omniop-spark-extension/cpp/src/io/arrowadapter/OmniHdfs.cc +++ b/omnioperator/omniop-spark-extension/cpp/src/io/arrowadapter/OmniHdfs.cc @@ -37,8 +37,8 @@ Result buildHdfsOptionsFromUri(const UriInfo &uri){ host = uri.scheme() + "://" + uri.host(); // configure endpoint - const auto port = uri.port(); - if (port == -1) { + int32_t port; + if (uri.port().empty() || (port = atoi(uri.port().c_str())) == -1) { // default port will be determined by hdfs FileSystem impl options.ConfigureEndPoint(host, 0); } else { diff --git a/omnioperator/omniop-spark-extension/cpp/src/io/orcfile/OmniOrcHdfsFile.cc b/omnioperator/omniop-spark-extension/cpp/src/io/orcfile/OmniOrcHdfsFile.cc index 4fe2d75a2..2c7528022 100644 --- a/omnioperator/omniop-spark-extension/cpp/src/io/orcfile/OmniOrcHdfsFile.cc +++ b/omnioperator/omniop-spark-extension/cpp/src/io/orcfile/OmniOrcHdfsFile.cc @@ -22,8 +22,6 @@ namespace orc { - - class OmniHdfsFileInputStream : public InputStream { private: UriInfo uri; @@ -63,8 +61,8 @@ namespace orc { //Checking if the user supplied the host if(!uri.host().empty()){ - std::string port = !uri.port_text().empty() ? - uri.port_text() : ""; + std::string port = !uri.port().empty() ? + uri.port() : ""; status = file_system->Connect(uri.host(), port); if (!status.ok()) { throw ParseError("Can't connect to " + uri.host() diff --git a/omnioperator/omniop-spark-extension/cpp/src/jni/OrcColumnarBatchJniReader.cpp b/omnioperator/omniop-spark-extension/cpp/src/jni/OrcColumnarBatchJniReader.cpp index 221a255ac..8876f2ec5 100644 --- a/omnioperator/omniop-spark-extension/cpp/src/jni/OrcColumnarBatchJniReader.cpp +++ b/omnioperator/omniop-spark-extension/cpp/src/jni/OrcColumnarBatchJniReader.cpp @@ -68,7 +68,7 @@ JNIEXPORT jlong JNICALL Java_com_huawei_boostkit_spark_jni_OrcColumnarBatchJniRe env->ReleaseStringUTFChars(hostJstr, hostPtr); jint port = (jint)env->CallIntMethod(jsonObj, jsonMethodInt, env->NewStringUTF("port")); - UriInfo uri(schemaStr, fileStr, hostStr, port); + UriInfo uri(schemaStr, fileStr, hostStr, std::to_string(port)); std::unique_ptr reader; reader = createReader(orc::readOmniFile(uri), readerOptions); diff --git a/omnioperator/omniop-spark-extension/cpp/src/jni/ParquetColumnarBatchJniReader.cpp b/omnioperator/omniop-spark-extension/cpp/src/jni/ParquetColumnarBatchJniReader.cpp index b02254630..682193168 100644 --- a/omnioperator/omniop-spark-extension/cpp/src/jni/ParquetColumnarBatchJniReader.cpp +++ b/omnioperator/omniop-spark-extension/cpp/src/jni/ParquetColumnarBatchJniReader.cpp @@ -75,7 +75,7 @@ JNIEXPORT jlong JNICALL Java_com_huawei_boostkit_spark_jni_ParquetColumnarBatchJ jint port = (jint)env->CallIntMethod(jsonObj, jsonMethodInt, env->NewStringUTF("port")); - UriInfo uriInfo(uriString, schemeString, pathString, hostString, port); + UriInfo uriInfo(uriString, schemeString, pathString, hostString, std::to_string(port)); // Get capacity for each record batch int64_t capacity = (int64_t)env->CallLongMethod(jsonObj, jsonMethodLong, env->NewStringUTF("capacity")); diff --git a/omnioperator/omniop-spark-extension/cpp/src/tablescan/ParquetReader.cpp b/omnioperator/omniop-spark-extension/cpp/src/tablescan/ParquetReader.cpp index 359345c7e..9d94e2a2b 100644 --- a/omnioperator/omniop-spark-extension/cpp/src/tablescan/ParquetReader.cpp +++ b/omnioperator/omniop-spark-extension/cpp/src/tablescan/ParquetReader.cpp @@ -74,7 +74,8 @@ std::string spark::reader::GetFileSystemKey(std::string& path, std::string& ugi) Filesystem* spark::reader::GetFileSystemPtr(UriInfo &uri, std::string& ugi, arrow::Status &status) { - auto key = GetFileSystemKey(const_cast(uri.ToString()), ugi); + std::string fullPath = uri.ToString(); + auto key = GetFileSystemKey(fullPath, ugi); // if not find key, create the filesystem ptr auto iter = restore_filesysptr.find(key); diff --git a/omnioperator/omniop-spark-extension/cpp/test/io/arrowadapter/OmniFileSystemTest.cc b/omnioperator/omniop-spark-extension/cpp/test/io/arrowadapter/OmniFileSystemTest.cc index cf10316b7..7d7fbbf08 100644 --- a/omnioperator/omniop-spark-extension/cpp/test/io/arrowadapter/OmniFileSystemTest.cc +++ b/omnioperator/omniop-spark-extension/cpp/test/io/arrowadapter/OmniFileSystemTest.cc @@ -67,17 +67,17 @@ protected: TEST_F(TestMockFS, FileSystemFromUriOrPath) { std::string path; - UriInfo uri1("mock", "", "", -1); + UriInfo uri1("mock", "", "", "-1"); ASSERT_OK_AND_ASSIGN(fs_, FileSystemFromUriOrPath(uri1, &path)); ASSERT_EQ(path, ""); CheckDirs({}); // Ensures it's a MockFileSystem - UriInfo uri2("mock", "foo/bar", "", -1); + UriInfo uri2("mock", "foo/bar", "", "-1"); ASSERT_OK_AND_ASSIGN(fs_, FileSystemFromUriOrPath(uri2, &path)); ASSERT_EQ(path, "foo/bar"); CheckDirs({}); - UriInfo ur3("mock", "/foo/bar", "", -1); + UriInfo ur3("mock", "/foo/bar", "", "-1"); ASSERT_OK_AND_ASSIGN(fs_, FileSystemFromUriOrPath(ur3, &path)); ASSERT_EQ(path, "foo/bar"); CheckDirs({}); @@ -174,27 +174,27 @@ TYPED_TEST_SUITE(TestLocalFs, PathFormatters); TYPED_TEST(TestLocalFs, FileSystemFromUriFile){ std::string path; ASSERT_OK_AND_ASSIGN(auto uri_string, arrow::internal::UriFromAbsolutePath(this->local_path_)); - UriInfo uri1(uri_string, "", uri_string, "", -1); + UriInfo uri1(uri_string, "", uri_string, "", "-1"); this->TestFileSystemFromUri(uri1); path = "/foo/bar"; - UriInfo uri2("file", path, "", -1); + UriInfo uri2("file", path, "", "-1"); this->TestLocalUri(uri2, path); path = "/some path/%percent"; - UriInfo uri3("file", path, "", -1); + UriInfo uri3("file", path, "", "-1"); this->TestLocalUri(uri3, path); path = "/some path/%中文魑魅魍魉"; - UriInfo uri4("file", path, "", -1); + UriInfo uri4("file", path, "", "-1"); this->TestLocalUri(uri4, path); } TYPED_TEST(TestLocalFs, FileSystemFromUriNoScheme){ - UriInfo uri1(this->local_path_, "", "", "", -1); + UriInfo uri1(this->local_path_, "", "", "", "-1"); this->TestFileSystemFromUri(uri1); - UriInfo uri2("foo/bar", "", "", "", -1); + UriInfo uri2("foo/bar", "", "", "", "-1"); this->TestInvalidUri(uri2); } diff --git a/omnioperator/omniop-spark-extension/cpp/test/io/orcfile/OmniOrcHdfsFileTest.cc b/omnioperator/omniop-spark-extension/cpp/test/io/orcfile/OmniOrcHdfsFileTest.cc index f8771f848..63119058a 100644 --- a/omnioperator/omniop-spark-extension/cpp/test/io/orcfile/OmniOrcHdfsFileTest.cc +++ b/omnioperator/omniop-spark-extension/cpp/test/io/orcfile/OmniOrcHdfsFileTest.cc @@ -34,7 +34,7 @@ TEST(OrcReader, createLocalFileReader) { cols.push_back(1); rowReaderOpts.include(cols); - UriInfo uriInfo("file", filename, "", -1); + UriInfo uriInfo("file", filename, "", ""); reader = orc::createReader(orc::readOmniFile(uriInfo), readerOpts); EXPECT_NE(nullptr, reader); } diff --git a/omnioperator/omniop-spark-extension/cpp/test/tablescan/parquet_scan_test.cpp b/omnioperator/omniop-spark-extension/cpp/test/tablescan/parquet_scan_test.cpp index cf1e045be..4f4a3d788 100644 --- a/omnioperator/omniop-spark-extension/cpp/test/tablescan/parquet_scan_test.cpp +++ b/omnioperator/omniop-spark-extension/cpp/test/tablescan/parquet_scan_test.cpp @@ -45,7 +45,7 @@ TEST(read, test_parquet_reader) ParquetReader *reader = new ParquetReader(); std::string ugi = "root@sample"; - UriInfo uri(filename, "", "", "", -1); + UriInfo uri(filename, "", "", "", "-1"); auto state1 = reader->InitRecordReader(uri, 1024, row_group_indices, column_indices, ugi); ASSERT_EQ(state1, Status::OK()); -- Gitee From ed9cef35fdf2afbe525bde849b96469c80e98a46 Mon Sep 17 00:00:00 2001 From: d00807371 Date: Mon, 8 Jan 2024 17:30:26 +0800 Subject: [PATCH 16/17] modify file name --- omnioperator/omniop-spark-extension/cpp/src/CMakeLists.txt | 6 +++--- .../{OmniFileSystem.cc => FileSystemAdapter.cc} | 6 +++--- .../arrowadapter/{OmniFileSystem.h => FileSystemAdapter.h} | 0 .../cpp/src/io/arrowadapter/{OmniHdfs.cc => HdfsAdapter.cc} | 2 +- .../cpp/src/io/arrowadapter/{OmniHdfs.h => HdfsAdapter.h} | 0 .../io/arrowadapter/{OmniLocalfs.cc => LocalfsAdapter.cc} | 2 +- .../src/io/arrowadapter/{OmniLocalfs.h => LocalfsAdapter.h} | 0 .../cpp/src/tablescan/ParquetReader.cpp | 2 +- .../cpp/test/io/arrowadapter/OmniFileSystemTest.cc | 2 +- 9 files changed, 10 insertions(+), 10 deletions(-) rename omnioperator/omniop-spark-extension/cpp/src/io/arrowadapter/{OmniFileSystem.cc => FileSystemAdapter.cc} (97%) rename omnioperator/omniop-spark-extension/cpp/src/io/arrowadapter/{OmniFileSystem.h => FileSystemAdapter.h} (100%) rename omnioperator/omniop-spark-extension/cpp/src/io/arrowadapter/{OmniHdfs.cc => HdfsAdapter.cc} (98%) rename omnioperator/omniop-spark-extension/cpp/src/io/arrowadapter/{OmniHdfs.h => HdfsAdapter.h} (100%) rename omnioperator/omniop-spark-extension/cpp/src/io/arrowadapter/{OmniLocalfs.cc => LocalfsAdapter.cc} (98%) rename omnioperator/omniop-spark-extension/cpp/src/io/arrowadapter/{OmniLocalfs.h => LocalfsAdapter.h} (100%) diff --git a/omnioperator/omniop-spark-extension/cpp/src/CMakeLists.txt b/omnioperator/omniop-spark-extension/cpp/src/CMakeLists.txt index bf96ffb38..455723aa1 100644 --- a/omnioperator/omniop-spark-extension/cpp/src/CMakeLists.txt +++ b/omnioperator/omniop-spark-extension/cpp/src/CMakeLists.txt @@ -19,10 +19,10 @@ set (SOURCE_FILES jni/jni_common.cpp jni/ParquetColumnarBatchJniReader.cpp tablescan/ParquetReader.cpp - io/arrowadapter/OmniFileSystem.cc + io/arrowadapter/FileSystemAdapter.cc io/arrowadapter/UtilInternal.cc - io/arrowadapter/OmniHdfs.cc - io/arrowadapter/OmniLocalfs.cc + io/arrowadapter/HdfsAdapter.cc + io/arrowadapter/LocalfsAdapter.cc io/UriInfo.cc ) diff --git a/omnioperator/omniop-spark-extension/cpp/src/io/arrowadapter/OmniFileSystem.cc b/omnioperator/omniop-spark-extension/cpp/src/io/arrowadapter/FileSystemAdapter.cc similarity index 97% rename from omnioperator/omniop-spark-extension/cpp/src/io/arrowadapter/OmniFileSystem.cc rename to omnioperator/omniop-spark-extension/cpp/src/io/arrowadapter/FileSystemAdapter.cc index 5753ff31e..6ea2a8725 100644 --- a/omnioperator/omniop-spark-extension/cpp/src/io/arrowadapter/OmniFileSystem.cc +++ b/omnioperator/omniop-spark-extension/cpp/src/io/arrowadapter/FileSystemAdapter.cc @@ -19,10 +19,10 @@ #include #include -#include "OmniFileSystem.h" +#include "FileSystemAdapter.h" #include "arrow/filesystem/hdfs.h" -#include "OmniHdfs.h" -#include "OmniLocalfs.h" +#include "HdfsAdapter.h" +#include "LocalfsAdapter.h" #include "arrow/filesystem/localfs.h" #include "arrow/filesystem/mockfs.h" #include "arrow/filesystem/path_util.h" diff --git a/omnioperator/omniop-spark-extension/cpp/src/io/arrowadapter/OmniFileSystem.h b/omnioperator/omniop-spark-extension/cpp/src/io/arrowadapter/FileSystemAdapter.h similarity index 100% rename from omnioperator/omniop-spark-extension/cpp/src/io/arrowadapter/OmniFileSystem.h rename to omnioperator/omniop-spark-extension/cpp/src/io/arrowadapter/FileSystemAdapter.h diff --git a/omnioperator/omniop-spark-extension/cpp/src/io/arrowadapter/OmniHdfs.cc b/omnioperator/omniop-spark-extension/cpp/src/io/arrowadapter/HdfsAdapter.cc similarity index 98% rename from omnioperator/omniop-spark-extension/cpp/src/io/arrowadapter/OmniHdfs.cc rename to omnioperator/omniop-spark-extension/cpp/src/io/arrowadapter/HdfsAdapter.cc index 851ff441b..efcf2a754 100644 --- a/omnioperator/omniop-spark-extension/cpp/src/io/arrowadapter/OmniHdfs.cc +++ b/omnioperator/omniop-spark-extension/cpp/src/io/arrowadapter/HdfsAdapter.cc @@ -21,7 +21,7 @@ #include "arrow/filesystem/hdfs.h" #include "arrow/util/value_parsing.h" -#include "OmniHdfs.h" +#include "HdfsAdapter.h" namespace arrow_adapter { diff --git a/omnioperator/omniop-spark-extension/cpp/src/io/arrowadapter/OmniHdfs.h b/omnioperator/omniop-spark-extension/cpp/src/io/arrowadapter/HdfsAdapter.h similarity index 100% rename from omnioperator/omniop-spark-extension/cpp/src/io/arrowadapter/OmniHdfs.h rename to omnioperator/omniop-spark-extension/cpp/src/io/arrowadapter/HdfsAdapter.h diff --git a/omnioperator/omniop-spark-extension/cpp/src/io/arrowadapter/OmniLocalfs.cc b/omnioperator/omniop-spark-extension/cpp/src/io/arrowadapter/LocalfsAdapter.cc similarity index 98% rename from omnioperator/omniop-spark-extension/cpp/src/io/arrowadapter/OmniLocalfs.cc rename to omnioperator/omniop-spark-extension/cpp/src/io/arrowadapter/LocalfsAdapter.cc index 71d8c6ab0..97b91b22a 100644 --- a/omnioperator/omniop-spark-extension/cpp/src/io/arrowadapter/OmniLocalfs.cc +++ b/omnioperator/omniop-spark-extension/cpp/src/io/arrowadapter/LocalfsAdapter.cc @@ -21,7 +21,7 @@ #include #include "arrow/filesystem/localfs.h" #include "arrow/util/io_util.h" -#include "OmniLocalfs.h" +#include "LocalfsAdapter.h" #include "arrow/result.h" namespace arrow_adapter { diff --git a/omnioperator/omniop-spark-extension/cpp/src/io/arrowadapter/OmniLocalfs.h b/omnioperator/omniop-spark-extension/cpp/src/io/arrowadapter/LocalfsAdapter.h similarity index 100% rename from omnioperator/omniop-spark-extension/cpp/src/io/arrowadapter/OmniLocalfs.h rename to omnioperator/omniop-spark-extension/cpp/src/io/arrowadapter/LocalfsAdapter.h diff --git a/omnioperator/omniop-spark-extension/cpp/src/tablescan/ParquetReader.cpp b/omnioperator/omniop-spark-extension/cpp/src/tablescan/ParquetReader.cpp index 9d94e2a2b..21e50b5ca 100644 --- a/omnioperator/omniop-spark-extension/cpp/src/tablescan/ParquetReader.cpp +++ b/omnioperator/omniop-spark-extension/cpp/src/tablescan/ParquetReader.cpp @@ -19,7 +19,7 @@ #include #include -#include "io/arrowadapter/OmniFileSystem.h" +#include "io/arrowadapter/FileSystemAdapter.h" #include #include "io/UriInfo.h" #include "jni/jni_common.h" diff --git a/omnioperator/omniop-spark-extension/cpp/test/io/arrowadapter/OmniFileSystemTest.cc b/omnioperator/omniop-spark-extension/cpp/test/io/arrowadapter/OmniFileSystemTest.cc index 7d7fbbf08..0df553a52 100644 --- a/omnioperator/omniop-spark-extension/cpp/test/io/arrowadapter/OmniFileSystemTest.cc +++ b/omnioperator/omniop-spark-extension/cpp/test/io/arrowadapter/OmniFileSystemTest.cc @@ -20,7 +20,7 @@ #include #include "gtest/gtest.h" -#include "io/arrowadapter/OmniFileSystem.h" +#include "io/arrowadapter/FileSystemAdapter.h" #include "arrow/filesystem/filesystem.h" #include "arrow/filesystem/mockfs.h" #include "arrow/util/checked_cast.h" -- Gitee From b913efc1637d2c01c0252ff497514c11e5da54c3 Mon Sep 17 00:00:00 2001 From: d00807371 Date: Mon, 8 Jan 2024 17:45:58 +0800 Subject: [PATCH 17/17] =?UTF-8?q?Revert=20"fix=20bug=EF=BC=9A=20"in"=20exp?= =?UTF-8?q?ression=20supports=20null"?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit This reverts commit 877bf43671fa8b429398d3a43cc04a5b0318c66d. --- .../boostkit/spark/jni/OrcColumnarBatchJniReader.java | 6 +----- 1 file changed, 1 insertion(+), 5 deletions(-) diff --git a/omnioperator/omniop-spark-extension/java/src/main/java/com/huawei/boostkit/spark/jni/OrcColumnarBatchJniReader.java b/omnioperator/omniop-spark-extension/java/src/main/java/com/huawei/boostkit/spark/jni/OrcColumnarBatchJniReader.java index 18f998b0f..94b7cd5ee 100644 --- a/omnioperator/omniop-spark-extension/java/src/main/java/com/huawei/boostkit/spark/jni/OrcColumnarBatchJniReader.java +++ b/omnioperator/omniop-spark-extension/java/src/main/java/com/huawei/boostkit/spark/jni/OrcColumnarBatchJniReader.java @@ -104,12 +104,8 @@ public class OrcColumnarBatchJniReader { jsonObject.put("literal", ""); } if ((pl.getLiteralList() != null) && (pl.getLiteralList().size() != 0)){ - List lst = new ArrayList<>(); + List lst = new ArrayList(); for (Object ob : pl.getLiteralList()) { - if(ob == null){ - lst.add(""); - continue; - } if (pl.getType() == PredicateLeaf.Type.DECIMAL) { int decimalP = schema.findSubtype(pl.getColumnName()).getPrecision(); int decimalS = schema.findSubtype(pl.getColumnName()).getScale(); -- Gitee