diff --git a/.github/workflows/flink-cdc-test.yml b/.github/workflows/flink-cdc-test.yml
index fce2dfea6..7f05d69f0 100644
--- a/.github/workflows/flink-cdc-test.yml
+++ b/.github/workflows/flink-cdc-test.yml
@@ -61,6 +61,9 @@ jobs:
- uses: Swatinem/rust-cache@v2
with:
workspaces: "./native-io -> target"
+ - uses: Swatinem/rust-cache@v2
+ with:
+ workspaces: "./native-metadata -> target"
- name: Pull images
run: |
docker pull -q bitnami/spark:3.3.1
@@ -69,10 +72,17 @@ jobs:
use-cross: true
command: build
args: '--manifest-path native-io/Cargo.toml --target x86_64-unknown-linux-gnu --release --all-features'
+ - uses: actions-rs/cargo@v1
+ with:
+ use-cross: true
+ command: build
+ args: '--manifest-path native-metadata/Cargo.toml --target x86_64-unknown-linux-gnu --release --all-features'
- name: Build with Maven
run: |
mkdir -p native-io/target/release
cp native-io/target/x86_64-unknown-linux-gnu/release/liblakesoul_io_c.so native-io/target/release
+ mkdir -p native-metadata/target/release
+ cp native-metadata/target/x86_64-unknown-linux-gnu/release/liblakesoul_metadata_c.so native-metadata/target/release
MAVEN_OPTS="-Xmx4000m" mvn -q -B package -f pom.xml -Pcross-build -DskipTests
- name: Get jar names
run: |
diff --git a/.github/workflows/maven-test.yml b/.github/workflows/maven-test.yml
index af8d4404e..b70089324 100644
--- a/.github/workflows/maven-test.yml
+++ b/.github/workflows/maven-test.yml
@@ -26,7 +26,35 @@ on:
workflow_dispatch:
jobs:
- build-linux-x86_64:
+ build-native-metadata-linux-x86_64:
+ runs-on: ubuntu-latest
+ steps:
+ - uses: actions/checkout@v3
+ - name: Set up JDK 8
+ uses: actions/setup-java@v3
+ with:
+ java-version: '8'
+ distribution: 'temurin'
+ cache: maven
+ - uses: actions-rs/toolchain@v1
+ with:
+ profile: minimal
+ toolchain: nightly-2023-05-20
+ default: true
+ - uses: Swatinem/rust-cache@v2
+ with:
+ workspaces: "./native-metadata -> target"
+ - uses: actions-rs/cargo@v1
+ with:
+ use-cross: true
+ command: build
+ args: '--manifest-path native-metadata/Cargo.toml --target x86_64-unknown-linux-gnu --release --all-features'
+ - uses: actions/upload-artifact@master
+ with:
+ name: lakesoul-nativemetadata-x86_64-unknown-linux-gnu-maven-test
+ path: ./native-metadata/target/x86_64-unknown-linux-gnu/release/liblakesoul_metadata_c.so
+
+ build-native-io-linux-x86_64:
runs-on: ubuntu-latest
steps:
- uses: actions/checkout@v3
@@ -56,7 +84,7 @@ jobs:
spark-test-1:
runs-on: ubuntu-latest
- needs: [ build-linux-x86_64 ]
+ needs: [ build-native-io-linux-x86_64, build-native-metadata-linux-x86_64 ]
services:
# Label used to access the service container
@@ -96,6 +124,10 @@ jobs:
uses: arduino/setup-protoc@v2
with:
version: "23.x"
+ - uses: actions/download-artifact@v3
+ with:
+ name: lakesoul-nativemetadata-x86_64-unknown-linux-gnu-maven-test
+ path: ./native-metadata/target/release/
- uses: actions/download-artifact@v3
with:
name: lakesoul-nativeio-x86_64-unknown-linux-gnu-maven-test
@@ -119,7 +151,7 @@ jobs:
spark-test-2:
runs-on: ubuntu-latest
- needs: [ build-linux-x86_64 ]
+ needs: [ build-native-io-linux-x86_64, build-native-metadata-linux-x86_64 ]
services:
# Label used to access the service container
@@ -159,6 +191,10 @@ jobs:
uses: arduino/setup-protoc@v2
with:
version: "23.x"
+ - uses: actions/download-artifact@v3
+ with:
+ name: lakesoul-nativemetadata-x86_64-unknown-linux-gnu-maven-test
+ path: ./native-metadata/target/release/
- uses: actions/download-artifact@v3
with:
name: lakesoul-nativeio-x86_64-unknown-linux-gnu-maven-test
@@ -182,7 +218,7 @@ jobs:
spark-test-rbac:
runs-on: ubuntu-latest
- needs: [ build-linux-x86_64 ]
+ needs: [ build-native-io-linux-x86_64, build-native-metadata-linux-x86_64 ]
services:
# Label used to access the service container
@@ -225,6 +261,10 @@ jobs:
uses: arduino/setup-protoc@v2
with:
version: "23.x"
+ - uses: actions/download-artifact@v3
+ with:
+ name: lakesoul-nativemetadata-x86_64-unknown-linux-gnu-maven-test
+ path: ./native-metadata/target/release/
- uses: actions/download-artifact@v3
with:
name: lakesoul-nativeio-x86_64-unknown-linux-gnu-maven-test
@@ -248,7 +288,7 @@ jobs:
flink-test-1:
runs-on: ubuntu-latest
- needs: [ build-linux-x86_64 ]
+ needs: [ build-native-io-linux-x86_64, build-native-metadata-linux-x86_64 ]
services:
# Label used to access the service container
@@ -288,6 +328,10 @@ jobs:
uses: arduino/setup-protoc@v2
with:
version: "23.x"
+ - uses: actions/download-artifact@v3
+ with:
+ name: lakesoul-nativemetadata-x86_64-unknown-linux-gnu-maven-test
+ path: ./native-metadata/target/release/
- uses: actions/download-artifact@v3
with:
name: lakesoul-nativeio-x86_64-unknown-linux-gnu-maven-test
@@ -311,7 +355,7 @@ jobs:
flink-test-rbac:
runs-on: ubuntu-latest
- needs: [ build-linux-x86_64 ]
+ needs: [ build-native-io-linux-x86_64, build-native-metadata-linux-x86_64 ]
services:
# Label used to access the service container
@@ -362,6 +406,10 @@ jobs:
uses: arduino/setup-protoc@v2
with:
version: "23.x"
+ - uses: actions/download-artifact@v3
+ with:
+ name: lakesoul-nativemetadata-x86_64-unknown-linux-gnu-maven-test
+ path: ./native-metadata/target/release/
- uses: actions/download-artifact@v3
with:
name: lakesoul-nativeio-x86_64-unknown-linux-gnu-maven-test
diff --git a/.github/workflows/native-build.yml b/.github/workflows/native-build.yml
index ee480e2be..fd2f75245 100644
--- a/.github/workflows/native-build.yml
+++ b/.github/workflows/native-build.yml
@@ -23,7 +23,7 @@ on:
workflow_dispatch:
jobs:
- build-linux-x86_64:
+ build-native-io-linux-x86_64:
runs-on: ubuntu-latest
steps:
- uses: actions/checkout@v3
@@ -51,7 +51,7 @@ jobs:
name: lakesoul-nativeio-x86_64-unknown-linux-gnu
path: ./native-io/target/x86_64-unknown-linux-gnu/release/liblakesoul_io_c.so
- build-windows-x86_64:
+ build-native-io-windows-x86_64:
runs-on: windows-latest
steps:
- uses: actions/checkout@v3
@@ -78,7 +78,7 @@ jobs:
name: lakesoul-nativeio-x86_64-pc-windows-msvc
path: ./native-io/target/release/lakesoul_io_c.dll
- build-macos-x86_64:
+ build-native-io-macos-x86_64:
runs-on: macos-latest
steps:
- name: Install automake
@@ -107,9 +107,101 @@ jobs:
name: lakesoul-nativeio-x86_64-apple-darwin
path: ./native-io/target/release/liblakesoul_io_c.dylib
+ build-native-metadata-linux-x86_64:
+ runs-on: ubuntu-latest
+ steps:
+ - uses: actions/checkout@v3
+ - name: Set up JDK 8
+ uses: actions/setup-java@v3
+ with:
+ java-version: '8'
+ distribution: 'temurin'
+ cache: maven
+ - uses: actions-rs/toolchain@v1
+ with:
+ profile: minimal
+ toolchain: nightly-2023-05-20
+ default: true
+ - uses: Swatinem/rust-cache@v2
+ with:
+ workspaces: "./native-metadata -> target"
+ - uses: actions-rs/cargo@v1
+ with:
+ use-cross: true
+ command: build
+ args: '--manifest-path native-metadata/Cargo.toml --target x86_64-unknown-linux-gnu --release --all-features'
+ - uses: actions/upload-artifact@master
+ with:
+ name: lakesoul-nativemetadata-x86_64-unknown-linux-gnu
+ path: ./native-metadata/target/x86_64-unknown-linux-gnu/release/liblakesoul_metadata_c.so
+
+ build-native-metadata-windows-x86_64:
+ runs-on: windows-latest
+ steps:
+ - uses: actions/checkout@v3
+ - name: Set up JDK 8
+ uses: actions/setup-java@v3
+ with:
+ java-version: '8'
+ distribution: 'temurin'
+ cache: maven
+ - name: Install Protoc
+ uses: arduino/setup-protoc@v2
+ with:
+ version: "23.x"
+ - uses: actions-rs/toolchain@v1
+ with:
+ profile: minimal
+ toolchain: nightly-2023-05-20
+ default: true
+ - uses: Swatinem/rust-cache@v2
+ with:
+ workspaces: "./native-metadata -> target"
+ - uses: actions-rs/cargo@v1
+ with:
+ command: build
+ args: '--manifest-path native-metadata/Cargo.toml --release --all-features'
+ - uses: actions/upload-artifact@master
+ with:
+ name: lakesoul-nativemetadata-x86_64-pc-windows-msvc
+ path: ./native-metadata/target/release/lakesoul_metadata_c.dll
+
+ build-native-metadata-macos-x86_64:
+ runs-on: macos-latest
+ steps:
+ - name: Install automake
+ run: brew install automake
+ - uses: actions/checkout@v3
+ - name: Set up JDK 8
+ uses: actions/setup-java@v3
+ with:
+ java-version: '8'
+ distribution: 'temurin'
+ cache: maven
+ - name: Install Protoc
+ uses: arduino/setup-protoc@v2
+ with:
+ version: "23.x"
+ - uses: actions-rs/toolchain@v1
+ with:
+ profile: minimal
+ toolchain: nightly-2023-05-20
+ default: true
+ - uses: Swatinem/rust-cache@v2
+ with:
+ workspaces: "./native-metadata -> target"
+ - uses: actions-rs/cargo@v1
+ with:
+ command: build
+ args: '--manifest-path native-metadata/Cargo.toml --release --all-features'
+ - uses: actions/upload-artifact@master
+ with:
+ name: lakesoul-nativemetadata-x86_64-apple-darwin
+ path: ./native-metadata/target/release/liblakesoul_metadata_c.dylib
+
build-maven-package:
runs-on: ubuntu-latest
- needs: [ build-linux-x86_64, build-windows-x86_64, build-macos-x86_64 ]
+ needs: [ build-native-io-linux-x86_64, build-native-io-windows-x86_64, build-native-io-macos-x86_64, build-native-metadata-linux-x86_64, build-native-metadata-windows-x86_64, build-native-metadata-macos-x86_64 ]
steps:
- uses: actions/checkout@v3
- uses: actions/download-artifact@v3
@@ -124,6 +216,18 @@ jobs:
with:
name: lakesoul-nativeio-x86_64-pc-windows-msvc
path: ./native-io/target/release/
+ - uses: actions/download-artifact@v3
+ with:
+ name: lakesoul-nativemetadata-x86_64-unknown-linux-gnu
+ path: ./native-metadata/target/release/
+ - uses: actions/download-artifact@v3
+ with:
+ name: lakesoul-nativemetadata-x86_64-apple-darwin
+ path: ./native-metadata/target/release/
+ - uses: actions/download-artifact@v3
+ with:
+ name: lakesoul-nativemetadata-x86_64-pc-windows-msvc
+ path: ./native-metadata/target/release/
- name: Set up JDK 8
uses: actions/setup-java@v3
with:
diff --git a/.github/workflows/rust-clippy.yml b/.github/workflows/rust-clippy.yml
index caa104ec4..9bf143754 100644
--- a/.github/workflows/rust-clippy.yml
+++ b/.github/workflows/rust-clippy.yml
@@ -33,9 +33,15 @@ jobs:
toolchain: nightly-2023-05-20
components: clippy
default: true
+ - name: Install Protoc
+ uses: arduino/setup-protoc@v2
+ with:
+ version: "23.x"
- uses: Swatinem/rust-cache@v2
with:
workspaces: "./native-io -> target"
key: "ubuntu-latest-clippy"
- - name: Run Clippy
+ - name: Run Clippy Native-io
run: cd native-io && cargo clippy --all-features --package lakesoul-io-c
+ - name: Run Clippy Native-metadata
+ run: cd native-metadata && cargo clippy --all-features --package lakesoul-metadata-c
diff --git a/.gitignore b/.gitignore
index 152e5143f..3135709fb 100644
--- a/.gitignore
+++ b/.gitignore
@@ -17,7 +17,6 @@ dependency-reduced-pom.xml
/lakesoul-spark/spark-warehouse/
dependency-reduced-pom.xml
/lakesoul-spark/spark-warehouse/
-lakesoul-common/src/main/java/com/dmetasoul/lakesoul/meta/entity/
/script/benchmark/work-dir/*.jar
*.DS_Store
native-io/lakesoul-io-c/lakesoul_c_bindings.h
diff --git a/lakesoul-common/pom.xml b/lakesoul-common/pom.xml
index df3f7527d..5237075a6 100644
--- a/lakesoul-common/pom.xml
+++ b/lakesoul-common/pom.xml
@@ -61,7 +61,7 @@ SPDX-License-Identifier: Apache-2.0
false
false
${basedir}/../native-metadata/proto/src
- ${basedir}/src/main/java
+ ${basedir}/target/generated-sources
@@ -140,12 +140,193 @@ SPDX-License-Identifier: Apache-2.0
compile
+
com.google.protobuf
protobuf-java
3.22.0
+
+
+
+ com.github.jnr
+ jnr-ffi
+ 2.2.13
+
+
+ com.github.jnr
+ jffi
+
+
+
+
+
+
+ com.github.jnr
+ jffi
+ 1.3.11
+
+
+ com.github.jnr
+ jffi
+ 1.3.11
+ runtime
+ native
+
+
+
+
+ native-dev
+
+ true
+
+
+
+
+ ${basedir}/../native-metadata/target/release/
+
+ *.dylib
+ *.so
+ *.dll
+
+
+ deps
+ .fingerprint
+ build
+ examples
+ *.d
+
+
+
+
+
+ org.codehaus.mojo
+ exec-maven-plugin
+ 3.1.0
+
+
+ compile-native
+ generate-resources
+
+ exec
+
+
+ cargo
+ build --release --all-features --package lakesoul-metadata-c
+
+ ${basedir}/../native-metadata
+
+
+
+ clean-native
+ clean
+
+ exec
+
+
+ cargo
+
+ clean
+
+ ${basedir}/../native-metadata/
+
+
+
+
+
+
+
+
+
+ release-linux-x86-64
+
+ false
+
+
+
+
+ ${basedir}/../native-metadata/target/x86_64-unknown-linux-gnu/release/
+
+ *.so
+
+
+ deps
+ .fingerprint
+ build
+ examples
+ *.d
+
+
+
+
+
+ org.codehaus.mojo
+ exec-maven-plugin
+ 3.1.0
+
+
+ compile-native
+ generate-resources
+
+ exec
+
+
+ cross
+ build --target x86_64-unknown-linux-gnu --release --all-features
+ --package lakesoul-metadata-c
+
+ ${basedir}/../native-metadata/
+
+
+
+ clean-native
+ clean
+
+ exec
+
+
+ cargo
+
+ clean
+
+ ${basedir}/../native-metadata/
+
+
+
+
+
+
+
+
+
+ cross-build
+
+ false
+
+
+
+
+ ${basedir}/../native-metadata/target/release/
+
+ *.dylib
+ *.so
+ *.dll
+
+
+ deps
+ .fingerprint
+ build
+ examples
+ *.d
+
+
+
+
+
+
+
+
\ No newline at end of file
diff --git a/lakesoul-common/src/main/java/com/dmetasoul/lakesoul/meta/DBManager.java b/lakesoul-common/src/main/java/com/dmetasoul/lakesoul/meta/DBManager.java
index f6bba9478..94e8feaf5 100644
--- a/lakesoul-common/src/main/java/com/dmetasoul/lakesoul/meta/DBManager.java
+++ b/lakesoul-common/src/main/java/com/dmetasoul/lakesoul/meta/DBManager.java
@@ -8,6 +8,8 @@
import com.alibaba.fastjson.JSONObject;
import com.dmetasoul.lakesoul.meta.dao.*;
import com.dmetasoul.lakesoul.meta.entity.*;
+import com.dmetasoul.lakesoul.meta.jnr.NativeMetadataJavaClient;
+import com.dmetasoul.lakesoul.meta.jnr.NativeUtils;
import com.dmetasoul.lakesoul.meta.rbac.AuthZContext;
import com.dmetasoul.lakesoul.meta.rbac.AuthZEnforcer;
import org.apache.commons.lang3.StringUtils;
@@ -145,12 +147,12 @@ public List listTables() {
return tablePathIdDao.listAllPath();
}
- public List listTableNamesByNamespace(String table_namespace) {
- return tableNameIdDao.listAllNameByNamespace(table_namespace);
+ public List listTableNamesByNamespace(String tableNamespace) {
+ return tableNameIdDao.listAllNameByNamespace(tableNamespace);
}
- public List listTablePathsByNamespace(String table_namespace) {
- return tablePathIdDao.listAllPathByNamespace(table_namespace);
+ public List listTablePathsByNamespace(String tableNamespace) {
+ return tablePathIdDao.listAllPathByNamespace(tableNamespace);
}
public TableInfo getTableInfoByPath(String tablePath) {
@@ -175,7 +177,7 @@ public List getOnePartitionVersions(String tableId, String partit
}
public long getLastedTimestamp(String tableId, String partitionDesc) {
- return partitionInfoDao.getLastedTimestamp(tableId, partitionDesc);
+ return partitionInfoDao.getLatestTimestamp(tableId, partitionDesc);
}
public int getLastedVersionUptoTime(String tableId, String partitionDesc, long utcMills) {
@@ -203,7 +205,7 @@ public List getDeleteFilePath(String tableId, String partitionDesc, long
public void deleteSinglePartitionMetaInfo(String tableId, String partitionDesc, long utcMills,
List fileOps, List deleteFilePathList) {
List filterPartitionInfo = getFilterPartitionInfo(tableId, partitionDesc, utcMills);
- List snapshotList = new ArrayList<>();
+ List snapshotList = new ArrayList<>();
filterPartitionInfo.forEach(p -> snapshotList.addAll(p.getSnapshotList()));
List filterDataCommitInfo =
dataCommitInfoDao.selectByTableIdPartitionDescCommitList(tableId, partitionDesc, snapshotList);
@@ -327,12 +329,13 @@ public void deleteShortTableName(String tableName, String tablePath, String tabl
public void updateTableProperties(String tableId, String properties) {
TableInfo tableInfo = tableInfoDao.selectByTableId(tableId);
- JSONObject originProperties = JSON.parseObject(tableInfo.getProperties());
JSONObject newProperties = JSONObject.parseObject(properties);
-
- if (tableInfo.getProperties() != null && originProperties.containsKey("domain")) {
- // do not modify domain in properties for this table
- newProperties.put("domain", originProperties.get("domain"));
+ if (tableInfo != null) {
+ JSONObject originProperties = JSON.parseObject(tableInfo.getProperties());
+ if (tableInfo.getProperties() != null && originProperties.containsKey("domain")) {
+ // do not modify domain in properties for this table
+ newProperties.put("domain", originProperties.get("domain"));
+ }
}
tableInfoDao.updatePropertiesById(tableId, newProperties.toJSONString());
}
@@ -384,7 +387,7 @@ public boolean commitData(MetaInfo metaInfo, boolean changeSchema, CommitOp comm
String partitionDesc = partitionInfo.getPartitionDesc();
rawMap.put(partitionDesc, partitionInfo);
partitionDescList.add(partitionDesc);
- snapshotList.addAll(partitionInfo.getSnapshotList());
+ snapshotList.addAll(partitionInfo.getSnapshotList().stream().map(uuid -> DBUtil.toJavaUUID(uuid).toString()).collect(Collectors.toList()));
}
Map curMap = getCurPartitionMap(tableId, partitionDescList);
@@ -681,8 +684,8 @@ public boolean mergeConflict(String tableId, List partitionDescList, Map
private PartitionInfo.Builder updateSubmitPartitionSnapshot(PartitionInfo rawPartitionInfo, PartitionInfo.Builder curPartitionInfo,
PartitionInfo readPartition) {
- List snapshot = new ArrayList<>(rawPartitionInfo.getSnapshotList());
- List curSnapshot = new ArrayList<>(curPartitionInfo.getSnapshotList());
+ List snapshot = new ArrayList<>(rawPartitionInfo.getSnapshotList());
+ List curSnapshot = new ArrayList<>(curPartitionInfo.getSnapshotList());
if (readPartition != null) {
curSnapshot.removeAll(readPartition.getSnapshotList());
}
@@ -702,7 +705,7 @@ private PartitionInfo getOrCreateCurPartitionInfo(Map cur
.setVersion(-1)
.setDomain(getTableDomain(tableId))
.build();
- }else{
+ } else {
curPartitionInfo = curPartitionInfo.toBuilder()
.setDomain(getTableDomain(tableId))
.build();
@@ -723,14 +726,14 @@ private Map getCurPartitionMap(String tableId, List getTableSinglePartitionDataInfo(PartitionInfo partitionInfo) {
String tableId = partitionInfo.getTableId();
String partitionDesc = partitionInfo.getPartitionDesc();
- List snapshotList = partitionInfo.getSnapshotList();
+ List snapshotList = partitionInfo.getSnapshotList();
return dataCommitInfoDao.selectByTableIdPartitionDescCommitList(tableId, partitionDesc, snapshotList);
}
public List getPartitionSnapshot(String tableId, String partitionDesc, int version) {
PartitionInfo partitionInfo = partitionInfoDao.findByKey(tableId, partitionDesc, version);
- List commitList = partitionInfo.getSnapshotList();
+ List commitList = partitionInfo.getSnapshotList();
return dataCommitInfoDao.selectByTableIdPartitionDescCommitList(tableId, partitionDesc, commitList);
}
@@ -753,13 +756,13 @@ public DataCommitInfo selectByTableId(String tableId) {
}
public List getDataCommitInfosFromUUIDs(String tableId, String partitionDesc,
- List dataCommitUUIDs) {
+ List dataCommitUUIDs) {
return dataCommitInfoDao.selectByTableIdPartitionDescCommitList(tableId, partitionDesc, dataCommitUUIDs);
}
public void rollbackPartitionByVersion(String tableId, String partitionDesc, int version) {
PartitionInfo partitionInfo = partitionInfoDao.findByKey(tableId, partitionDesc, version);
- if (partitionInfo.getTableId() == null) {
+ if (partitionInfo == null) {
return;
}
PartitionInfo curPartitionInfo = partitionInfoDao.selectLatestPartitionInfo(tableId, partitionDesc);
@@ -769,23 +772,23 @@ public void rollbackPartitionByVersion(String tableId, String partitionDesc, int
.build());
}
- private String getTableDomain(String tableId){
- if(!AuthZEnforcer.authZEnabled()){
+ private String getTableDomain(String tableId) {
+ if (!AuthZEnforcer.authZEnabled()) {
return "public";
}
TableInfo tableInfo = this.getTableInfoByTableId(tableId);
- if(tableInfo == null){
+ if (tableInfo == null) {
throw new IllegalStateException("target tableinfo does not exists");
}
return getNameSpaceDomain(tableInfo.getTableNamespace());
}
- private String getNameSpaceDomain(String namespace){
- if(!AuthZEnforcer.authZEnabled()){
+ private String getNameSpaceDomain(String namespace) {
+ if (!AuthZEnforcer.authZEnabled()) {
return "public";
}
Namespace namespaceInfo = getNamespaceByNamespace(namespace);
- if(namespaceInfo == null) {
+ if (namespaceInfo == null) {
throw new IllegalStateException("target namespace does not exists");
}
return namespaceInfo.getDomain();
@@ -794,9 +797,9 @@ private String getNameSpaceDomain(String namespace){
public void commitDataCommitInfo(DataCommitInfo dataCommitInfo) {
String tableId = dataCommitInfo.getTableId();
String partitionDesc = dataCommitInfo.getPartitionDesc().replaceAll("/", LAKESOUL_RANGE_PARTITION_SPLITTER);
- String commitId = dataCommitInfo.getCommitId();
+ Uuid commitId = dataCommitInfo.getCommitId();
CommitOp commitOp = dataCommitInfo.getCommitOp();
- DataCommitInfo metaCommitInfo = dataCommitInfoDao.selectByPrimaryKey(tableId, partitionDesc, commitId);
+ DataCommitInfo metaCommitInfo = dataCommitInfoDao.selectByPrimaryKey(tableId, partitionDesc, DBUtil.toJavaUUID(commitId).toString());
if (metaCommitInfo != null && metaCommitInfo.getCommitted()) {
LOG.info("DataCommitInfo with tableId={}, commitId={} committed already", tableId, commitId.toString());
return;
@@ -809,7 +812,7 @@ public void commitDataCommitInfo(DataCommitInfo dataCommitInfo) {
MetaInfo.Builder metaInfo = MetaInfo.newBuilder();
TableInfo tableInfo = tableInfoDao.selectByTableId(tableId);
- List snapshot = new ArrayList<>();
+ List snapshot = new ArrayList<>();
snapshot.add(commitId);
List partitionInfoList = new ArrayList<>();
@@ -869,9 +872,15 @@ public void deleteNamespace(String namespace) {
// just for test
public void cleanMeta() {
-
+ if (NativeUtils.NATIVE_METADATA_UPDATE_ENABLED) {
+ NativeMetadataJavaClient.cleanMeta();
+ if (!AuthZEnforcer.authZEnabled()) {
+ namespaceDao.insert(NamespaceDao.DEFAULT_NAMESPACE);
+ }
+ return;
+ }
namespaceDao.clean();
- if(!AuthZEnforcer.authZEnabled()){
+ if (!AuthZEnforcer.authZEnabled()) {
namespaceDao.insert(NamespaceDao.DEFAULT_NAMESPACE);
}
dataCommitInfoDao.clean();
diff --git a/lakesoul-common/src/main/java/com/dmetasoul/lakesoul/meta/DBUtil.java b/lakesoul-common/src/main/java/com/dmetasoul/lakesoul/meta/DBUtil.java
index ccde32a09..2098e935e 100644
--- a/lakesoul-common/src/main/java/com/dmetasoul/lakesoul/meta/DBUtil.java
+++ b/lakesoul-common/src/main/java/com/dmetasoul/lakesoul/meta/DBUtil.java
@@ -9,10 +9,13 @@
import com.alibaba.fastjson.JSONObject;
import com.dmetasoul.lakesoul.meta.entity.DataFileOp;
import com.dmetasoul.lakesoul.meta.entity.FileOp;
+import com.dmetasoul.lakesoul.meta.entity.Uuid;
import com.zaxxer.hikari.HikariConfig;
import org.apache.commons.lang3.StringUtils;
import java.io.IOException;
+import java.net.MalformedURLException;
+import java.net.URL;
import java.nio.file.Files;
import java.nio.file.Paths;
import java.sql.Connection;
@@ -95,6 +98,14 @@ public static DataBaseProperty getDBInfo() {
dataBaseProperty.setUrl(properties.getProperty(urlKey, urlDefault));
dataBaseProperty.setUsername(properties.getProperty(usernameKey, usernameDefault));
dataBaseProperty.setPassword(properties.getProperty(passwordKey, passwordDefault));
+ try {
+ URL url = new URL(properties.getProperty(urlKey, urlDefault).replaceFirst("jdbc:postgresql", "http"));
+ dataBaseProperty.setDbName(url.getPath().substring(1));
+ dataBaseProperty.setHost(url.getHost());
+ dataBaseProperty.setPort(String.valueOf(url.getPort()));
+ } catch (MalformedURLException e) {
+ throw new RuntimeException(e);
+ }
return dataBaseProperty;
}
@@ -305,4 +316,23 @@ public String getRangeKeyString() {
return String.join(LAKESOUL_RANGE_PARTITION_SPLITTER, rangeKeys);
}
}
+
+ public static UUID toJavaUUID(Uuid uuid) {
+ return new UUID(uuid.getHigh(), uuid.getLow());
+ }
+
+ public static Uuid toProtoUuid(UUID uuid) {
+ return Uuid.newBuilder().setHigh(uuid.getMostSignificantBits()).setLow(uuid.getLeastSignificantBits()).build();
+ }
+
+ public static String protoUuidToJniString(Uuid uuid) {
+ StringBuilder sb = new StringBuilder();
+ String high = Long.toUnsignedString(uuid.getHigh(), 16);
+ sb.append(new String(new char[16 - high.length()]).replace("\0", "0"));
+ sb.append(high);
+ String low = Long.toUnsignedString(uuid.getLow(), 16);
+ sb.append(new String(new char[16 - low.length()]).replace("\0", "0"));
+ sb.append(low);
+ return sb.toString();
+ }
}
diff --git a/lakesoul-common/src/main/java/com/dmetasoul/lakesoul/meta/DataBaseProperty.java b/lakesoul-common/src/main/java/com/dmetasoul/lakesoul/meta/DataBaseProperty.java
index 305c0b48a..4e411e699 100644
--- a/lakesoul-common/src/main/java/com/dmetasoul/lakesoul/meta/DataBaseProperty.java
+++ b/lakesoul-common/src/main/java/com/dmetasoul/lakesoul/meta/DataBaseProperty.java
@@ -11,6 +11,9 @@ public class DataBaseProperty implements Serializable {
private String url;
private String username;
private String password;
+ private String dbName;
+ private String host;
+ private String port;
private int maxCommitAttempt;
public String getDriver() {
@@ -65,4 +68,28 @@ public String toString() {
password +
"\n";
}
+
+ public String getHost() {
+ return host;
+ }
+
+ public void setHost(String host) {
+ this.host = host;
+ }
+
+ public String getPort() {
+ return port;
+ }
+
+ public void setPort(String port) {
+ this.port = port;
+ }
+
+ public String getDbName() {
+ return dbName;
+ }
+
+ public void setDbName(String dbName) {
+ this.dbName = dbName;
+ }
}
diff --git a/lakesoul-common/src/main/java/com/dmetasoul/lakesoul/meta/dao/DataCommitInfoDao.java b/lakesoul-common/src/main/java/com/dmetasoul/lakesoul/meta/dao/DataCommitInfoDao.java
index e80677575..46cf6c7fb 100644
--- a/lakesoul-common/src/main/java/com/dmetasoul/lakesoul/meta/dao/DataCommitInfoDao.java
+++ b/lakesoul-common/src/main/java/com/dmetasoul/lakesoul/meta/dao/DataCommitInfoDao.java
@@ -6,22 +6,26 @@
import com.dmetasoul.lakesoul.meta.DBConnector;
import com.dmetasoul.lakesoul.meta.DBUtil;
-import com.dmetasoul.lakesoul.meta.entity.CommitOp;
-import com.dmetasoul.lakesoul.meta.entity.DataCommitInfo;
+import com.dmetasoul.lakesoul.meta.entity.*;
+import com.dmetasoul.lakesoul.meta.jnr.NativeMetadataJavaClient;
+import com.dmetasoul.lakesoul.meta.jnr.NativeUtils;
import java.sql.Connection;
import java.sql.PreparedStatement;
import java.sql.ResultSet;
import java.sql.SQLException;
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.List;
-import java.util.UUID;
+import java.util.*;
import java.util.stream.Collectors;
public class DataCommitInfoDao {
public void insert(DataCommitInfo dataCommitInfo) {
+ if (NativeUtils.NATIVE_METADATA_UPDATE_ENABLED) {
+ Integer count = NativeMetadataJavaClient.insert(
+ NativeUtils.CodedDaoType.InsertDataCommitInfo,
+ JniWrapper.newBuilder().addDataCommitInfo(dataCommitInfo).build());
+ return;
+ }
Connection conn = null;
PreparedStatement pstmt = null;
try {
@@ -39,6 +43,12 @@ public void insert(DataCommitInfo dataCommitInfo) {
}
public void deleteByPrimaryKey(String tableId, String partitionDesc, UUID commitId) {
+ if (NativeUtils.NATIVE_METADATA_UPDATE_ENABLED) {
+ Integer count = NativeMetadataJavaClient.update(
+ NativeUtils.CodedDaoType.DeleteOneDataCommitInfoByTableIdAndPartitionDescAndCommitId,
+ Arrays.asList(tableId, partitionDesc, commitId.toString()));
+ return;
+ }
Connection conn = null;
PreparedStatement pstmt = null;
String sql = "delete from data_commit_info where table_id = ? and partition_desc = ? and commit_id = ? ";
@@ -56,7 +66,15 @@ public void deleteByPrimaryKey(String tableId, String partitionDesc, UUID commit
}
}
- public void deleteByTableIdPartitionDescCommitList(String tableId, String partitionDesc, List commitIdList) {
+ public void deleteByTableIdPartitionDescCommitList(String tableId, String partitionDesc, List commitIdList) {
+ if (NativeUtils.NATIVE_METADATA_UPDATE_ENABLED) {
+ Integer count = NativeMetadataJavaClient.update(
+ NativeUtils.CodedDaoType.DeleteDataCommitInfoByTableIdAndPartitionDescAndCommitIdList,
+ Arrays.asList(tableId, partitionDesc,
+ commitIdList.stream().map(DBUtil::protoUuidToJniString).collect(Collectors.joining(""))
+ ));
+ return;
+ }
Connection conn = null;
PreparedStatement pstmt = null;
if (commitIdList.size() < 1) {
@@ -71,8 +89,8 @@ public void deleteByTableIdPartitionDescCommitList(String tableId, String partit
pstmt.setString(1, tableId);
pstmt.setString(2, partitionDesc);
int index = 3;
- for (String uuid : commitIdList) {
- pstmt.setString(index++, uuid.toString());
+ for (Uuid uuid : commitIdList) {
+ pstmt.setString(index++, DBUtil.toJavaUUID(uuid).toString());
}
pstmt.execute();
} catch (SQLException e) {
@@ -83,6 +101,12 @@ public void deleteByTableIdPartitionDescCommitList(String tableId, String partit
}
public void deleteByTableIdAndPartitionDesc(String tableId, String partitionDesc) {
+ if (NativeUtils.NATIVE_METADATA_UPDATE_ENABLED) {
+ Integer count = NativeMetadataJavaClient.update(
+ NativeUtils.CodedDaoType.DeleteDataCommitInfoByTableIdAndPartitionDesc,
+ Arrays.asList(tableId, partitionDesc));
+ return;
+ }
Connection conn = null;
PreparedStatement pstmt = null;
String sql = "delete from data_commit_info where table_id = ? and partition_desc = ?";
@@ -100,6 +124,12 @@ public void deleteByTableIdAndPartitionDesc(String tableId, String partitionDesc
}
public void deleteByTableId(String tableId) {
+ if (NativeUtils.NATIVE_METADATA_UPDATE_ENABLED) {
+ Integer count = NativeMetadataJavaClient.update(
+ NativeUtils.CodedDaoType.DeleteDataCommitInfoByTableId,
+ Collections.singletonList(tableId));
+ return;
+ }
Connection conn = null;
PreparedStatement pstmt = null;
String sql = "delete from data_commit_info where table_id = ?";
@@ -116,6 +146,14 @@ public void deleteByTableId(String tableId) {
}
public DataCommitInfo selectByPrimaryKey(String tableId, String partitionDesc, String commitId) {
+ if (NativeUtils.NATIVE_METADATA_QUERY_ENABLED) {
+ JniWrapper jniWrapper = NativeMetadataJavaClient.query(
+ NativeUtils.CodedDaoType.SelectOneDataCommitInfoByTableIdAndPartitionDescAndCommitId,
+ Arrays.asList(tableId, partitionDesc, commitId));
+ if (jniWrapper == null) return null;
+ List dataCommitInfoList = jniWrapper.getDataCommitInfoList();
+ return dataCommitInfoList.isEmpty() ? null : dataCommitInfoList.get(0);
+ }
Connection conn = null;
PreparedStatement pstmt = null;
ResultSet rs = null;
@@ -141,6 +179,14 @@ public DataCommitInfo selectByPrimaryKey(String tableId, String partitionDesc, S
}
public DataCommitInfo selectByTableId(String tableId) {
+ if (NativeUtils.NATIVE_METADATA_QUERY_ENABLED) {
+ JniWrapper jniWrapper = NativeMetadataJavaClient.query(
+ NativeUtils.CodedDaoType.SelectOneDataCommitInfoByTableId,
+ Collections.singletonList(tableId));
+ if (jniWrapper == null) return null;
+ List dataCommitInfoList = jniWrapper.getDataCommitInfoList();
+ return dataCommitInfoList.isEmpty() ? null : dataCommitInfoList.get(0);
+ }
Connection conn = null;
PreparedStatement pstmt = null;
ResultSet rs = null;
@@ -164,7 +210,19 @@ public DataCommitInfo selectByTableId(String tableId) {
}
public List selectByTableIdPartitionDescCommitList(String tableId, String partitionDesc,
- List commitIdList) {
+ List commitIdList) {
+ if (NativeUtils.NATIVE_METADATA_QUERY_ENABLED) {
+ if (commitIdList.isEmpty()) {
+ return Collections.emptyList();
+ }
+ JniWrapper jniWrapper = NativeMetadataJavaClient.query(
+ NativeUtils.CodedDaoType.ListDataCommitInfoByTableIdAndPartitionDescAndCommitList,
+ Arrays.asList(tableId, partitionDesc,
+ commitIdList.stream().map(DBUtil::protoUuidToJniString).collect(Collectors.joining(""))
+ ));
+ if (jniWrapper == null) return null;
+ return jniWrapper.getDataCommitInfoList();
+ }
Connection conn = null;
PreparedStatement pstmt = null;
ResultSet rs = null;
@@ -172,7 +230,7 @@ public List selectByTableIdPartitionDescCommitList(String tableI
if (commitIdList.size() < 1) {
return commitInfoList;
}
- String uuidListOrderString = commitIdList.stream().collect(Collectors.joining(","));
+ String uuidListOrderString = commitIdList.stream().map(uuid -> DBUtil.toJavaUUID(uuid).toString()).collect(Collectors.joining(","));
String sql = String.format("select * from data_commit_info where table_id = ? and partition_desc = ? and " +
"commit_id in (%s) order by position(commit_id::text in ?) ", String.join(",", Collections.nCopies(commitIdList.size(), "?")));
@@ -182,8 +240,8 @@ public List selectByTableIdPartitionDescCommitList(String tableI
pstmt.setString(1, tableId);
pstmt.setString(2, partitionDesc);
int index = 3;
- for (String uuid : commitIdList) {
- pstmt.setString(index++, uuid);
+ for (Uuid uuid : commitIdList) {
+ pstmt.setString(index++, DBUtil.toJavaUUID(uuid).toString());
}
pstmt.setString(index, uuidListOrderString);
@@ -201,10 +259,12 @@ public List selectByTableIdPartitionDescCommitList(String tableI
}
public static DataCommitInfo dataCommitInfoFromResultSet(ResultSet rs) throws SQLException {
+ UUID uuid = UUID.fromString(rs.getString("commit_id"));
+ Uuid commitId = DBUtil.toProtoUuid(uuid);
return DataCommitInfo.newBuilder()
.setTableId(rs.getString("table_id"))
.setPartitionDesc(rs.getString("partition_desc"))
- .setCommitId(rs.getString("commit_id"))
+ .setCommitId(commitId)
.addAllFileOps(DBUtil.changeStringToDataFileOpList(rs.getString("file_ops")))
.setCommitOp(CommitOp.valueOf(rs.getString("commit_op")))
.setTimestamp(rs.getLong("timestamp"))
@@ -214,6 +274,13 @@ public static DataCommitInfo dataCommitInfoFromResultSet(ResultSet rs) throws SQ
}
public boolean batchInsert(List listData) {
+ if (NativeUtils.NATIVE_METADATA_UPDATE_ENABLED) {
+ if (listData.isEmpty()) return true;
+ Integer count = NativeMetadataJavaClient.insert(
+ NativeUtils.CodedDaoType.TransactionInsertDataCommitInfo,
+ JniWrapper.newBuilder().addAllDataCommitInfo(listData).build());
+ return count > 0;
+ }
Connection conn = null;
PreparedStatement pstmt = null;
boolean result = true;
@@ -246,7 +313,7 @@ public boolean batchInsert(List listData) {
private void dataCommitInsert(PreparedStatement pstmt, DataCommitInfo dataCommitInfo) throws SQLException {
pstmt.setString(1, dataCommitInfo.getTableId());
pstmt.setString(2, dataCommitInfo.getPartitionDesc());
- pstmt.setString(3, dataCommitInfo.getCommitId().toString());
+ pstmt.setString(3, DBUtil.toJavaUUID(dataCommitInfo.getCommitId()).toString());
pstmt.setString(4, DBUtil.changeDataFileOpListToString(dataCommitInfo.getFileOpsList()));
pstmt.setString(5, dataCommitInfo.getCommitOp().toString());
pstmt.setLong(6, dataCommitInfo.getTimestamp());
diff --git a/lakesoul-common/src/main/java/com/dmetasoul/lakesoul/meta/dao/NamespaceDao.java b/lakesoul-common/src/main/java/com/dmetasoul/lakesoul/meta/dao/NamespaceDao.java
index 133b8d5f9..3ec6f399b 100644
--- a/lakesoul-common/src/main/java/com/dmetasoul/lakesoul/meta/dao/NamespaceDao.java
+++ b/lakesoul-common/src/main/java/com/dmetasoul/lakesoul/meta/dao/NamespaceDao.java
@@ -6,8 +6,10 @@
import com.dmetasoul.lakesoul.meta.DBConfig;
import com.dmetasoul.lakesoul.meta.DBConnector;
-import com.dmetasoul.lakesoul.meta.DBUtil;
+import com.dmetasoul.lakesoul.meta.entity.JniWrapper;
import com.dmetasoul.lakesoul.meta.entity.Namespace;
+import com.dmetasoul.lakesoul.meta.jnr.NativeMetadataJavaClient;
+import com.dmetasoul.lakesoul.meta.jnr.NativeUtils;
import com.dmetasoul.lakesoul.meta.rbac.AuthZContext;
import com.dmetasoul.lakesoul.meta.rbac.AuthZEnforcer;
import dev.failsafe.internal.util.Lists;
@@ -17,10 +19,19 @@
import java.sql.ResultSet;
import java.sql.SQLException;
import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
import java.util.List;
+import java.util.stream.Collectors;
public class NamespaceDao {
public void insert(Namespace namespace) {
+ if (NativeUtils.NATIVE_METADATA_UPDATE_ENABLED) {
+ Integer count = NativeMetadataJavaClient.insert(
+ NativeUtils.CodedDaoType.InsertNamespace,
+ JniWrapper.newBuilder().addNamespace(namespace).build());
+ return;
+ }
Connection conn = null;
PreparedStatement pstmt = null;
try {
@@ -40,6 +51,14 @@ public void insert(Namespace namespace) {
}
public Namespace findByNamespace(String name) {
+ if (NativeUtils.NATIVE_METADATA_QUERY_ENABLED) {
+ JniWrapper jniWrapper = NativeMetadataJavaClient.query(
+ NativeUtils.CodedDaoType.SelectNamespaceByNamespace,
+ Collections.singletonList(name));
+ if (jniWrapper == null) return null;
+ List namespaceList = jniWrapper.getNamespaceList();
+ return namespaceList.isEmpty() ? null : namespaceList.get(0);
+ }
Connection conn = null;
PreparedStatement pstmt = null;
ResultSet rs = null;
@@ -61,6 +80,12 @@ public Namespace findByNamespace(String name) {
}
public void deleteByNamespace(String namespace) {
+ if (NativeUtils.NATIVE_METADATA_UPDATE_ENABLED) {
+ Integer count = NativeMetadataJavaClient.update(
+ NativeUtils.CodedDaoType.DeleteNamespaceByNamespace,
+ Collections.singletonList(namespace));
+ return;
+ }
Connection conn = null;
PreparedStatement pstmt = null;
String sql = String.format("delete from namespace where namespace = '%s' ", namespace);
@@ -76,6 +101,14 @@ public void deleteByNamespace(String namespace) {
}
public List listNamespaces() {
+ if (NativeUtils.NATIVE_METADATA_QUERY_ENABLED) {
+ JniWrapper jniWrapper = NativeMetadataJavaClient.query(
+ NativeUtils.CodedDaoType.ListNamespaces,
+ Collections.emptyList());
+ if (jniWrapper == null) return null;
+ List namespaceList = jniWrapper.getNamespaceList();
+ return namespaceList.stream().map(Namespace::getNamespace).collect(Collectors.toList());
+ }
Connection conn = null;
PreparedStatement pstmt = null;
ResultSet rs = null;
@@ -98,6 +131,11 @@ public List listNamespaces() {
}
public int updatePropertiesByNamespace(String namespace, String properties) {
+ if (NativeUtils.NATIVE_METADATA_UPDATE_ENABLED) {
+ return NativeMetadataJavaClient.update(
+ NativeUtils.CodedDaoType.UpdateNamespacePropertiesByNamespace,
+ Arrays.asList(namespace, properties));
+ }
int result = 0;
Connection conn = null;
PreparedStatement pstmt = null;
@@ -136,7 +174,7 @@ public static Namespace namespaceFromResultSet(ResultSet rs) throws SQLException
return Namespace.newBuilder()
.setNamespace(rs.getString("namespace"))
.setProperties(rs.getString("properties"))
- .setComment(comment == null ? "" : comment )
+ .setComment(comment == null ? "" : comment)
.setDomain(rs.getString("domain"))
.build();
}
@@ -150,4 +188,3 @@ public static Namespace namespaceFromResultSet(ResultSet rs) throws SQLException
.build();
}
-
diff --git a/lakesoul-common/src/main/java/com/dmetasoul/lakesoul/meta/dao/PartitionInfoDao.java b/lakesoul-common/src/main/java/com/dmetasoul/lakesoul/meta/dao/PartitionInfoDao.java
index caa783dbb..2af350be1 100644
--- a/lakesoul-common/src/main/java/com/dmetasoul/lakesoul/meta/dao/PartitionInfoDao.java
+++ b/lakesoul-common/src/main/java/com/dmetasoul/lakesoul/meta/dao/PartitionInfoDao.java
@@ -6,9 +6,9 @@
import com.dmetasoul.lakesoul.meta.DBConnector;
import com.dmetasoul.lakesoul.meta.DBUtil;
-import com.dmetasoul.lakesoul.meta.entity.CommitOp;
-import com.dmetasoul.lakesoul.meta.entity.PartitionInfo;
-import com.dmetasoul.lakesoul.meta.rbac.AuthZEnforcer;
+import com.dmetasoul.lakesoul.meta.entity.*;
+import com.dmetasoul.lakesoul.meta.jnr.NativeMetadataJavaClient;
+import com.dmetasoul.lakesoul.meta.jnr.NativeUtils;
import java.sql.*;
import java.util.*;
@@ -17,6 +17,12 @@
public class PartitionInfoDao {
public void insert(PartitionInfo partitionInfo) {
+ if (NativeUtils.NATIVE_METADATA_UPDATE_ENABLED) {
+ Integer count = NativeMetadataJavaClient.insert(
+ NativeUtils.CodedDaoType.InsertPartitionInfo,
+ JniWrapper.newBuilder().addPartitionInfo(partitionInfo).build());
+ return;
+ }
Connection conn = null;
PreparedStatement pstmt = null;
try {
@@ -32,6 +38,13 @@ public void insert(PartitionInfo partitionInfo) {
}
public boolean transactionInsert(List partitionInfoList, List snapshotList) {
+ if (NativeUtils.NATIVE_METADATA_UPDATE_ENABLED) {
+ if (partitionInfoList.isEmpty()) return true;
+ Integer count = NativeMetadataJavaClient.insert(
+ NativeUtils.CodedDaoType.TransactionInsertPartitionInfo,
+ JniWrapper.newBuilder().addAllPartitionInfo(partitionInfoList).build());
+ return count > 0;
+ }
boolean flag = true;
Connection conn = null;
PreparedStatement pstmt = null;
@@ -73,7 +86,7 @@ public boolean transactionInsert(List partitionInfoList, List findByTableIdAndParList(String tableId, List partitionDescList) {
+ if (NativeUtils.NATIVE_METADATA_QUERY_ENABLED) {
+ if (partitionDescList.isEmpty()) return Collections.emptyList();
+ JniWrapper jniWrapper = NativeMetadataJavaClient.query(
+ NativeUtils.CodedDaoType.ListPartitionDescByTableIdAndParList,
+ Arrays.asList(tableId,
+ String.join(NativeUtils.PARTITION_DESC_DELIM, partitionDescList)));
+ if (jniWrapper == null) return null;
+ return jniWrapper.getPartitionInfoList();
+ }
Connection conn = null;
PreparedStatement pstmt = null;
ResultSet rs = null;
@@ -164,6 +202,7 @@ public List findByTableIdAndParList(String tableId, List
}
}
rs = pstmt.executeQuery();
+ System.out.println(pstmt);
while (rs.next()) {
rsList.add(partitionInfoFromResultSetWithoutTimestamp(rs));
}
@@ -176,6 +215,14 @@ public List findByTableIdAndParList(String tableId, List
}
public PartitionInfo selectLatestPartitionInfo(String tableId, String partitionDesc) {
+ if (NativeUtils.NATIVE_METADATA_QUERY_ENABLED) {
+ JniWrapper jniWrapper = NativeMetadataJavaClient.query(
+ NativeUtils.CodedDaoType.SelectOnePartitionVersionByTableIdAndDesc,
+ Arrays.asList(tableId, partitionDesc));
+ if (jniWrapper == null) return null;
+ List partitionInfoList = jniWrapper.getPartitionInfoList();
+ return partitionInfoList.isEmpty() ? null : partitionInfoList.get(0);
+ }
String sql = String.format(
"select m.table_id, t.partition_desc, m.version, m.commit_op, m.snapshot, m.expression, m.domain from (" +
"select table_id,partition_desc,max(version) from partition_info " +
@@ -186,7 +233,22 @@ public PartitionInfo selectLatestPartitionInfo(String tableId, String partitionD
return getPartitionInfo(sql);
}
- public long getLastedTimestamp(String tableId, String partitionDesc) {
+ public long getLatestTimestamp(String tableId, String partitionDesc) {
+ if (NativeUtils.NATIVE_METADATA_QUERY_ENABLED) {
+ if (null == partitionDesc || "".equals(partitionDesc)) {
+ List result = NativeMetadataJavaClient.queryScalar(
+ NativeUtils.CodedDaoType.GetLatestTimestampFromPartitionInfoWithoutPartitionDesc,
+ Collections.singletonList(tableId));
+ if (result == null || result.isEmpty()) return -1;
+ return Long.parseLong(result.get(0));
+ } else {
+ List result = NativeMetadataJavaClient.queryScalar(
+ NativeUtils.CodedDaoType.GetLatestTimestampFromPartitionInfo,
+ Arrays.asList(tableId, partitionDesc));
+ if (result == null || result.isEmpty()) return -1;
+ return Long.parseLong(result.get(0));
+ }
+ }
Connection conn = null;
PreparedStatement pstmt = null;
ResultSet rs = null;
@@ -217,6 +279,13 @@ public long getLastedTimestamp(String tableId, String partitionDesc) {
}
public int getLastedVersionUptoTime(String tableId, String partitionDesc, long utcMills) {
+ if (NativeUtils.NATIVE_METADATA_QUERY_ENABLED) {
+ List result = NativeMetadataJavaClient.queryScalar(
+ NativeUtils.CodedDaoType.GetLatestVersionUpToTimeFromPartitionInfo,
+ Arrays.asList(tableId, partitionDesc, Long.toString(utcMills)));
+ if (result == null || result.isEmpty()) return -1;
+ return Integer.parseInt(result.get(0));
+ }
Connection conn = null;
PreparedStatement pstmt = null;
ResultSet rs = null;
@@ -247,6 +316,13 @@ public int getLastedVersionUptoTime(String tableId, String partitionDesc, long u
}
public long getLastedVersionTimestampUptoTime(String tableId, String partitionDesc, long utcMills) {
+ if (NativeUtils.NATIVE_METADATA_QUERY_ENABLED) {
+ List result = NativeMetadataJavaClient.queryScalar(
+ NativeUtils.CodedDaoType.GetLatestVersionTimestampUpToTimeFromPartitionInfo,
+ Arrays.asList(tableId, partitionDesc, Long.toString(utcMills)));
+ if (result == null || result.isEmpty()) return 0;
+ return Long.parseLong(result.get(0));
+ }
Connection conn = null;
PreparedStatement pstmt = null;
ResultSet rs = null;
@@ -277,6 +353,13 @@ public long getLastedVersionTimestampUptoTime(String tableId, String partitionDe
}
public List getPartitionVersions(String tableId, String partitionDesc) {
+ if (NativeUtils.NATIVE_METADATA_QUERY_ENABLED) {
+ JniWrapper jniWrapper = NativeMetadataJavaClient.query(
+ NativeUtils.CodedDaoType.ListPartitionByTableIdAndDesc,
+ Arrays.asList(tableId, partitionDesc));
+ if (jniWrapper == null) return null;
+ return jniWrapper.getPartitionInfoList();
+ }
Connection conn = null;
PreparedStatement pstmt = null;
ResultSet rs = null;
@@ -300,6 +383,13 @@ public List getPartitionVersions(String tableId, String partition
}
public List getPartitionDescByTableId(String tableId) {
+ if (NativeUtils.NATIVE_METADATA_QUERY_ENABLED) {
+ JniWrapper jniWrapper = NativeMetadataJavaClient.query(
+ NativeUtils.CodedDaoType.ListPartitionByTableId,
+ Collections.singletonList(tableId));
+ if (jniWrapper == null) return null;
+ return jniWrapper.getPartitionInfoList();
+ }
Connection conn = null;
PreparedStatement pstmt = null;
ResultSet rs = null;
@@ -327,7 +417,14 @@ public List getPartitionDescByTableId(String tableId) {
}
public PartitionInfo findByKey(String tableId, String partitionDesc, int version) {
-
+ if (NativeUtils.NATIVE_METADATA_QUERY_ENABLED) {
+ JniWrapper jniWrapper = NativeMetadataJavaClient.query(
+ NativeUtils.CodedDaoType.SelectPartitionVersionByTableIdAndDescAndVersion,
+ Arrays.asList(tableId, partitionDesc, Integer.toString(version)));
+ if (jniWrapper == null) return null;
+ List partitionInfoList = jniWrapper.getPartitionInfoList();
+ return partitionInfoList.isEmpty() ? null : partitionInfoList.get(0);
+ }
String sql = String.format(
"select * from partition_info where table_id = '%s' and partition_desc = '%s' and version = %d",
tableId, partitionDesc, version);
@@ -336,6 +433,13 @@ public PartitionInfo findByKey(String tableId, String partitionDesc, int version
public List getPartitionsFromVersion(String tableId, String partitionDesc, int startVersion,
int endVersion) {
+ if (NativeUtils.NATIVE_METADATA_QUERY_ENABLED) {
+ JniWrapper jniWrapper = NativeMetadataJavaClient.query(
+ NativeUtils.CodedDaoType.ListPartitionVersionByTableIdAndPartitionDescAndVersionRange,
+ Arrays.asList(tableId, partitionDesc, Integer.toString(startVersion), Integer.toString(endVersion)));
+ if (jniWrapper == null) return null;
+ return jniWrapper.getPartitionInfoList();
+ }
String sql = String.format(
"select * from partition_info where table_id = '%s' and partition_desc = '%s' and version >= %d and " +
"version <= %d",
@@ -344,6 +448,13 @@ public List getPartitionsFromVersion(String tableId, String parti
}
public List getOnePartition(String tableId, String partitionDesc) {
+ if (NativeUtils.NATIVE_METADATA_QUERY_ENABLED) {
+ JniWrapper jniWrapper = NativeMetadataJavaClient.query(
+ NativeUtils.CodedDaoType.SelectOnePartitionVersionByTableIdAndDesc,
+ Arrays.asList(tableId, partitionDesc));
+ if (jniWrapper == null) return null;
+ return jniWrapper.getPartitionInfoList();
+ }
String sql =
String.format("select * from partition_info where table_id = '%s' and partition_desc = '%s' limit 1",
tableId, partitionDesc);
@@ -352,6 +463,13 @@ public List getOnePartition(String tableId, String partitionDesc)
public List getPartitionsFromTimestamp(String tableId, String partitionDesc, long startTimestamp,
long endTimestamp) {
+ if (NativeUtils.NATIVE_METADATA_QUERY_ENABLED) {
+ JniWrapper jniWrapper = NativeMetadataJavaClient.query(
+ NativeUtils.CodedDaoType.ListPartitionVersionByTableIdAndPartitionDescAndTimestampRange,
+ Arrays.asList(tableId, partitionDesc, Long.toString(startTimestamp), Long.toString(endTimestamp)));
+ if (jniWrapper == null) return null;
+ return jniWrapper.getPartitionInfoList();
+ }
String sql = String.format(
"select * from partition_info where table_id = '%s' and partition_desc = '%s' and timestamp >= %d and" +
" timestamp < %d",
@@ -360,6 +478,13 @@ public List getPartitionsFromTimestamp(String tableId, String par
}
public List getAllPartitionDescByTableId(String tableId) {
+ if (NativeUtils.NATIVE_METADATA_QUERY_ENABLED) {
+ JniWrapper jniWrapper = NativeMetadataJavaClient.query(
+ NativeUtils.CodedDaoType.ListPartitionByTableId,
+ Collections.singletonList(tableId));
+ if (jniWrapper == null) return null;
+ return jniWrapper.getPartitionInfoList().stream().map(PartitionInfo::getPartitionDesc).collect(Collectors.toList());
+ }
Connection conn = null;
PreparedStatement pstmt = null;
ResultSet rs = null;
@@ -382,6 +507,13 @@ public List getAllPartitionDescByTableId(String tableId) {
public Set getCommitOpsBetweenVersions(String tableId, String partitionDesc, int firstVersion,
int secondVersion) {
+ if (NativeUtils.NATIVE_METADATA_QUERY_ENABLED) {
+ JniWrapper jniWrapper = NativeMetadataJavaClient.query(
+ NativeUtils.CodedDaoType.ListCommitOpsBetweenVersions,
+ Arrays.asList(tableId, partitionDesc, Integer.toString(firstVersion), Long.toString(secondVersion)));
+ if (jniWrapper == null) return null;
+ return jniWrapper.getPartitionInfoList().stream().map(PartitionInfo::getCommitOp).collect(Collectors.toSet());
+ }
Connection conn = null;
PreparedStatement pstmt = null;
ResultSet rs = null;
@@ -473,7 +605,7 @@ public static PartitionInfo partitionInfoFromResultSet(ResultSet rs) throws SQLE
.setDomain(rs.getString("domain"))
.setTimestamp(rs.getLong("timestamp"));
Array snapshotArray = rs.getArray("snapshot");
- partitionInfo.addAllSnapshot(Arrays.stream((UUID[]) snapshotArray.getArray()).map(UUID::toString).collect(Collectors.toList()));
+ partitionInfo.addAllSnapshot(Arrays.stream((UUID[]) snapshotArray.getArray()).map(DBUtil::toProtoUuid).collect(Collectors.toList()));
partitionInfo.setExpression(rs.getString("expression"));
return partitionInfo.build();
}
@@ -485,7 +617,7 @@ public static PartitionInfo partitionInfoFromResultSetWithoutTimestamp(ResultSet
.setVersion(rs.getInt("version"))
.setCommitOp(CommitOp.valueOf(rs.getString("commit_op")));
Array snapshotArray = rs.getArray("snapshot");
- partitionInfo.addAllSnapshot(Arrays.stream((UUID[]) snapshotArray.getArray()).map(UUID::toString).collect(Collectors.toList()));
+ partitionInfo.addAllSnapshot(Arrays.stream((UUID[]) snapshotArray.getArray()).map(DBUtil::toProtoUuid).collect(Collectors.toList()));
partitionInfo.setExpression(rs.getString("expression"));
return partitionInfo.build();
}
diff --git a/lakesoul-common/src/main/java/com/dmetasoul/lakesoul/meta/dao/TableInfoDao.java b/lakesoul-common/src/main/java/com/dmetasoul/lakesoul/meta/dao/TableInfoDao.java
index c73d06a22..fbd0cc2a9 100644
--- a/lakesoul-common/src/main/java/com/dmetasoul/lakesoul/meta/dao/TableInfoDao.java
+++ b/lakesoul-common/src/main/java/com/dmetasoul/lakesoul/meta/dao/TableInfoDao.java
@@ -4,37 +4,72 @@
package com.dmetasoul.lakesoul.meta.dao;
-import com.alibaba.fastjson.JSONObject;
import com.dmetasoul.lakesoul.meta.DBConnector;
-import com.dmetasoul.lakesoul.meta.DBUtil;
+import com.dmetasoul.lakesoul.meta.entity.JniWrapper;
import com.dmetasoul.lakesoul.meta.entity.TableInfo;
+import com.dmetasoul.lakesoul.meta.jnr.NativeMetadataJavaClient;
+import com.dmetasoul.lakesoul.meta.jnr.NativeUtils;
import org.apache.commons.lang3.StringUtils;
import java.sql.Connection;
import java.sql.PreparedStatement;
import java.sql.ResultSet;
import java.sql.SQLException;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
public class TableInfoDao {
public TableInfo selectByTableId(String tableId) {
-
+ if (NativeUtils.NATIVE_METADATA_QUERY_ENABLED) {
+ JniWrapper jniWrapper = NativeMetadataJavaClient.query(
+ NativeUtils.CodedDaoType.SelectTableInfoByTableId,
+ Collections.singletonList(tableId));
+ if (jniWrapper == null) return null;
+ List tableInfoList = jniWrapper.getTableInfoList();
+ return tableInfoList.isEmpty() ? null : tableInfoList.get(0);
+ }
String sql = String.format("select * from table_info where table_id = '%s'", tableId);
return getTableInfo(sql);
}
public TableInfo selectByTableNameAndNameSpace(String tableName, String namespace) {
+ if (NativeUtils.NATIVE_METADATA_QUERY_ENABLED) {
+ JniWrapper jniWrapper = NativeMetadataJavaClient.query(
+ NativeUtils.CodedDaoType.SelectTableInfoByTableNameAndNameSpace,
+ Arrays.asList(tableName, namespace));
+ if (jniWrapper == null) return null;
+ List tableInfoList = jniWrapper.getTableInfoList();
+ return tableInfoList.isEmpty() ? null : tableInfoList.get(0);
+ }
String sql = String.format("select * from table_info where table_name = '%s'" +
" and table_namespace='%s'", tableName, namespace);
return getTableInfo(sql);
}
public TableInfo selectByTablePath(String tablePath) {
+ if (NativeUtils.NATIVE_METADATA_QUERY_ENABLED) {
+ JniWrapper jniWrapper = NativeMetadataJavaClient.query(
+ NativeUtils.CodedDaoType.SelectTableInfoByTablePath,
+ Collections.singletonList(tablePath));
+ if (jniWrapper == null) return null;
+ List tableInfoList = jniWrapper.getTableInfoList();
+ return tableInfoList.isEmpty() ? null : tableInfoList.get(0);
+ }
String sql = String.format("select * from table_info where table_path = '%s'", tablePath);
return getTableInfo(sql);
}
public TableInfo selectByIdAndTablePath(String tableId, String tablePath) {
+ if (NativeUtils.NATIVE_METADATA_QUERY_ENABLED) {
+ JniWrapper jniWrapper = NativeMetadataJavaClient.query(
+ NativeUtils.CodedDaoType.SelectTableInfoByIdAndTablePath,
+ Arrays.asList(tableId, tablePath));
+ if (jniWrapper == null) return null;
+ List tableInfoList = jniWrapper.getTableInfoList();
+ return tableInfoList.isEmpty() ? null : tableInfoList.get(0);
+ }
String sql = String.format("select * from table_info where table_id = '%s' and table_path = '%s' ", tableId,
tablePath);
return getTableInfo(sql);
@@ -67,6 +102,12 @@ private TableInfo getTableInfo(String sql) {
}
public void insert(TableInfo tableInfo) {
+ if (NativeUtils.NATIVE_METADATA_UPDATE_ENABLED) {
+ Integer count = NativeMetadataJavaClient.insert(
+ NativeUtils.CodedDaoType.InsertTableInfo,
+ JniWrapper.newBuilder().addTableInfo(tableInfo).build());
+ return;
+ }
Connection conn = null;
PreparedStatement pstmt = null;
try {
@@ -90,22 +131,30 @@ public void insert(TableInfo tableInfo) {
}
}
- public void deleteByTableId(String tableId) {
- Connection conn = null;
- PreparedStatement pstmt = null;
- String sql = String.format("delete from table_info where table_id = '%s' ", tableId);
- try {
- conn = DBConnector.getConn();
- pstmt = conn.prepareStatement(sql);
- pstmt.execute();
- } catch (SQLException e) {
- throw new RuntimeException(e);
- } finally {
- DBConnector.closeConn(pstmt, conn);
- }
- }
+// public void deleteByTableId(String tableId) {
+// Connection conn = null;
+// PreparedStatement pstmt = null;
+// String sql = String.format("delete from table_info where table_id = '%s' ", tableId);
+// try {
+// conn = DBConnector.getConn();
+// pstmt = conn.prepareStatement(sql);
+// pstmt.execute();
+// } catch (SQLException e) {
+// throw new RuntimeException(e);
+// } finally {
+// DBConnector.closeConn(pstmt, conn);
+// }
+// }
public void deleteByIdAndPath(String tableId, String tablePath) {
+ if (NativeUtils.NATIVE_METADATA_UPDATE_ENABLED) {
+
+ Integer count = NativeMetadataJavaClient.update(
+ NativeUtils.CodedDaoType.DeleteTableInfoByIdAndPath,
+ Arrays.asList(tableId, tablePath));
+ System.out.println("DeleteTableInfoByIdAndPath " + tableId + " " + tablePath + " result = " + count);
+ return;
+ }
Connection conn = null;
PreparedStatement pstmt = null;
String sql =
@@ -122,6 +171,11 @@ public void deleteByIdAndPath(String tableId, String tablePath) {
}
public int updatePropertiesById(String tableId, String properties) {
+ if (NativeUtils.NATIVE_METADATA_UPDATE_ENABLED) {
+ return NativeMetadataJavaClient.update(
+ NativeUtils.CodedDaoType.UpdateTableInfoPropertiesById,
+ Arrays.asList(tableId, properties));
+ }
int result = 0;
Connection conn = null;
PreparedStatement pstmt = null;
@@ -142,6 +196,11 @@ public int updatePropertiesById(String tableId, String properties) {
}
public int updateByTableId(String tableId, String tableName, String tablePath, String tableSchema) {
+ if (NativeUtils.NATIVE_METADATA_UPDATE_ENABLED) {
+ return NativeMetadataJavaClient.update(
+ NativeUtils.CodedDaoType.UpdateTableInfoById,
+ Arrays.asList(tableId, tableName, tablePath, tableSchema));
+ }
int result = 0;
if (StringUtils.isBlank(tableName) && StringUtils.isBlank(tablePath) && StringUtils.isBlank(tableSchema)) {
return result;
diff --git a/lakesoul-common/src/main/java/com/dmetasoul/lakesoul/meta/dao/TableNameIdDao.java b/lakesoul-common/src/main/java/com/dmetasoul/lakesoul/meta/dao/TableNameIdDao.java
index 608c99e32..9b51eb3ff 100644
--- a/lakesoul-common/src/main/java/com/dmetasoul/lakesoul/meta/dao/TableNameIdDao.java
+++ b/lakesoul-common/src/main/java/com/dmetasoul/lakesoul/meta/dao/TableNameIdDao.java
@@ -5,7 +5,10 @@
package com.dmetasoul.lakesoul.meta.dao;
import com.dmetasoul.lakesoul.meta.DBConnector;
+import com.dmetasoul.lakesoul.meta.entity.JniWrapper;
import com.dmetasoul.lakesoul.meta.entity.TableNameId;
+import com.dmetasoul.lakesoul.meta.jnr.NativeMetadataJavaClient;
+import com.dmetasoul.lakesoul.meta.jnr.NativeUtils;
import org.apache.commons.lang3.StringUtils;
import java.sql.Connection;
@@ -13,11 +16,22 @@
import java.sql.ResultSet;
import java.sql.SQLException;
import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
import java.util.List;
+import java.util.stream.Collectors;
public class TableNameIdDao {
public TableNameId findByTableName(String tableName, String tableNamespace) {
+ if (NativeUtils.NATIVE_METADATA_QUERY_ENABLED) {
+ JniWrapper jniWrapper = NativeMetadataJavaClient.query(
+ NativeUtils.CodedDaoType.SelectTableNameIdByTableName,
+ Arrays.asList(tableName, tableNamespace));
+ if (jniWrapper == null) return null;
+ List tableNameIdList = jniWrapper.getTableNameIdList();
+ return tableNameIdList.isEmpty() ? null : tableNameIdList.get(0);
+ }
Connection conn = null;
PreparedStatement pstmt = null;
ResultSet rs = null;
@@ -39,12 +53,20 @@ public TableNameId findByTableName(String tableName, String tableNamespace) {
return tableNameId;
}
- public List listAllNameByNamespace(String table_namespace) {
+ public List listAllNameByNamespace(String tableNamespace) {
+ if (NativeUtils.NATIVE_METADATA_QUERY_ENABLED) {
+ JniWrapper jniWrapper = NativeMetadataJavaClient.query(
+ NativeUtils.CodedDaoType.ListTableNameByNamespace,
+ Collections.singletonList(tableNamespace));
+ if (jniWrapper == null) return null;
+ List tableNameIdList = jniWrapper.getTableNameIdList();
+ return tableNameIdList.stream().map(TableNameId::getTableName).collect(Collectors.toList());
+ }
Connection conn = null;
PreparedStatement pstmt = null;
ResultSet rs = null;
String sql =
- String.format("select table_name from table_name_id where table_namespace = '%s'", table_namespace);
+ String.format("select table_name from table_name_id where table_namespace = '%s'", tableNamespace);
List list = new ArrayList<>();
try {
conn = DBConnector.getConn();
@@ -63,6 +85,12 @@ public List listAllNameByNamespace(String table_namespace) {
}
public void insert(TableNameId tableNameId) {
+ if (NativeUtils.NATIVE_METADATA_UPDATE_ENABLED) {
+ Integer count = NativeMetadataJavaClient.insert(
+ NativeUtils.CodedDaoType.InsertTableNameId,
+ JniWrapper.newBuilder().addTableNameId(tableNameId).build());
+ return;
+ }
Connection conn = null;
PreparedStatement pstmt = null;
try {
@@ -82,6 +110,10 @@ public void insert(TableNameId tableNameId) {
}
public void delete(String tableName, String tableNamespace) {
+ if (NativeUtils.NATIVE_METADATA_UPDATE_ENABLED) {
+ Integer count = NativeMetadataJavaClient.update(NativeUtils.CodedDaoType.DeleteTableNameIdByTableNameAndNamespace, Arrays.asList(tableName, tableNamespace));
+ return;
+ }
Connection conn = null;
PreparedStatement pstmt = null;
String sql =
@@ -99,6 +131,12 @@ public void delete(String tableName, String tableNamespace) {
}
public void deleteByTableId(String tableId) {
+ if (NativeUtils.NATIVE_METADATA_UPDATE_ENABLED) {
+ Integer count = NativeMetadataJavaClient.update(
+ NativeUtils.CodedDaoType.DeleteTableNameIdByTableId,
+ Collections.singletonList(tableId));
+ return;
+ }
Connection conn = null;
PreparedStatement pstmt = null;
String sql = String.format("delete from table_name_id where table_id = '%s' ", tableId);
diff --git a/lakesoul-common/src/main/java/com/dmetasoul/lakesoul/meta/dao/TablePathIdDao.java b/lakesoul-common/src/main/java/com/dmetasoul/lakesoul/meta/dao/TablePathIdDao.java
index 6cc7e95d1..b87916352 100644
--- a/lakesoul-common/src/main/java/com/dmetasoul/lakesoul/meta/dao/TablePathIdDao.java
+++ b/lakesoul-common/src/main/java/com/dmetasoul/lakesoul/meta/dao/TablePathIdDao.java
@@ -5,7 +5,10 @@
package com.dmetasoul.lakesoul.meta.dao;
import com.dmetasoul.lakesoul.meta.DBConnector;
+import com.dmetasoul.lakesoul.meta.entity.JniWrapper;
import com.dmetasoul.lakesoul.meta.entity.TablePathId;
+import com.dmetasoul.lakesoul.meta.jnr.NativeMetadataJavaClient;
+import com.dmetasoul.lakesoul.meta.jnr.NativeUtils;
import org.apache.commons.lang3.StringUtils;
import java.sql.Connection;
@@ -13,11 +16,21 @@
import java.sql.ResultSet;
import java.sql.SQLException;
import java.util.ArrayList;
+import java.util.Collections;
import java.util.List;
+import java.util.stream.Collectors;
public class TablePathIdDao {
public TablePathId findByTablePath(String tablePath) {
+ if (NativeUtils.NATIVE_METADATA_QUERY_ENABLED) {
+ JniWrapper jniWrapper = NativeMetadataJavaClient.query(
+ NativeUtils.CodedDaoType.SelectTablePathIdByTablePath,
+ Collections.singletonList(tablePath));
+ if (jniWrapper == null) return null;
+ List tablePathIdList = jniWrapper.getTablePathIdList();
+ return tablePathIdList.isEmpty() ? null : tablePathIdList.get(0);
+ }
Connection conn = null;
PreparedStatement pstmt = null;
ResultSet rs = null;
@@ -59,11 +72,11 @@ public List listAll() {
return list;
}
- public List listAllByNamespace(String table_namespace) {
+ public List listAllByNamespace(String tableNamespace) {
Connection conn = null;
PreparedStatement pstmt = null;
ResultSet rs = null;
- String sql = String.format("select * from table_path_id where table_namespace = '%s'", table_namespace);
+ String sql = String.format("select * from table_path_id where table_namespace = '%s'", tableNamespace);
List list = new ArrayList<>();
try {
conn = DBConnector.getConn();
@@ -81,6 +94,12 @@ public List listAllByNamespace(String table_namespace) {
}
public List listAllPath() {
+ if (NativeUtils.NATIVE_METADATA_QUERY_ENABLED) {
+ JniWrapper jniWrapper = NativeMetadataJavaClient.query(
+ NativeUtils.CodedDaoType.ListAllTablePath,
+ Collections.emptyList());
+ return jniWrapper.getTablePathIdList().stream().map(TablePathId::getTablePath).collect(Collectors.toList());
+ }
Connection conn = null;
PreparedStatement pstmt = null;
ResultSet rs = null;
@@ -102,11 +121,17 @@ public List listAllPath() {
return list;
}
- public List listAllPathByNamespace(String table_namespace) {
+ public List listAllPathByNamespace(String tableNamespace) {
+ if (NativeUtils.NATIVE_METADATA_QUERY_ENABLED) {
+ JniWrapper jniWrapper = NativeMetadataJavaClient.query(
+ NativeUtils.CodedDaoType.ListAllPathTablePathByNamespace,
+ Collections.singletonList(tableNamespace));
+ return jniWrapper.getTablePathIdList().stream().map(TablePathId::getTablePath).collect(Collectors.toList());
+ }
Connection conn = null;
PreparedStatement pstmt = null;
ResultSet rs = null;
- String sql = String.format("select table_path from table_path_id where table_namespace = '%s'", table_namespace);
+ String sql = String.format("select table_path from table_path_id where table_namespace = '%s'", tableNamespace);
List list = new ArrayList<>();
try {
conn = DBConnector.getConn();
@@ -125,6 +150,12 @@ public List listAllPathByNamespace(String table_namespace) {
}
public void insert(TablePathId tablePathId) {
+ if (NativeUtils.NATIVE_METADATA_UPDATE_ENABLED) {
+ Integer count = NativeMetadataJavaClient.insert(
+ NativeUtils.CodedDaoType.InsertTablePathId,
+ JniWrapper.newBuilder().addTablePathId(tablePathId).build());
+ return;
+ }
Connection conn = null;
PreparedStatement pstmt = null;
try {
@@ -143,6 +174,13 @@ public void insert(TablePathId tablePathId) {
}
public void delete(String tablePath) {
+ if (NativeUtils.NATIVE_METADATA_UPDATE_ENABLED) {
+ Integer count = NativeMetadataJavaClient.update(
+ NativeUtils.CodedDaoType.DeleteTablePathIdByTablePath,
+ Collections.singletonList(tablePath));
+ System.out.println("DeleteTablePathIdByTablePath " + tablePath + " result = " + count);
+ return;
+ }
Connection conn = null;
PreparedStatement pstmt = null;
String sql = String.format("delete from table_path_id where table_path = '%s' ", tablePath);
@@ -158,6 +196,12 @@ public void delete(String tablePath) {
}
public void deleteByTableId(String tableId) {
+ if (NativeUtils.NATIVE_METADATA_UPDATE_ENABLED) {
+ Integer count = NativeMetadataJavaClient.update(
+ NativeUtils.CodedDaoType.DeleteTablePathIdByTableId,
+ Collections.singletonList(tableId));
+ return;
+ }
Connection conn = null;
PreparedStatement pstmt = null;
String sql = String.format("delete from table_path_id where table_id = '%s' ", tableId);
diff --git a/lakesoul-common/src/main/java/com/dmetasoul/lakesoul/meta/jnr/JnrLoader.java b/lakesoul-common/src/main/java/com/dmetasoul/lakesoul/meta/jnr/JnrLoader.java
new file mode 100644
index 000000000..0e824dfd6
--- /dev/null
+++ b/lakesoul-common/src/main/java/com/dmetasoul/lakesoul/meta/jnr/JnrLoader.java
@@ -0,0 +1,71 @@
+// SPDX-FileCopyrightText: 2023 LakeSoul Contributors
+//
+// SPDX-License-Identifier: Apache-2.0
+package com.dmetasoul.lakesoul.meta.jnr;
+
+import jnr.ffi.LibraryLoader;
+import jnr.ffi.LibraryOption;
+
+import java.io.File;
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.io.InputStream;
+import java.nio.file.Files;
+import java.nio.file.StandardCopyOption;
+import java.util.HashMap;
+import java.util.Map;
+
+public class JnrLoader {
+
+ private LibLakeSoulMetaData libLakeSoulMetaData = null;
+
+ private boolean hasLoaded = false;
+
+ public static final JnrLoader INSTANCE = new JnrLoader();
+
+ public static LibLakeSoulMetaData get() {
+ JnrLoader.tryLoad();
+ return INSTANCE.libLakeSoulMetaData;
+ }
+
+ public synchronized static void tryLoad() {
+ if (INSTANCE.hasLoaded) {
+ return;
+ }
+
+ String libName = System.mapLibraryName("lakesoul_metadata_c");
+
+ String finalPath = null;
+
+ if (JnrLoader.class.getClassLoader().getResource(libName) != null) {
+ try {
+ File temp = File.createTempFile(libName + "_", ".tmp", new File(System.getProperty("java.io.tmpdir")));
+ temp.deleteOnExit();
+ try (final InputStream is = JnrLoader.class.getClassLoader().getResourceAsStream(libName)) {
+ if (is == null) {
+ throw new FileNotFoundException(libName);
+ }
+ Files.copy(is, temp.toPath(), StandardCopyOption.REPLACE_EXISTING);
+ finalPath = temp.getAbsolutePath();
+ }
+ } catch (IOException e) {
+ throw new IllegalStateException("error loading native libraries: " + e);
+ }
+ }
+
+ if (finalPath != null) {
+ Map libraryOptions = new HashMap<>();
+ libraryOptions.put(LibraryOption.LoadNow, true);
+ libraryOptions.put(LibraryOption.IgnoreError, true);
+
+ JnrLoader.INSTANCE.libLakeSoulMetaData = LibraryLoader.loadLibrary(
+ LibLakeSoulMetaData.class,
+ libraryOptions,
+ finalPath
+ );
+
+ }
+
+ INSTANCE.hasLoaded = true;
+ }
+}
diff --git a/lakesoul-common/src/main/java/com/dmetasoul/lakesoul/meta/jnr/LibLakeSoulMetaData.java b/lakesoul-common/src/main/java/com/dmetasoul/lakesoul/meta/jnr/LibLakeSoulMetaData.java
new file mode 100644
index 000000000..c8e900396
--- /dev/null
+++ b/lakesoul-common/src/main/java/com/dmetasoul/lakesoul/meta/jnr/LibLakeSoulMetaData.java
@@ -0,0 +1,71 @@
+// SPDX-FileCopyrightText: 2023 LakeSoul Contributors
+//
+// SPDX-License-Identifier: Apache-2.0
+package com.dmetasoul.lakesoul.meta.jnr;
+
+import com.google.protobuf.InvalidProtocolBufferException;
+import jnr.ffi.Pointer;
+import jnr.ffi.annotations.Delegate;
+import jnr.ffi.annotations.LongLong;
+
+import java.awt.*;
+
+public interface LibLakeSoulMetaData {
+
+ Pointer create_tokio_runtime();
+
+ void free_tokio_runtime(Pointer runtime);
+
+ Pointer create_prepared_statement();
+
+ void free_prepared_statement(Pointer prepared);
+
+ Pointer create_tokio_postgres_client(BooleanCallback booleanCallback, String config, Pointer runtime);
+
+ void free_tokio_postgres_client(Pointer client);
+
+ void execute_query(IntegerCallback integerCallback, Pointer runtime, Pointer client, Pointer prepared, Integer type, String texts, @LongLong long addr);
+
+ void execute_update(IntegerCallback integerCallback, Pointer runtime, Pointer client, Pointer prepared, Integer type, String texts);
+
+ void execute_query_scalar(StringCallback stringCallback, Pointer runtime, Pointer client, Pointer prepared, Integer type, String texts);
+
+ void execute_insert(IntegerCallback integerCallback, Pointer runtime, Pointer client, Pointer prepared, Integer type, @LongLong long addr, int length);
+
+ void clean_meta_for_test(IntegerCallback integerCallback, Pointer runtime, Pointer client);
+
+ void hello_world(Callback bytesCallback);
+
+ void namespace(byte[] bytes, Integer len);
+
+ Pointer create_native_client(
+ BooleanCallback booleanCallback,
+ String config);
+
+ interface Callback {
+ @Delegate
+ void invoke(T result, String msg);
+ }
+
+ interface VoidCallback { // type representing callback
+ @Delegate
+ void invoke(String err); // function name doesn't matter, it just needs to be the only function and have @Delegate
+ }
+
+ interface BooleanCallback { // type representing callback
+ @Delegate
+ void invoke(Boolean status, String err); // function name doesn't matter, it just needs to be the only function and have @Delegate
+ }
+
+ interface IntegerCallback { // type representing callback
+ @Delegate
+ void invoke(Integer status, String err); // function name doesn't matter, it just needs to be the only function and have @Delegate
+ }
+
+ interface StringCallback { // type representing callback
+ @Delegate
+ void invoke(String status, String err); // function name doesn't matter, it just needs to be the only function and have @Delegate
+ }
+
+
+}
diff --git a/lakesoul-common/src/main/java/com/dmetasoul/lakesoul/meta/jnr/NativeMetadataJavaClient.java b/lakesoul-common/src/main/java/com/dmetasoul/lakesoul/meta/jnr/NativeMetadataJavaClient.java
new file mode 100644
index 000000000..83581b1cb
--- /dev/null
+++ b/lakesoul-common/src/main/java/com/dmetasoul/lakesoul/meta/jnr/NativeMetadataJavaClient.java
@@ -0,0 +1,463 @@
+// SPDX-FileCopyrightText: 2023 LakeSoul Contributors
+//
+// SPDX-License-Identifier: Apache-2.0
+package com.dmetasoul.lakesoul.meta.jnr;
+
+import com.dmetasoul.lakesoul.meta.DBUtil;
+import com.dmetasoul.lakesoul.meta.DataBaseProperty;
+import com.dmetasoul.lakesoul.meta.entity.JniWrapper;
+import com.google.protobuf.InvalidProtocolBufferException;
+import jnr.ffi.ObjectReferenceManager;
+import jnr.ffi.Pointer;
+import jnr.ffi.Runtime;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.sql.SQLException;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+import java.util.function.BiConsumer;
+import java.util.stream.Collectors;
+
+import static com.dmetasoul.lakesoul.meta.jnr.NativeUtils.*;
+
+public class NativeMetadataJavaClient implements AutoCloseable {
+
+ private static final Logger LOG = LoggerFactory.getLogger(NativeMetadataJavaClient.class);
+
+ private final long timeout;
+ private long bufferSize;
+
+ private long largeBufferSize;
+
+ private Pointer tokioPostgresClient = null;
+
+ private Pointer sharedBuffer = null;
+
+ private Pointer largeSharedBuffer = null;
+
+ private Pointer tokioRuntime = null;
+
+ private Pointer preparedStatement = null;
+
+
+ protected final LibLakeSoulMetaData libLakeSoulMetaData;
+
+ protected final ObjectReferenceManager booleanCallbackObjectReferenceManager;
+
+ protected final ObjectReferenceManager stringCallbackObjectReferenceManager;
+
+ protected final ObjectReferenceManager integerCallbackObjectReferenceManager;
+
+ private static NativeMetadataJavaClient instance = null;
+
+ private final ReentrantReadWriteLock lock;
+
+ private static DataBaseProperty dataBaseProperty = null;
+
+ public static void setDataBaseProperty(DataBaseProperty dataBaseProperty) {
+ NativeMetadataJavaClient.dataBaseProperty = dataBaseProperty;
+ }
+
+ public NativeMetadataJavaClient() {
+ this(5000L, 1 << 12, 1 << 16);
+ }
+
+ public NativeMetadataJavaClient(long timeout, int bufferSize, int largeBufferSize) {
+ this.timeout = timeout;
+ libLakeSoulMetaData = JnrLoader.get();
+ booleanCallbackObjectReferenceManager = Runtime.getRuntime(libLakeSoulMetaData).newObjectReferenceManager();
+ stringCallbackObjectReferenceManager = Runtime.getRuntime(libLakeSoulMetaData).newObjectReferenceManager();
+ integerCallbackObjectReferenceManager = Runtime.getRuntime(libLakeSoulMetaData).newObjectReferenceManager();
+ this.bufferSize = bufferSize;
+ this.largeBufferSize = largeBufferSize;
+ sharedBuffer = Runtime.getRuntime(libLakeSoulMetaData).getMemoryManager().allocateDirect(bufferSize);
+ largeSharedBuffer = Runtime.getRuntime(libLakeSoulMetaData).getMemoryManager().allocateDirect(largeBufferSize);
+ lock = new ReentrantReadWriteLock();
+ initialize();
+ }
+
+ public static NativeMetadataJavaClient getInstance() {
+ if (instance == null) {
+ instance = new NativeMetadataJavaClient();
+ }
+ return instance;
+ }
+
+
+ public Pointer getTokioPostgresClient() {
+ return tokioPostgresClient;
+ }
+
+ public ObjectReferenceManager getbooleanCallbackObjectReferenceManager() {
+ return booleanCallbackObjectReferenceManager;
+ }
+
+ public ObjectReferenceManager getStringCallbackObjectReferenceManager() {
+ return stringCallbackObjectReferenceManager;
+ }
+
+ public ObjectReferenceManager getIntegerCallbackObjectReferenceManager() {
+ return integerCallbackObjectReferenceManager;
+ }
+
+ public Runtime getRuntime() {
+ return Runtime.getRuntime(libLakeSoulMetaData);
+ }
+
+ public LibLakeSoulMetaData getLibLakeSoulMetaData() {
+ return libLakeSoulMetaData;
+ }
+
+ static class ReferencedBooleanCallback implements LibLakeSoulMetaData.BooleanCallback {
+ public final BiConsumer callback;
+ private final Pointer key;
+ private final ObjectReferenceManager referenceManager;
+
+ public ReferencedBooleanCallback(BiConsumer callback, ObjectReferenceManager referenceManager) {
+ this.callback = callback;
+ this.referenceManager = referenceManager;
+ key = this.referenceManager.add(this);
+ }
+
+ @Override
+ public void invoke(Boolean result, String msg) {
+ callback.accept(result, msg);
+ close();
+ }
+
+ public void close() {
+ if (key != null) {
+ referenceManager.remove(key);
+ }
+ }
+ }
+
+ static class ReferencedIntegerCallback implements LibLakeSoulMetaData.IntegerCallback {
+ public final BiConsumer callback;
+ private final Pointer key;
+ private final ObjectReferenceManager referenceManager;
+
+ public ReferencedIntegerCallback(BiConsumer callback, ObjectReferenceManager referenceManager) {
+ this.callback = callback;
+ this.referenceManager = referenceManager;
+ key = this.referenceManager.add(this);
+ }
+
+ @Override
+ public void invoke(Integer result, String msg) {
+ callback.accept(result, msg);
+ close();
+ }
+
+ public void close() {
+ if (key != null) {
+ referenceManager.remove(key);
+ }
+ }
+ }
+
+ static class ReferencedStringCallback implements LibLakeSoulMetaData.StringCallback, AutoCloseable {
+ public final BiConsumer callback;
+ private final Pointer key;
+ private final ObjectReferenceManager referenceManager;
+
+ public ReferencedStringCallback(BiConsumer callback, ObjectReferenceManager referenceManager) {
+ this.callback = callback;
+ this.referenceManager = referenceManager;
+ key = this.referenceManager.add(this);
+ }
+
+ @Override
+ public void invoke(String result, String msg) {
+ callback.accept(result, msg);
+ close();
+ }
+
+ @Override
+ public void close() {
+ if (key != null) {
+ referenceManager.remove(key);
+ }
+ }
+ }
+
+ private void initialize() {
+ DataBaseProperty dataBaseProperty = NativeMetadataJavaClient.dataBaseProperty;
+ if (dataBaseProperty == null) {
+ dataBaseProperty = DBUtil.getDBInfo();
+ }
+ tokioRuntime = libLakeSoulMetaData.create_tokio_runtime();
+
+ String config = String.format(
+ "host=%s port=%s dbname=%s user=%s password=%s",
+ dataBaseProperty.getHost(),
+ dataBaseProperty.getPort(),
+ dataBaseProperty.getDbName(),
+ dataBaseProperty.getUsername(),
+ dataBaseProperty.getPassword());
+ final CompletableFuture future = new CompletableFuture<>();
+ tokioPostgresClient = libLakeSoulMetaData.create_tokio_postgres_client(
+ new ReferencedBooleanCallback((bool, msg) -> {
+ if (msg.isEmpty()) {
+ future.complete(bool);
+ } else {
+ System.err.println(msg);
+ future.completeExceptionally(new IOException(msg));
+ }
+ }, getbooleanCallbackObjectReferenceManager()),
+ config,
+ tokioRuntime
+ );
+ preparedStatement = libLakeSoulMetaData.create_prepared_statement();
+ try {
+ future.get(timeout, TimeUnit.MILLISECONDS);
+ } catch (InterruptedException | ExecutionException e) {
+ throw new RuntimeException(e);
+ } catch (TimeoutException e) {
+ LOG.error("Configuring postgres with {} timeout", dataBaseProperty);
+ throw new RuntimeException(e);
+ }
+ }
+
+
+ public JniWrapper executeQuery(Integer queryType, List params) {
+ getReadLock();
+ final CompletableFuture future = new CompletableFuture<>();
+ try {
+ getLibLakeSoulMetaData().execute_query(
+ new ReferencedIntegerCallback((result, msg) -> {
+ if (msg.isEmpty()) {
+ future.complete(result);
+ } else {
+ future.completeExceptionally(new SQLException(msg));
+ }
+ }, getIntegerCallbackObjectReferenceManager()),
+ tokioRuntime,
+ tokioPostgresClient,
+ preparedStatement,
+ queryType,
+ String.join(PARAM_DELIM, params),
+ queryType < DAO_TYPE_QUERY_LIST_OFFSET ? sharedBuffer.address() : largeSharedBuffer.address()
+ );
+ Integer len = future.get(timeout, TimeUnit.MILLISECONDS);
+ if (len < 0) return null;
+ byte[] bytes = new byte[len];
+ if (queryType < DAO_TYPE_QUERY_LIST_OFFSET)
+ sharedBuffer.get(0, bytes, 0, len);
+ else
+ largeSharedBuffer.get(0, bytes, 0, len);
+ try {
+ return JniWrapper.parseFrom(bytes);
+ } catch (InvalidProtocolBufferException e) {
+ throw new RuntimeException(e);
+ }
+ } catch (InterruptedException | ExecutionException e) {
+ throw new RuntimeException(e);
+ } catch (TimeoutException e) {
+ LOG.error("Execute Query {} with {} timeout", queryType, params);
+ throw new RuntimeException(e);
+ } finally {
+ unlockReadLock();
+ }
+ }
+
+ private void getReadLock() {
+ lock.readLock().lock();
+ }
+
+ private void unlockReadLock() {
+ lock.readLock().unlock();
+ }
+
+ private void getWriteLock() {
+ lock.writeLock().lock();
+ }
+
+ private void unlockWriteLock() {
+ lock.writeLock().unlock();
+ }
+
+
+ public Integer executeInsert(Integer insertType, JniWrapper jniWrapper) {
+ getWriteLock();
+ try {
+ final CompletableFuture future = new CompletableFuture<>();
+
+ byte[] bytes = jniWrapper.toByteArray();
+ if (insertType < DAO_TYPE_TRANSACTION_INSERT_LIST_OFFSET)
+ sharedBuffer.put(0, bytes, 0, bytes.length);
+ else
+ largeSharedBuffer.put(0, bytes, 0, bytes.length);
+
+ getLibLakeSoulMetaData().execute_insert(
+ new ReferencedIntegerCallback((result, msg) -> {
+ if (msg.isEmpty()) {
+ future.complete(result);
+ } else {
+ future.completeExceptionally(new SQLException(msg));
+ }
+ }, getIntegerCallbackObjectReferenceManager()),
+ tokioRuntime,
+ tokioPostgresClient,
+ preparedStatement,
+ insertType,
+ insertType < DAO_TYPE_TRANSACTION_INSERT_LIST_OFFSET ? sharedBuffer.address() : largeSharedBuffer.address(),
+ bytes.length
+ );
+ return future.get(timeout, TimeUnit.MILLISECONDS);
+ } catch (InterruptedException | ExecutionException e) {
+ throw new RuntimeException(e);
+ } catch (TimeoutException e) {
+ LOG.error("Execute Insert {} with {} timeout", insertType, jniWrapper);
+ throw new RuntimeException(e);
+ } finally {
+ unlockWriteLock();
+ }
+ }
+
+ public Integer executeUpdate(Integer updateType, List params) {
+ getWriteLock();
+ try {
+ final CompletableFuture future = new CompletableFuture<>();
+
+ getLibLakeSoulMetaData().execute_update(
+ new ReferencedIntegerCallback((result, msg) -> {
+ if (msg.isEmpty()) {
+ future.complete(result);
+ } else {
+ future.completeExceptionally(new SQLException(msg));
+ }
+ }, getIntegerCallbackObjectReferenceManager()),
+ tokioRuntime,
+ tokioPostgresClient,
+ preparedStatement,
+ updateType,
+ String.join(PARAM_DELIM, params)
+ );
+ return future.get(timeout, TimeUnit.MILLISECONDS);
+ } catch (InterruptedException | ExecutionException e) {
+ throw new RuntimeException(e);
+ } catch (TimeoutException e) {
+ LOG.error("Execute Update {} with {} timeout", updateType, params);
+ throw new RuntimeException(e);
+ } finally {
+ unlockWriteLock();
+ }
+ }
+
+ public List executeQueryScalar(Integer updateType, List params) {
+ getReadLock();
+ try {
+ final CompletableFuture future = new CompletableFuture<>();
+
+ getLibLakeSoulMetaData().execute_query_scalar(
+ new ReferencedStringCallback((result, msg) -> {
+ if (msg.isEmpty()) {
+ future.complete(result);
+ } else {
+ future.completeExceptionally(new SQLException(msg));
+ }
+ }, getStringCallbackObjectReferenceManager()),
+ tokioRuntime,
+ tokioPostgresClient,
+ preparedStatement,
+ updateType,
+ String.join(PARAM_DELIM, params)
+ );
+ String result = future.get(timeout, TimeUnit.MILLISECONDS);
+ if (result.isEmpty()) return Collections.emptyList();
+ return Arrays.stream(result.split(PARAM_DELIM)).collect(Collectors.toList());
+ } catch (InterruptedException | ExecutionException e) {
+ throw new RuntimeException(e);
+ } catch (TimeoutException e) {
+ LOG.error("Execute Update {} with {} timeout", updateType, params);
+ throw new RuntimeException(e);
+ } finally {
+ unlockReadLock();
+ }
+ }
+
+ public static Integer insert(NativeUtils.CodedDaoType insertType, JniWrapper jniWrapper) {
+ return getInstance().executeInsert(insertType.getCode(), jniWrapper);
+ }
+
+ public static JniWrapper query(NativeUtils.CodedDaoType queryType, List params) {
+ if (params.size() != queryType.getParamsNum()) {
+ throw new RuntimeException("Params Num mismatch for " + queryType.name() + ", params=" + params + " paramsNum=" + params.size());
+ }
+ return getInstance().executeQuery(queryType.getCode(), params);
+ }
+
+ public static Integer update(NativeUtils.CodedDaoType updateType, List params) {
+ if (params.size() != updateType.getParamsNum()) {
+ throw new RuntimeException("Params Num mismatch for " + updateType.name() + ", params=" + params + " paramsNum=" + params.size());
+ }
+ return getInstance().executeUpdate(updateType.getCode(), params);
+ }
+
+ public static List queryScalar(NativeUtils.CodedDaoType queryScalarType, List params) {
+ if (params.size() != queryScalarType.getParamsNum()) {
+ throw new RuntimeException("Params Num mismatch for " + queryScalarType.name() + ", params=" + params + " paramsNum=" + params.size());
+ }
+ return getInstance().executeQueryScalar(queryScalarType.getCode(), params);
+ }
+
+ public static int cleanMeta() {
+ final CompletableFuture future = new CompletableFuture<>();
+
+ NativeMetadataJavaClient instance = getInstance();
+ instance.getWriteLock();
+ try {
+ instance.getLibLakeSoulMetaData().clean_meta_for_test(
+ new ReferencedIntegerCallback((result, msg) -> {
+ if (msg.isEmpty()) {
+ future.complete(result);
+ } else {
+ future.completeExceptionally(new SQLException(msg));
+ }
+ }, instance.getIntegerCallbackObjectReferenceManager()),
+ instance.tokioRuntime,
+ instance.tokioPostgresClient
+ );
+ return future.get(instance.timeout, TimeUnit.MILLISECONDS);
+ } catch (InterruptedException | ExecutionException e) {
+ throw new RuntimeException(e);
+ } catch (TimeoutException e) {
+ LOG.error("Clean Meta timeout");
+ throw new RuntimeException(e);
+ } finally {
+ instance.unlockWriteLock();
+ }
+ }
+
+ @Override
+ public void close() {
+ if (tokioRuntime != null) {
+ libLakeSoulMetaData.free_tokio_runtime(tokioRuntime);
+ tokioRuntime = null;
+ }
+ if (tokioPostgresClient != null) {
+ libLakeSoulMetaData.free_tokio_postgres_client(tokioPostgresClient);
+ tokioPostgresClient = null;
+ }
+ if (preparedStatement != null) {
+ libLakeSoulMetaData.free_prepared_statement(preparedStatement);
+ preparedStatement = null;
+ }
+ }
+
+ public static void closeAll() {
+ if (instance != null) {
+ instance.close();
+ instance = null;
+ }
+ }
+}
diff --git a/lakesoul-common/src/main/java/com/dmetasoul/lakesoul/meta/jnr/NativeUtils.java b/lakesoul-common/src/main/java/com/dmetasoul/lakesoul/meta/jnr/NativeUtils.java
new file mode 100644
index 000000000..1b63eae89
--- /dev/null
+++ b/lakesoul-common/src/main/java/com/dmetasoul/lakesoul/meta/jnr/NativeUtils.java
@@ -0,0 +1,124 @@
+// SPDX-FileCopyrightText: 2023 LakeSoul Contributors
+//
+// SPDX-License-Identifier: Apache-2.0
+
+package com.dmetasoul.lakesoul.meta.jnr;
+
+public class NativeUtils {
+
+ public static boolean NATIVE_METADATA_QUERY_ENABLED = true;
+
+ public static boolean NATIVE_METADATA_UPDATE_ENABLED = true;
+
+ public static int DAO_TYPE_QUERY_ONE_OFFSET = 0;
+ public static int DAO_TYPE_QUERY_LIST_OFFSET = 100;
+ public static int DAO_TYPE_INSERT_ONE_OFFSET = 200;
+ public static int DAO_TYPE_TRANSACTION_INSERT_LIST_OFFSET = 300;
+ public static int DAO_TYPE_QUERY_SCALAR_OFFSET = 400;
+ public static int DAO_TYPE_UPDATE_OFFSET = 500;
+
+ public static final String PARAM_DELIM = "__DELIM__";
+
+ public static final String PARTITION_DESC_DELIM = "_DELIM_";
+
+ public enum CodedDaoType {
+ // ==== Query One ====
+ SelectNamespaceByNamespace(DAO_TYPE_QUERY_ONE_OFFSET, 1),
+ SelectTablePathIdByTablePath(DAO_TYPE_QUERY_ONE_OFFSET + 1, 1),
+ SelectTableInfoByTableId(DAO_TYPE_QUERY_ONE_OFFSET + 2, 1),
+ SelectTableNameIdByTableName(DAO_TYPE_QUERY_ONE_OFFSET + 3, 2),
+ SelectTableInfoByTableNameAndNameSpace(DAO_TYPE_QUERY_ONE_OFFSET + 4, 2),
+ SelectTableInfoByTablePath(DAO_TYPE_QUERY_ONE_OFFSET + 5, 1),
+ SelectTableInfoByIdAndTablePath(DAO_TYPE_QUERY_ONE_OFFSET + 6, 2),
+
+ SelectOnePartitionVersionByTableIdAndDesc(DAO_TYPE_QUERY_ONE_OFFSET + 7, 2),
+ SelectPartitionVersionByTableIdAndDescAndVersion(DAO_TYPE_QUERY_ONE_OFFSET + 8, 3),
+
+ SelectOneDataCommitInfoByTableIdAndPartitionDescAndCommitId(DAO_TYPE_QUERY_ONE_OFFSET + 9, 3),
+ SelectOneDataCommitInfoByTableId(DAO_TYPE_QUERY_ONE_OFFSET + 10, 3),
+
+ // ==== Query List ====
+
+ ListNamespaces(DAO_TYPE_QUERY_LIST_OFFSET),
+ ListTableNameByNamespace(DAO_TYPE_QUERY_LIST_OFFSET + 1, 1),
+ ListAllTablePath(DAO_TYPE_QUERY_LIST_OFFSET + 2),
+ ListAllPathTablePathByNamespace(DAO_TYPE_QUERY_LIST_OFFSET + 3, 1),
+
+ // Query Partition List
+ ListPartitionByTableId(DAO_TYPE_QUERY_LIST_OFFSET + 4, 1),
+ ListPartitionDescByTableIdAndParList(DAO_TYPE_QUERY_LIST_OFFSET + 5, 2),
+ ListPartitionByTableIdAndDesc(DAO_TYPE_QUERY_LIST_OFFSET + 6, 2),
+ ListPartitionVersionByTableIdAndPartitionDescAndVersionRange(DAO_TYPE_QUERY_LIST_OFFSET + 7, 4),
+ ListPartitionVersionByTableIdAndPartitionDescAndTimestampRange(DAO_TYPE_QUERY_LIST_OFFSET + 8, 4),
+ ListCommitOpsBetweenVersions(DAO_TYPE_QUERY_LIST_OFFSET + 9, 4),
+
+ // Query DataCommitInfo List
+ ListDataCommitInfoByTableIdAndPartitionDescAndCommitList(DAO_TYPE_QUERY_LIST_OFFSET + 10, 3),
+
+ // ==== Insert One ====
+ InsertNamespace(DAO_TYPE_INSERT_ONE_OFFSET),
+ InsertTablePathId(DAO_TYPE_INSERT_ONE_OFFSET + 1),
+ InsertTableNameId(DAO_TYPE_INSERT_ONE_OFFSET + 2),
+ InsertTableInfo(DAO_TYPE_INSERT_ONE_OFFSET + 3),
+ InsertPartitionInfo(DAO_TYPE_INSERT_ONE_OFFSET + 4),
+ InsertDataCommitInfo(DAO_TYPE_INSERT_ONE_OFFSET + 5),
+
+ // ==== Transaction Insert List ====
+ TransactionInsertPartitionInfo(DAO_TYPE_TRANSACTION_INSERT_LIST_OFFSET),
+ TransactionInsertDataCommitInfo(DAO_TYPE_TRANSACTION_INSERT_LIST_OFFSET + 1),
+
+ // ==== Query SCALAR ====
+ GetLatestTimestampFromPartitionInfo(DAO_TYPE_QUERY_SCALAR_OFFSET, 2),
+ GetLatestTimestampFromPartitionInfoWithoutPartitionDesc(DAO_TYPE_QUERY_SCALAR_OFFSET + 1, 1),
+ GetLatestVersionUpToTimeFromPartitionInfo(DAO_TYPE_QUERY_SCALAR_OFFSET + 2, 3),
+ GetLatestVersionTimestampUpToTimeFromPartitionInfo(DAO_TYPE_QUERY_SCALAR_OFFSET + 3, 3),
+
+ // ==== Update ====
+ // Update Namespace
+ DeleteNamespaceByNamespace(DAO_TYPE_UPDATE_OFFSET, 1),
+ UpdateNamespacePropertiesByNamespace(DAO_TYPE_UPDATE_OFFSET + 1, 2),
+
+ // Update TableInfo
+ DeleteTableInfoByIdAndPath(DAO_TYPE_UPDATE_OFFSET + 2, 2),
+ UpdateTableInfoPropertiesById(DAO_TYPE_UPDATE_OFFSET + 3, 2),
+ UpdateTableInfoById(DAO_TYPE_UPDATE_OFFSET + 4, 4),
+
+ // Update TablePathId
+ DeleteTablePathIdByTablePath(DAO_TYPE_UPDATE_OFFSET + 5, 1),
+ DeleteTablePathIdByTableId(DAO_TYPE_UPDATE_OFFSET + 6, 1),
+ // Update TableNameId
+ DeleteTableNameIdByTableNameAndNamespace(DAO_TYPE_UPDATE_OFFSET + 7, 2),
+ DeleteTableNameIdByTableId(DAO_TYPE_UPDATE_OFFSET + 8, 1),
+ // Update PartitionInfo
+ DeletePartitionInfoByTableIdAndPartitionDesc(DAO_TYPE_UPDATE_OFFSET + 9, 2),
+ DeletePartitionInfoByTableId(DAO_TYPE_UPDATE_OFFSET + 10, 1),
+ DeletePreviousVersionPartition(DAO_TYPE_UPDATE_OFFSET + 11, 3),
+ // Update DataCommitInfo
+ DeleteOneDataCommitInfoByTableIdAndPartitionDescAndCommitId(DAO_TYPE_UPDATE_OFFSET + 12, 3),
+ DeleteDataCommitInfoByTableIdAndPartitionDescAndCommitIdList(DAO_TYPE_UPDATE_OFFSET + 13, 3),
+ DeleteDataCommitInfoByTableIdAndPartitionDesc(DAO_TYPE_UPDATE_OFFSET + 14, 2),
+ DeleteDataCommitInfoByTableId(DAO_TYPE_UPDATE_OFFSET + 15, 1),
+ ;
+
+ private final int code;
+ private final int paramsNum;
+
+ CodedDaoType(int code) {
+ this(code, 0);
+ }
+
+ CodedDaoType(int code, int paramsNum) {
+ this.code = code;
+ this.paramsNum = paramsNum;
+ }
+
+ public int getCode() {
+ return code;
+ }
+
+ public int getParamsNum() {
+ return paramsNum;
+ }
+ }
+
+}
diff --git a/lakesoul-flink/src/main/java/org/apache/flink/lakesoul/sink/committer/LakeSoulSinkCommitter.java b/lakesoul-flink/src/main/java/org/apache/flink/lakesoul/sink/committer/LakeSoulSinkCommitter.java
index eeda97d0d..8d216616b 100644
--- a/lakesoul-flink/src/main/java/org/apache/flink/lakesoul/sink/committer/LakeSoulSinkCommitter.java
+++ b/lakesoul-flink/src/main/java/org/apache/flink/lakesoul/sink/committer/LakeSoulSinkCommitter.java
@@ -5,6 +5,7 @@
package org.apache.flink.lakesoul.sink.committer;
import com.dmetasoul.lakesoul.meta.DBManager;
+import com.dmetasoul.lakesoul.meta.DBUtil;
import com.dmetasoul.lakesoul.meta.entity.*;
import org.apache.flink.api.connector.sink.Committer;
import org.apache.flink.core.fs.FileStatus;
@@ -107,7 +108,7 @@ public List commit(List uuid.toString).toBuffer))
+ metaPartitionInfo.addAllSnapshot(JavaConverters.bufferAsJavaList(partition_info.read_files.map(DBUtil.toProtoUuid).toBuffer))
val dataCommitInfoList = dbManager.getTableSinglePartitionDataInfo(metaPartitionInfo.build).asScala.toArray
for (metaDataCommitInfo <- dataCommitInfoList) {
val fileOps = metaDataCommitInfo.getFileOpsList.asScala.toArray
@@ -178,16 +178,16 @@ object DataOperation {
loop.break()
}
if (startVersionTimestamp == dataItem.getTimestamp) {
- preVersionUUIDs ++= dataItem.getSnapshotList.asScala.map(str => UUID.fromString(str))
+ preVersionUUIDs ++= dataItem.getSnapshotList.asScala.map(DBUtil.toJavaUUID)
} else {
if ("CompactionCommit".equals(dataItem.getCommitOp)) {
- val compactShotList = dataItem.getSnapshotList.asScala.map(str => UUID.fromString(str)).toArray
+ val compactShotList = dataItem.getSnapshotList.asScala.map(DBUtil.toJavaUUID).toArray
compactionUUIDs += compactShotList(0)
if (compactShotList.length > 1) {
incrementalAllUUIDs ++= compactShotList.slice(1, compactShotList.length)
}
} else {
- incrementalAllUUIDs ++= dataItem.getSnapshotList.asScala.map(str => UUID.fromString(str))
+ incrementalAllUUIDs ++= dataItem.getSnapshotList.asScala.map(DBUtil.toJavaUUID)
}
}
}
@@ -199,7 +199,7 @@ object DataOperation {
val resultUUID = tmpUUIDs -- compactionUUIDs
val file_arr_buf = new ArrayBuffer[DataFileInfo]()
val dataCommitInfoList = dbManager
- .getDataCommitInfosFromUUIDs(table_id, partition_desc, Lists.newArrayList(resultUUID.map(uuid => uuid.toString).asJava)).asScala.toArray
+ .getDataCommitInfosFromUUIDs(table_id, partition_desc, Lists.newArrayList(resultUUID.map(DBUtil.toProtoUuid).asJava)).asScala.toArray
fillFiles(file_arr_buf, dataCommitInfoList)
}
}
diff --git a/lakesoul-flink/src/main/scala/com/dmetasoul/lakesoul/meta/MetaVersion.scala b/lakesoul-flink/src/main/scala/com/dmetasoul/lakesoul/meta/MetaVersion.scala
index c2009c37d..eb80327c1 100644
--- a/lakesoul-flink/src/main/scala/com/dmetasoul/lakesoul/meta/MetaVersion.scala
+++ b/lakesoul-flink/src/main/scala/com/dmetasoul/lakesoul/meta/MetaVersion.scala
@@ -126,7 +126,7 @@ object MetaVersion {
table_id = info.getTableId,
range_value = range_value,
version = info.getVersion,
- read_files = info.getSnapshotList.asScala.map(str => UUID.fromString(str)).toArray,
+ read_files = info.getSnapshotList.asScala.map(DBUtil.toJavaUUID).toArray,
expression = info.getExpression,
commit_op = info.getCommitOp.name
)
@@ -139,7 +139,7 @@ object MetaVersion {
table_id = info.getTableId,
range_value = range_value,
version = info.getVersion,
- read_files = info.getSnapshotList.asScala.map(str => UUID.fromString(str)).toArray,
+ read_files = info.getSnapshotList.asScala.map(DBUtil.toJavaUUID).toArray,
expression = info.getExpression,
commit_op = info.getCommitOp.name
)
@@ -193,7 +193,7 @@ object MetaVersion {
table_id = res.getTableId,
range_value = res.getPartitionDesc,
version = res.getVersion,
- read_files = res.getSnapshotList.asScala.map(str => UUID.fromString(str)).toArray,
+ read_files = res.getSnapshotList.asScala.map(DBUtil.toJavaUUID).toArray,
expression = res.getExpression,
commit_op = res.getCommitOp.name
)
diff --git a/lakesoul-flink/src/test/java/com/dmetasoul/lakesoul/meta/LakeSoulRBACTest.java b/lakesoul-flink/src/test/java/com/dmetasoul/lakesoul/meta/LakeSoulRBACTest.java
index d9bef86b6..a23d9c56e 100644
--- a/lakesoul-flink/src/test/java/com/dmetasoul/lakesoul/meta/LakeSoulRBACTest.java
+++ b/lakesoul-flink/src/test/java/com/dmetasoul/lakesoul/meta/LakeSoulRBACTest.java
@@ -4,6 +4,7 @@
package com.dmetasoul.lakesoul.meta;
+import com.dmetasoul.lakesoul.meta.jnr.NativeMetadataJavaClient;
import org.apache.flink.lakesoul.test.LakeSoulFlinkTestBase;
import org.apache.flink.table.catalog.exceptions.CatalogException;
import org.apache.flink.table.catalog.exceptions.DatabaseNotExistException;
@@ -14,7 +15,7 @@
import java.util.List;
public class LakeSoulRBACTest extends LakeSoulFlinkTestBase {
- final String ADMIN1 = "admin1";
+ final String ADMIN1 = "admin1";
final String ADMIN1_PASS = "admin1";
final String ADMIN2 = "admin2";
final String ADMIN2_PASS = "admin2";
@@ -23,21 +24,22 @@ public class LakeSoulRBACTest extends LakeSoulFlinkTestBase {
final String DOMAIN1 = "domain1";
final String DOMAIN2 = "domain2";
- private void login(String username , String password, String domain) {
+ private void login(String username, String password, String domain) {
System.setProperty(DBUtil.usernameKey, username);
System.setProperty(DBUtil.passwordKey, password);
System.setProperty(DBUtil.domainKey, domain);
DBConnector.closeAllConnections();
+ NativeMetadataJavaClient.closeAll();
}
@Test
- public void testDifferentDomain(){
+ public void testDifferentDomain() {
getTableEnv().useCatalog("lakesoul");
login(ADMIN1, ADMIN1_PASS, DOMAIN1);
// create
sql("create database if not exists database1");
- assert(sql("show databases").size() == 2);
+ assert (sql("show databases").size() == 2);
// drop: coming soon
// spark.sql("drop database database1").collect()
// val df2 = spark.sql("show databases").toDF()
@@ -53,12 +55,12 @@ public void testDifferentDomain(){
+ " with ('format' = 'lakesoul', 'path' = '"
+ getTempDirUri("/lakeSource/table2")
+ "')");
- assert( sql("show tables").size() == 2);
+ assert (sql("show tables").size() == 2);
// drop table
sql("drop table table1");
sql("drop table table2");
- assert(sql("show tables").size() == 0);
+ assert (sql("show tables").size() == 0);
// write and read data
sql("create table if not exists table1 ( id int, foo string, bar string )"
@@ -67,7 +69,7 @@ public void testDifferentDomain(){
+ "')");
sql("insert into table1 values(1, 'foo1', 'bar1')");
sql("insert into table1 values(2, 'foo2', 'bar2')");
- assert(sql("select * from table1").size() == 2);
+ assert (sql("select * from table1").size() == 2);
// create & drop database
sql("insert into table1 values(3, 'foo3', 'bar3')");
@@ -75,13 +77,13 @@ public void testDifferentDomain(){
try {
sql("use database1");
throw new RuntimeException("test state was unexcepted");
- }catch (Exception e){
- assert(e instanceof CatalogException);
+ } catch (Exception e) {
+ assert (e instanceof CatalogException);
}
- try{
+ try {
sql("create database if not exists database2");
throw new RuntimeException("test state was unexcepted");
- }catch (Exception e) {
+ } catch (Exception e) {
System.out.println(e.getMessage());
assert e.getMessage().contains("Could not execute CREATE DATABASE");
}
@@ -96,14 +98,14 @@ public void testDifferentDomain(){
+ getTempDirUri("/lakeSource/table3")
+ "')");
throw new RuntimeException("test state was unexcepted");
- }catch (Exception e){
+ } catch (Exception e) {
System.out.println(e.getMessage());
assert e.getCause() instanceof DatabaseNotExistException;
}
try {
sql("drop table database1.table1");
throw new RuntimeException("test state was unexcepted");
- }catch (Exception e){
+ } catch (Exception e) {
System.out.println(e.getMessage());
assert e.getMessage().contains("Table with identifier 'lakesoul.database1.table1' does not exist.");
}
@@ -112,17 +114,17 @@ public void testDifferentDomain(){
try {
sql("insert into database1.table1 values(4, 'foo4', 'bar4')");
throw new RuntimeException("test state was unexcepted");
- }catch (Exception e){
+ } catch (Exception e) {
System.out.println(e.getMessage());
- assert(e.getMessage().contains("Cannot find table '`lakesoul`.`database1`.`table1`' in any of the catalogs"));
+ assert (e.getMessage().contains("Cannot find table '`lakesoul`.`database1`.`table1`' in any of the catalogs"));
}
try {
sql("select * from database1.table1");
throw new RuntimeException("test state was unexcepted");
- }catch (Exception e){
+ } catch (Exception e) {
System.out.println(e.getMessage());
- assert(e.getMessage().contains("Object 'database1' not found within 'lakesoul'"));
+ assert (e.getMessage().contains("Object 'database1' not found within 'lakesoul'"));
}
// clear test
@@ -132,7 +134,7 @@ public void testDifferentDomain(){
}
@Test
- public void testDifferentRole(){
+ public void testDifferentRole() {
getTableEnv().useCatalog("lakesoul");
login(ADMIN1, ADMIN1_PASS, DOMAIN1);
// create
@@ -142,17 +144,17 @@ public void testDifferentRole(){
login(USER1, USER1_PASS, DOMAIN1);
// create table & drop database
sql("use database1");
- try{
+ try {
sql("create database if not exists database3");
throw new RuntimeException("test state was unexcepted");
- }catch (Exception e) {
+ } catch (Exception e) {
System.out.println(e.getMessage());
assert e.getMessage().contains("Could not execute CREATE DATABASE");
}
- try{
+ try {
sql("drop database database1");
throw new RuntimeException("test state was unexcepted");
- }catch (Exception e) {
+ } catch (Exception e) {
System.out.println(e.getMessage());
assert e.getMessage().contains("Could not execute DROP DATABASE lakesoul.database1 RESTRICT");
}
@@ -166,10 +168,10 @@ public void testDifferentRole(){
+ " with ('format' = 'lakesoul', 'path' = '"
+ getTempDirUri("/lakeSource/table2")
+ "')");
- assert(sql("show tables").size() == 2);
+ assert (sql("show tables").size() == 2);
sql("drop table table1");
sql("drop table table2");
- assert(sql("show tables").size() == 0);
+ assert (sql("show tables").size() == 0);
// CRUD data
sql("create table if not exists table1 ( id int, foo string, bar string )"
@@ -178,7 +180,7 @@ public void testDifferentRole(){
+ "')");
sql("insert into table1 values(1, 'foo1', 'bar1')");
sql("insert into table1 values(2, 'foo2', 'bar2')");
- assert(sql("select * from table1").size() == 2);
+ assert (sql("select * from table1").size() == 2);
// clear test
sql("drop table table1");
diff --git a/lakesoul-flink/src/test/java/org/apache/flink/lakesoul/test/LakeSoulCatalogTest.java b/lakesoul-flink/src/test/java/org/apache/flink/lakesoul/test/LakeSoulCatalogTest.java
index d7bbd6992..c4adad210 100644
--- a/lakesoul-flink/src/test/java/org/apache/flink/lakesoul/test/LakeSoulCatalogTest.java
+++ b/lakesoul-flink/src/test/java/org/apache/flink/lakesoul/test/LakeSoulCatalogTest.java
@@ -102,8 +102,6 @@ public void createTableWithLike() {
Assertions.assertThat(JSON.parseObject(info.getProperties()).get("lakesoul_cdc_change_column")).isEqualTo(JSON.parseObject(info2.getProperties()).get("lakesoul_cdc_change_column"));
Assertions.assertThat(JSON.parseObject(info.getProperties()).get("path")).isEqualTo("/tmp/user_behaviorgg");
Assertions.assertThat(JSON.parseObject(info2.getProperties()).get("path")).isEqualTo("/tmp/like_table");
- System.out.println(info);
- System.out.println(info2);
tEnvs.executeSql("DROP TABLE user_behaviorgg");
tEnvs.executeSql("DROP TABLE like_table");
diff --git a/lakesoul-flink/src/test/java/org/apache/flink/lakesoul/test/LakeSoulFlinkTestBase.java b/lakesoul-flink/src/test/java/org/apache/flink/lakesoul/test/LakeSoulFlinkTestBase.java
index 02cbbd2a4..4d12cb93c 100644
--- a/lakesoul-flink/src/test/java/org/apache/flink/lakesoul/test/LakeSoulFlinkTestBase.java
+++ b/lakesoul-flink/src/test/java/org/apache/flink/lakesoul/test/LakeSoulFlinkTestBase.java
@@ -32,7 +32,6 @@ public class LakeSoulFlinkTestBase extends AbstractTestBase {
@BeforeClass
public static void startDBManager() {
- System.out.println("Initializing DBManager");
LakeSoulFlinkTestBase.dbManager = new DBManager();
dbManager.cleanMeta();
LakeSoulFlinkTestBase.catalog = new LakeSoulCatalog();
@@ -52,7 +51,7 @@ protected TableEnvironment getTableEnv() {
TableEnvironment env = TableEnvironment.create(settings);
env.getConfig()
- .getConfiguration()
+ .getConfiguration()
;
env.registerCatalog("lakesoul", catalog);
@@ -87,9 +86,9 @@ protected void assertSameElements(Iterable expected, Iterable actual)
protected void assertSameElements(String message, Iterable expected, Iterable actual) {
Assertions.assertThat(actual)
- .isNotNull()
- .as(message)
- .containsExactlyInAnyOrderElementsOf(expected);
+ .isNotNull()
+ .as(message)
+ .containsExactlyInAnyOrderElementsOf(expected);
}
}
diff --git a/lakesoul-spark/pom.xml b/lakesoul-spark/pom.xml
index 8445526f2..07666791d 100644
--- a/lakesoul-spark/pom.xml
+++ b/lakesoul-spark/pom.xml
@@ -325,6 +325,9 @@ SPDX-License-Identifier: Apache-2.0
jdbc:postgresql://127.0.0.1:5432/lakesoul_test_${surefire.forkNumber}?stringtype=unspecified
+
+ lakesoul_test_${surefire.forkNumber}
+
.test_workdir_${surefire.forkNumber}
diff --git a/lakesoul-spark/src/main/scala/com/dmetasoul/lakesoul/meta/DataOperation.scala b/lakesoul-spark/src/main/scala/com/dmetasoul/lakesoul/meta/DataOperation.scala
index b3641c517..940bcaa81 100644
--- a/lakesoul-spark/src/main/scala/com/dmetasoul/lakesoul/meta/DataOperation.scala
+++ b/lakesoul-spark/src/main/scala/com/dmetasoul/lakesoul/meta/DataOperation.scala
@@ -4,7 +4,7 @@
package com.dmetasoul.lakesoul.meta
-import com.dmetasoul.lakesoul.meta.entity.{DataFileOp, FileOp}
+import com.dmetasoul.lakesoul.meta.entity.{DataFileOp, FileOp, Uuid}
import com.google.common.collect.Lists
import org.apache.spark.internal.Logging
import org.apache.spark.sql.lakesoul.LakeSoulOptions
@@ -40,7 +40,7 @@ object DataOperation extends Logging {
val metaPartitionInfo = entity.PartitionInfo.newBuilder
metaPartitionInfo.setTableId(partition_info.table_id)
metaPartitionInfo.setPartitionDesc(partition_info.range_value)
- metaPartitionInfo.addAllSnapshot(JavaConverters.bufferAsJavaList(partition_info.read_files.map(uuid => uuid.toString).toBuffer))
+ metaPartitionInfo.addAllSnapshot(JavaConverters.bufferAsJavaList(partition_info.read_files.map(uuid => DBUtil.toProtoUuid(uuid)).toBuffer))
val dataCommitInfoList = MetaVersion.dbManager.getTableSinglePartitionDataInfo(metaPartitionInfo.build).asScala.toArray
for (metaDataCommitInfo <- dataCommitInfoList) {
val fileOps = metaDataCommitInfo.getFileOpsList.asScala.toArray
@@ -146,16 +146,16 @@ object DataOperation extends Logging {
loop.break()
}
if (startVersionTimestamp == dataItem.getTimestamp) {
- preVersionUUIDs ++= dataItem.getSnapshotList.asScala.map(id => UUID.fromString(id))
+ preVersionUUIDs ++= dataItem.getSnapshotList.asScala.map(id => DBUtil.toJavaUUID(id))
} else {
if ("CompactionCommit".equals(dataItem.getCommitOp)) {
- val compactShotList = dataItem.getSnapshotList.asScala.map(id => UUID.fromString(id)).toArray
+ val compactShotList = dataItem.getSnapshotList.asScala.map(id => DBUtil.toJavaUUID(id)).toArray
compactionUUIDs += compactShotList(0)
- if (compactShotList.size > 1) {
- incrementalAllUUIDs ++= compactShotList.slice(1, compactShotList.size)
+ if (compactShotList.length > 1) {
+ incrementalAllUUIDs ++= compactShotList.slice(1, compactShotList.length)
}
} else {
- incrementalAllUUIDs ++= dataItem.getSnapshotList.asScala.map(id => UUID.fromString(id))
+ incrementalAllUUIDs ++= dataItem.getSnapshotList.asScala.map(id => DBUtil.toJavaUUID(id))
}
}
}
@@ -169,7 +169,7 @@ object DataOperation extends Logging {
val file_arr_buf = new ArrayBuffer[DataFileInfo]()
val file_res_arr_buf = new ArrayBuffer[DataFileInfo]()
val dupCheck = new mutable.HashSet[String]()
- val dataCommitInfoList = MetaVersion.dbManager.getDataCommitInfosFromUUIDs(table_id, partition_desc, Lists.newArrayList(resultUUID.map(uuid => uuid.toString).asJava)).asScala.toArray
+ val dataCommitInfoList = MetaVersion.dbManager.getDataCommitInfosFromUUIDs(table_id, partition_desc, Lists.newArrayList(resultUUID.map(DBUtil.toProtoUuid).asJava)).asScala.toArray
dataCommitInfoList.foreach(data_commit_info => {
val fileOps = data_commit_info.getFileOpsList.asScala.toArray
fileOps.foreach(file => {
@@ -224,7 +224,7 @@ object DataOperation extends Logging {
metaDataCommitInfo.setTableId(table_id)
metaDataCommitInfo.setPartitionDesc(range_value)
metaDataCommitInfo.setCommitOp(entity.CommitOp.valueOf(commit_type))
- metaDataCommitInfo.setCommitId(commit_id.toString)
+ metaDataCommitInfo.setCommitId(DBUtil.toProtoUuid(commit_id))
metaDataCommitInfo.addAllFileOps(JavaConverters.bufferAsJavaList(file_arr_buf))
metaDataCommitInfo.setTimestamp(modification_time)
metaDataCommitInfoList.add(metaDataCommitInfo.build)
diff --git a/lakesoul-spark/src/main/scala/com/dmetasoul/lakesoul/meta/MetaCommit.scala b/lakesoul-spark/src/main/scala/com/dmetasoul/lakesoul/meta/MetaCommit.scala
index 49210b8c6..95ed898a4 100644
--- a/lakesoul-spark/src/main/scala/com/dmetasoul/lakesoul/meta/MetaCommit.scala
+++ b/lakesoul-spark/src/main/scala/com/dmetasoul/lakesoul/meta/MetaCommit.scala
@@ -5,7 +5,7 @@
package com.dmetasoul.lakesoul.meta
import com.alibaba.fastjson.JSONObject
-import com.dmetasoul.lakesoul.meta.entity.FileOp
+import com.dmetasoul.lakesoul.meta.entity.{FileOp, Uuid}
import org.apache.spark.internal.Logging
import org.apache.spark.sql.lakesoul.exception.LakeSoulErrors
import org.apache.spark.sql.lakesoul.utils._
@@ -47,7 +47,7 @@ object MetaCommit extends Logging {
val partitionInfo = entity.PartitionInfo.newBuilder
partitionInfo.setTableId(table_info.table_id)
partitionInfo.setPartitionDesc(partition_info.range_value)
- partitionInfo.addAllSnapshot(JavaConverters.bufferAsJavaList(partition_info.read_files.map(uuid => uuid.toString).toBuffer))
+ partitionInfo.addAllSnapshot(JavaConverters.bufferAsJavaList(partition_info.read_files.map(uuid => DBUtil.toProtoUuid(uuid)).toBuffer))
partitionInfo.setCommitOp(commit_type)
javaPartitionInfoList.add(partitionInfo.build)
}
@@ -60,7 +60,7 @@ object MetaCommit extends Logging {
partitionInfo.setTableId(table_info.table_id)
partitionInfo.setPartitionDesc(partition.range_value)
partitionInfo.setVersion(partition.version)
- partitionInfo.addAllSnapshot(JavaConverters.bufferAsJavaList(partition.read_files.map(uuid => uuid.toString).toBuffer))
+ partitionInfo.addAllSnapshot(JavaConverters.bufferAsJavaList(partition.read_files.map(uuid => Uuid.newBuilder.setHigh(uuid.getMostSignificantBits).setLow(uuid.getLeastSignificantBits).build).toBuffer))
partitionInfo.setCommitOp(commit_type)
readPartitionInfoList.add(partitionInfo.build)
}
@@ -96,7 +96,8 @@ object MetaCommit extends Logging {
metaDataCommitInfo.setTableId(table_id)
metaDataCommitInfo.setPartitionDesc(dataCommitInfo.range_value)
metaDataCommitInfo.setCommitOp(entity.CommitOp.valueOf(commitType))
- metaDataCommitInfo.setCommitId(dataCommitInfo.commit_id.toString)
+ val uuid = dataCommitInfo.commit_id
+ metaDataCommitInfo.setCommitId(Uuid.newBuilder.setHigh(uuid.getMostSignificantBits).setLow(uuid.getLeastSignificantBits).build)
val fileOps = new util.ArrayList[entity.DataFileOp]()
for (file_info <- dataCommitInfo.file_ops) {
val metaDataFileInfo = entity.DataFileOp.newBuilder
diff --git a/lakesoul-spark/src/main/scala/com/dmetasoul/lakesoul/meta/MetaVersion.scala b/lakesoul-spark/src/main/scala/com/dmetasoul/lakesoul/meta/MetaVersion.scala
index 9b111cca5..431e79f7f 100644
--- a/lakesoul-spark/src/main/scala/com/dmetasoul/lakesoul/meta/MetaVersion.scala
+++ b/lakesoul-spark/src/main/scala/com/dmetasoul/lakesoul/meta/MetaVersion.scala
@@ -125,7 +125,7 @@ object MetaVersion {
table_id = info.getTableId,
range_value = range_value,
version = info.getVersion,
- read_files = info.getSnapshotList.asScala.map(str => UUID.fromString(str)).toArray,
+ read_files = info.getSnapshotList.asScala.map(uuid => DBUtil.toJavaUUID(uuid)).toArray,
expression = info.getExpression,
commit_op = info.getCommitOp.name()
)
@@ -138,7 +138,7 @@ object MetaVersion {
table_id = info.getTableId,
range_value = range_value,
version = info.getVersion,
- read_files = info.getSnapshotList.asScala.map(str => UUID.fromString(str)).toArray,
+ read_files = info.getSnapshotList.asScala.map(uuid => DBUtil.toJavaUUID(uuid)).toArray,
expression = info.getExpression,
commit_op = info.getCommitOp.name()
)
@@ -192,7 +192,7 @@ object MetaVersion {
table_id = res.getTableId,
range_value = res.getPartitionDesc,
version = res.getVersion,
- read_files = res.getSnapshotList.asScala.map(str => UUID.fromString(str)).toArray,
+ read_files = res.getSnapshotList.asScala.map(uuid => DBUtil.toJavaUUID(uuid)).toArray,
expression = res.getExpression,
commit_op = res.getCommitOp.name
)
diff --git a/lakesoul-spark/src/main/scala/com/dmetasoul/lakesoul/meta/StreamingRecord.scala b/lakesoul-spark/src/main/scala/com/dmetasoul/lakesoul/meta/StreamingRecord.scala
index bf2ce01f4..d825fd689 100644
--- a/lakesoul-spark/src/main/scala/com/dmetasoul/lakesoul/meta/StreamingRecord.scala
+++ b/lakesoul-spark/src/main/scala/com/dmetasoul/lakesoul/meta/StreamingRecord.scala
@@ -12,7 +12,7 @@ object StreamingRecord {
def getBatchId(tableId: String, queryId: String): Long = {
try {
- val commitId = UUID.fromString(dbManager.selectByTableId(tableId).getCommitId)
+ val commitId = DBUtil.toJavaUUID(dbManager.selectByTableId(tableId).getCommitId)
if (commitId.getMostSignificantBits.equals(UUID.fromString(queryId).getMostSignificantBits)) {
commitId.getLeastSignificantBits
} else {
diff --git a/lakesoul-spark/src/main/scala/org/apache/spark/sql/lakesoul/LakeSoulUtils.scala b/lakesoul-spark/src/main/scala/org/apache/spark/sql/lakesoul/LakeSoulUtils.scala
index 3d08f2b99..99d7580d9 100644
--- a/lakesoul-spark/src/main/scala/org/apache/spark/sql/lakesoul/LakeSoulUtils.scala
+++ b/lakesoul-spark/src/main/scala/org/apache/spark/sql/lakesoul/LakeSoulUtils.scala
@@ -212,7 +212,7 @@ object LakeSoulTableRelationV2 {
object LakeSoulTableV2ScanRelation {
def unapply(plan: LogicalPlan): Option[DataSourceV2ScanRelation] = plan match {
- case dsv2@DataSourceV2Relation(t: LakeSoulTableV2, _, _, _, _) => Some( createScanRelation(t, dsv2))
+ case dsv2@DataSourceV2Relation(t: LakeSoulTableV2, _, _, _, _) => Some(createScanRelation(t, dsv2))
case _ => None
}
diff --git a/lakesoul-spark/src/main/scala/org/apache/spark/sql/lakesoul/Snapshot.scala b/lakesoul-spark/src/main/scala/org/apache/spark/sql/lakesoul/Snapshot.scala
index 32445983c..1926b268c 100644
--- a/lakesoul-spark/src/main/scala/org/apache/spark/sql/lakesoul/Snapshot.scala
+++ b/lakesoul-spark/src/main/scala/org/apache/spark/sql/lakesoul/Snapshot.scala
@@ -55,4 +55,5 @@ class Snapshot(table_info: TableInfo,
def getPartitionInfoArray: Array[PartitionInfo] = partition_info_arr
+ override def toString: String = table_info + partition_info_arr.mkString(",")
}
diff --git a/lakesoul-spark/src/main/scala/org/apache/spark/sql/lakesoul/SnapshotManagement.scala b/lakesoul-spark/src/main/scala/org/apache/spark/sql/lakesoul/SnapshotManagement.scala
index 6fc3bc341..7de9b6ae6 100644
--- a/lakesoul-spark/src/main/scala/org/apache/spark/sql/lakesoul/SnapshotManagement.scala
+++ b/lakesoul-spark/src/main/scala/org/apache/spark/sql/lakesoul/SnapshotManagement.scala
@@ -55,7 +55,6 @@ class SnapshotManagement(path: String, namespace: String) extends Logging {
private def getCurrentSnapshot: Snapshot = {
-
if (LakeSoulSourceUtils.isLakeSoulTableExists(table_path)) {
createSnapshot
} else {
@@ -97,13 +96,13 @@ class SnapshotManagement(path: String, namespace: String) extends Logging {
}
/**
- * Execute a piece of code within a new [[TransactionCommit]]. Reads/write sets will
- * be recorded for this table, and all other tables will be read
- * at a snapshot that is pinned on the first access.
- *
- * @note This uses thread-local variable to make the active transaction visible. So do not use
- * multi-threaded code in the provided thunk.
- */
+ * Execute a piece of code within a new [[TransactionCommit]]. Reads/write sets will
+ * be recorded for this table, and all other tables will be read
+ * at a snapshot that is pinned on the first access.
+ *
+ * @note This uses thread-local variable to make the active transaction visible. So do not use
+ * multi-threaded code in the provided thunk.
+ */
def withNewTransaction[T](thunk: TransactionCommit => T): T = {
try {
val tc = startTransaction()
@@ -115,9 +114,9 @@ class SnapshotManagement(path: String, namespace: String) extends Logging {
}
/**
- * Checks whether this table only accepts appends. If so it will throw an error in operations that
- * can remove data such as DELETE/UPDATE/MERGE.
- */
+ * Checks whether this table only accepts appends. If so it will throw an error in operations that
+ * can remove data such as DELETE/UPDATE/MERGE.
+ */
def assertRemovable(): Unit = {
if (LakeSoulConfig.IS_APPEND_ONLY.fromTableInfo(snapshot.getTableInfo)) {
throw LakeSoulErrors.modifyAppendOnlyTableException
@@ -138,9 +137,9 @@ class SnapshotManagement(path: String, namespace: String) extends Logging {
object SnapshotManagement {
/**
- * We create only a single [[SnapshotManagement]] for any given path to avoid wasted work
- * in reconstructing.
- */
+ * We create only a single [[SnapshotManagement]] for any given path to avoid wasted work
+ * in reconstructing.
+ */
private val snapshotManagementCache = {
val expireMin = if (SparkSession.getActiveSession.isDefined) {
SparkSession.getActiveSession.get.conf.get(LakeSoulSQLConf.SNAPSHOT_CACHE_EXPIRE)
diff --git a/lakesoul-spark/src/main/scala/org/apache/spark/sql/lakesoul/catalog/LakeSoulTableV2.scala b/lakesoul-spark/src/main/scala/org/apache/spark/sql/lakesoul/catalog/LakeSoulTableV2.scala
index 918059f5e..a420f5c49 100644
--- a/lakesoul-spark/src/main/scala/org/apache/spark/sql/lakesoul/catalog/LakeSoulTableV2.scala
+++ b/lakesoul-spark/src/main/scala/org/apache/spark/sql/lakesoul/catalog/LakeSoulTableV2.scala
@@ -46,13 +46,14 @@ case class LakeSoulTableV2(spark: SparkSession,
}
- private lazy val (rootPath, partitionFilters) =
+ private lazy val (rootPath, partitionFilters) = {
if (catalogTable.isDefined) {
// Fast path for reducing path munging overhead
(SparkUtil.makeQualifiedTablePath(new Path(catalogTable.get.location)), Nil)
} else {
LakeSoulDataSource.parsePathIdentifier(spark, path.toString)
}
+ }
// The loading of the SnapshotManagement is lazy in order to reduce the amount of FileSystem calls,
// in cases where we will fallback to the V1 behavior.
@@ -99,7 +100,7 @@ case class LakeSoulTableV2(spark: SparkSession,
override def capabilities(): java.util.Set[TableCapability] = {
var caps = Set(
BATCH_READ, V1_BATCH_WRITE, OVERWRITE_DYNAMIC,
- OVERWRITE_BY_FILTER, TRUNCATE ,MICRO_BATCH_READ
+ OVERWRITE_BY_FILTER, TRUNCATE, MICRO_BATCH_READ
)
if (spark.conf.get(LakeSoulSQLConf.SCHEMA_AUTO_MIGRATE)) {
caps += ACCEPT_ANY_SCHEMA
@@ -131,9 +132,9 @@ case class LakeSoulTableV2(spark: SparkSession,
}
/**
- * Creates a V1 BaseRelation from this Table to allow read APIs to go through V1 DataSource code
- * paths.
- */
+ * Creates a V1 BaseRelation from this Table to allow read APIs to go through V1 DataSource code
+ * paths.
+ */
def toBaseRelation: BaseRelation = {
val partitionPredicates = LakeSoulDataSource.verifyAndCreatePartitionFilters(
path.toString, snapshotManagement.snapshot, partitionFilters)
diff --git a/lakesoul-spark/src/test/scala/com/dmetasoul/lakesoul/meta/RBACOperatinSuite.scala b/lakesoul-spark/src/test/scala/com/dmetasoul/lakesoul/meta/RBACOperatinSuite.scala
index 3839d2261..42dcb9419 100644
--- a/lakesoul-spark/src/test/scala/com/dmetasoul/lakesoul/meta/RBACOperatinSuite.scala
+++ b/lakesoul-spark/src/test/scala/com/dmetasoul/lakesoul/meta/RBACOperatinSuite.scala
@@ -4,6 +4,7 @@
package com.dmetasoul.lakesoul.meta
+import com.dmetasoul.lakesoul.meta.jnr.NativeMetadataJavaClient
import com.dmetasoul.lakesoul.meta.rbac.AuthZEnforcer
import org.apache.hadoop.fs.Path
import org.apache.spark.sql.catalyst.analysis.{NoSuchNamespaceException, UnresolvedTableOrView}
@@ -42,8 +43,8 @@ class RBACOperatinSuite extends QueryTest
System.setProperty(DBUtil.usernameKey, username)
System.setProperty(DBUtil.passwordKey, password)
System.setProperty(DBUtil.domainKey, domain)
- DBConnector.closeAllConnections()
-
+ DBConnector.closeAllConnections
+ NativeMetadataJavaClient.closeAll
}
test("testDifferentDomain") {
@@ -53,10 +54,10 @@ class RBACOperatinSuite extends QueryTest
val df = spark.sql("show databases").toDF()
assert(df.count() == 2)
// drop: coming soon
-// spark.sql("drop database database1").collect()
-// val df2 = spark.sql("show databases").toDF()
-// assert(df2.count() == 1)
-// assert(df2.collectAsList().get(0).getString(1).equals("default"))
+ // spark.sql("drop database database1").collect()
+ // val df2 = spark.sql("show databases").toDF()
+ // assert(df2.count() == 1)
+ // assert(df2.collectAsList().get(0).getString(1).equals("default"))
// create tables
spark.sql("use database1;")
spark.sql("create table if not exists table1 ( id int, foo string, bar string ) using lakesoul ")
@@ -78,7 +79,7 @@ class RBACOperatinSuite extends QueryTest
assert(df5.count() == 2)
// update data
- spark.sql("update table1 set foo = 'foo3', bar = 'bar3' where id = 2")
+ spark.sql("update table1 set foo = 'foo3', bar = 'bar3' where id = 2")
val df6 = spark.sql("select (id, foo, bar) from table1 where id = 2").toDF()
val row = df6.collectAsList().get(0).get(0).asInstanceOf[GenericRowWithSchema];
assert(row.getString(1).equals("foo3"))
@@ -91,14 +92,14 @@ class RBACOperatinSuite extends QueryTest
- // create & drop database
+ // create & drop database
spark.sql("insert into table1 values(3, 'foo3', 'bar3')")
login(ADMIN2, ADMIN2_PASS, DOMAIN1)
val err0 = intercept[Exception] {
spark.sql("use database1;")
}
assert(err0.isInstanceOf[NoSuchNamespaceException])
- val err1 = intercept[Exception] {
+ val err1 = intercept[Exception] {
spark.sql("create database if not exists database2")
}
println(err1.getMessage)
diff --git a/lakesoul-spark/src/test/scala/org/apache/spark/sql/lakesoul/DDLSuite.scala b/lakesoul-spark/src/test/scala/org/apache/spark/sql/lakesoul/DDLSuite.scala
index 08dee193d..47914d33b 100644
--- a/lakesoul-spark/src/test/scala/org/apache/spark/sql/lakesoul/DDLSuite.scala
+++ b/lakesoul-spark/src/test/scala/org/apache/spark/sql/lakesoul/DDLSuite.scala
@@ -89,6 +89,7 @@ abstract class DDLTestBase extends QueryTest with SQLTestUtils {
.write.format("lakesoul").mode("append").save(location.get)
}
}
+ waitForTasksToFinish()
}
}
diff --git a/lakesoul-spark/src/test/scala/org/apache/spark/sql/lakesoul/DataFrameWriterV2Suite.scala b/lakesoul-spark/src/test/scala/org/apache/spark/sql/lakesoul/DataFrameWriterV2Suite.scala
index 6984cc95d..1571cc19c 100644
--- a/lakesoul-spark/src/test/scala/org/apache/spark/sql/lakesoul/DataFrameWriterV2Suite.scala
+++ b/lakesoul-spark/src/test/scala/org/apache/spark/sql/lakesoul/DataFrameWriterV2Suite.scala
@@ -40,7 +40,7 @@ trait DataFrameWriterV2Tests
val catalog = spark.sessionState.catalogManager.currentCatalog.asInstanceOf[LakeSoulCatalog]
catalog
.listTables(Array("default"))
- .foreach { ti => catalog.dropTable(ti)}
+ .foreach { ti => catalog.dropTable(ti) }
}
def catalog: TableCatalog = {
diff --git a/lakesoul-spark/src/test/scala/org/apache/spark/sql/lakesoul/InsertIntoTableSuite.scala b/lakesoul-spark/src/test/scala/org/apache/spark/sql/lakesoul/InsertIntoTableSuite.scala
index 4ba23f470..a27e231df 100644
--- a/lakesoul-spark/src/test/scala/org/apache/spark/sql/lakesoul/InsertIntoTableSuite.scala
+++ b/lakesoul-spark/src/test/scala/org/apache/spark/sql/lakesoul/InsertIntoTableSuite.scala
@@ -5,6 +5,7 @@
package org.apache.spark.sql.lakesoul
// scalastyle:off import.ordering.noEmptyLine
+
import org.apache.spark.sql._
import org.apache.spark.sql.functions.{col, lit, struct}
import org.apache.spark.sql.internal.SQLConf
@@ -24,6 +25,8 @@ import scala.collection.JavaConverters._
@RunWith(classOf[JUnitRunner])
class InsertIntoSQLSuite extends InsertIntoTests(false, true)
with LakeSoulSQLCommandTest {
+ override def suiteName: String = "InsertIntoSQLSuite"
+
override protected def doInsert(tableName: String, insert: DataFrame, mode: SaveMode): Unit = {
val tmpView = "tmp_view"
withTempView(tmpView) {
@@ -37,6 +40,8 @@ class InsertIntoSQLSuite extends InsertIntoTests(false, true)
@RunWith(classOf[JUnitRunner])
class InsertIntoSQLByPathSuite extends InsertIntoTests(false, true)
with LakeSoulSQLCommandTest {
+ override def suiteName: String = "InsertIntoSQLByPathSuite"
+
override protected def doInsert(tableName: String, insert: DataFrame, mode: SaveMode): Unit = {
val tmpView = "tmp_view"
withTempView(tmpView) {
@@ -76,6 +81,8 @@ class InsertIntoSQLByPathSuite extends InsertIntoTests(false, true)
@RunWith(classOf[JUnitRunner])
class InsertIntoDataFrameSuite extends InsertIntoTests(false, false)
with LakeSoulSQLCommandTest {
+ override def suiteName: String = "InsertIntoDataFrameSuite"
+
override protected def doInsert(tableName: String, insert: DataFrame, mode: SaveMode): Unit = {
val dfw = insert.write.format(v2Format)
if (mode != null) {
@@ -88,6 +95,8 @@ class InsertIntoDataFrameSuite extends InsertIntoTests(false, false)
@RunWith(classOf[JUnitRunner])
class InsertIntoDataFrameByPathSuite extends InsertIntoTests(false, false)
with LakeSoulSQLCommandTest {
+ override def suiteName: String = "InsertIntoDataFrameByPathSuite"
+
override protected def doInsert(tableName: String, insert: DataFrame, mode: SaveMode): Unit = {
val dfw = insert.write.format(v2Format)
if (mode != null) {
@@ -138,6 +147,8 @@ abstract class InsertIntoTests(
import testImplicits._
+ override def suiteName: String = ???
+
override def afterEach(): Unit = {
val catalog = spark.sessionState.catalogManager.currentCatalog.asInstanceOf[LakeSoulCatalog]
catalog.listTables(Array("default")).foreach(t => {
@@ -188,7 +199,7 @@ abstract class InsertIntoTests(
}
test("insertInto: append non partitioned table and read with filter") {
- val t1 = "default.tbl"
+ val t1 = "default.tbl" + System.currentTimeMillis()
withTable(t1) {
sql(s"CREATE TABLE $t1 (id bigint, data string) USING $v2Format")
val df = Seq((1L, "a"), (2L, "b"), (3L, "c")).toDF("id", "data")
@@ -199,7 +210,7 @@ abstract class InsertIntoTests(
}
test("insertInto: append partitioned table and read with partition filter") {
- val t1 = "default.tbl"
+ val t1 = "default.tbl" + System.currentTimeMillis()
withTable(t1) {
sql(s"CREATE TABLE $t1 (id bigint, data string) USING $v2Format PARTITIONED BY(id)")
val df = Seq((1L, "a"), (2L, "b"), (3L, "c")).toDF("id", "data").select("data", "id")
@@ -482,6 +493,7 @@ abstract class InsertIntoTests(
if (diff.nonEmpty) {
fail(diff.mkString("\n"))
}
+ waitForTasksToFinish()
}
}
@@ -538,10 +550,12 @@ trait InsertIntoSQLOnlyTests
/** Check that the results in `tableName` match the `expected` DataFrame. */
protected def verifyTable(tableName: String, expected: DataFrame): Unit = {
checkAnswer(spark.table(tableName), expected)
+ waitForTasksToFinish()
}
protected def verifyTable(tableName: String, expected: DataFrame, colNames: Seq[String]): Unit = {
checkAnswer(spark.table(tableName).select(colNames.map(col): _*), expected)
+ waitForTasksToFinish()
}
protected val v2Format: String = "lakesoul"
diff --git a/lakesoul-spark/src/test/scala/org/apache/spark/sql/lakesoul/LakeSoulSinkSuite.scala b/lakesoul-spark/src/test/scala/org/apache/spark/sql/lakesoul/LakeSoulSinkSuite.scala
index a14cce8d3..45a28c6fd 100644
--- a/lakesoul-spark/src/test/scala/org/apache/spark/sql/lakesoul/LakeSoulSinkSuite.scala
+++ b/lakesoul-spark/src/test/scala/org/apache/spark/sql/lakesoul/LakeSoulSinkSuite.scala
@@ -52,24 +52,24 @@ class LakeSoulSinkSuite extends StreamTest with LakeSoulTestUtils {
checkDatasetUnorderly(outputDf.as[Int], 1)
val snapshotManagement = SnapshotManagement(outputDir.getCanonicalPath)
-// val tableId = snapshotManagement.snapshot.getTableInfo.table_id
-// var info = StreamingRecord.getStreamingInfo(tableId)
-// assert(info._1.equals(query.id.toString) && info._2 == 0L)
+ // val tableId = snapshotManagement.snapshot.getTableInfo.table_id
+ // var info = StreamingRecord.getStreamingInfo(tableId)
+ // assert(info._1.equals(query.id.toString) && info._2 == 0L)
inputData.addData(2)
query.processAllAvailable()
checkDatasetUnorderly(outputDf.as[Int], 1, 2)
-// info = StreamingRecord.getStreamingInfo(tableId)
-// assert(info._1.equals(query.id.toString) && info._2 == 1L)
+ // info = StreamingRecord.getStreamingInfo(tableId)
+ // assert(info._1.equals(query.id.toString) && info._2 == 1L)
inputData.addData(3)
query.processAllAvailable()
checkDatasetUnorderly(outputDf.as[Int], 1, 2, 3)
-// info = StreamingRecord.getStreamingInfo(tableId)
-// assert(info._1.equals(query.id.toString) && info._2 == 2L)
+ // info = StreamingRecord.getStreamingInfo(tableId)
+ // assert(info._1.equals(query.id.toString) && info._2 == 2L)
} finally {
query.stop()
}
@@ -97,23 +97,23 @@ class LakeSoulSinkSuite extends StreamTest with LakeSoulTestUtils {
checkDatasetUnorderly(outputDf.as[Long], 1L)
val snapshotManagement = SnapshotManagement(outputDir.getCanonicalPath)
-// val tableId = snapshotManagement.snapshot.getTableInfo.table_id
-// var info = StreamingRecord.getStreamingInfo(tableId)
-// assert(info._1.equals(query.id.toString) && info._2 == 0L)
+ // val tableId = snapshotManagement.snapshot.getTableInfo.table_id
+ // var info = StreamingRecord.getStreamingInfo(tableId)
+ // assert(info._1.equals(query.id.toString) && info._2 == 0L)
inputData.addData(2)
query.processAllAvailable()
checkDatasetUnorderly(outputDf.as[Long], 2L)
-// info = StreamingRecord.getStreamingInfo(tableId)
-// assert(info._1.equals(query.id.toString) && info._2 == 1L)
+ // info = StreamingRecord.getStreamingInfo(tableId)
+ // assert(info._1.equals(query.id.toString) && info._2 == 1L)
inputData.addData(3)
query.processAllAvailable()
checkDatasetUnorderly(outputDf.as[Long], 3L)
-// info = StreamingRecord.getStreamingInfo(tableId)
-// assert(info._1.equals(query.id.toString) && info._2 == 2L)
+ // info = StreamingRecord.getStreamingInfo(tableId)
+ // assert(info._1.equals(query.id.toString) && info._2 == 2L)
} finally {
query.stop()
}
diff --git a/lakesoul-spark/src/test/scala/org/apache/spark/sql/lakesoul/TableCreationTests.scala b/lakesoul-spark/src/test/scala/org/apache/spark/sql/lakesoul/TableCreationTests.scala
index 1f5c712e5..1a74f18a2 100644
--- a/lakesoul-spark/src/test/scala/org/apache/spark/sql/lakesoul/TableCreationTests.scala
+++ b/lakesoul-spark/src/test/scala/org/apache/spark/sql/lakesoul/TableCreationTests.scala
@@ -42,9 +42,9 @@ trait TableCreationTests
val format = "lakesoul"
protected def createTableByPath(path: File,
- df: DataFrame,
- tableName: String,
- partitionedBy: Seq[String] = Nil): Unit = {
+ df: DataFrame,
+ tableName: String,
+ partitionedBy: Seq[String] = Nil): Unit = {
df.write
.partitionBy(partitionedBy: _*)
.mode(SaveMode.Append)
@@ -673,7 +673,7 @@ trait TableCreationTests
assert(location.isDefined)
assert(location.get == path.get)
val partDir = new File(new File(location.get), "a=1")
-// assert(partDir.listFiles().nonEmpty)
+ // assert(partDir.listFiles().nonEmpty)
checkDatasetUnorderly(
sql("SELECT a,b FROM lakesoul_test").as[(Long, String)],
@@ -1217,7 +1217,7 @@ trait TableCreationTests
.create()
val tableInfo = SnapshotManagement(SparkUtil.makeQualifiedTablePath(new Path(path)).toString).getTableInfoOnly
- tableInfo.configuration should contain ("lakesoul_cdc_change_column" -> "change_kind")
+ tableInfo.configuration should contain("lakesoul_cdc_change_column" -> "change_kind")
})
}
}
@@ -1233,7 +1233,7 @@ trait TableCreationTests
.option("lakesoul_cdc_change_column", "change_kind")
.save(path)
val tableInfo = SnapshotManagement(SparkUtil.makeQualifiedTablePath(new Path(path)).toString).getTableInfoOnly
- tableInfo.configuration should contain ("lakesoul_cdc_change_column" -> "change_kind")
+ tableInfo.configuration should contain("lakesoul_cdc_change_column" -> "change_kind")
})
}
}
diff --git a/lakesoul-spark/src/test/scala/org/apache/spark/sql/lakesoul/commands/CompactionSuite.scala b/lakesoul-spark/src/test/scala/org/apache/spark/sql/lakesoul/commands/CompactionSuite.scala
index 73500935e..39867eaf3 100644
--- a/lakesoul-spark/src/test/scala/org/apache/spark/sql/lakesoul/commands/CompactionSuite.scala
+++ b/lakesoul-spark/src/test/scala/org/apache/spark/sql/lakesoul/commands/CompactionSuite.scala
@@ -47,7 +47,7 @@ class CompactionSuite extends QueryTest
.format("lakesoul")
.save(tableName)
- assert(SnapshotManagement(SparkUtil.makeQualifiedTablePath(new Path(tableName)).toString).snapshot.getPartitionInfoArray.forall(_.read_files.size==1))
+ assert(SnapshotManagement(SparkUtil.makeQualifiedTablePath(new Path(tableName)).toString).snapshot.getPartitionInfoArray.forall(_.read_files.size == 1))
})
}
@@ -66,8 +66,8 @@ class CompactionSuite extends QueryTest
.save(tableName)
val sm = SnapshotManagement(SparkUtil.makeQualifiedTablePath(new Path(tableName)).toString)
- var rangeGroup = SparkUtil.allDataInfo(sm.updateSnapshot()).groupBy(_.range_partitions)
- assert(rangeGroup.forall(_._2.groupBy(_.file_bucket_id).forall(_._2.length == 1)))
+ var rangeGroup = SparkUtil.allDataInfo(sm.updateSnapshot()).groupBy(_.range_partitions)
+ assert(rangeGroup.forall(_._2.groupBy(_.file_bucket_id).forall(_._2.length == 1)))
val df2 = Seq((1, 1, 1), (2, 1, 1), (3, 1, 1), (1, 2, 2), (1, 3, 3))
@@ -77,8 +77,8 @@ class CompactionSuite extends QueryTest
LakeSoulTable.forPath(tableName).upsert(df2)
}
- rangeGroup = SparkUtil.allDataInfo(sm.updateSnapshot()).groupBy(_.range_partitions)
- assert(!rangeGroup.forall(_._2.groupBy(_.file_bucket_id).forall(_._2.length == 1)))
+ rangeGroup = SparkUtil.allDataInfo(sm.updateSnapshot()).groupBy(_.range_partitions)
+ assert(!rangeGroup.forall(_._2.groupBy(_.file_bucket_id).forall(_._2.length == 1)))
LakeSoulTable.forPath(tableName).compaction(true)
@@ -111,7 +111,7 @@ class CompactionSuite extends QueryTest
val sm = SnapshotManagement(SparkUtil.makeQualifiedTablePath(new Path(tableName)).toString)
- val rangeInfo = SparkUtil.allDataInfo(sm.snapshot).filter(_.range_partitions.equals("range=1"))
+ val rangeInfo = SparkUtil.allDataInfo(sm.snapshot).filter(_.range_partitions.equals("range=1"))
assert(!rangeInfo.groupBy(_.file_bucket_id).forall(_._2.length == 1))
@@ -208,7 +208,6 @@ class CompactionSuite extends QueryTest
}
-
test("simple compaction with merge operator") {
withTempDir(file => {
val tableName = file.getCanonicalPath
diff --git a/lakesoul-spark/src/test/scala/org/apache/spark/sql/lakesoul/test/LakeSoulSQLCommandTest.scala b/lakesoul-spark/src/test/scala/org/apache/spark/sql/lakesoul/test/LakeSoulSQLCommandTest.scala
index f979f89d6..3d4fa84b9 100644
--- a/lakesoul-spark/src/test/scala/org/apache/spark/sql/lakesoul/test/LakeSoulSQLCommandTest.scala
+++ b/lakesoul-spark/src/test/scala/org/apache/spark/sql/lakesoul/test/LakeSoulSQLCommandTest.scala
@@ -36,7 +36,7 @@ trait LakeSoulTestUtils extends Logging {
tableNames.foreach { name =>
spark.sql(s"DROP TABLE IF EXISTS $name")
if (name.split("\\.").length == 1) {
- val databaseName = if (name.startsWith(testDatabase+".")) name else s"$testDatabase.$name"
+ val databaseName = if (name.startsWith(testDatabase + ".")) name else s"$testDatabase.$name"
spark.sql(s"DROP TABLE IF EXISTS $databaseName")
}
}
@@ -59,7 +59,7 @@ trait LakeSoulTestUtils extends Logging {
}
def createDF(seq: Seq[Product], names: Seq[String],
- types: Seq[String], nullables: Option[Seq[Boolean]] = None): DataFrame = {
+ types: Seq[String], nullables: Option[Seq[Boolean]] = None): DataFrame = {
val fields = nullables match {
case None =>
names.zip(types).map(nt => StructField(nt._1, CatalystSqlParser.parseDataType(nt._2), nullable = false))
diff --git a/native-io/lakesoul-io-java/src/main/java/com/dmetasoul/lakesoul/lakesoul/io/jnr/JnrLoader.java b/native-io/lakesoul-io-java/src/main/java/com/dmetasoul/lakesoul/lakesoul/io/jnr/JnrLoader.java
index 469d25422..4ff02e61c 100644
--- a/native-io/lakesoul-io-java/src/main/java/com/dmetasoul/lakesoul/lakesoul/io/jnr/JnrLoader.java
+++ b/native-io/lakesoul-io-java/src/main/java/com/dmetasoul/lakesoul/lakesoul/io/jnr/JnrLoader.java
@@ -6,14 +6,12 @@
import jnr.ffi.LibraryLoader;
import jnr.ffi.LibraryOption;
-import org.apache.arrow.c.jni.JniWrapper;
import java.io.File;
import java.io.FileNotFoundException;
import java.io.IOException;
import java.io.InputStream;
import java.nio.file.Files;
-import java.nio.file.Paths;
import java.nio.file.StandardCopyOption;
import java.util.HashMap;
import java.util.Map;
@@ -39,12 +37,12 @@ public synchronized static void tryLoad() {
String libName = System.mapLibraryName("lakesoul_io_c");
String finalPath = null;
-
+
if (JnrLoader.class.getClassLoader().getResource(libName) != null) {
try {
File temp = File.createTempFile(libName + "_", ".tmp", new File(System.getProperty("java.io.tmpdir")));
temp.deleteOnExit();
- try (final InputStream is = JniWrapper.class.getClassLoader().getResourceAsStream(libName)) {
+ try (final InputStream is = JnrLoader.class.getClassLoader().getResourceAsStream(libName)) {
if (is == null) {
throw new FileNotFoundException(libName);
}
diff --git a/native-metadata/.gitignore b/native-metadata/.gitignore
index 850830b05..2d00c94e4 100644
--- a/native-metadata/.gitignore
+++ b/native-metadata/.gitignore
@@ -3,5 +3,4 @@
# SPDX-License-Identifier: Apache-2.0
/target/
-/.vscode/
-Cargo.lock
\ No newline at end of file
+/.vscode/
\ No newline at end of file
diff --git a/native-metadata/Cargo.lock b/native-metadata/Cargo.lock
new file mode 100644
index 000000000..c9106cb2b
--- /dev/null
+++ b/native-metadata/Cargo.lock
@@ -0,0 +1,1288 @@
+# This file is automatically @generated by Cargo.
+# It is not intended for manual editing.
+version = 3
+
+[[package]]
+name = "addr2line"
+version = "0.20.0"
+source = "registry+https://github.com/rust-lang/crates.io-index"
+checksum = "f4fa78e18c64fce05e902adecd7a5eed15a5e0a3439f7b0e169f0252214865e3"
+dependencies = [
+ "gimli",
+]
+
+[[package]]
+name = "adler"
+version = "1.0.2"
+source = "registry+https://github.com/rust-lang/crates.io-index"
+checksum = "f26201604c87b1e01bd3d98f8d5d9a8fcbb815e8cedb41ffccbeb4bf593a35fe"
+
+[[package]]
+name = "aho-corasick"
+version = "1.0.2"
+source = "registry+https://github.com/rust-lang/crates.io-index"
+checksum = "43f6cb1bf222025340178f382c426f13757b2960e89779dfcb319c32542a5a41"
+dependencies = [
+ "memchr",
+]
+
+[[package]]
+name = "anyhow"
+version = "1.0.71"
+source = "registry+https://github.com/rust-lang/crates.io-index"
+checksum = "9c7d0618f0e0b7e8ff11427422b64564d5fb0be1940354bfe2e0529b18a9d9b8"
+
+[[package]]
+name = "array-init"
+version = "2.1.0"
+source = "registry+https://github.com/rust-lang/crates.io-index"
+checksum = "3d62b7694a562cdf5a74227903507c56ab2cc8bdd1f781ed5cb4cf9c9f810bfc"
+
+[[package]]
+name = "async-trait"
+version = "0.1.71"
+source = "registry+https://github.com/rust-lang/crates.io-index"
+checksum = "a564d521dd56509c4c47480d00b80ee55f7e385ae48db5744c67ad50c92d2ebf"
+dependencies = [
+ "proc-macro2",
+ "quote",
+ "syn 2.0.25",
+]
+
+[[package]]
+name = "autocfg"
+version = "1.1.0"
+source = "registry+https://github.com/rust-lang/crates.io-index"
+checksum = "d468802bab17cbc0cc575e9b053f41e72aa36bfa6b7f55e3529ffa43161b97fa"
+
+[[package]]
+name = "autotools"
+version = "0.2.6"
+source = "registry+https://github.com/rust-lang/crates.io-index"
+checksum = "aef8da1805e028a172334c3b680f93e71126f2327622faef2ec3d893c0a4ad77"
+dependencies = [
+ "cc",
+]
+
+[[package]]
+name = "backtrace"
+version = "0.3.68"
+source = "registry+https://github.com/rust-lang/crates.io-index"
+checksum = "4319208da049c43661739c5fade2ba182f09d1dc2299b32298d3a31692b17e12"
+dependencies = [
+ "addr2line",
+ "cc",
+ "cfg-if",
+ "libc",
+ "miniz_oxide",
+ "object",
+ "rustc-demangle",
+]
+
+[[package]]
+name = "base64"
+version = "0.21.2"
+source = "registry+https://github.com/rust-lang/crates.io-index"
+checksum = "604178f6c5c21f02dc555784810edfb88d34ac2c73b2eae109655649ee73ce3d"
+
+[[package]]
+name = "bitflags"
+version = "1.3.2"
+source = "registry+https://github.com/rust-lang/crates.io-index"
+checksum = "bef38d45163c2f1dde094a7dfd33ccf595c92905c8f8f4fdc18d06fb1037718a"
+
+[[package]]
+name = "block-buffer"
+version = "0.10.4"
+source = "registry+https://github.com/rust-lang/crates.io-index"
+checksum = "3078c7629b62d3f0439517fa394996acacc5cbc91c5a20d8c658e77abd503a71"
+dependencies = [
+ "generic-array",
+]
+
+[[package]]
+name = "byteorder"
+version = "1.4.3"
+source = "registry+https://github.com/rust-lang/crates.io-index"
+checksum = "14c189c53d098945499cdfa7ecc63567cf3886b3332b312a5b4585d8d3a6a610"
+
+[[package]]
+name = "bytes"
+version = "1.4.0"
+source = "registry+https://github.com/rust-lang/crates.io-index"
+checksum = "89b2fd2a0dcf38d7971e2194b6b6eebab45ae01067456a7fd93d5547a61b70be"
+
+[[package]]
+name = "cc"
+version = "1.0.79"
+source = "registry+https://github.com/rust-lang/crates.io-index"
+checksum = "50d30906286121d95be3d479533b458f87493b30a4b5f79a607db8f5d11aa91f"
+
+[[package]]
+name = "cfg-if"
+version = "1.0.0"
+source = "registry+https://github.com/rust-lang/crates.io-index"
+checksum = "baf1de4339761588bc0619e3cbc0120ee582ebb74b53b4efbf79117bd2da40fd"
+
+[[package]]
+name = "cpufeatures"
+version = "0.2.9"
+source = "registry+https://github.com/rust-lang/crates.io-index"
+checksum = "a17b76ff3a4162b0b27f354a0c87015ddad39d35f9c0c36607a3bdd175dde1f1"
+dependencies = [
+ "libc",
+]
+
+[[package]]
+name = "crypto-common"
+version = "0.1.6"
+source = "registry+https://github.com/rust-lang/crates.io-index"
+checksum = "1bfb12502f3fc46cca1bb51ac28df9d618d813cdc3d2f25b9fe775a34af26bb3"
+dependencies = [
+ "generic-array",
+ "typenum",
+]
+
+[[package]]
+name = "digest"
+version = "0.10.7"
+source = "registry+https://github.com/rust-lang/crates.io-index"
+checksum = "9ed9a281f7bc9b7576e61468ba615a66a5c8cfdff42420a70aa82701a3b1e292"
+dependencies = [
+ "block-buffer",
+ "crypto-common",
+ "subtle",
+]
+
+[[package]]
+name = "either"
+version = "1.8.1"
+source = "registry+https://github.com/rust-lang/crates.io-index"
+checksum = "7fcaabb2fef8c910e7f4c7ce9f67a1283a1715879a7c230ca9d6d1ae31f16d91"
+
+[[package]]
+name = "equivalent"
+version = "1.0.1"
+source = "registry+https://github.com/rust-lang/crates.io-index"
+checksum = "5443807d6dff69373d433ab9ef5378ad8df50ca6298caf15de6e52e24aaf54d5"
+
+[[package]]
+name = "errno"
+version = "0.3.1"
+source = "registry+https://github.com/rust-lang/crates.io-index"
+checksum = "4bcfec3a70f97c962c307b2d2c56e358cf1d00b558d74262b5f929ee8cc7e73a"
+dependencies = [
+ "errno-dragonfly",
+ "libc",
+ "windows-sys",
+]
+
+[[package]]
+name = "errno-dragonfly"
+version = "0.1.2"
+source = "registry+https://github.com/rust-lang/crates.io-index"
+checksum = "aa68f1b12764fab894d2755d2518754e71b4fd80ecfb822714a1206c2aab39bf"
+dependencies = [
+ "cc",
+ "libc",
+]
+
+[[package]]
+name = "fallible-iterator"
+version = "0.2.0"
+source = "registry+https://github.com/rust-lang/crates.io-index"
+checksum = "4443176a9f2c162692bd3d352d745ef9413eec5782a80d8fd6f8a1ac692a07f7"
+
+[[package]]
+name = "fastrand"
+version = "1.9.0"
+source = "registry+https://github.com/rust-lang/crates.io-index"
+checksum = "e51093e27b0797c359783294ca4f0a911c270184cb10f85783b118614a1501be"
+dependencies = [
+ "instant",
+]
+
+[[package]]
+name = "fixedbitset"
+version = "0.4.2"
+source = "registry+https://github.com/rust-lang/crates.io-index"
+checksum = "0ce7134b9999ecaf8bcd65542e436736ef32ddca1b3e06094cb6ec5755203b80"
+
+[[package]]
+name = "futures-channel"
+version = "0.3.28"
+source = "registry+https://github.com/rust-lang/crates.io-index"
+checksum = "955518d47e09b25bbebc7a18df10b81f0c766eaf4c4f1cccef2fca5f2a4fb5f2"
+dependencies = [
+ "futures-core",
+ "futures-sink",
+]
+
+[[package]]
+name = "futures-core"
+version = "0.3.28"
+source = "registry+https://github.com/rust-lang/crates.io-index"
+checksum = "4bca583b7e26f571124fe5b7561d49cb2868d79116cfa0eefce955557c6fee8c"
+
+[[package]]
+name = "futures-macro"
+version = "0.3.28"
+source = "registry+https://github.com/rust-lang/crates.io-index"
+checksum = "89ca545a94061b6365f2c7355b4b32bd20df3ff95f02da9329b34ccc3bd6ee72"
+dependencies = [
+ "proc-macro2",
+ "quote",
+ "syn 2.0.25",
+]
+
+[[package]]
+name = "futures-sink"
+version = "0.3.28"
+source = "registry+https://github.com/rust-lang/crates.io-index"
+checksum = "f43be4fe21a13b9781a69afa4985b0f6ee0e1afab2c6f454a8cf30e2b2237b6e"
+
+[[package]]
+name = "futures-task"
+version = "0.3.28"
+source = "registry+https://github.com/rust-lang/crates.io-index"
+checksum = "76d3d132be6c0e6aa1534069c705a74a5997a356c0dc2f86a47765e5617c5b65"
+
+[[package]]
+name = "futures-util"
+version = "0.3.28"
+source = "registry+https://github.com/rust-lang/crates.io-index"
+checksum = "26b01e40b772d54cf6c6d721c1d1abd0647a0106a12ecaa1c186273392a69533"
+dependencies = [
+ "futures-core",
+ "futures-macro",
+ "futures-sink",
+ "futures-task",
+ "pin-project-lite",
+ "pin-utils",
+ "slab",
+]
+
+[[package]]
+name = "generic-array"
+version = "0.14.7"
+source = "registry+https://github.com/rust-lang/crates.io-index"
+checksum = "85649ca51fd72272d7821adaf274ad91c288277713d9c18820d8499a7ff69e9a"
+dependencies = [
+ "typenum",
+ "version_check",
+]
+
+[[package]]
+name = "getrandom"
+version = "0.2.10"
+source = "registry+https://github.com/rust-lang/crates.io-index"
+checksum = "be4136b2a15dd319360be1c07d9933517ccf0be8f16bf62a3bee4f0d618df427"
+dependencies = [
+ "cfg-if",
+ "libc",
+ "wasi",
+]
+
+[[package]]
+name = "gimli"
+version = "0.27.3"
+source = "registry+https://github.com/rust-lang/crates.io-index"
+checksum = "b6c80984affa11d98d1b88b66ac8853f143217b399d3c74116778ff8fdb4ed2e"
+
+[[package]]
+name = "hashbrown"
+version = "0.12.3"
+source = "registry+https://github.com/rust-lang/crates.io-index"
+checksum = "8a9ee70c43aaf417c914396645a0fa852624801b24ebb7ae78fe8272889ac888"
+
+[[package]]
+name = "hashbrown"
+version = "0.14.0"
+source = "registry+https://github.com/rust-lang/crates.io-index"
+checksum = "2c6201b9ff9fd90a5a3bac2e56a830d0caa509576f0e503818ee82c181b3437a"
+
+[[package]]
+name = "heck"
+version = "0.4.1"
+source = "registry+https://github.com/rust-lang/crates.io-index"
+checksum = "95505c38b4572b2d910cecb0281560f54b440a19336cbbcb27bf6ce6adc6f5a8"
+
+[[package]]
+name = "hermit-abi"
+version = "0.3.2"
+source = "registry+https://github.com/rust-lang/crates.io-index"
+checksum = "443144c8cdadd93ebf52ddb4056d257f5b52c04d3c804e657d19eb73fc33668b"
+
+[[package]]
+name = "hmac"
+version = "0.12.1"
+source = "registry+https://github.com/rust-lang/crates.io-index"
+checksum = "6c49c37c09c17a53d937dfbb742eb3a961d65a994e6bcdcf37e7399d0cc8ab5e"
+dependencies = [
+ "digest",
+]
+
+[[package]]
+name = "indexmap"
+version = "1.9.3"
+source = "registry+https://github.com/rust-lang/crates.io-index"
+checksum = "bd070e393353796e801d209ad339e89596eb4c8d430d18ede6a1cced8fafbd99"
+dependencies = [
+ "autocfg",
+ "hashbrown 0.12.3",
+]
+
+[[package]]
+name = "indexmap"
+version = "2.0.0"
+source = "registry+https://github.com/rust-lang/crates.io-index"
+checksum = "d5477fe2230a79769d8dc68e0eabf5437907c0457a5614a9e8dddb67f65eb65d"
+dependencies = [
+ "equivalent",
+ "hashbrown 0.14.0",
+]
+
+[[package]]
+name = "instant"
+version = "0.1.12"
+source = "registry+https://github.com/rust-lang/crates.io-index"
+checksum = "7a5bbe824c507c5da5956355e86a746d82e0e1464f65d862cc5e71da70e94b2c"
+dependencies = [
+ "cfg-if",
+]
+
+[[package]]
+name = "io-lifetimes"
+version = "1.0.11"
+source = "registry+https://github.com/rust-lang/crates.io-index"
+checksum = "eae7b9aee968036d54dce06cebaefd919e4472e753296daccd6d344e3e2df0c2"
+dependencies = [
+ "hermit-abi",
+ "libc",
+ "windows-sys",
+]
+
+[[package]]
+name = "itertools"
+version = "0.10.5"
+source = "registry+https://github.com/rust-lang/crates.io-index"
+checksum = "b0fd2260e829bddf4cb6ea802289de2f86d6a7a690192fbe91b3f46e0f2c8473"
+dependencies = [
+ "either",
+]
+
+[[package]]
+name = "itoa"
+version = "1.0.9"
+source = "registry+https://github.com/rust-lang/crates.io-index"
+checksum = "af150ab688ff2122fcef229be89cb50dd66af9e01a4ff320cc137eecc9bacc38"
+
+[[package]]
+name = "lakesoul-metadata"
+version = "0.1.0"
+dependencies = [
+ "num_enum",
+ "postgres",
+ "postgres-types",
+ "prost",
+ "proto",
+ "serde_json",
+ "tokio",
+ "tokio-postgres",
+ "uuid",
+]
+
+[[package]]
+name = "lakesoul-metadata-c"
+version = "0.1.0"
+dependencies = [
+ "lakesoul-metadata",
+ "prost",
+ "proto",
+]
+
+[[package]]
+name = "lazy_static"
+version = "1.4.0"
+source = "registry+https://github.com/rust-lang/crates.io-index"
+checksum = "e2abad23fbc42b3700f2f279844dc832adb2b2eb069b2df918f455c4e18cc646"
+
+[[package]]
+name = "libc"
+version = "0.2.147"
+source = "registry+https://github.com/rust-lang/crates.io-index"
+checksum = "b4668fb0ea861c1df094127ac5f1da3409a82116a4ba74fca2e58ef927159bb3"
+
+[[package]]
+name = "linux-raw-sys"
+version = "0.3.8"
+source = "registry+https://github.com/rust-lang/crates.io-index"
+checksum = "ef53942eb7bf7ff43a617b3e2c1c4a5ecf5944a7c1bc12d7ee39bbb15e5c1519"
+
+[[package]]
+name = "lock_api"
+version = "0.4.10"
+source = "registry+https://github.com/rust-lang/crates.io-index"
+checksum = "c1cc9717a20b1bb222f333e6a92fd32f7d8a18ddc5a3191a11af45dcbf4dcd16"
+dependencies = [
+ "autocfg",
+ "scopeguard",
+]
+
+[[package]]
+name = "log"
+version = "0.4.19"
+source = "registry+https://github.com/rust-lang/crates.io-index"
+checksum = "b06a4cde4c0f271a446782e3eff8de789548ce57dbc8eca9292c27f4a42004b4"
+
+[[package]]
+name = "md-5"
+version = "0.10.5"
+source = "registry+https://github.com/rust-lang/crates.io-index"
+checksum = "6365506850d44bff6e2fbcb5176cf63650e48bd45ef2fe2665ae1570e0f4b9ca"
+dependencies = [
+ "digest",
+]
+
+[[package]]
+name = "memchr"
+version = "2.5.0"
+source = "registry+https://github.com/rust-lang/crates.io-index"
+checksum = "2dffe52ecf27772e601905b7522cb4ef790d2cc203488bbd0e2fe85fcb74566d"
+
+[[package]]
+name = "miniz_oxide"
+version = "0.7.1"
+source = "registry+https://github.com/rust-lang/crates.io-index"
+checksum = "e7810e0be55b428ada41041c41f32c9f1a42817901b4ccf45fa3d4b6561e74c7"
+dependencies = [
+ "adler",
+]
+
+[[package]]
+name = "mio"
+version = "0.8.8"
+source = "registry+https://github.com/rust-lang/crates.io-index"
+checksum = "927a765cd3fc26206e66b296465fa9d3e5ab003e651c1b3c060e7956d96b19d2"
+dependencies = [
+ "libc",
+ "wasi",
+ "windows-sys",
+]
+
+[[package]]
+name = "multimap"
+version = "0.8.3"
+source = "registry+https://github.com/rust-lang/crates.io-index"
+checksum = "e5ce46fe64a9d73be07dcbe690a38ce1b293be448fd8ce1e6c1b8062c9f72c6a"
+
+[[package]]
+name = "num_cpus"
+version = "1.16.0"
+source = "registry+https://github.com/rust-lang/crates.io-index"
+checksum = "4161fcb6d602d4d2081af7c3a45852d875a03dd337a6bfdd6e06407b61342a43"
+dependencies = [
+ "hermit-abi",
+ "libc",
+]
+
+[[package]]
+name = "num_enum"
+version = "0.5.11"
+source = "registry+https://github.com/rust-lang/crates.io-index"
+checksum = "1f646caf906c20226733ed5b1374287eb97e3c2a5c227ce668c1f2ce20ae57c9"
+dependencies = [
+ "num_enum_derive",
+]
+
+[[package]]
+name = "num_enum_derive"
+version = "0.5.11"
+source = "registry+https://github.com/rust-lang/crates.io-index"
+checksum = "dcbff9bc912032c62bf65ef1d5aea88983b420f4f839db1e9b0c281a25c9c799"
+dependencies = [
+ "proc-macro-crate",
+ "proc-macro2",
+ "quote",
+ "syn 1.0.109",
+]
+
+[[package]]
+name = "object"
+version = "0.31.1"
+source = "registry+https://github.com/rust-lang/crates.io-index"
+checksum = "8bda667d9f2b5051b8833f59f3bf748b28ef54f850f4fcb389a252aa383866d1"
+dependencies = [
+ "memchr",
+]
+
+[[package]]
+name = "once_cell"
+version = "1.18.0"
+source = "registry+https://github.com/rust-lang/crates.io-index"
+checksum = "dd8b5dd2ae5ed71462c540258bedcb51965123ad7e7ccf4b9a8cafaa4a63576d"
+
+[[package]]
+name = "parking_lot"
+version = "0.12.1"
+source = "registry+https://github.com/rust-lang/crates.io-index"
+checksum = "3742b2c103b9f06bc9fff0a37ff4912935851bee6d36f3c02bcc755bcfec228f"
+dependencies = [
+ "lock_api",
+ "parking_lot_core",
+]
+
+[[package]]
+name = "parking_lot_core"
+version = "0.9.8"
+source = "registry+https://github.com/rust-lang/crates.io-index"
+checksum = "93f00c865fe7cabf650081affecd3871070f26767e7b2070a3ffae14c654b447"
+dependencies = [
+ "cfg-if",
+ "libc",
+ "redox_syscall",
+ "smallvec",
+ "windows-targets",
+]
+
+[[package]]
+name = "percent-encoding"
+version = "2.3.0"
+source = "registry+https://github.com/rust-lang/crates.io-index"
+checksum = "9b2a4787296e9989611394c33f193f676704af1686e70b8f8033ab5ba9a35a94"
+
+[[package]]
+name = "petgraph"
+version = "0.6.3"
+source = "registry+https://github.com/rust-lang/crates.io-index"
+checksum = "4dd7d28ee937e54fe3080c91faa1c3a46c06de6252988a7f4592ba2310ef22a4"
+dependencies = [
+ "fixedbitset",
+ "indexmap 1.9.3",
+]
+
+[[package]]
+name = "phf"
+version = "0.11.2"
+source = "registry+https://github.com/rust-lang/crates.io-index"
+checksum = "ade2d8b8f33c7333b51bcf0428d37e217e9f32192ae4772156f65063b8ce03dc"
+dependencies = [
+ "phf_shared",
+]
+
+[[package]]
+name = "phf_shared"
+version = "0.11.2"
+source = "registry+https://github.com/rust-lang/crates.io-index"
+checksum = "90fcb95eef784c2ac79119d1dd819e162b5da872ce6f3c3abe1e8ca1c082f72b"
+dependencies = [
+ "siphasher",
+]
+
+[[package]]
+name = "pin-project-lite"
+version = "0.2.10"
+source = "registry+https://github.com/rust-lang/crates.io-index"
+checksum = "4c40d25201921e5ff0c862a505c6557ea88568a4e3ace775ab55e93f2f4f9d57"
+
+[[package]]
+name = "pin-utils"
+version = "0.1.0"
+source = "registry+https://github.com/rust-lang/crates.io-index"
+checksum = "8b870d8c151b6f2fb93e84a13146138f05d02ed11c7e7c54f8826aaaf7c9f184"
+
+[[package]]
+name = "postgres"
+version = "0.19.5"
+source = "registry+https://github.com/rust-lang/crates.io-index"
+checksum = "0bed5017bc2ff49649c0075d0d7a9d676933c1292480c1d137776fb205b5cd18"
+dependencies = [
+ "bytes",
+ "fallible-iterator",
+ "futures-util",
+ "log",
+ "tokio",
+ "tokio-postgres",
+]
+
+[[package]]
+name = "postgres-derive"
+version = "0.4.4"
+source = "registry+https://github.com/rust-lang/crates.io-index"
+checksum = "070ffaa78859c779b19f9358ce035480479cf2619e968593ffbe72abcb6e0fcf"
+dependencies = [
+ "proc-macro2",
+ "quote",
+ "syn 2.0.25",
+]
+
+[[package]]
+name = "postgres-protocol"
+version = "0.6.5"
+source = "registry+https://github.com/rust-lang/crates.io-index"
+checksum = "78b7fa9f396f51dffd61546fd8573ee20592287996568e6175ceb0f8699ad75d"
+dependencies = [
+ "base64",
+ "byteorder",
+ "bytes",
+ "fallible-iterator",
+ "hmac",
+ "md-5",
+ "memchr",
+ "rand",
+ "sha2",
+ "stringprep",
+]
+
+[[package]]
+name = "postgres-types"
+version = "0.2.5"
+source = "registry+https://github.com/rust-lang/crates.io-index"
+checksum = "f028f05971fe20f512bcc679e2c10227e57809a3af86a7606304435bc8896cd6"
+dependencies = [
+ "array-init",
+ "bytes",
+ "fallible-iterator",
+ "postgres-derive",
+ "postgres-protocol",
+ "serde",
+ "serde_json",
+ "uuid",
+]
+
+[[package]]
+name = "ppv-lite86"
+version = "0.2.17"
+source = "registry+https://github.com/rust-lang/crates.io-index"
+checksum = "5b40af805b3121feab8a3c29f04d8ad262fa8e0561883e7653e024ae4479e6de"
+
+[[package]]
+name = "prettyplease"
+version = "0.1.25"
+source = "registry+https://github.com/rust-lang/crates.io-index"
+checksum = "6c8646e95016a7a6c4adea95bafa8a16baab64b583356217f2c85db4a39d9a86"
+dependencies = [
+ "proc-macro2",
+ "syn 1.0.109",
+]
+
+[[package]]
+name = "proc-macro-crate"
+version = "1.3.1"
+source = "registry+https://github.com/rust-lang/crates.io-index"
+checksum = "7f4c021e1093a56626774e81216a4ce732a735e5bad4868a03f3ed65ca0c3919"
+dependencies = [
+ "once_cell",
+ "toml_edit",
+]
+
+[[package]]
+name = "proc-macro2"
+version = "1.0.64"
+source = "registry+https://github.com/rust-lang/crates.io-index"
+checksum = "78803b62cbf1f46fde80d7c0e803111524b9877184cfe7c3033659490ac7a7da"
+dependencies = [
+ "unicode-ident",
+]
+
+[[package]]
+name = "prost"
+version = "0.11.9"
+source = "registry+https://github.com/rust-lang/crates.io-index"
+checksum = "0b82eaa1d779e9a4bc1c3217db8ffbeabaae1dca241bf70183242128d48681cd"
+dependencies = [
+ "bytes",
+ "prost-derive",
+]
+
+[[package]]
+name = "prost-build"
+version = "0.11.9"
+source = "registry+https://github.com/rust-lang/crates.io-index"
+checksum = "119533552c9a7ffacc21e099c24a0ac8bb19c2a2a3f363de84cd9b844feab270"
+dependencies = [
+ "bytes",
+ "heck",
+ "itertools",
+ "lazy_static",
+ "log",
+ "multimap",
+ "petgraph",
+ "prettyplease",
+ "prost",
+ "prost-types",
+ "regex",
+ "syn 1.0.109",
+ "tempfile",
+ "which",
+]
+
+[[package]]
+name = "prost-derive"
+version = "0.11.9"
+source = "registry+https://github.com/rust-lang/crates.io-index"
+checksum = "e5d2d8d10f3c6ded6da8b05b5fb3b8a5082514344d56c9f871412d29b4e075b4"
+dependencies = [
+ "anyhow",
+ "itertools",
+ "proc-macro2",
+ "quote",
+ "syn 1.0.109",
+]
+
+[[package]]
+name = "prost-types"
+version = "0.11.9"
+source = "registry+https://github.com/rust-lang/crates.io-index"
+checksum = "213622a1460818959ac1181aaeb2dc9c7f63df720db7d788b3e24eacd1983e13"
+dependencies = [
+ "prost",
+]
+
+[[package]]
+name = "proto"
+version = "0.1.0"
+dependencies = [
+ "bytes",
+ "prost",
+ "prost-build",
+ "protobuf-src",
+]
+
+[[package]]
+name = "protobuf-src"
+version = "1.1.0+21.5"
+source = "registry+https://github.com/rust-lang/crates.io-index"
+checksum = "c7ac8852baeb3cc6fb83b93646fb93c0ffe5d14bf138c945ceb4b9948ee0e3c1"
+dependencies = [
+ "autotools",
+]
+
+[[package]]
+name = "quote"
+version = "1.0.29"
+source = "registry+https://github.com/rust-lang/crates.io-index"
+checksum = "573015e8ab27661678357f27dc26460738fd2b6c86e46f386fde94cb5d913105"
+dependencies = [
+ "proc-macro2",
+]
+
+[[package]]
+name = "rand"
+version = "0.8.5"
+source = "registry+https://github.com/rust-lang/crates.io-index"
+checksum = "34af8d1a0e25924bc5b7c43c079c942339d8f0a8b57c39049bef581b46327404"
+dependencies = [
+ "libc",
+ "rand_chacha",
+ "rand_core",
+]
+
+[[package]]
+name = "rand_chacha"
+version = "0.3.1"
+source = "registry+https://github.com/rust-lang/crates.io-index"
+checksum = "e6c10a63a0fa32252be49d21e7709d4d4baf8d231c2dbce1eaa8141b9b127d88"
+dependencies = [
+ "ppv-lite86",
+ "rand_core",
+]
+
+[[package]]
+name = "rand_core"
+version = "0.6.4"
+source = "registry+https://github.com/rust-lang/crates.io-index"
+checksum = "ec0be4795e2f6a28069bec0b5ff3e2ac9bafc99e6a9a7dc3547996c5c816922c"
+dependencies = [
+ "getrandom",
+]
+
+[[package]]
+name = "redox_syscall"
+version = "0.3.5"
+source = "registry+https://github.com/rust-lang/crates.io-index"
+checksum = "567664f262709473930a4bf9e51bf2ebf3348f2e748ccc50dea20646858f8f29"
+dependencies = [
+ "bitflags",
+]
+
+[[package]]
+name = "regex"
+version = "1.9.1"
+source = "registry+https://github.com/rust-lang/crates.io-index"
+checksum = "b2eae68fc220f7cf2532e4494aded17545fce192d59cd996e0fe7887f4ceb575"
+dependencies = [
+ "aho-corasick",
+ "memchr",
+ "regex-automata",
+ "regex-syntax",
+]
+
+[[package]]
+name = "regex-automata"
+version = "0.3.3"
+source = "registry+https://github.com/rust-lang/crates.io-index"
+checksum = "39354c10dd07468c2e73926b23bb9c2caca74c5501e38a35da70406f1d923310"
+dependencies = [
+ "aho-corasick",
+ "memchr",
+ "regex-syntax",
+]
+
+[[package]]
+name = "regex-syntax"
+version = "0.7.4"
+source = "registry+https://github.com/rust-lang/crates.io-index"
+checksum = "e5ea92a5b6195c6ef2a0295ea818b312502c6fc94dde986c5553242e18fd4ce2"
+
+[[package]]
+name = "rustc-demangle"
+version = "0.1.23"
+source = "registry+https://github.com/rust-lang/crates.io-index"
+checksum = "d626bb9dae77e28219937af045c257c28bfd3f69333c512553507f5f9798cb76"
+
+[[package]]
+name = "rustix"
+version = "0.37.23"
+source = "registry+https://github.com/rust-lang/crates.io-index"
+checksum = "4d69718bf81c6127a49dc64e44a742e8bb9213c0ff8869a22c308f84c1d4ab06"
+dependencies = [
+ "bitflags",
+ "errno",
+ "io-lifetimes",
+ "libc",
+ "linux-raw-sys",
+ "windows-sys",
+]
+
+[[package]]
+name = "ryu"
+version = "1.0.15"
+source = "registry+https://github.com/rust-lang/crates.io-index"
+checksum = "1ad4cc8da4ef723ed60bced201181d83791ad433213d8c24efffda1eec85d741"
+
+[[package]]
+name = "scopeguard"
+version = "1.1.0"
+source = "registry+https://github.com/rust-lang/crates.io-index"
+checksum = "d29ab0c6d3fc0ee92fe66e2d99f700eab17a8d57d1c1d3b748380fb20baa78cd"
+
+[[package]]
+name = "serde"
+version = "1.0.171"
+source = "registry+https://github.com/rust-lang/crates.io-index"
+checksum = "30e27d1e4fd7659406c492fd6cfaf2066ba8773de45ca75e855590f856dc34a9"
+
+[[package]]
+name = "serde_json"
+version = "1.0.103"
+source = "registry+https://github.com/rust-lang/crates.io-index"
+checksum = "d03b412469450d4404fe8499a268edd7f8b79fecb074b0d812ad64ca21f4031b"
+dependencies = [
+ "itoa",
+ "ryu",
+ "serde",
+]
+
+[[package]]
+name = "sha2"
+version = "0.10.7"
+source = "registry+https://github.com/rust-lang/crates.io-index"
+checksum = "479fb9d862239e610720565ca91403019f2f00410f1864c5aa7479b950a76ed8"
+dependencies = [
+ "cfg-if",
+ "cpufeatures",
+ "digest",
+]
+
+[[package]]
+name = "signal-hook-registry"
+version = "1.4.1"
+source = "registry+https://github.com/rust-lang/crates.io-index"
+checksum = "d8229b473baa5980ac72ef434c4415e70c4b5e71b423043adb4ba059f89c99a1"
+dependencies = [
+ "libc",
+]
+
+[[package]]
+name = "siphasher"
+version = "0.3.10"
+source = "registry+https://github.com/rust-lang/crates.io-index"
+checksum = "7bd3e3206899af3f8b12af284fafc038cc1dc2b41d1b89dd17297221c5d225de"
+
+[[package]]
+name = "slab"
+version = "0.4.8"
+source = "registry+https://github.com/rust-lang/crates.io-index"
+checksum = "6528351c9bc8ab22353f9d776db39a20288e8d6c37ef8cfe3317cf875eecfc2d"
+dependencies = [
+ "autocfg",
+]
+
+[[package]]
+name = "smallvec"
+version = "1.11.0"
+source = "registry+https://github.com/rust-lang/crates.io-index"
+checksum = "62bb4feee49fdd9f707ef802e22365a35de4b7b299de4763d44bfea899442ff9"
+
+[[package]]
+name = "socket2"
+version = "0.4.9"
+source = "registry+https://github.com/rust-lang/crates.io-index"
+checksum = "64a4a911eed85daf18834cfaa86a79b7d266ff93ff5ba14005426219480ed662"
+dependencies = [
+ "libc",
+ "winapi",
+]
+
+[[package]]
+name = "socket2"
+version = "0.5.3"
+source = "registry+https://github.com/rust-lang/crates.io-index"
+checksum = "2538b18701741680e0322a2302176d3253a35388e2e62f172f64f4f16605f877"
+dependencies = [
+ "libc",
+ "windows-sys",
+]
+
+[[package]]
+name = "stringprep"
+version = "0.1.2"
+source = "registry+https://github.com/rust-lang/crates.io-index"
+checksum = "8ee348cb74b87454fff4b551cbf727025810a004f88aeacae7f85b87f4e9a1c1"
+dependencies = [
+ "unicode-bidi",
+ "unicode-normalization",
+]
+
+[[package]]
+name = "subtle"
+version = "2.5.0"
+source = "registry+https://github.com/rust-lang/crates.io-index"
+checksum = "81cdd64d312baedb58e21336b31bc043b77e01cc99033ce76ef539f78e965ebc"
+
+[[package]]
+name = "syn"
+version = "1.0.109"
+source = "registry+https://github.com/rust-lang/crates.io-index"
+checksum = "72b64191b275b66ffe2469e8af2c1cfe3bafa67b529ead792a6d0160888b4237"
+dependencies = [
+ "proc-macro2",
+ "quote",
+ "unicode-ident",
+]
+
+[[package]]
+name = "syn"
+version = "2.0.25"
+source = "registry+https://github.com/rust-lang/crates.io-index"
+checksum = "15e3fc8c0c74267e2df136e5e5fb656a464158aa57624053375eb9c8c6e25ae2"
+dependencies = [
+ "proc-macro2",
+ "quote",
+ "unicode-ident",
+]
+
+[[package]]
+name = "tempfile"
+version = "3.6.0"
+source = "registry+https://github.com/rust-lang/crates.io-index"
+checksum = "31c0432476357e58790aaa47a8efb0c5138f137343f3b5f23bd36a27e3b0a6d6"
+dependencies = [
+ "autocfg",
+ "cfg-if",
+ "fastrand",
+ "redox_syscall",
+ "rustix",
+ "windows-sys",
+]
+
+[[package]]
+name = "tinyvec"
+version = "1.6.0"
+source = "registry+https://github.com/rust-lang/crates.io-index"
+checksum = "87cc5ceb3875bb20c2890005a4e226a4651264a5c75edb2421b52861a0a0cb50"
+dependencies = [
+ "tinyvec_macros",
+]
+
+[[package]]
+name = "tinyvec_macros"
+version = "0.1.1"
+source = "registry+https://github.com/rust-lang/crates.io-index"
+checksum = "1f3ccbac311fea05f86f61904b462b55fb3df8837a366dfc601a0161d0532f20"
+
+[[package]]
+name = "tokio"
+version = "1.29.1"
+source = "registry+https://github.com/rust-lang/crates.io-index"
+checksum = "532826ff75199d5833b9d2c5fe410f29235e25704ee5f0ef599fb51c21f4a4da"
+dependencies = [
+ "autocfg",
+ "backtrace",
+ "bytes",
+ "libc",
+ "mio",
+ "num_cpus",
+ "parking_lot",
+ "pin-project-lite",
+ "signal-hook-registry",
+ "socket2 0.4.9",
+ "tokio-macros",
+ "windows-sys",
+]
+
+[[package]]
+name = "tokio-macros"
+version = "2.1.0"
+source = "registry+https://github.com/rust-lang/crates.io-index"
+checksum = "630bdcf245f78637c13ec01ffae6187cca34625e8c63150d424b59e55af2675e"
+dependencies = [
+ "proc-macro2",
+ "quote",
+ "syn 2.0.25",
+]
+
+[[package]]
+name = "tokio-postgres"
+version = "0.7.8"
+source = "registry+https://github.com/rust-lang/crates.io-index"
+checksum = "6e89f6234aa8fd43779746012fcf53603cdb91fdd8399aa0de868c2d56b6dde1"
+dependencies = [
+ "async-trait",
+ "byteorder",
+ "bytes",
+ "fallible-iterator",
+ "futures-channel",
+ "futures-util",
+ "log",
+ "parking_lot",
+ "percent-encoding",
+ "phf",
+ "pin-project-lite",
+ "postgres-protocol",
+ "postgres-types",
+ "socket2 0.5.3",
+ "tokio",
+ "tokio-util",
+]
+
+[[package]]
+name = "tokio-util"
+version = "0.7.8"
+source = "registry+https://github.com/rust-lang/crates.io-index"
+checksum = "806fe8c2c87eccc8b3267cbae29ed3ab2d0bd37fca70ab622e46aaa9375ddb7d"
+dependencies = [
+ "bytes",
+ "futures-core",
+ "futures-sink",
+ "pin-project-lite",
+ "tokio",
+ "tracing",
+]
+
+[[package]]
+name = "toml_datetime"
+version = "0.6.3"
+source = "registry+https://github.com/rust-lang/crates.io-index"
+checksum = "7cda73e2f1397b1262d6dfdcef8aafae14d1de7748d66822d3bfeeb6d03e5e4b"
+
+[[package]]
+name = "toml_edit"
+version = "0.19.14"
+source = "registry+https://github.com/rust-lang/crates.io-index"
+checksum = "f8123f27e969974a3dfba720fdb560be359f57b44302d280ba72e76a74480e8a"
+dependencies = [
+ "indexmap 2.0.0",
+ "toml_datetime",
+ "winnow",
+]
+
+[[package]]
+name = "tracing"
+version = "0.1.37"
+source = "registry+https://github.com/rust-lang/crates.io-index"
+checksum = "8ce8c33a8d48bd45d624a6e523445fd21ec13d3653cd51f681abf67418f54eb8"
+dependencies = [
+ "cfg-if",
+ "pin-project-lite",
+ "tracing-core",
+]
+
+[[package]]
+name = "tracing-core"
+version = "0.1.31"
+source = "registry+https://github.com/rust-lang/crates.io-index"
+checksum = "0955b8137a1df6f1a2e9a37d8a6656291ff0297c1a97c24e0d8425fe2312f79a"
+dependencies = [
+ "once_cell",
+]
+
+[[package]]
+name = "typenum"
+version = "1.16.0"
+source = "registry+https://github.com/rust-lang/crates.io-index"
+checksum = "497961ef93d974e23eb6f433eb5fe1b7930b659f06d12dec6fc44a8f554c0bba"
+
+[[package]]
+name = "unicode-bidi"
+version = "0.3.13"
+source = "registry+https://github.com/rust-lang/crates.io-index"
+checksum = "92888ba5573ff080736b3648696b70cafad7d250551175acbaa4e0385b3e1460"
+
+[[package]]
+name = "unicode-ident"
+version = "1.0.10"
+source = "registry+https://github.com/rust-lang/crates.io-index"
+checksum = "22049a19f4a68748a168c0fc439f9516686aa045927ff767eca0a85101fb6e73"
+
+[[package]]
+name = "unicode-normalization"
+version = "0.1.22"
+source = "registry+https://github.com/rust-lang/crates.io-index"
+checksum = "5c5713f0fc4b5db668a2ac63cdb7bb4469d8c9fed047b1d0292cc7b0ce2ba921"
+dependencies = [
+ "tinyvec",
+]
+
+[[package]]
+name = "uuid"
+version = "1.4.0"
+source = "registry+https://github.com/rust-lang/crates.io-index"
+checksum = "d023da39d1fde5a8a3fe1f3e01ca9632ada0a63e9797de55a879d6e2236277be"
+dependencies = [
+ "getrandom",
+ "rand",
+ "uuid-macro-internal",
+]
+
+[[package]]
+name = "uuid-macro-internal"
+version = "1.4.0"
+source = "registry+https://github.com/rust-lang/crates.io-index"
+checksum = "8614dda80b9075fbca36bc31b58d1447715b1236af98dee21db521c47a0cc2c0"
+dependencies = [
+ "proc-macro2",
+ "quote",
+ "syn 2.0.25",
+]
+
+[[package]]
+name = "version_check"
+version = "0.9.4"
+source = "registry+https://github.com/rust-lang/crates.io-index"
+checksum = "49874b5167b65d7193b8aba1567f5c7d93d001cafc34600cee003eda787e483f"
+
+[[package]]
+name = "wasi"
+version = "0.11.0+wasi-snapshot-preview1"
+source = "registry+https://github.com/rust-lang/crates.io-index"
+checksum = "9c8d87e72b64a3b4db28d11ce29237c246188f4f51057d65a7eab63b7987e423"
+
+[[package]]
+name = "which"
+version = "4.4.0"
+source = "registry+https://github.com/rust-lang/crates.io-index"
+checksum = "2441c784c52b289a054b7201fc93253e288f094e2f4be9058343127c4226a269"
+dependencies = [
+ "either",
+ "libc",
+ "once_cell",
+]
+
+[[package]]
+name = "winapi"
+version = "0.3.9"
+source = "registry+https://github.com/rust-lang/crates.io-index"
+checksum = "5c839a674fcd7a98952e593242ea400abe93992746761e38641405d28b00f419"
+dependencies = [
+ "winapi-i686-pc-windows-gnu",
+ "winapi-x86_64-pc-windows-gnu",
+]
+
+[[package]]
+name = "winapi-i686-pc-windows-gnu"
+version = "0.4.0"
+source = "registry+https://github.com/rust-lang/crates.io-index"
+checksum = "ac3b87c63620426dd9b991e5ce0329eff545bccbbb34f3be09ff6fb6ab51b7b6"
+
+[[package]]
+name = "winapi-x86_64-pc-windows-gnu"
+version = "0.4.0"
+source = "registry+https://github.com/rust-lang/crates.io-index"
+checksum = "712e227841d057c1ee1cd2fb22fa7e5a5461ae8e48fa2ca79ec42cfc1931183f"
+
+[[package]]
+name = "windows-sys"
+version = "0.48.0"
+source = "registry+https://github.com/rust-lang/crates.io-index"
+checksum = "677d2418bec65e3338edb076e806bc1ec15693c5d0104683f2efe857f61056a9"
+dependencies = [
+ "windows-targets",
+]
+
+[[package]]
+name = "windows-targets"
+version = "0.48.1"
+source = "registry+https://github.com/rust-lang/crates.io-index"
+checksum = "05d4b17490f70499f20b9e791dcf6a299785ce8af4d709018206dc5b4953e95f"
+dependencies = [
+ "windows_aarch64_gnullvm",
+ "windows_aarch64_msvc",
+ "windows_i686_gnu",
+ "windows_i686_msvc",
+ "windows_x86_64_gnu",
+ "windows_x86_64_gnullvm",
+ "windows_x86_64_msvc",
+]
+
+[[package]]
+name = "windows_aarch64_gnullvm"
+version = "0.48.0"
+source = "registry+https://github.com/rust-lang/crates.io-index"
+checksum = "91ae572e1b79dba883e0d315474df7305d12f569b400fcf90581b06062f7e1bc"
+
+[[package]]
+name = "windows_aarch64_msvc"
+version = "0.48.0"
+source = "registry+https://github.com/rust-lang/crates.io-index"
+checksum = "b2ef27e0d7bdfcfc7b868b317c1d32c641a6fe4629c171b8928c7b08d98d7cf3"
+
+[[package]]
+name = "windows_i686_gnu"
+version = "0.48.0"
+source = "registry+https://github.com/rust-lang/crates.io-index"
+checksum = "622a1962a7db830d6fd0a69683c80a18fda201879f0f447f065a3b7467daa241"
+
+[[package]]
+name = "windows_i686_msvc"
+version = "0.48.0"
+source = "registry+https://github.com/rust-lang/crates.io-index"
+checksum = "4542c6e364ce21bf45d69fdd2a8e455fa38d316158cfd43b3ac1c5b1b19f8e00"
+
+[[package]]
+name = "windows_x86_64_gnu"
+version = "0.48.0"
+source = "registry+https://github.com/rust-lang/crates.io-index"
+checksum = "ca2b8a661f7628cbd23440e50b05d705db3686f894fc9580820623656af974b1"
+
+[[package]]
+name = "windows_x86_64_gnullvm"
+version = "0.48.0"
+source = "registry+https://github.com/rust-lang/crates.io-index"
+checksum = "7896dbc1f41e08872e9d5e8f8baa8fdd2677f29468c4e156210174edc7f7b953"
+
+[[package]]
+name = "windows_x86_64_msvc"
+version = "0.48.0"
+source = "registry+https://github.com/rust-lang/crates.io-index"
+checksum = "1a515f5799fe4961cb532f983ce2b23082366b898e52ffbce459c86f67c8378a"
+
+[[package]]
+name = "winnow"
+version = "0.5.1"
+source = "registry+https://github.com/rust-lang/crates.io-index"
+checksum = "25b5872fa2e10bd067ae946f927e726d7d603eaeb6e02fa6a350e0722d2b8c11"
+dependencies = [
+ "memchr",
+]
diff --git a/native-metadata/Cross.toml b/native-metadata/Cross.toml
new file mode 100644
index 000000000..27c77686a
--- /dev/null
+++ b/native-metadata/Cross.toml
@@ -0,0 +1,6 @@
+# SPDX-FileCopyrightText: 2023 LakeSoul Contributors
+#
+# SPDX-License-Identifier: Apache-2.0
+
+[target.x86_64-unknown-linux-gnu]
+image = "dmetasoul/lakesoul-cross:v1.0.0"
diff --git a/native-metadata/lakesoul-metadata-c/Cargo.toml b/native-metadata/lakesoul-metadata-c/Cargo.toml
index 9f6f0b828..6e26989f0 100644
--- a/native-metadata/lakesoul-metadata-c/Cargo.toml
+++ b/native-metadata/lakesoul-metadata-c/Cargo.toml
@@ -14,3 +14,5 @@ crate-type = ["cdylib"]
[dependencies]
lakesoul-metadata = { path = "../lakesoul-metadata" }
+proto = { path = "../proto" }
+prost = "0.11"
diff --git a/native-metadata/lakesoul-metadata-c/src/lib.rs b/native-metadata/lakesoul-metadata-c/src/lib.rs
index c3eff20aa..cca63828d 100644
--- a/native-metadata/lakesoul-metadata-c/src/lib.rs
+++ b/native-metadata/lakesoul-metadata-c/src/lib.rs
@@ -1,21 +1,283 @@
-#![feature(c_size_t)]
-
// SPDX-FileCopyrightText: 2023 LakeSoul Contributors
//
// SPDX-License-Identifier: Apache-2.0
+#![feature(c_size_t)]
+#![allow(clippy::not_unsafe_ptr_arg_deref)]
extern crate core;
+use core::ffi::c_ptrdiff_t;
use std::ptr::NonNull;
+use std::ffi::{c_char, c_uchar, CString, CStr};
+
+use lakesoul_metadata::{Runtime, Builder, Client, PreparedStatementMap};
+use proto::proto::entity;
+use prost::Message;
+
+#[repr(C)]
+pub struct Result {
+ ptr: *mut OpaqueT,
+ err: *const c_char,
+}
+
+impl Result {
+ pub fn new(obj: T) -> Self {
+ Result {
+ ptr: convert_to_opaque_raw::(obj),
+ err: std::ptr::null(),
+ }
+ }
+
+ pub fn error(err_msg: &str) -> Self {
+ Result {
+ ptr: std::ptr::null_mut(),
+ err: CString::new(err_msg).unwrap().into_raw(),
+ }
+ }
+
+ pub fn free(&mut self) {
+ unsafe {
+ if !self.ptr.is_null() {
+ drop(from_opaque::(NonNull::new_unchecked(self.ptr)));
+ }
+ if !self.err.is_null() {
+ drop(CString::from_raw(self.err as *mut c_char));
+ }
+ }
+ }
+}
+
+#[repr(C)]
+pub struct PreparedStatement {
+ private: [u8; 0],
+}
+
#[repr(C)]
-pub struct Holder {
+pub struct TokioPostgresClient {
private: [u8; 0],
}
+#[repr(C)]
+pub struct TokioRuntime {
+ private: [u8; 0],
+}
+
+fn convert_to_opaque_raw(obj: F) -> *mut T {
+ Box::into_raw(Box::new(obj)) as *mut T
+}
+
+fn convert_to_nonnull(obj: T) -> NonNull {
+ unsafe { NonNull::new_unchecked(Box::into_raw(Box::new(obj))) }
+}
+
+fn from_opaque(obj: NonNull) -> T {
+ unsafe { *Box::from_raw(obj.as_ptr() as *mut T) }
+}
+
+fn from_nonnull(obj: NonNull) -> T {
+ unsafe { *Box::from_raw(obj.as_ptr()) }
+}
+
+fn string_from_ptr(ptr: *const c_char) -> String {
+ unsafe {
+ CStr::from_ptr(ptr).to_str().unwrap().to_string()
+ }
+}
+
+pub type ResultCallback = extern "C" fn(T, *const c_char);
+
+#[no_mangle]
+pub extern "C" fn execute_insert(
+ callback: extern "C" fn(i32, *const c_char),
+ runtime: NonNull>,
+ client: NonNull>,
+ prepared: NonNull>,
+ insert_type: i32,
+ addr: c_ptrdiff_t,
+ len: i32,
+) {
+ let runtime = unsafe {NonNull::new_unchecked(runtime.as_ref().ptr as *mut Runtime).as_ref()};
+ let client = unsafe {NonNull::new_unchecked(client.as_ref().ptr as *mut Client).as_mut()};
+ let prepared = unsafe {NonNull::new_unchecked(prepared.as_ref().ptr as *mut PreparedStatementMap).as_mut()};
+
+ let raw_parts = unsafe {std::slice::from_raw_parts(addr as *const u8, len as usize)};
+ let wrapper = entity::JniWrapper::decode(prost::bytes::Bytes::from(raw_parts)).unwrap();
+ let result = lakesoul_metadata::execute_insert(
+ runtime,
+ client,
+ prepared,
+ insert_type,
+ wrapper
+ );
+ match result {
+ Ok(count) => callback(count, CString::new("").unwrap().into_raw()),
+ Err(e) => callback(-1, CString::new(e.to_string().as_str()).unwrap().into_raw())
+ }
+}
+
+#[no_mangle]
+pub extern "C" fn execute_update(
+ callback: extern "C" fn(i32, *const c_char),
+ runtime: NonNull>,
+ client: NonNull>,
+ prepared: NonNull>,
+ update_type: i32,
+ joined_string: *const c_char
+) {
+ let runtime = unsafe {NonNull::new_unchecked(runtime.as_ref().ptr as *mut Runtime).as_ref()};
+ let client = unsafe {NonNull::new_unchecked(client.as_ref().ptr as *mut Client).as_mut()};
+ let prepared = unsafe {NonNull::new_unchecked(prepared.as_ref().ptr as *mut PreparedStatementMap).as_mut()};
+
+ let result = lakesoul_metadata::execute_update(
+ runtime,
+ client,
+ prepared,
+ update_type,
+ string_from_ptr(joined_string),
+ );
+ match result {
+ Ok(count) => callback(count, CString::new("").unwrap().into_raw()),
+ Err(e) => callback(-1, CString::new(e.to_string().as_str()).unwrap().into_raw())
+ }
+}
+
+#[no_mangle]
+pub extern "C" fn execute_query_scalar(
+ callback: extern "C" fn(*const c_char, *const c_char),
+ runtime: NonNull>,
+ client: NonNull>,
+ prepared: NonNull>,
+ update_type: i32,
+ joined_string: *const c_char
+) {
+ let runtime = unsafe {NonNull::new_unchecked(runtime.as_ref().ptr as *mut Runtime).as_ref()};
+ let client = unsafe {NonNull::new_unchecked(client.as_ref().ptr as *mut Client).as_mut()};
+ let prepared = unsafe {NonNull::new_unchecked(prepared.as_ref().ptr as *mut PreparedStatementMap).as_mut()};
+
+ let result = lakesoul_metadata::execute_query_scalar(
+ runtime,
+ client,
+ prepared,
+ update_type,
+ string_from_ptr(joined_string),
+ );
+ match result {
+ Ok(Some(result)) => callback(CString::new(result.as_str()).unwrap().into_raw(), CString::new("").unwrap().into_raw()),
+ Ok(None) => callback(CString::new("").unwrap().into_raw(), CString::new("").unwrap().into_raw()),
+ Err(e) => callback(CString::new("").unwrap().into_raw(), CString::new(e.to_string().as_str()).unwrap().into_raw())
+ }
+}
+
+
+
+#[no_mangle]
+pub extern "C" fn execute_query(
+ callback: extern "C" fn(i32, *const c_char),
+ runtime: NonNull>,
+ client: NonNull>,
+ prepared: NonNull>,
+ query_type: i32,
+ joined_string: *const c_char,
+ addr: c_ptrdiff_t,
+){
+ let runtime = unsafe {NonNull::new_unchecked(runtime.as_ref().ptr as *mut Runtime).as_ref()};
+ let client = unsafe {NonNull::new_unchecked(client.as_ref().ptr as *mut Client).as_ref()};
+ let prepared = unsafe {NonNull::new_unchecked(prepared.as_ref().ptr as *mut PreparedStatementMap).as_mut()};
+
+ let result = lakesoul_metadata::execute_query(
+ runtime,
+ client,
+ prepared,
+ query_type,
+ string_from_ptr(joined_string),
+ );
+ match result {
+ Ok(u8_vec) => {
+ let addr = addr as *mut c_uchar;
+ let _ = u8_vec
+ .iter()
+ .enumerate()
+ .map(
+ |(idx, byte)|
+ unsafe{std::ptr::write::(addr.wrapping_add(idx), *byte)})
+ .collect::>();
+ let len = u8_vec.len() as i32;
+ callback( len, CString::new("").unwrap().into_raw());
+ }
+ Err(e) => {
+ callback(0, CString::new(e.to_string().as_str()).unwrap().into_raw());
+ }
+ }
+}
+
+#[no_mangle]
+pub extern "C" fn clean_meta_for_test(
+ callback: extern "C" fn(i32, *const c_char),
+ runtime: NonNull>,
+ client: NonNull>,
+) {
+ let runtime = unsafe {NonNull::new_unchecked(runtime.as_ref().ptr as *mut Runtime).as_ref()};
+ let client = unsafe {NonNull::new_unchecked(client.as_ref().ptr as *mut Client).as_ref()};
+ let result = lakesoul_metadata::clean_meta_for_test(
+ runtime,
+ client);
+ match result {
+ Ok(count) => callback(count, CString::new("").unwrap().into_raw()),
+ Err(e) => callback(-1, CString::new(e.to_string().as_str()).unwrap().into_raw())
+ }
+}
+
+#[no_mangle]
+pub extern "C" fn create_tokio_runtime() -> NonNull> {
+ let runtime = Builder::new_multi_thread()
+ .enable_all()
+ .worker_threads(2)
+ .max_blocking_threads(8)
+ .build()
+ .unwrap();
+ convert_to_nonnull(Result::::new(runtime))
+}
+
+#[no_mangle]
+pub extern "C" fn free_tokio_runtime(runtime: NonNull>) {
+ from_nonnull(runtime).free::();
+}
#[no_mangle]
-pub extern "C" fn do_something() -> NonNull {
- let obj = "a".to_owned();
- unsafe { NonNull::new_unchecked(Box::into_raw(Box::new(obj)) as *mut Holder) }
+pub extern "C" fn create_tokio_postgres_client(
+ callback: extern "C" fn(bool, *const c_char),
+ config: *const c_char,
+ runtime: NonNull>,
+) -> NonNull> {
+ let config = string_from_ptr(config);
+ let runtime = unsafe {NonNull::new_unchecked(runtime.as_ref().ptr as *mut Runtime).as_ref()};
+
+ let result = match lakesoul_metadata::create_connection(runtime, config) {
+ Ok(client) => {
+ callback(true, CString::new("").unwrap().into_raw());
+ Result::::new(client)
+ }
+ Err(e) => {
+ callback(false, CString::new(e.to_string().as_str()).unwrap().into_raw());
+ Result::::error(format!("{}", e).as_str())
+ }
+ };
+ convert_to_nonnull(result)
}
+
+#[no_mangle]
+pub extern "C" fn free_tokio_postgres_client(client: NonNull>) {
+ from_nonnull(client).free::();
+}
+
+#[no_mangle]
+pub extern "C" fn create_prepared_statement() -> NonNull> {
+ let prepared = PreparedStatementMap::new();
+ convert_to_nonnull(Result::::new(prepared))
+}
+
+#[no_mangle]
+pub extern "C" fn free_prepared_statement(prepared: NonNull>) {
+ from_nonnull(prepared).free::();
+}
\ No newline at end of file
diff --git a/native-metadata/lakesoul-metadata/Cargo.toml b/native-metadata/lakesoul-metadata/Cargo.toml
index 4374e3698..b2574d21d 100644
--- a/native-metadata/lakesoul-metadata/Cargo.toml
+++ b/native-metadata/lakesoul-metadata/Cargo.toml
@@ -6,19 +6,20 @@
name = "lakesoul-metadata"
version = "0.1.0"
edition = "2021"
-build = "build.rs"
# See more keys and their definitions at https://doc.rust-lang.org/cargo/reference/manifest.html
[dependencies]
postgres="0.19.5"
-tokio-postgres = "0.7.8"
-tokio = { version = "1", features = ["full"] }
+tokio-postgres = {version = "0.7.8", features=["default", "with-serde_json-1", "with-uuid-1", "array-impls"]}
+postgres-types = {version = "0.2.5", features=["derive"]}
+tokio = { version = "1", features = ["full"] }
+proto = { path = "../proto" }
prost = "0.11"
-# Only necessary if using Protobuf well-known types:
-prost-types = "0.11"
-[build-dependencies]
-prost-build = "0.11"
\ No newline at end of file
+num_enum = "0.5.1"
+uuid = { version = "1.4.0", features = ["v4", "fast-rng", "macro-diagnostics"]}
+serde_json = { version = "1.0"}
+
diff --git a/native-metadata/lakesoul-metadata/build.rs b/native-metadata/lakesoul-metadata/build.rs
deleted file mode 100644
index 9ddf41f3d..000000000
--- a/native-metadata/lakesoul-metadata/build.rs
+++ /dev/null
@@ -1,9 +0,0 @@
-// SPDX-FileCopyrightText: 2023 LakeSoul Contributors
-//
-// SPDX-License-Identifier: Apache-2.0
-
-use std::io::Result;
-fn main() -> Result<()> {
- prost_build::compile_protos(&["src/DataCommitInfo.proto"], &["src/"])?;
- Ok(())
-}
\ No newline at end of file
diff --git a/native-metadata/lakesoul-metadata/src/lib.rs b/native-metadata/lakesoul-metadata/src/lib.rs
new file mode 100644
index 000000000..0e1f0d88a
--- /dev/null
+++ b/native-metadata/lakesoul-metadata/src/lib.rs
@@ -0,0 +1,1426 @@
+// SPDX-FileCopyrightText: 2023 LakeSoul Contributors
+//
+// SPDX-License-Identifier: Apache-2.0
+
+#![feature(io_error_other)]
+#![feature(split_array)]
+use std::collections::HashMap;
+use std::str::FromStr;
+
+use proto::proto::entity;
+use prost::Message;
+
+pub use tokio::runtime::{Builder, Runtime};
+
+pub use tokio_postgres::{NoTls, Client, Statement};
+use postgres_types::{ToSql, FromSql};
+
+pub const DAO_TYPE_QUERY_ONE_OFFSET : i32 = 0;
+pub const DAO_TYPE_QUERY_LIST_OFFSET : i32 = 100;
+pub const DAO_TYPE_INSERT_ONE_OFFSET : i32 = 200;
+pub const DAO_TYPE_TRANSACTION_INSERT_LIST_OFFSET : i32 = 300;
+pub const DAO_TYPE_QUERY_SCALAR_OFFSET : i32 = 400;
+pub const DAO_TYPE_UPDATE_OFFSET : i32 = 500;
+
+pub const PARAM_DELIM: &str = "__DELIM__";
+pub const PARTITION_DESC_DELIM: &str = "_DELIM_";
+
+
+
+enum ResultType {
+ Namespace,
+ TableInfo,
+ TableNameId,
+ TablePathId,
+ PartitionInfo,
+ DataCommitInfo,
+ TablePathIdWithOnlyPath,
+ PartitionInfoWithOnlyCommitOp,
+ PartitionInfoWithoutTimestamp,
+}
+
+#[derive(FromSql, ToSql, Debug, PartialEq)]
+#[postgres(name = "data_file_op")]
+struct DataFileOp {
+ path: String,
+ file_op: String,
+ size: i64,
+ file_exist_cols: String,
+}
+
+impl DataFileOp {
+ fn from_proto_data_file_op(
+ data_file_op: &entity::DataFileOp
+ ) -> Self {
+ DataFileOp{
+ path: data_file_op.path.clone(),
+ file_op: proto::proto::entity::FileOp::from_i32(data_file_op.file_op).unwrap().as_str_name().to_string(),
+ size: data_file_op.size,
+ file_exist_cols: data_file_op.file_exist_cols.clone()
+ }
+ }
+
+ fn as_proto_data_file_op(&self) -> entity::DataFileOp {
+ entity::DataFileOp {
+ path: self.path.clone(),
+ file_op: proto::proto::entity::FileOp::from_str_name(self.file_op.as_str()).unwrap() as i32,
+ size: self.size,
+ file_exist_cols: self.file_exist_cols.clone()
+ }
+ }
+}
+
+#[derive(Clone, Copy, Debug, PartialEq, Eq, Hash, PartialOrd, Ord, num_enum::TryFromPrimitive)]
+#[repr(i32)]
+pub enum DaoType{
+ SelectNamespaceByNamespace = DAO_TYPE_QUERY_ONE_OFFSET,
+ SelectTablePathIdByTablePath = DAO_TYPE_QUERY_ONE_OFFSET + 1,
+ SelectTableInfoByTableId = DAO_TYPE_QUERY_ONE_OFFSET + 2,
+ SelectTableNameIdByTableName = DAO_TYPE_QUERY_ONE_OFFSET + 3,
+ SelectTableInfoByTableNameAndNameSpace = DAO_TYPE_QUERY_ONE_OFFSET + 4,
+ SelectTableInfoByTablePath = DAO_TYPE_QUERY_ONE_OFFSET + 5,
+ SelectTableInfoByIdAndTablePath = DAO_TYPE_QUERY_ONE_OFFSET + 6,
+
+ SelectOnePartitionVersionByTableIdAndDesc = DAO_TYPE_QUERY_ONE_OFFSET + 7,
+ SelectPartitionVersionByTableIdAndDescAndVersion = DAO_TYPE_QUERY_ONE_OFFSET + 8,
+
+ SelectOneDataCommitInfoByTableIdAndPartitionDescAndCommitId = DAO_TYPE_QUERY_ONE_OFFSET + 9,
+
+ // ==== Query List ====
+
+ ListNamespaces = DAO_TYPE_QUERY_LIST_OFFSET,
+ ListTableNameByNamespace = DAO_TYPE_QUERY_LIST_OFFSET + 1,
+ ListAllTablePath = DAO_TYPE_QUERY_LIST_OFFSET + 2,
+ ListAllPathTablePathByNamespace = DAO_TYPE_QUERY_LIST_OFFSET + 3,
+
+ // Query Partition List
+ ListPartitionByTableId = DAO_TYPE_QUERY_LIST_OFFSET + 4,
+ ListPartitionDescByTableIdAndParList = DAO_TYPE_QUERY_LIST_OFFSET + 5,
+ ListPartitionByTableIdAndDesc = DAO_TYPE_QUERY_LIST_OFFSET + 6,
+ ListPartitionVersionByTableIdAndPartitionDescAndVersionRange = DAO_TYPE_QUERY_LIST_OFFSET + 7,
+ ListPartitionVersionByTableIdAndPartitionDescAndTimestampRange = DAO_TYPE_QUERY_LIST_OFFSET + 8,
+ ListCommitOpsBetweenVersions = DAO_TYPE_QUERY_LIST_OFFSET + 9,
+
+ // Query DataCommitInfo List
+ ListDataCommitInfoByTableIdAndPartitionDescAndCommitList = DAO_TYPE_QUERY_LIST_OFFSET + 10,
+
+ // ==== Insert One ====
+ InsertNamespace = DAO_TYPE_INSERT_ONE_OFFSET,
+ InsertTablePathId = DAO_TYPE_INSERT_ONE_OFFSET + 1,
+ InsertTableNameId = DAO_TYPE_INSERT_ONE_OFFSET + 2,
+ InsertTableInfo = DAO_TYPE_INSERT_ONE_OFFSET + 3,
+ InsertPartitionInfo = DAO_TYPE_INSERT_ONE_OFFSET + 4,
+ InsertDataCommitInfo = DAO_TYPE_INSERT_ONE_OFFSET + 5,
+
+ // ==== Transaction Insert List ====
+ TransactionInsertPartitionInfo = DAO_TYPE_TRANSACTION_INSERT_LIST_OFFSET,
+ TransactionInsertDataCommitInfo = DAO_TYPE_TRANSACTION_INSERT_LIST_OFFSET + 1,
+
+ // ==== Query SCALAR ====
+ GetLatestTimestampFromPartitionInfo = DAO_TYPE_QUERY_SCALAR_OFFSET,
+ GetLatestTimestampFromPartitionInfoWithoutPartitionDesc = DAO_TYPE_QUERY_SCALAR_OFFSET + 1,
+ GetLatestVersionUpToTimeFromPartitionInfo = DAO_TYPE_QUERY_SCALAR_OFFSET + 2,
+ GetLatestVersionTimestampUpToTimeFromPartitionInfo = DAO_TYPE_QUERY_SCALAR_OFFSET + 3,
+
+ // ==== Update ====
+ // Update Namespace
+ DeleteNamespaceByNamespace = DAO_TYPE_UPDATE_OFFSET,
+ UpdateNamespacePropertiesByNamespace = DAO_TYPE_UPDATE_OFFSET + 1,
+
+ // Update TableInfo
+ DeleteTableInfoByIdAndPath = DAO_TYPE_UPDATE_OFFSET + 2,
+ UpdateTableInfoPropertiesById = DAO_TYPE_UPDATE_OFFSET + 3,
+ UpdateTableInfoById = DAO_TYPE_UPDATE_OFFSET + 4,
+
+ // Update TablePathId
+ DeleteTablePathIdByTablePath = DAO_TYPE_UPDATE_OFFSET + 5,
+ DeleteTablePathIdByTableId = DAO_TYPE_UPDATE_OFFSET + 6,
+ // Update TableNameId
+ DeleteTableNameIdByTableNameAndNamespace = DAO_TYPE_UPDATE_OFFSET + 7,
+ DeleteTableNameIdByTableId = DAO_TYPE_UPDATE_OFFSET + 8,
+ // Update PartitionInfo
+ DeletePartitionInfoByTableIdAndPartitionDesc = DAO_TYPE_UPDATE_OFFSET + 9,
+ DeletePartitionInfoByTableId = DAO_TYPE_UPDATE_OFFSET + 10,
+ DeletePreviousVersionPartition = DAO_TYPE_UPDATE_OFFSET + 11,
+ // Update DataCommitInfo
+ DeleteOneDataCommitInfoByTableIdAndPartitionDescAndCommitId = DAO_TYPE_UPDATE_OFFSET + 12,
+ DeleteDataCommitInfoByTableIdAndPartitionDescAndCommitIdList = DAO_TYPE_UPDATE_OFFSET + 13,
+ DeleteDataCommitInfoByTableIdAndPartitionDesc = DAO_TYPE_UPDATE_OFFSET + 14,
+ DeleteDataCommitInfoByTableId = DAO_TYPE_UPDATE_OFFSET + 15,
+}
+
+
+pub type PreparedStatementMap = HashMap;
+
+
+fn get_prepared_statement(
+ runtime: &Runtime,
+ client: &Client,
+ prepared :&mut PreparedStatementMap,
+ dao_type: &DaoType,
+) -> Result {
+ if let Some(statement) = prepared.get(dao_type) {
+ Ok(statement.clone())
+ } else {
+ let result = runtime.block_on(async {
+ let statement = match dao_type {
+ // Select Namespace
+ DaoType::SelectNamespaceByNamespace =>
+ "select namespace, properties, comment, domain
+ from namespace
+ where namespace = $1::TEXT",
+ DaoType::ListNamespaces =>
+ "select namespace, properties, comment, domain
+ from namespace",
+
+ // Select TablePathId
+ DaoType::SelectTablePathIdByTablePath =>
+ "select table_path, table_id, table_namespace, domain
+ from table_path_id
+ where table_path = $1::TEXT",
+ DaoType::ListAllTablePath =>
+ "select table_path, table_id, table_namespace, domain
+ from table_path_id",
+ DaoType::ListAllPathTablePathByNamespace =>
+ "select table_path
+ from table_path_id
+ where table_namespace = $1::TEXT ",
+
+ // Select TableNameId
+ DaoType::SelectTableNameIdByTableName =>
+ "select table_name, table_id, table_namespace, domain
+ from table_name_id
+ where table_name = $1::TEXT and table_namespace = $2::TEXT",
+ DaoType::ListTableNameByNamespace =>
+ "select table_name, table_id, table_namespace, domain
+ from table_name_id
+ where table_namespace = $1::TEXT",
+
+ // Select TableInfo
+ DaoType::SelectTableInfoByTableId =>
+ "select table_id, table_name, table_path, table_schema, properties, partitions, table_namespace, domain
+ from table_info
+ where table_id = $1::TEXT",
+ DaoType::SelectTableInfoByTableNameAndNameSpace =>
+ "select table_id, table_name, table_path, table_schema, properties, partitions, table_namespace, domain
+ from table_info
+ where table_name = $1::TEXT and table_namespace=$2::TEXT",
+ DaoType::SelectTableInfoByTablePath =>
+ "select table_id, table_name, table_path, table_schema, properties, partitions, table_namespace, domain
+ from table_info
+ where table_path = $1::TEXT",
+ DaoType::SelectTableInfoByIdAndTablePath =>
+ "select table_id, table_name, table_path, table_schema, properties, partitions, table_namespace, domain
+ from table_info
+ where table_id = $1::TEXT and table_path=$2::TEXT",
+
+ // Select PartitionInfo
+ DaoType::SelectPartitionVersionByTableIdAndDescAndVersion =>
+ "select table_id, partition_desc, version, commit_op, snapshot, expression, domain
+ from partition_info from partition_info
+ where table_id = $1::TEXT and partition_desc = $2::TEXT and version = $3::INT",
+ DaoType::SelectOnePartitionVersionByTableIdAndDesc =>
+ "select m.table_id, t.partition_desc, m.version, m.commit_op, m.snapshot, m.expression, m.domain from (
+ select table_id,partition_desc,max(version) from partition_info
+ where table_id = $1::TEXT and partition_desc = $2::TEXT group by table_id, partition_desc) t
+ left join partition_info m on t.table_id = m.table_id
+ and t.partition_desc = m.partition_desc and t.max = m.version",
+ DaoType::ListPartitionByTableIdAndDesc =>
+ "select table_id, partition_desc, version, commit_op, snapshot, timestamp, expression, domain
+ from partition_info
+ where table_id = $1::TEXT and partition_desc = $2::TEXT ",
+ DaoType::ListPartitionByTableId =>
+ "select m.table_id, t.partition_desc, m.version, m.commit_op, m.snapshot, m.expression, m.domain
+ from (
+ select table_id,partition_desc,max(version)
+ from partition_info
+ where table_id = $1::TEXT
+ group by table_id,partition_desc) t
+ left join partition_info m
+ on t.table_id = m.table_id and t.partition_desc = m.partition_desc and t.max = m.version",
+ DaoType::ListPartitionVersionByTableIdAndPartitionDescAndTimestampRange =>
+ "select table_id, partition_desc, version, commit_op, snapshot, timestamp, expression, domain
+ from partition_info
+ where table_id = $1::TEXT and partition_desc = $2::TEXT and timestamp >= $3::BIGINT and timestamp < $4::BIGINT",
+ DaoType::ListCommitOpsBetweenVersions =>
+ "select distinct(commit_op)
+ from partition_info
+ where table_id = $1::TEXT and partition_desc = $2::TEXT and version between $3::INT and $4::INT",
+ DaoType::ListPartitionVersionByTableIdAndPartitionDescAndVersionRange =>
+ "select table_id, partition_desc, version, commit_op, snapshot, timestamp, expression, domain
+ from partition_info
+ where table_id = $1::TEXT and partition_desc = $2::TEXT and version >= $3::INT and version <= $4::INT",
+
+ // Select DataCommitInfo
+ DaoType::SelectOneDataCommitInfoByTableIdAndPartitionDescAndCommitId =>
+ "select table_id, partition_desc, commit_id, file_ops, commit_op, timestamp, committed, domain
+ from data_commit_info
+ where table_id = $1::TEXT and partition_desc = $2::TEXT and commit_id = $3::UUID",
+
+
+ // Insert
+ DaoType::InsertNamespace =>
+ "insert into namespace(
+ namespace,
+ properties,
+ comment,
+ domain)
+ values($1::TEXT, $2::JSON, $3::TEXT, $4::TEXT)",
+ DaoType::InsertTableInfo =>
+ "insert into table_info(
+ table_id,
+ table_name,
+ table_path,
+ table_schema,
+ properties,
+ partitions,
+ table_namespace,
+ domain)
+ values($1::TEXT, $2::TEXT, $3::TEXT, $4::TEXT, $5::JSON, $6::TEXT, $7::TEXT, $8::TEXT)",
+ DaoType::InsertTableNameId =>
+ "insert into table_name_id(
+ table_id,
+ table_name,
+ table_namespace,
+ domain)
+ values($1::TEXT, $2::TEXT, $3::TEXT, $4::TEXT)",
+ DaoType::InsertTablePathId =>
+ "insert into table_path_id(
+ table_id,
+ table_path,
+ table_namespace,
+ domain)
+ values($1::TEXT, $2::TEXT, $3::TEXT, $4::TEXT)",
+ DaoType::InsertPartitionInfo =>
+ "insert into partition_info(
+ table_id,
+ partition_desc,
+ version,
+ commit_op,
+ snapshot,
+ expression,
+ domain
+ )
+ values($1::TEXT, $2::TEXT, $3::INT, $4::TEXT, $5::_UUID, $6::TEXT, $7::TEXT)",
+ DaoType::InsertDataCommitInfo =>
+ "insert into data_commit_info(
+ table_id,
+ partition_desc,
+ commit_id,
+ file_ops,
+ commit_op,
+ timestamp,
+ committed,
+ domain
+ )
+ values($1::TEXT, $2::TEXT, $3::UUID, $4::_data_file_op, $5::TEXT, $6::BIGINT, $7::BOOL, $8::TEXT)",
+
+ // Query Scalar
+ DaoType::GetLatestTimestampFromPartitionInfo =>
+ "select max(timestamp) as timestamp
+ from partition_info
+ where table_id = $1::TEXT and partition_desc = $2::TEXT",
+ DaoType::GetLatestTimestampFromPartitionInfoWithoutPartitionDesc =>
+ "select max(timestamp) as timestamp
+ from partition_info
+ where table_id = $1::TEXT",
+ DaoType::GetLatestVersionUpToTimeFromPartitionInfo =>
+ "select max(version) as version
+ from partition_info
+ where table_id = $1::TEXT and partition_desc = $2::TEXT and timestamp < $3::BIGINT",
+ DaoType::GetLatestVersionTimestampUpToTimeFromPartitionInfo =>
+ "select max(timestamp) as timestamp
+ from partition_info
+ where table_id = $1::TEXT and partition_desc = $2::TEXT and timestamp < $3::BIGINT",
+
+ // Update / Delete
+ DaoType::DeleteNamespaceByNamespace =>
+ "delete from namespace
+ where namespace = $1::TEXT ",
+ DaoType::UpdateNamespacePropertiesByNamespace =>
+ "update namespace
+ set properties = $2::JSON where namespace = $1::TEXT",
+
+ DaoType::DeleteTableNameIdByTableNameAndNamespace =>
+ "delete from table_name_id
+ where table_name = $1::TEXT and table_namespace = $2::TEXT",
+ DaoType::DeleteTableNameIdByTableId =>
+ "delete from table_name_id
+ where table_id = $1::TEXT",
+
+ DaoType::DeleteTableInfoByIdAndPath =>
+ "delete from table_info
+ where table_id = $1::TEXT and table_path = $2::TEXT",
+ DaoType::UpdateTableInfoPropertiesById =>
+ "update table_info
+ set properties = $2::JSON where table_id = $1::TEXT",
+
+
+ DaoType::DeleteTablePathIdByTablePath =>
+ "delete from table_path_id
+ where table_path = $1::TEXT ",
+ DaoType::DeleteTablePathIdByTableId =>
+ "delete from table_path_id
+ where table_id = $1::TEXT ",
+
+ DaoType::DeleteOneDataCommitInfoByTableIdAndPartitionDescAndCommitId =>
+ "delete from data_commit_info
+ where table_id = $1::TEXT and partition_desc = $2::TEXT and commit_id = $3::UUID ",
+ DaoType::DeleteDataCommitInfoByTableIdAndPartitionDesc =>
+ "delete from data_commit_info
+ where table_id = $1::TEXT and partition_desc = $2::TEXT",
+ DaoType::DeleteDataCommitInfoByTableId =>
+ "delete from data_commit_info
+ where table_id = $1::TEXT",
+
+ DaoType::DeletePartitionInfoByTableId =>
+ "delete from partition_info
+ where table_id = $1::TEXT",
+ DaoType::DeletePartitionInfoByTableIdAndPartitionDesc =>
+ "delete from partition_info
+ where table_id = $1::TEXT and partition_desc = $2::TEXT",
+ DaoType::DeletePreviousVersionPartition =>
+ "delete from partition_info
+ where table_id = $1::TEXT and partition_desc = $2::TEXT and timestamp <= $3::BIGINT",
+
+
+ // not prepared
+ DaoType::UpdateTableInfoById |
+ DaoType::TransactionInsertDataCommitInfo |
+ DaoType::TransactionInsertPartitionInfo |
+ DaoType::ListDataCommitInfoByTableIdAndPartitionDescAndCommitList |
+ DaoType::DeleteDataCommitInfoByTableIdAndPartitionDescAndCommitIdList |
+ DaoType::ListPartitionDescByTableIdAndParList => "",
+
+ // _ => todo!(),
+
+ };
+ client.prepare(statement).await
+ });
+ match result {
+ Ok(statement) => {
+ prepared.insert(*dao_type, statement.clone());
+ Ok(statement)
+ }
+ Err(err) => Err(err)
+ }
+ }
+}
+
+
+pub fn execute_query(
+ runtime: &Runtime,
+ client: &Client,
+ prepared: &mut PreparedStatementMap,
+ query_type: i32,
+ joined_string: String,
+) -> Result, std::io::Error> {
+ if query_type >= DAO_TYPE_INSERT_ONE_OFFSET {
+ eprintln!("Invalid query_type_index: {:?}", query_type);
+ return Err(std::io::Error::from(std::io::ErrorKind::InvalidInput))
+ }
+ let query_type = DaoType::try_from(query_type).unwrap();
+ let statement = match get_prepared_statement(runtime, client, prepared, &query_type) {
+ Ok(statement) => statement,
+ Err(err) => return Err(convert_to_io_error(err))
+ };
+
+ let params = joined_string
+ .split(PARAM_DELIM)
+ .collect::>()
+ .iter()
+ .map(|str|str.to_string())
+ .collect::>();
+
+ let rows = match query_type {
+ DaoType::ListNamespaces |
+ DaoType::ListAllTablePath if params.len() == 1 && params[0].is_empty() => {
+ let result = runtime.block_on(async{
+ client.query(&statement, &[]).await
+ });
+ match result {
+ Ok(rows) => rows,
+ Err(e) => return Err(convert_to_io_error(e)),
+ }
+ }
+ DaoType::ListTableNameByNamespace if params.len() == 1 => {
+ let result = runtime.block_on(async{
+ client.query(&statement, &[¶ms[0]]).await
+ });
+ match result {
+ Ok(rows) => rows,
+ Err(e) => return Err(convert_to_io_error(e)),
+ }
+ }
+ DaoType::SelectNamespaceByNamespace |
+ DaoType::SelectTableInfoByTableId |
+ DaoType::SelectTablePathIdByTablePath |
+ DaoType::SelectTableInfoByTablePath if params.len() == 1 => {
+ let result = runtime.block_on(async{
+ client.query_opt(&statement, &[¶ms[0]]).await
+ });
+ match result {
+ Ok(Some(row)) => vec![row],
+ Ok(None) => vec![],
+ Err(e) => return Err(convert_to_io_error(e)),
+ }
+ }
+ DaoType::ListPartitionByTableId |
+ DaoType::ListAllPathTablePathByNamespace if params.len() == 1 => {
+ let result = runtime.block_on(async{
+ client.query(&statement, &[¶ms[0]]).await
+ });
+ match result {
+ Ok(rows) => rows,
+ Err(e) => return Err(convert_to_io_error(e)),
+ }
+ }
+ DaoType::SelectOnePartitionVersionByTableIdAndDesc |
+ DaoType::ListPartitionByTableIdAndDesc if params.len() == 2 => {
+ let result = runtime.block_on(async{
+ client.query(&statement, &[¶ms[0], ¶ms[1]]).await
+ });
+ match result {
+ Ok(rows) => rows,
+ Err(e) => return Err(convert_to_io_error(e)),
+ }
+ }
+ DaoType::SelectTableNameIdByTableName |
+ DaoType::SelectTableInfoByTableNameAndNameSpace |
+ DaoType::SelectTableInfoByIdAndTablePath if params.len() == 2 => {
+ let result = runtime.block_on(async{
+ client.query_opt(&statement, &[¶ms[0], ¶ms[1]]).await
+ });
+ match result {
+ Ok(Some(row)) => vec![row],
+ Ok(None) => vec![],
+ Err(e) => return Err(convert_to_io_error(e)),
+ }
+ }
+ DaoType::SelectOneDataCommitInfoByTableIdAndPartitionDescAndCommitId if params.len() == 3 => {
+ let result = runtime.block_on(async{
+ client.query_opt(&statement, &[¶ms[0], ¶ms[1], &uuid::Uuid::from_str(¶ms[2]).unwrap()]).await
+ });
+ match result {
+ Ok(Some(row)) => vec![row],
+ Ok(None) => vec![],
+ Err(e) => return Err(convert_to_io_error(e)),
+ }
+ }
+ DaoType::SelectPartitionVersionByTableIdAndDescAndVersion if params.len() == 3 => {
+ let result = runtime.block_on(async{
+ client.query(&statement, &[¶ms[0], ¶ms[1], &i32::from_str(¶ms[2]).unwrap()]).await
+ });
+ match result {
+ Ok(rows) => rows,
+ Err(e) => return Err(convert_to_io_error(e)),
+ }
+ }
+ DaoType::ListCommitOpsBetweenVersions |
+ DaoType::ListPartitionVersionByTableIdAndPartitionDescAndVersionRange if params.len() == 4 => {
+ let result = runtime.block_on(async{
+ client.query(&statement, &[¶ms[0], ¶ms[1], &i32::from_str(¶ms[2]).unwrap(), &i32::from_str(¶ms[3]).unwrap()]).await
+ });
+ match result {
+ Ok(rows) => rows,
+ Err(e) => return Err(convert_to_io_error(e)),
+ }
+ }
+ DaoType::ListPartitionDescByTableIdAndParList if params.len() == 2 => {
+ let partitions = "'".to_owned() + ¶ms[1]
+ .replace('\'', "''")
+ .split(PARTITION_DESC_DELIM)
+ .collect::>()
+ .join("','") + "'";
+ let statement = format!("select m.table_id, t.partition_desc, m.version, m.commit_op, m.snapshot, m.expression, m.domain from (
+ select table_id,partition_desc,max(version) from partition_info
+ where table_id = $1::TEXT and partition_desc in ({})
+ group by table_id,partition_desc) t
+ left join partition_info m on t.table_id = m.table_id and t.partition_desc = m.partition_desc and t.max = m.version", partitions);
+ let result = runtime.block_on(async{
+ let statement = client.prepare(&statement).await?;
+ client.query(&statement, &[¶ms[0]]).await
+ });
+ match result {
+ Ok(rows) => rows,
+ Err(e) => return Err(convert_to_io_error(e)),
+ }
+ }
+ DaoType::ListPartitionVersionByTableIdAndPartitionDescAndTimestampRange if params.len() == 4 => {
+ let result = runtime.block_on(async{
+ client.query(&statement, &[¶ms[0], ¶ms[1], &i64::from_str(¶ms[2]).unwrap(), &i64::from_str(¶ms[3]).unwrap()]).await
+ });
+ match result {
+ Ok(rows) => rows,
+ Err(e) => return Err(convert_to_io_error(e)),
+ }
+ }
+ DaoType::ListDataCommitInfoByTableIdAndPartitionDescAndCommitList if params.len() == 3 => {
+ let concated_uuid = ¶ms[2];
+ if concated_uuid.len() % 32 != 0 {
+ eprintln!("Invalid params of query_type={:?}, params={:?}", query_type, params);
+ return Err(std::io::Error::from(std::io::ErrorKind::InvalidInput));
+ }
+ let uuid_num = concated_uuid.len() / 32;
+ let mut uuid_list = Vec::::with_capacity(uuid_num);
+ let mut idx = 0;
+ for _ in 0..uuid_num {
+ let high = u64::from_str_radix(&concated_uuid[idx..idx+16], 16).unwrap();
+ let low = u64::from_str_radix(&concated_uuid[idx+16..idx+32], 16).unwrap();
+ uuid_list.push(uuid::Uuid::from_u64_pair(high, low).to_string());
+ idx += 32;
+ }
+
+ let uuid_str_list = "'".to_owned() + &uuid_list.join("','") + "'";
+
+ let uuid_list_str = uuid_list.join("");
+
+
+ let statement = format!("select table_id, partition_desc, commit_id, file_ops, commit_op, timestamp, committed, domain
+ from data_commit_info
+ where table_id = $1::TEXT and partition_desc = $2::TEXT
+ and commit_id in ({})
+ order by position(commit_id::text in '{}')", uuid_str_list, uuid_list_str);
+
+ let result = runtime.block_on(async{
+ let statement = client.prepare(&statement).await?;
+ client.query(&statement, &[¶ms[0], ¶ms[1]]).await
+ });
+ match result {
+ Ok(rows) => rows,
+ Err(e) => return Err(convert_to_io_error(e)),
+ }
+ }
+ _ => {
+ eprintln!("Invalid params num of query_type={:?}, params={:?}", query_type, params);
+ return Err(std::io::Error::from(std::io::ErrorKind::InvalidInput));
+ }
+ };
+
+ let result_type = match query_type {
+ DaoType::SelectNamespaceByNamespace |
+ DaoType::ListNamespaces => ResultType::Namespace,
+
+ DaoType::SelectTableInfoByTableId |
+ DaoType::SelectTableInfoByTableNameAndNameSpace |
+ DaoType::SelectTableInfoByTablePath |
+ DaoType::SelectTableInfoByIdAndTablePath => ResultType::TableInfo,
+
+ DaoType::SelectTablePathIdByTablePath |
+ DaoType::ListAllTablePath => ResultType::TablePathId,
+
+ DaoType::SelectTableNameIdByTableName |
+ DaoType::ListTableNameByNamespace => ResultType::TableNameId,
+
+ DaoType::ListPartitionByTableId |
+ DaoType::ListPartitionDescByTableIdAndParList |
+ DaoType::SelectPartitionVersionByTableIdAndDescAndVersion |
+ DaoType::SelectOnePartitionVersionByTableIdAndDesc => ResultType::PartitionInfoWithoutTimestamp,
+
+ DaoType::ListPartitionByTableIdAndDesc |
+ DaoType::ListPartitionVersionByTableIdAndPartitionDescAndTimestampRange |
+ DaoType::ListPartitionVersionByTableIdAndPartitionDescAndVersionRange => ResultType::PartitionInfo,
+
+ DaoType::SelectOneDataCommitInfoByTableIdAndPartitionDescAndCommitId |
+ DaoType::ListDataCommitInfoByTableIdAndPartitionDescAndCommitList => ResultType::DataCommitInfo,
+
+ DaoType::ListAllPathTablePathByNamespace => ResultType::TablePathIdWithOnlyPath ,
+
+ DaoType::ListCommitOpsBetweenVersions => ResultType::PartitionInfoWithOnlyCommitOp,
+ _ => {
+ eprintln!("Invalid query_type={:?} when parsing query result type", query_type);
+ return Err(std::io::Error::from(std::io::ErrorKind::InvalidInput));
+ }
+ };
+
+ let wrapper = match result_type {
+ ResultType::TableNameId => {
+ let table_name_id :Vec =
+ rows
+ .iter()
+ .map(|row|proto::proto::entity::TableNameId {
+ table_name: row.get(0),
+ table_id: row.get(1),
+ table_namespace: row.get(2),
+ domain: row.get(3),
+ })
+ .collect();
+ proto::proto::entity::JniWrapper {
+ table_name_id,
+ ..Default::default()
+ }
+ }
+ ResultType::TablePathId => {
+ let table_path_id :Vec =
+ rows
+ .iter()
+ .map(|row|proto::proto::entity::TablePathId {
+ table_path: row.get(0),
+ table_id: row.get(1),
+ table_namespace: row.get(2),
+ domain: row.get(3),
+ })
+ .collect();
+ proto::proto::entity::JniWrapper {
+ table_path_id,
+ ..Default::default()
+ }
+ }
+ ResultType::TablePathIdWithOnlyPath => {
+ let table_path_id :Vec =
+ rows
+ .iter()
+ .map(|row|proto::proto::entity::TablePathId {
+ table_path: row.get(0),
+ ..Default::default()
+ })
+ .collect();
+ proto::proto::entity::JniWrapper {
+ table_path_id,
+ ..Default::default()
+ }
+ }
+
+ ResultType::Namespace => {
+ let namespace:Vec =
+ rows
+ .iter()
+ .map(|row|proto::proto::entity::Namespace {
+ namespace: row.get(0),
+ properties: row.get::<_, serde_json::Value>(1).to_string(),
+ comment: row.get(2),
+ domain: row.get(3)
+ })
+ .collect();
+ proto::proto::entity::JniWrapper {
+ namespace,
+ ..Default::default()
+ }
+ },
+ ResultType::TableInfo => {
+ let table_info:Vec =
+ rows
+ .iter()
+ .map(|row|proto::proto::entity::TableInfo {
+ table_id: row.get(0),
+ table_name: row.get(1),
+ table_path: row.get(2),
+ table_schema: row.get(3),
+ properties: row.get::<_, serde_json::Value>(4).to_string(),
+ partitions: row.get(5),
+ table_namespace: row.get(6),
+ domain: row.get(7)
+ })
+ .collect();
+ proto::proto::entity::JniWrapper {
+ table_info,
+ ..Default::default()
+ }
+ }
+ ResultType::PartitionInfo => {
+ let partition_info:Vec =
+ rows
+ .iter()
+ .map(|row|{
+ proto::proto::entity::PartitionInfo {
+ table_id: row.get(0),
+ partition_desc: row.get(1),
+ version: row.get::<_, i32>(2),
+ commit_op: proto::proto::entity::CommitOp::from_str_name(row.get(3)).unwrap() as i32,
+ snapshot: row.get::<_, Vec>(4)
+ .iter()
+ .map(|uuid| {
+ let (high, low) = uuid.as_u64_pair();
+ entity::Uuid{high, low}
+ })
+ .collect::>(),
+ timestamp: row.get::<_, i64>(5),
+ expression: row.get(6),
+ domain: row.get(7),
+ }
+ })
+ .collect();
+ proto::proto::entity::JniWrapper {
+ partition_info,
+ ..Default::default()
+ }
+ }
+
+ ResultType::PartitionInfoWithoutTimestamp => {
+ let partition_info:Vec =
+ rows
+ .iter()
+ .map(|row|{
+ proto::proto::entity::PartitionInfo {
+ table_id: row.get(0),
+ partition_desc: row.get(1),
+ version: row.get::<_, i32>(2),
+ commit_op: proto::proto::entity::CommitOp::from_str_name(row.get(3)).unwrap() as i32,
+ snapshot: row.get::<_, Vec>(4)
+ .iter()
+ .map(|uuid| {
+ let (high, low) = uuid.as_u64_pair();
+ entity::Uuid{high, low}
+ })
+ .collect::>(),
+ expression: row.get(5),
+ domain: row.get(6),
+ ..Default::default()
+ }
+ })
+ .collect();
+ proto::proto::entity::JniWrapper {
+ partition_info,
+ ..Default::default()
+ }
+ }
+ ResultType::PartitionInfoWithOnlyCommitOp => {
+ let partition_info:Vec =
+ rows
+ .iter()
+ .map(|row|{
+ proto::proto::entity::PartitionInfo {
+ commit_op: proto::proto::entity::CommitOp::from_str_name(row.get(0)).unwrap() as i32,
+ ..Default::default()
+ }
+ })
+ .collect();
+ proto::proto::entity::JniWrapper {
+ partition_info,
+ ..Default::default()
+ }
+ }
+ ResultType::DataCommitInfo => {
+ let data_commit_info:Vec =
+ rows
+ .iter()
+ .map(|row|{
+ proto::proto::entity::DataCommitInfo {
+ table_id: row.get(0),
+ partition_desc: row.get(1),
+ commit_id: {
+ let (high, low)=row.get::<_, uuid::Uuid>(2).as_u64_pair();
+ Some(entity::Uuid{high, low})
+ },
+ file_ops: row.get::<_, Vec>(3)
+ .iter()
+ .map(|data_file_op| data_file_op.as_proto_data_file_op())
+ .collect::>(),
+ commit_op: proto::proto::entity::CommitOp::from_str_name(row.get(4)).unwrap() as i32,
+ timestamp: row.get(5),
+ committed: row.get(6),
+ domain: row.get(7),
+ }
+ })
+ .collect();
+ proto::proto::entity::JniWrapper {
+ data_commit_info,
+ ..Default::default()
+ }
+ }
+ };
+ Ok(wrapper.encode_to_vec())
+}
+
+
+pub fn execute_insert(
+ runtime: &Runtime,
+ client: &mut Client,
+ prepared: &mut PreparedStatementMap,
+ insert_type: i32,
+ wrapper: entity::JniWrapper,
+) -> Result {
+ if !(DAO_TYPE_INSERT_ONE_OFFSET..DAO_TYPE_QUERY_SCALAR_OFFSET).contains(&insert_type){
+ eprintln!("Invalid insert_type_index: {:?}", insert_type);
+ return Err(std::io::Error::from(std::io::ErrorKind::InvalidInput))
+ }
+ let insert_type = DaoType::try_from(insert_type).unwrap();
+ let statement = match get_prepared_statement(runtime, client, prepared, &insert_type) {
+ Ok(statement) => statement,
+ Err(err) => return Err(convert_to_io_error(err))
+ };
+
+ let result = match insert_type {
+ DaoType::InsertNamespace if wrapper.namespace.len() == 1 => {
+ let namespace = wrapper.namespace.get(0).unwrap();
+ let properties:serde_json::Value = serde_json::from_str(&namespace.properties)?;
+ runtime.block_on(async{
+ client.execute(
+ &statement,
+ &[
+ &namespace.namespace,
+ &properties,
+ &namespace.comment,
+ &namespace.domain,
+ ]
+ ).await
+ })
+ }
+ DaoType::InsertTableInfo if wrapper.table_info.len() == 1=> {
+ let table_info = wrapper.table_info.get(0).unwrap();
+ let properties:serde_json::Value = serde_json::from_str(&table_info.properties)?;
+ runtime.block_on(async{
+ client.execute(
+ &statement,
+ &[
+ &table_info.table_id,
+ &table_info.table_name,
+ &table_info.table_path,
+ &table_info.table_schema,
+ &properties,
+ &table_info.partitions,
+ &table_info.table_namespace,
+ &table_info.domain,
+ ]
+ ).await
+ })
+ }
+ DaoType::InsertTableNameId if wrapper.table_name_id.len() == 1 => {
+ let table_name_id = wrapper.table_name_id.get(0).unwrap();
+ runtime.block_on(async{
+ client.execute(
+ &statement,
+ &[
+ &table_name_id.table_id,
+ &table_name_id.table_name,
+ &table_name_id.table_namespace,
+ &table_name_id.domain,
+ ]
+ ).await
+ })
+ }
+ DaoType::InsertTablePathId if wrapper.table_path_id.len() == 1 => {
+ let table_path_id = wrapper.table_path_id.get(0).unwrap();
+ runtime.block_on(async{
+ client.execute(
+ &statement,
+ &[
+ &table_path_id.table_id,
+ &table_path_id.table_path,
+ &table_path_id.table_namespace,
+ &table_path_id.domain,
+ ]
+ ).await
+ })
+ }
+ DaoType::InsertPartitionInfo if wrapper.partition_info.len() == 1 =>{
+ let partition_info = wrapper.partition_info.get(0).unwrap();
+ let snapshot = partition_info.snapshot
+ .iter()
+ .map(|_uuid| uuid::Uuid::from_u64_pair(_uuid.high, _uuid.low))
+ .collect::>();
+
+ runtime.block_on(async{
+ client.execute(
+ &statement,
+ &[
+ &partition_info.table_id,
+ &partition_info.partition_desc,
+ &partition_info.version,
+ &partition_info.commit_op().as_str_name(),
+ &snapshot,
+ &partition_info.expression,
+ &partition_info.domain
+ ]
+ ).await
+ })
+ }
+ DaoType::InsertDataCommitInfo if wrapper.data_commit_info.len() == 1 =>{
+ let data_commit_info = wrapper.data_commit_info.get(0).unwrap();
+ let file_ops = data_commit_info.file_ops
+ .iter()
+ .map(DataFileOp::from_proto_data_file_op)
+ .collect::>();
+ let commit_id = data_commit_info.commit_id.as_ref().unwrap();
+ let _uuid = uuid::Uuid::from_u64_pair(commit_id.high, commit_id.low);
+
+ runtime.block_on(async{
+ client.execute(
+ &statement,
+ &[
+ &data_commit_info.table_id,
+ &data_commit_info.partition_desc,
+ &_uuid,
+ &file_ops,
+ &data_commit_info.commit_op().as_str_name(),
+ &data_commit_info.timestamp,
+ &data_commit_info.committed,
+ &data_commit_info.domain
+ ]
+ ).await
+ })
+ }
+ DaoType::TransactionInsertPartitionInfo => {
+ let partition_info_list = wrapper.partition_info;
+ let result = runtime.block_on(async{
+ let transaction = client.transaction().await.unwrap();
+ let prepared = transaction.prepare("insert into partition_info(
+ table_id,
+ partition_desc,
+ version,
+ commit_op,
+ snapshot,
+ expression,
+ domain
+ )
+ values($1::TEXT, $2::TEXT, $3::INT, $4::TEXT, $5::_UUID, $6::TEXT, $7::TEXT)").await;
+ let statement = match prepared {
+ Ok(statement) => statement,
+ Err(e) => return Err(e)
+ };
+
+ for i in 0..partition_info_list.len() {
+ let partition_info = partition_info_list.get(i).unwrap();
+ let snapshot = partition_info.snapshot
+ .iter()
+ .map(|_uuid| uuid::Uuid::from_u64_pair(_uuid.high, _uuid.low))
+ .collect::>();
+
+ let result = transaction.execute(
+ &statement,
+ &[
+ &partition_info.table_id,
+ &partition_info.partition_desc,
+ &partition_info.version,
+ &partition_info.commit_op().as_str_name(),
+ &snapshot,
+ &partition_info.expression,
+ &partition_info.domain
+ ]
+ ).await;
+
+ if let Some(e) = result.err() {
+ eprintln!("transaction insert error, err = {:?}", e);
+ return match transaction.rollback().await{
+ Ok(()) => Ok(0u64),
+ Err(e) => Err(e)
+ };
+ };
+
+ for uuid in &snapshot {
+ let result = transaction.execute(
+ "update data_commit_info set committed = 'true' where commit_id = $1::UUID",
+ &[&uuid]
+ ).await;
+
+ if let Some(e) = result.err() {
+ eprintln!("update committed error, err = {:?}", e);
+ return match transaction.rollback().await{
+ Ok(()) => Ok(0u64),
+ Err(e) => Err(e)
+ };
+ }
+ };
+ };
+ match transaction.commit().await{
+ Ok(()) => Ok(partition_info_list.len() as u64),
+ Err(e) => Err(e)
+ }
+ });
+ match result {
+ Ok(count) => Ok(count),
+ Err(e) => {
+ return Err(convert_to_io_error(e))
+ }
+ }
+ }
+ DaoType::TransactionInsertDataCommitInfo => {
+ let data_commit_info_list = wrapper.data_commit_info;
+ let result = runtime.block_on(async{
+ let transaction = client.transaction().await.unwrap();
+ let prepared = transaction.prepare("insert into data_commit_info(
+ table_id,
+ partition_desc,
+ commit_id,
+ file_ops,
+ commit_op,
+ timestamp,
+ committed,
+ domain
+ )
+ values($1::TEXT, $2::TEXT, $3::UUID, $4::_data_file_op, $5::TEXT, $6::BIGINT, $7::BOOL, $8::TEXT)").await;
+ let statement = match prepared {
+ Ok(statement) => statement,
+ Err(e) => return Err(e)
+ };
+
+ for i in 0..data_commit_info_list.len() {
+ let data_commit_info = data_commit_info_list.get(i).unwrap();
+ let file_ops = data_commit_info.file_ops
+ .iter()
+ .map(DataFileOp::from_proto_data_file_op)
+ .collect::>();
+ let commit_id = data_commit_info.commit_id.as_ref().unwrap();
+ let _uuid = uuid::Uuid::from_u64_pair(commit_id.high, commit_id.low);
+
+ let result = transaction.execute(
+ &statement,
+ &[
+ &data_commit_info.table_id,
+ &data_commit_info.partition_desc,
+ &_uuid,
+ &file_ops,
+ &data_commit_info.commit_op().as_str_name(),
+ &data_commit_info.timestamp,
+ &data_commit_info.committed,
+ &data_commit_info.domain
+ ]
+ ).await;
+
+ if let Some(e) = result.err() {
+ eprintln!("transaction insert error, err = {:?}", e);
+ return match transaction.rollback().await{
+ Ok(()) => Ok(0u64),
+ Err(e) => Err(e)
+ };
+ };
+
+ };
+ match transaction.commit().await{
+ Ok(()) => Ok(data_commit_info_list.len() as u64),
+ Err(e) => Err(e)
+ }
+ });
+ match result {
+ Ok(count) => Ok(count),
+ Err(e) => {
+ return Err(convert_to_io_error(e))
+ }
+ }
+ }
+ _ => {
+ eprintln!("InvalidInput of type={:?}: {:?}", insert_type, wrapper);
+ return Err(std::io::Error::from(std::io::ErrorKind::InvalidInput))
+ }
+ };
+ match result {
+ Ok(count) => Ok(count as i32),
+ Err(e) => Err(convert_to_io_error(e)),
+ }
+}
+
+pub fn execute_update(
+ runtime: &Runtime,
+ client: &mut Client,
+ prepared: &mut PreparedStatementMap,
+ update_type: i32,
+ joined_string: String,
+) -> Result {
+ if update_type < DAO_TYPE_UPDATE_OFFSET {
+ eprintln!("Invalid update_type_index: {:?}", update_type);
+ return Err(std::io::Error::from(std::io::ErrorKind::InvalidInput))
+ }
+ let update_type = DaoType::try_from(update_type).unwrap();
+ let statement = match get_prepared_statement(runtime, client, prepared, &update_type) {
+ Ok(statement) => statement,
+ Err(err) => return Err(convert_to_io_error(err))
+ };
+
+ let params = joined_string
+ .split(PARAM_DELIM)
+ .collect::>()
+ .iter()
+ .map(|str|str.to_string())
+ .collect::>();
+
+ let result = match update_type {
+ DaoType::DeleteNamespaceByNamespace |
+ DaoType::DeletePartitionInfoByTableId |
+ DaoType::DeleteDataCommitInfoByTableId |
+ DaoType::DeleteTableNameIdByTableId |
+ DaoType::DeleteTablePathIdByTableId |
+ DaoType::DeleteTablePathIdByTablePath if params.len() == 1 => {
+ runtime.block_on(async{
+ client.execute(&statement, &[¶ms[0]]).await
+ })
+ }
+ DaoType::DeleteTableInfoByIdAndPath |
+ DaoType::DeleteTableNameIdByTableNameAndNamespace |
+ DaoType::DeletePartitionInfoByTableIdAndPartitionDesc |
+ DaoType::DeleteDataCommitInfoByTableIdAndPartitionDesc if params.len() == 2 => {
+ runtime.block_on(async{
+ client.execute(&statement, &[¶ms[0], ¶ms[1]]).await
+ })
+ }
+ DaoType::UpdateTableInfoPropertiesById |
+ DaoType::UpdateNamespacePropertiesByNamespace if params.len() == 2 => {
+ let properties:serde_json::Value = serde_json::from_str(¶ms[1]).unwrap();
+ runtime.block_on(async{
+ client.execute(&statement, &[¶ms[0], &properties]).await
+ })
+ }
+ DaoType::DeletePreviousVersionPartition if params.len() == 3 => {
+ let ts = i64::from_str(¶ms[2]).unwrap();
+ runtime.block_on(async{
+ client.execute(&statement, &[¶ms[0], ¶ms[1], &ts]).await
+ })
+ }
+ DaoType::DeleteOneDataCommitInfoByTableIdAndPartitionDescAndCommitId if params.len() == 3 => {
+ let commit_id:uuid::Uuid = uuid::Uuid::from_str(¶ms[2]).unwrap();
+ runtime.block_on(async{
+ client.execute(&statement, &[¶ms[0], ¶ms[1], &commit_id]).await
+ })
+ }
+ DaoType::UpdateTableInfoById if params.len() == 4 => {
+ let mut statement = "update table_info set ".to_owned();
+ let mut idx = 2;
+ let mut filter_params = Vec::::with_capacity(3);
+ if !params[1].is_empty() {
+ statement += format!("table_name = ${}::TEXT ", idx).as_str();
+ idx += 1;
+ filter_params.push(params[1].clone());
+ }
+ if !params[2].is_empty() {
+ if idx > 2 {statement += ",";}
+ statement += format!("table_path = ${}::TEXT ", idx).as_str();
+ idx += 1;
+ filter_params.push(params[2].clone());
+ }
+ if !params[3].is_empty() {
+ if idx > 2 {statement += ",";}
+ statement += format!("table_schema = ${}::TEXT ", idx).as_str();
+ idx += 1;
+ filter_params.push(params[3].clone());
+ }
+ statement += " where table_id = $1::TEXT";
+ runtime.block_on(async{
+ match idx {
+ 3 => client.execute(&statement, &[¶ms[0], &filter_params[0]]).await,
+ 4 => client.execute(&statement, &[¶ms[0], &filter_params[0], &filter_params[1]]).await,
+ 5 => client.execute(&statement, &[¶ms[0], &filter_params[0], &filter_params[1], &filter_params[2]]).await,
+ _ => todo!(),
+ }
+ })
+ }
+ DaoType::DeleteDataCommitInfoByTableIdAndPartitionDescAndCommitIdList if params.len() == 3 => {
+ let concated_uuid = ¶ms[2];
+ if concated_uuid.len() % 32 != 0 {
+ eprintln!("Invalid params of update_type={:?}, params={:?}", update_type, params);
+ return Err(std::io::Error::from(std::io::ErrorKind::InvalidInput));
+ }
+ let uuid_num = concated_uuid.len() / 32;
+ let mut uuid_list = Vec::::with_capacity(uuid_num);
+ let mut idx = 0;
+ for _ in 0..uuid_num {
+ let high = u64::from_str_radix(&concated_uuid[idx..idx+16], 16).unwrap();
+ let low = u64::from_str_radix(&concated_uuid[idx+16..idx+32], 16).unwrap();
+ uuid_list.push(uuid::Uuid::from_u64_pair(high, low).to_string());
+ idx += 32;
+ }
+
+ let uuid_str_list = "'".to_owned() + &uuid_list.join("','") + "'";
+
+ let statement = format!(
+ "delete from data_commit_info
+ where table_id = $1::TEXT and partition_desc = $2::TEXT and commit_id in commit_id in ({}) ", uuid_str_list);
+
+ runtime.block_on(async{
+ let statement = client.prepare(&statement).await?;
+ client.execute(&statement, &[¶ms[0], ¶ms[1]]).await
+ })
+ }
+ _ => {
+ eprintln!("InvalidInput of type={:?}: {:?}", update_type, params);
+ return Err(std::io::Error::from(std::io::ErrorKind::InvalidInput))
+ }
+ };
+ match result {
+ Ok(count) => Ok(count as i32),
+ Err(e) => Err(convert_to_io_error(e)),
+ }
+}
+
+pub fn execute_query_scalar(
+ runtime: &Runtime,
+ client: &mut Client,
+ prepared: &mut PreparedStatementMap,
+ query_type: i32,
+ joined_string: String,
+) -> Result