diff --git a/build.sh b/build.sh index 37e13774f97dae..28e01b4a31cc9a 100755 --- a/build.sh +++ b/build.sh @@ -66,6 +66,8 @@ Usage: $0 --be-java-extensions build Backend java extensions. Default ON. --be-cdc-client build Cdc Client for backend. Default ON. --be-extension-ignore build be-java-extensions package, choose which modules to ignore. Multiple modules separated by commas. + --connector-es build Elasticsearch connector plugin. Default ON. + --no-connector-es exclude Elasticsearch connector from build. --enable-dynamic-arch enable dynamic CPU detection in OpenBLAS. Default ON. --disable-dynamic-arch disable dynamic CPU detection in OpenBLAS. --clean clean and build target @@ -200,6 +202,8 @@ if ! OPTS="$(getopt \ -l 'be-java-extensions' \ -l 'be-cdc-client' \ -l 'be-extension-ignore:' \ + -l 'connector-es' \ + -l 'no-connector-es' \ -l 'enable-dynamic-arch' \ -l 'disable-dynamic-arch' \ -l 'clean' \ @@ -228,6 +232,7 @@ BUILD_BE_CDC_CLIENT=0 BUILD_OBS_DEPENDENCIES=1 BUILD_COS_DEPENDENCIES=1 BUILD_HIVE_UDF=0 +BUILD_CONNECTOR_ES=1 ENABLE_DYNAMIC_ARCH='ON' CLEAN=0 HELP=0 @@ -249,6 +254,7 @@ if [[ "$#" == 1 ]]; then BUILD_INDEX_TOOL='OFF' BUILD_BENCHMARK='OFF' BUILD_HIVE_UDF=1 + BUILD_CONNECTOR_ES=1 BUILD_BE_JAVA_EXTENSIONS=1 BUILD_BE_CDC_CLIENT=1 CLEAN=0 @@ -313,6 +319,14 @@ else --be-cdc-client) BUILD_BE_CDC_CLIENT=1 shift + ;; + --connector-es) + BUILD_CONNECTOR_ES=1 + shift + ;; + --no-connector-es) + BUILD_CONNECTOR_ES=0 + shift ;; --exclude-obs-dependencies) BUILD_OBS_DEPENDENCIES=0 @@ -380,6 +394,7 @@ else BUILD_INDEX_TOOL='ON' BUILD_TASK_EXECUTOR_SIMULATOR='OFF' BUILD_HIVE_UDF=1 + BUILD_CONNECTOR_ES=1 BUILD_BE_JAVA_EXTENSIONS=1 BUILD_BE_CDC_CLIENT=1 CLEAN=0 @@ -600,6 +615,7 @@ echo "Get params: BUILD_BE_JAVA_EXTENSIONS -- ${BUILD_BE_JAVA_EXTENSIONS} BUILD_BE_CDC_CLIENT -- ${BUILD_BE_CDC_CLIENT} BUILD_HIVE_UDF -- ${BUILD_HIVE_UDF} + BUILD_CONNECTOR_ES -- ${BUILD_CONNECTOR_ES} BUILD_JUICEFS -- ${BUILD_JUICEFS} PARALLEL -- ${PARALLEL} CLEAN -- ${CLEAN} @@ -645,6 +661,9 @@ if [[ "${BUILD_FE}" -eq 1 ]]; then modules+=("fe-${WITH_TDE_DIR}") fi fi +if [[ "${BUILD_CONNECTOR_ES}" -eq 1 ]]; then + modules+=("fe-connectors/connector-es") +fi if [[ "${BUILD_HIVE_UDF}" -eq 1 ]]; then modules+=("hive-udf") fi @@ -948,6 +967,12 @@ if [[ "${BUILD_FE}" -eq 1 ]]; then mkdir -p "${DORIS_OUTPUT}/fe/conf/ssl" mkdir -p "${DORIS_OUTPUT}/fe/plugins/jdbc_drivers/" mkdir -p "${DORIS_OUTPUT}/fe/plugins/java_udf/" + mkdir -p "${DORIS_OUTPUT}/fe/lib/connectors/" + if [[ "${BUILD_CONNECTOR_ES}" -eq 1 ]]; then + mkdir -p "${DORIS_OUTPUT}/fe/lib/connectors/es/" + cp -r -p "${DORIS_HOME}/fe/fe-connectors/connector-es/target/doris-connector-es.jar" \ + "${DORIS_OUTPUT}/fe/lib/connectors/es/" + fi mkdir -p "${DORIS_OUTPUT}/fe/plugins/connectors/" mkdir -p "${DORIS_OUTPUT}/fe/plugins/hadoop_conf/" mkdir -p "${DORIS_OUTPUT}/fe/plugins/java_extensions/" diff --git a/docker/thirdparties/run-thirdparties-docker.sh b/docker/thirdparties/run-thirdparties-docker.sh index b4a98e27fc8d04..fcf4a5d2a8a9ab 100755 --- a/docker/thirdparties/run-thirdparties-docker.sh +++ b/docker/thirdparties/run-thirdparties-docker.sh @@ -382,6 +382,14 @@ prepare_juicefs_meta_for_hive() { return 0 fi + # Clean stale bucket data before formatting. When meta is not formatted, + # any leftover data in the bucket directory is orphaned from a previous run + # and will cause "juicefs format" to fail with "Storage ... is not empty". + if [[ -d "${bucket_dir}" ]]; then + echo "Cleaning stale JuiceFS bucket directory: ${bucket_dir}" + sudo rm -rf "${bucket_dir:?}"/* + fi + if ! run_juicefs_cli \ format --storage file --bucket "${bucket_dir}" "${jfs_meta}" "${jfs_cluster_name}"; then # If format reports conflict on rerun, verify by status and continue. diff --git a/fe/fe-connectors/AGENTS.md b/fe/fe-connectors/AGENTS.md new file mode 100644 index 00000000000000..bb3ff25ecb021b --- /dev/null +++ b/fe/fe-connectors/AGENTS.md @@ -0,0 +1,401 @@ +# AGENTS.md — Doris FE Connector Plugins + +This directory contains connector plugins for Apache Doris external data sources. Each connector is an independent Maven module that implements the `CatalogProvider` SPI interface, enabling Doris to dynamically discover and load external data source support. + +## Architecture Overview + +``` +fe/fe-core/src/.../datasource/spi/ +├── CatalogProvider.java # SPI interface: factory for creating ExternalCatalog +├── ConnectorMetadata.java # SPI interface: all data source operations +├── ScanNodeProvider.java # SPI interface: scan node creation for query planning +├── CatalogProviderRegistry.java # In-memory registry of loaded providers +└── CatalogPluginLoader.java # Scans lib/connectors/ and loads JAR plugins + +fe/fe-connectors/ +├── AGENTS.md # This guide +└── connector-es/ # Reference implementation (Elasticsearch) + ├── pom.xml + └── src/main/java/.../es/ + ├── EsCatalogProvider.java # implements CatalogProvider (pure factory) + ├── EsConnectorMetadata.java # implements ConnectorMetadata (all ES operations) + ├── EsScanNodeProvider.java # implements ScanNodeProvider + ├── EsExternalCatalog.java # extends ExternalCatalog + ├── EsExternalDatabase.java + ├── EsExternalTable.java + ├── source/EsScanNode.java # extends ScanNode + └── ... +``` + +### Key Interfaces + +| Interface | Responsibility | +|---|---| +| `CatalogProvider` | Pure factory: `getType()` + `createCatalog()`. Registered via ServiceLoader. | +| `ConnectorMetadata` | All data source operations: metadata queries, Doris object construction, scan node creation, lifecycle management. Owned by `ExternalCatalog`, NOT the reverse — `ConnectorMetadata` must NOT hold a back-reference to `ExternalCatalog`. | +| `ScanNodeProvider` | Creates `ScanNode` instances for query plan generation. Returned by `ConnectorMetadata.getScanNodeProvider()`. | + +### Ownership & Dependency Rules + +``` +ExternalCatalog ──holds──> ConnectorMetadata ──provides──> ScanNodeProvider + │ │ + │ (lifecycle, cache, │ (metadata queries, object creation, + │ EditLog, DDL orchestration)│ scan node factory, ES/HMS/... client) + │ │ + │ ╳ Must NOT hold back-reference to ExternalCatalog + │ Receives ExternalCatalog as parameter only in + │ createDatabase() and createTable() for Doris + │ internal object construction. +``` + +### Runtime Loading Flow + +1. FE startup → `Env.java` calls `CatalogPluginLoader.loadPlugins()` +2. `CatalogPluginLoader` scans `${DORIS_HOME}/lib/connectors/` subdirectories +3. Each subdirectory's JARs are loaded via an isolated `URLClassLoader` +4. `ServiceLoader` discovers provider implementations +5. Providers are registered in `CatalogProviderRegistry` +6. `CatalogFactory.createCatalog()` checks the registry before falling back to the hardcoded switch-case + +### Initialization Flow + +1. `CatalogFactory` calls `CatalogProvider.createCatalog()` to create the `ExternalCatalog` +2. On first access, `ExternalCatalog.makeSureInitialized()` → `initLocalObjectsImpl()` +3. The subclass (e.g., `EsExternalCatalog`) creates data source clients and a `ConnectorMetadata` instance +4. `this.connectorMetadata = new EsConnectorMetadata(esRestClient, config...)` +5. Base class `ExternalCatalog` delegates metadata operations to `connectorMetadata` + +## Creating a New Connector Plugin + +### Step 1: Create Module Directory + +```bash +mkdir -p fe/fe-connectors/connector-{name}/src/main/java/org/apache/doris/datasource/{name} +mkdir -p fe/fe-connectors/connector-{name}/src/main/java/org/apache/doris/datasource/{name}/source +mkdir -p fe/fe-connectors/connector-{name}/src/main/resources/META-INF/services +``` + +Replace `{name}` with the connector name (e.g., `jdbc`, `iceberg`, `paimon`). + +### Step 2: Create `pom.xml` + +Copy and adapt from `connector-es/pom.xml`. Key points: + +```xml + + org.apache.doris + ${revision} + fe + ../../pom.xml + +connector-{name} +Doris FE Connector Plugin - {Name} + + + + + ${project.groupId} + fe-core + ${project.version} + provided + + + + + org.projectlombok + lombok + provided + + + + + + + + + doris-connector-{name} + ${project.basedir}/target/ + + + org.apache.maven.plugins + maven-shade-plugin + + + package + shade + + + + org.apache.doris:fe-core + org.apache.doris:fe-common + org.apache.doris:fe-catalog + + + + + + + + org.apache.maven.plugins + maven-javadoc-plugin + true + + + +``` + +**Important rules:** +- `fe-core` MUST be `provided` scope — it is the parent classloader at runtime +- `lombok` MUST be `provided` scope +- Connector-specific third-party libraries should be non-provided (default scope), so the shade plugin bundles them into the fat JAR +- The shade plugin excludes all Doris modules to avoid duplication + +### Step 3: Implement `CatalogProvider` + +Create `{Name}CatalogProvider.java` implementing `org.apache.doris.datasource.spi.CatalogProvider`: + +```java +package org.apache.doris.datasource.{name}; + +import org.apache.doris.datasource.ExternalCatalog; +import org.apache.doris.datasource.spi.CatalogProvider; + +import java.util.Map; + +public class {Name}CatalogProvider implements CatalogProvider { + + @Override + public String getType() { + // Must match the "type" property in CREATE CATALOG statement + return "{name}"; + } + + @Override + public ExternalCatalog createCatalog(long catalogId, String name, String resource, + Map props, String comment) { + return new {Name}ExternalCatalog(catalogId, name, resource, props, comment); + } +} +``` + +`CatalogProvider` is a **pure factory** with only two methods: `getType()` and `createCatalog()`. + +### Step 4: Implement `ConnectorMetadata` + +Create `{Name}ConnectorMetadata.java` implementing `org.apache.doris.datasource.spi.ConnectorMetadata`: + +```java +package org.apache.doris.datasource.{name}; + +import org.apache.doris.datasource.ExternalCatalog; +import org.apache.doris.datasource.ExternalDatabase; +import org.apache.doris.datasource.ExternalTable; +import org.apache.doris.datasource.spi.ConnectorMetadata; +import org.apache.doris.datasource.spi.ScanNodeProvider; + +import java.util.List; + +public class {Name}ConnectorMetadata implements ConnectorMetadata { + + // Hold data source clients directly, NOT the ExternalCatalog + private final {Name}Client client; + + public {Name}ConnectorMetadata({Name}Client client) { + this.client = client; + } + + @Override + public List listSchemaNames() { ... } + + @Override + public List listTableNames(String schemaName) { ... } + + @Override + public boolean tableExists(String schemaName, String tableName) { ... } + + @Override + public ExternalDatabase createDatabase(ExternalCatalog catalog, + long dbId, String localDbName, String remoteDbName) { + // catalog is passed as parameter, NOT stored as a field + return new {Name}ExternalDatabase(catalog, dbId, localDbName, remoteDbName); + } + + @Override + public ExternalTable createTable(ExternalCatalog catalog, + ExternalDatabase db, + long tblId, String localName, String remoteName) { + return new {Name}ExternalTable(tblId, localName, remoteName, + ({Name}ExternalCatalog) catalog, ({Name}ExternalDatabase) db); + } + + @Override + public ScanNodeProvider getScanNodeProvider() { + return new {Name}ScanNodeProvider(); + } + + @Override + public void close() { + // Close data source clients + } +} +``` + +**Critical rule:** `ConnectorMetadata` must NOT hold a back-reference to `ExternalCatalog`. It should hold data source clients (e.g., REST client, JDBC client) directly. The `ExternalCatalog` reference is only passed as a method parameter to `createDatabase()` and `createTable()` for Doris internal object construction. + +### Step 5: Implement `ExternalCatalog` Subclass + +```java +public class {Name}ExternalCatalog extends ExternalCatalog { + + @Override + protected void initLocalObjectsImpl() { + // 1. Create data source client + {Name}Client client = new {Name}Client(...); + + // 2. Create ConnectorMetadata and assign to base class field + this.connectorMetadata = new {Name}ConnectorMetadata(client); + } +} +``` + +The subclass creates the `ConnectorMetadata` in `initLocalObjectsImpl()` and assigns it to `this.connectorMetadata`. The base class `ExternalCatalog` handles all delegation automatically. + +### Step 6: Register SPI Service + +Create the file: +``` +src/main/resources/META-INF/services/org.apache.doris.datasource.spi.CatalogProvider +``` + +Content (must include the Apache license header): +``` +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# ... +# under the License. + +org.apache.doris.datasource.{name}.{Name}CatalogProvider +``` + +### Step 7: Implement Remaining Classes + +At minimum, you need: + +| Class | Extends/Implements | Purpose | +|---|---|---| +| `{Name}ExternalCatalog` | `ExternalCatalog` | Catalog lifecycle, creates `ConnectorMetadata` in `initLocalObjectsImpl()` | +| `{Name}ConnectorMetadata` | `ConnectorMetadata` | All data source operations, owns data source client | +| `{Name}ScanNodeProvider` | `ScanNodeProvider` | Creates scan nodes | +| `{Name}ExternalDatabase` | `ExternalDatabase<{Name}ExternalTable>` | Database-level metadata | +| `{Name}ExternalTable` | `ExternalTable` | Table-level metadata, schema | +| `{Name}ScanNode` | `ScanNode` | Query execution plan node | + +All source files must use the package `org.apache.doris.datasource.{name}` (or `org.apache.doris.datasource.{name}.source` for ScanNode). + +### Step 8: Register in Parent `pom.xml` + +Add the module to `fe/pom.xml`: +```xml +fe-connectors/connector-{name} +``` + +### Step 9: Update `build.sh` + +All connectors are built by default. You only need to add `--no-connector-{name}` for users who want to exclude it. + +Four changes required in `build.sh`: + +1. **Add getopt entries:** +```bash + -l 'no-connector-{name}' \ +``` + +2. **Add variable (default ON):** +```bash +BUILD_CONNECTOR_{NAME}=1 +``` + +3. **Add case handler for exclusion:** +```bash +--no-connector-{name}) + BUILD_CONNECTOR_{NAME}=0 + shift + ;; +``` + +4. **Add module assembly and output copy:** +```bash +# In the FE_MODULES assembly section: +if [[ "${BUILD_CONNECTOR_{NAME}}" -eq 1 ]]; then + modules+=("fe-connectors/connector-{name}") +fi + +# In the FE output copy section: +if [[ "${BUILD_CONNECTOR_{NAME}}" -eq 1 ]]; then + mkdir -p "${DORIS_OUTPUT}/fe/lib/connectors/{name}/" + cp -r -p "${DORIS_HOME}/fe/fe-connectors/connector-{name}/target/doris-connector-{name}.jar" \ + "${DORIS_OUTPUT}/fe/lib/connectors/{name}/" +fi +``` + +### Step 10: Update `GsonUtils.java` for Backward Compatibility + +If migrating an existing connector, add a `registerCompatibleSubtype` entry in `GsonUtils.java` so that old EditLog entries with the original class name can be deserialized as `ExternalCatalog`: + +```java +// {Name}: old EditLog "{Name}ExternalCatalog" maps to ExternalCatalog +.registerCompatibleSubtype(ExternalCatalog.class, {Name}ExternalCatalog.class.getSimpleName()) +``` + +This ensures that metadata written before the migration can still be loaded. + +## Migrating an Existing Connector from fe-core + +When moving an existing data source from `fe-core` to a connector plugin: + +1. **Move source files:** `git mv` from `fe-core/src/.../datasource/{name}/` to `fe-connectors/connector-{name}/src/.../datasource/{name}/` +2. **Create `{Name}CatalogProvider`** implementing `CatalogProvider` (pure factory) +3. **Create `{Name}ConnectorMetadata`** implementing `ConnectorMetadata` (holds data source client directly, no back-reference to ExternalCatalog) +4. **Update `{Name}ExternalCatalog.initLocalObjectsImpl()`:** Create data source client and `ConnectorMetadata`, assign to `this.connectorMetadata` +5. **Remove duplicate method overrides** in `{Name}ExternalCatalog`: `listTableNamesFromRemote()`, `tableExist()`, `listDatabaseNames()` — the base class delegates to `connectorMetadata` automatically +6. **Update `PhysicalPlanTranslator.java`:** Use `ExternalCatalog.getConnectorMetadata().getScanNodeProvider()` to create scan nodes +7. **Update `GsonUtils.java`:** Add `registerCompatibleSubtype` for backward compatibility +8. **Keep the fallback switch-case in `CatalogFactory.java`** for backward compatibility during the transition period; it can be removed after the connector plugin is stable +9. **Follow all steps in "Creating a New Connector Plugin"** (pom.xml, SPI service file, build.sh, parent pom) + +## Build and Test + +```bash +# Full build (all connectors included by default) +./build.sh --fe --be -j8 + +# Build everything except a specific connector +./build.sh --fe --be --no-connector-{name} -j8 +``` + +Output artifacts: +``` +output/fe/lib/connectors/{name}/doris-connector-{name}.jar +``` + +## Checklist for New Connector PR + +- [ ] `fe/fe-connectors/connector-{name}/pom.xml` created with correct parent, dependencies (`fe-core` provided, `lombok` provided), shade plugin +- [ ] `{Name}CatalogProvider` implements `CatalogProvider` (only `getType()` + `createCatalog()`) +- [ ] `{Name}ConnectorMetadata` implements `ConnectorMetadata` (holds data source client directly, no back-reference) +- [ ] `{Name}ExternalCatalog.initLocalObjectsImpl()` creates `ConnectorMetadata` and assigns to `this.connectorMetadata` +- [ ] `META-INF/services/org.apache.doris.datasource.spi.CatalogProvider` file created with license header +- [ ] `{Name}ExternalDatabase`, `{Name}ExternalTable`, `{Name}ScanNode` implemented +- [ ] `{Name}ScanNodeProvider` implemented +- [ ] No duplicate metadata method overrides in `{Name}ExternalCatalog` +- [ ] `fe/pom.xml` updated with new module +- [ ] `build.sh` updated: getopt, variable, case handler, module assembly, output copy +- [ ] `GsonUtils.java` updated with `registerCompatibleSubtype` (if migrating existing connector) +- [ ] All source files have Apache 2.0 license headers +- [ ] All resource files have Apache 2.0 license headers (use `#` comment style) +- [ ] Import order is lexicographical (CI enforces this via checkstyle) diff --git a/fe/fe-connectors/connector-es/pom.xml b/fe/fe-connectors/connector-es/pom.xml new file mode 100644 index 00000000000000..eaeaf2f1642f86 --- /dev/null +++ b/fe/fe-connectors/connector-es/pom.xml @@ -0,0 +1,104 @@ + + + + 4.0.0 + + org.apache.doris + ${revision} + fe + ../../pom.xml + + connector-es + jar + Doris FE Connector Plugin - Elasticsearch + Elasticsearch connector plugin for Apache Doris, loaded via SPI + + + + + ${project.groupId} + fe-core + ${project.version} + provided + + + + + org.projectlombok + lombok + provided + + + + + org.junit.jupiter + junit-jupiter + test + + + + + doris-connector-es + ${project.basedir}/target/ + + + + org.apache.maven.plugins + maven-shade-plugin + + + package + + shade + + + + + *:* + + META-INF/*.SF + META-INF/*.DSA + META-INF/*.RSA + + + + + + + org.apache.doris:fe-core + org.apache.doris:fe-common + org.apache.doris:fe-catalog + + + + + + + + org.apache.maven.plugins + maven-javadoc-plugin + + true + + + + + diff --git a/fe/fe-connectors/connector-es/src/main/java/org/apache/doris/datasource/es/DorisEsException.java b/fe/fe-connectors/connector-es/src/main/java/org/apache/doris/datasource/es/DorisEsException.java new file mode 100644 index 00000000000000..9d46e86acd3233 --- /dev/null +++ b/fe/fe-connectors/connector-es/src/main/java/org/apache/doris/datasource/es/DorisEsException.java @@ -0,0 +1,26 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +package org.apache.doris.datasource.es; + + +public class DorisEsException extends RuntimeException { + + public DorisEsException(String msg) { + super(msg); + } +} diff --git a/fe/fe-connectors/connector-es/src/main/java/org/apache/doris/datasource/es/EsCatalogProvider.java b/fe/fe-connectors/connector-es/src/main/java/org/apache/doris/datasource/es/EsCatalogProvider.java new file mode 100644 index 00000000000000..193ea5cf654c29 --- /dev/null +++ b/fe/fe-connectors/connector-es/src/main/java/org/apache/doris/datasource/es/EsCatalogProvider.java @@ -0,0 +1,43 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +package org.apache.doris.datasource.es; + +import org.apache.doris.datasource.ExternalCatalog; +import org.apache.doris.datasource.spi.CatalogProvider; + +import java.util.Map; + +/** + * CatalogProvider (factory) implementation for Elasticsearch. + * + *

Pure factory: creates the {@link EsExternalCatalog} instance. + * All metadata and scan node operations are handled by {@link EsConnectorMetadata}.

+ */ +public class EsCatalogProvider implements CatalogProvider { + + @Override + public String getType() { + return "es"; + } + + @Override + public ExternalCatalog createCatalog(long catalogId, String name, String resource, + Map props, String comment) { + return new EsExternalCatalog(catalogId, name, resource, props, comment); + } +} diff --git a/fe/fe-connectors/connector-es/src/main/java/org/apache/doris/datasource/es/EsConnectorMetadata.java b/fe/fe-connectors/connector-es/src/main/java/org/apache/doris/datasource/es/EsConnectorMetadata.java new file mode 100644 index 00000000000000..3446d239da5db8 --- /dev/null +++ b/fe/fe-connectors/connector-es/src/main/java/org/apache/doris/datasource/es/EsConnectorMetadata.java @@ -0,0 +1,91 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +package org.apache.doris.datasource.es; + +import org.apache.doris.datasource.ExternalCatalog; +import org.apache.doris.datasource.ExternalDatabase; +import org.apache.doris.datasource.ExternalTable; +import org.apache.doris.datasource.spi.ConnectorMetadata; +import org.apache.doris.datasource.spi.ScanNodeProvider; + +import com.google.common.collect.Lists; + +import java.util.List; + +/** + * ConnectorMetadata implementation for Elasticsearch. + * + *

Provides all data source operations for ES: metadata queries, + * Doris internal object construction, and scan node creation.

+ * + *

This class owns the {@link EsRestClient} and interacts with the ES cluster + * directly, without holding a back-reference to {@link EsExternalCatalog}.

+ */ +public class EsConnectorMetadata implements ConnectorMetadata { + + private final EsRestClient esRestClient; + private final boolean includeHiddenIndex; + + public EsConnectorMetadata(EsRestClient esRestClient, boolean includeHiddenIndex) { + this.esRestClient = esRestClient; + this.includeHiddenIndex = includeHiddenIndex; + } + + public EsRestClient getEsRestClient() { + return esRestClient; + } + + @Override + public List listSchemaNames() { + return Lists.newArrayList(EsExternalCatalog.DEFAULT_DB); + } + + @Override + public List listTableNames(String schemaName) { + return esRestClient.listTable(includeHiddenIndex); + } + + @Override + public boolean tableExists(String schemaName, String tableName) { + return esRestClient.existIndex(esRestClient.getClient(), tableName); + } + + @Override + public ExternalDatabase createDatabase(ExternalCatalog catalog, + long dbId, String localDbName, String remoteDbName) { + return new EsExternalDatabase(catalog, dbId, localDbName, remoteDbName); + } + + @Override + public ExternalTable createTable(ExternalCatalog catalog, + ExternalDatabase db, + long tblId, String localName, String remoteName) { + return new EsExternalTable(tblId, localName, remoteName, + (EsExternalCatalog) catalog, (EsExternalDatabase) db); + } + + @Override + public ScanNodeProvider getScanNodeProvider() { + return new EsScanNodeProvider(); + } + + @Override + public void close() { + // EsRestClient doesn't have an explicit close, but if it did, we'd close it here. + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/es/EsExternalCatalog.java b/fe/fe-connectors/connector-es/src/main/java/org/apache/doris/datasource/es/EsExternalCatalog.java similarity index 88% rename from fe/fe-core/src/main/java/org/apache/doris/datasource/es/EsExternalCatalog.java rename to fe/fe-connectors/connector-es/src/main/java/org/apache/doris/datasource/es/EsExternalCatalog.java index 60371365c50536..e59c6fe2f429ab 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/es/EsExternalCatalog.java +++ b/fe/fe-connectors/connector-es/src/main/java/org/apache/doris/datasource/es/EsExternalCatalog.java @@ -22,10 +22,8 @@ import org.apache.doris.datasource.CatalogProperty; import org.apache.doris.datasource.ExternalCatalog; import org.apache.doris.datasource.InitCatalogLog; -import org.apache.doris.datasource.SessionContext; import com.google.common.collect.ImmutableList; -import com.google.common.collect.Lists; import com.google.common.collect.Maps; import lombok.Getter; import org.apache.commons.lang3.StringUtils; @@ -43,7 +41,6 @@ public class EsExternalCatalog extends ExternalCatalog { public static final String DEFAULT_DB = "default_db"; private static final Logger LOG = LogManager.getLogger(EsExternalCatalog.class); - private EsRestClient esRestClient; private static final List REQUIRED_PROPERTIES = ImmutableList.of( EsResource.HOSTS ); @@ -126,26 +123,16 @@ public boolean enableIncludeHiddenIndex() { @Override protected void initLocalObjectsImpl() { - esRestClient = new EsRestClient(getNodes(), getUsername(), getPassword(), enableSsl()); - if (!esRestClient.health()) { + EsRestClient client = new EsRestClient(getNodes(), getUsername(), getPassword(), enableSsl()); + if (!client.health()) { throw new DorisEsException("Failed to connect to ES cluster," + " please check your ES cluster or your ES catalog configuration."); } + this.connectorMetadata = new EsConnectorMetadata(client, enableIncludeHiddenIndex()); } - @Override - protected List listTableNamesFromRemote(SessionContext ctx, String dbName) { - return esRestClient.listTable(enableIncludeHiddenIndex()); - } - - @Override - public boolean tableExist(SessionContext ctx, String dbName, String tblName) { - return esRestClient.existIndex(this.esRestClient.getClient(), tblName); - } - - @Override - protected List listDatabaseNames() { - return Lists.newArrayList(DEFAULT_DB); + public EsRestClient getEsRestClient() { + return ((EsConnectorMetadata) getConnectorMetadata()).getEsRestClient(); } @Override diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/es/EsExternalDatabase.java b/fe/fe-connectors/connector-es/src/main/java/org/apache/doris/datasource/es/EsExternalDatabase.java similarity index 100% rename from fe/fe-core/src/main/java/org/apache/doris/datasource/es/EsExternalDatabase.java rename to fe/fe-connectors/connector-es/src/main/java/org/apache/doris/datasource/es/EsExternalDatabase.java diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/es/EsExternalTable.java b/fe/fe-connectors/connector-es/src/main/java/org/apache/doris/datasource/es/EsExternalTable.java similarity index 100% rename from fe/fe-core/src/main/java/org/apache/doris/datasource/es/EsExternalTable.java rename to fe/fe-connectors/connector-es/src/main/java/org/apache/doris/datasource/es/EsExternalTable.java diff --git a/fe/fe-connectors/connector-es/src/main/java/org/apache/doris/datasource/es/EsMajorVersion.java b/fe/fe-connectors/connector-es/src/main/java/org/apache/doris/datasource/es/EsMajorVersion.java new file mode 100644 index 00000000000000..ccca6e1ee665c9 --- /dev/null +++ b/fe/fe-connectors/connector-es/src/main/java/org/apache/doris/datasource/es/EsMajorVersion.java @@ -0,0 +1,120 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +package org.apache.doris.datasource.es; + + +/** + * Elasticsearch major version information, useful to check client's query compatibility with the Rest API. + *

+ * reference es-hadoop: + */ +public class EsMajorVersion { + + public static final EsMajorVersion V_0_X = new EsMajorVersion((byte) 0, "0.x"); + public static final EsMajorVersion V_1_X = new EsMajorVersion((byte) 1, "1.x"); + public static final EsMajorVersion V_2_X = new EsMajorVersion((byte) 2, "2.x"); + public static final EsMajorVersion V_5_X = new EsMajorVersion((byte) 5, "5.x"); + public static final EsMajorVersion V_6_X = new EsMajorVersion((byte) 6, "6.x"); + public static final EsMajorVersion V_7_X = new EsMajorVersion((byte) 7, "7.x"); + public static final EsMajorVersion V_8_X = new EsMajorVersion((byte) 8, "8.x"); + public static final EsMajorVersion LATEST = V_8_X; + + public final byte major; + private final String version; + + private EsMajorVersion(byte major, String version) { + this.major = major; + this.version = version; + } + + public boolean after(EsMajorVersion version) { + return version.major < major; + } + + public boolean on(EsMajorVersion version) { + return version.major == major; + } + + public boolean notOn(EsMajorVersion version) { + return !on(version); + } + + public boolean onOrAfter(EsMajorVersion version) { + return version.major <= major; + } + + public boolean before(EsMajorVersion version) { + return version.major > major; + } + + public boolean onOrBefore(EsMajorVersion version) { + return version.major >= major; + } + + public static EsMajorVersion parse(String version) throws DorisEsException { + if (version.startsWith("0.")) { + return new EsMajorVersion((byte) 0, version); + } + if (version.startsWith("1.")) { + return new EsMajorVersion((byte) 1, version); + } + if (version.startsWith("2.")) { + return new EsMajorVersion((byte) 2, version); + } + if (version.startsWith("5.")) { + return new EsMajorVersion((byte) 5, version); + } + if (version.startsWith("6.")) { + return new EsMajorVersion((byte) 6, version); + } + if (version.startsWith("7.")) { + return new EsMajorVersion((byte) 7, version); + } + // used for the next released ES version + if (version.startsWith("8.")) { + return new EsMajorVersion((byte) 8, version); + } + throw new DorisEsException( + "Unsupported/Unknown ES Cluster version [" + version + "]." + "Highest supported version is [" + + LATEST.version + "]."); + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + + EsMajorVersion version = (EsMajorVersion) o; + + return major == version.major && version.equals(version.version); + } + + @Override + public int hashCode() { + return major; + } + + @Override + public String toString() { + return version; + } +} diff --git a/fe/fe-connectors/connector-es/src/main/java/org/apache/doris/datasource/es/EsMetaStateTracker.java b/fe/fe-connectors/connector-es/src/main/java/org/apache/doris/datasource/es/EsMetaStateTracker.java new file mode 100644 index 00000000000000..43c112a8f73258 --- /dev/null +++ b/fe/fe-connectors/connector-es/src/main/java/org/apache/doris/datasource/es/EsMetaStateTracker.java @@ -0,0 +1,65 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +package org.apache.doris.datasource.es; + +import org.apache.doris.catalog.EsTable; +import org.apache.doris.common.UserException; + +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; + +import java.util.LinkedList; +import java.util.List; + +/** + * It is responsible for this class to schedule all network request sent to remote ES Cluster + * Request sequence + * 1. GET {index}/_mapping + * 2. GET {index}/_search_shards + *

+ * note: step 1 is not necessary + */ +public class EsMetaStateTracker { + + private static final Logger LOG = LogManager.getLogger(EsMetaStateTracker.class); + + private List builtinSearchPhase = new LinkedList<>(); + private SearchContext searchContext; + + public EsMetaStateTracker(EsRestClient client, EsTable esTable) { + builtinSearchPhase.add(new MappingPhase(client)); + builtinSearchPhase.add(new PartitionPhase(client)); + searchContext = new SearchContext(esTable); + } + + public SearchContext searchContext() throws UserException { + if (searchContext == null) { + LOG.warn("ES meta state track failed, please check ES health status."); + throw new UserException("ES meta state track failed, please check ES health status."); + } + return searchContext; + } + + public void run() throws DorisEsException { + for (SearchPhase searchPhase : builtinSearchPhase) { + searchPhase.preProcess(searchContext); + searchPhase.execute(searchContext); + searchPhase.postProcess(searchContext); + } + } +} diff --git a/fe/fe-connectors/connector-es/src/main/java/org/apache/doris/datasource/es/EsNodeInfo.java b/fe/fe-connectors/connector-es/src/main/java/org/apache/doris/datasource/es/EsNodeInfo.java new file mode 100644 index 00000000000000..04ffa4b460afc3 --- /dev/null +++ b/fe/fe-connectors/connector-es/src/main/java/org/apache/doris/datasource/es/EsNodeInfo.java @@ -0,0 +1,240 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +package org.apache.doris.datasource.es; + +import org.apache.doris.thrift.TNetworkAddress; + +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; + +import java.util.List; +import java.util.Map; +import java.util.Objects; + +/** + * This class represents one node with the http and potential thrift publish address + */ +public class EsNodeInfo { + private final String id; + private final String name; + private final String host; + private final String ip; + private TNetworkAddress publishAddress; + private final boolean hasHttp; + private final boolean isClient; + private final boolean isData; + private final boolean isIngest; + private boolean hasThrift; + private TNetworkAddress thriftAddress; + + private static final Logger LOG = LogManager.getLogger(EsNodeInfo.class); + + public EsNodeInfo(String id, Map map, boolean httpSslEnabled) { + this.id = id; + EsMajorVersion version = EsMajorVersion.parse((String) map.get("version")); + this.name = (String) map.get("name"); + this.host = (String) map.get("host"); + this.ip = (String) map.get("ip"); + if (version.before(EsMajorVersion.V_5_X)) { + Map attributes = (Map) map.get("attributes"); + if (attributes == null) { + this.isClient = false; + this.isData = true; + } else { + String data = (String) attributes.get("data"); + this.isClient = data == null ? true : !Boolean.parseBoolean(data); + this.isData = data == null ? true : Boolean.parseBoolean(data); + } + this.isIngest = false; + } else { + List roles = (List) map.get("roles"); + this.isClient = roles.contains("data") == false; + this.isData = roles.contains("data"); + this.isIngest = roles.contains("ingest"); + } + Map httpMap = (Map) map.get("http"); + if (httpMap != null) { + String address = (String) httpMap.get("publish_address"); + if (address != null) { + // if network.publish_host is set to hostname like localhost, + // the publish_address contains hostname like "localhost/127.0.0.1:9200" + address = address.substring(address.lastIndexOf('/') + 1); + String[] scratch = address.split(":"); + this.publishAddress = new TNetworkAddress((httpSslEnabled ? "https://" : "") + scratch[0], Integer.parseInt(scratch[1])); + this.hasHttp = true; + } else { + this.publishAddress = null; + this.hasHttp = false; + } + } else { + this.publishAddress = null; + this.hasHttp = false; + } + + Map attributesMap = (Map) map.get("attributes"); + if (attributesMap != null) { + String thriftPortStr = (String) attributesMap.get("thrift_port"); + if (thriftPortStr != null) { + try { + int thriftPort = Integer.valueOf(thriftPortStr); + hasThrift = true; + thriftAddress = new TNetworkAddress(this.ip, thriftPort); + } catch (Exception e) { + hasThrift = false; + } + } else { + hasThrift = false; + } + } else { + hasThrift = false; + } + } + + public EsNodeInfo(String id, String seed) { + this.id = id; + String[] scratch = seed.split(":"); + int port = 80; + if (scratch.length == 3) { + String portStr = scratch[2]; + if (portStr.contains("/")) { + portStr = portStr.substring(0, portStr.indexOf('/')); + } + port = Integer.parseInt(portStr); + } + String remoteHost = scratch[0] + ":" + scratch[1]; + this.name = remoteHost; + this.host = remoteHost; + this.ip = remoteHost; + this.isClient = true; + this.isData = true; + this.isIngest = true; + this.publishAddress = new TNetworkAddress(remoteHost, port); + this.hasHttp = true; + } + + public boolean hasHttp() { + return hasHttp; + } + + public boolean isClient() { + return isClient; + } + + public boolean isData() { + return isData; + } + + public boolean isIngest() { + return isIngest; + } + + public String getId() { + return id; + } + + public String getName() { + return name; + } + + public String getHost() { + return host; + } + + public TNetworkAddress getPublishAddress() { + return publishAddress; + } + + public boolean isHasThrift() { + return hasThrift; + } + + public TNetworkAddress getThriftAddress() { + return thriftAddress; + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + + EsNodeInfo nodeInfo = (EsNodeInfo) o; + + if (hasHttp != nodeInfo.hasHttp) { + return false; + } + if (isClient != nodeInfo.isClient) { + return false; + } + if (isData != nodeInfo.isData) { + return false; + } + if (!id.equals(nodeInfo.id)) { + return false; + } + if (!name.equals(nodeInfo.name)) { + return false; + } + if (!host.equals(nodeInfo.host)) { + return false; + } + if (!ip.equals(nodeInfo.ip)) { + return false; + } + if (hasThrift != nodeInfo.hasThrift) { + return false; + } + return (Objects.equals(publishAddress, nodeInfo.publishAddress)) + && (Objects.equals(thriftAddress, nodeInfo.thriftAddress)); + } + + @Override + public int hashCode() { + int result = id.hashCode(); + result = 31 * result + name.hashCode(); + result = 31 * result + host.hashCode(); + result = 31 * result + ip.hashCode(); + result = 31 * result + (publishAddress != null ? publishAddress.hashCode() : 0); + result = 31 * result + (thriftAddress != null ? thriftAddress.hashCode() : 0); + result = 31 * result + (hasHttp ? 1 : 0); + result = 31 * result + (hasThrift ? 1 : 0); + result = 31 * result + (isClient ? 1 : 0); + result = 31 * result + (isData ? 1 : 0); + return result; + } + + @Override + public String toString() { + return "EsNodeInfo{" + + "id='" + id + '\'' + + ", name='" + name + '\'' + + ", host='" + host + '\'' + + ", ip='" + ip + '\'' + + ", publishAddress=" + publishAddress + + ", hasHttp=" + hasHttp + + ", isClient=" + isClient + + ", isData=" + isData + + ", isIngest=" + isIngest + + ", hasThrift=" + hasThrift + + ", thriftAddress=" + thriftAddress + + '}'; + } +} diff --git a/fe/fe-connectors/connector-es/src/main/java/org/apache/doris/datasource/es/EsRepository.java b/fe/fe-connectors/connector-es/src/main/java/org/apache/doris/datasource/es/EsRepository.java new file mode 100644 index 00000000000000..b67c36e72e477a --- /dev/null +++ b/fe/fe-connectors/connector-es/src/main/java/org/apache/doris/datasource/es/EsRepository.java @@ -0,0 +1,108 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +package org.apache.doris.datasource.es; + + +import org.apache.doris.catalog.Database; +import org.apache.doris.catalog.Env; +import org.apache.doris.catalog.EsTable; +import org.apache.doris.catalog.Table; +import org.apache.doris.catalog.TableIf.TableType; +import org.apache.doris.common.Config; +import org.apache.doris.common.util.MasterDaemon; + +import com.google.common.collect.Maps; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; + +import java.util.List; +import java.util.Map; + + +/** + * It is responsible for loading all ES external table's meta-data such as `fields`, `partitions` periodically, + * playing the `repo` role at Doris On ES + */ +public class EsRepository extends MasterDaemon { + + private static final Logger LOG = LogManager.getLogger(EsRepository.class); + + private Map esTables; + + private Map esClients; + + public EsRepository() { + super("es repository", Config.es_state_sync_interval_second * 1000); + esTables = Maps.newConcurrentMap(); + esClients = Maps.newConcurrentMap(); + } + + public void registerTable(EsTable esTable) { + if (Env.isCheckpointThread()) { + return; + } + esTables.put(esTable.getId(), esTable); + esClients.put(esTable.getId(), + new EsRestClient(esTable.getSeeds(), esTable.getUserName(), esTable.getPasswd(), + esTable.isHttpSslEnabled())); + LOG.info("register a new table [{}] to sync list", esTable); + } + + public void deRegisterTable(long tableId) { + esTables.remove(tableId); + esClients.remove(tableId); + LOG.info("deregister table [{}] from sync list", tableId); + } + + @Override + protected void runAfterCatalogReady() { + for (EsTable esTable : esTables.values()) { + try { + esTable.syncTableMetaData(); + } catch (Throwable e) { + LOG.warn("Exception happens when fetch index [{}] meta data from remote es cluster", + esTable.getName(), e); + esTable.setEsTablePartitions(null); + esTable.setLastMetaDataSyncException(e); + } + } + } + + // should call this method to init the state store after loading image + // the rest of tables will be added or removed by replaying edit log + // when fe is start to load image, should call this method to init the state store + public void loadTableFromCatalog() { + if (Env.isCheckpointThread()) { + return; + } + List dbIds = Env.getCurrentEnv().getInternalCatalog().getDbIds(); + for (Long dbId : dbIds) { + Database database = Env.getCurrentInternalCatalog().getDbNullable(dbId); + if (database == null) { + continue; + } + + List tables = database.getTables(); + for (Table table : tables) { + if (table.getType() == TableType.ELASTICSEARCH) { + registerTable((EsTable) table); + } + } + } + } +} diff --git a/fe/fe-connectors/connector-es/src/main/java/org/apache/doris/datasource/es/EsRestClient.java b/fe/fe-connectors/connector-es/src/main/java/org/apache/doris/datasource/es/EsRestClient.java new file mode 100644 index 00000000000000..e2c7fa0b688913 --- /dev/null +++ b/fe/fe-connectors/connector-es/src/main/java/org/apache/doris/datasource/es/EsRestClient.java @@ -0,0 +1,374 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +package org.apache.doris.datasource.es; + +import org.apache.doris.cloud.security.SecurityChecker; +import org.apache.doris.common.util.JsonUtil; + +import com.fasterxml.jackson.databind.JsonNode; +import com.fasterxml.jackson.databind.node.ArrayNode; +import com.google.common.collect.ImmutableList; +import okhttp3.Credentials; +import okhttp3.MediaType; +import okhttp3.OkHttpClient; +import okhttp3.Request; +import okhttp3.RequestBody; +import okhttp3.Response; +import org.apache.http.HttpHeaders; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; +import org.apache.logging.log4j.util.Strings; + +import java.io.IOException; +import java.security.SecureRandom; +import java.security.cert.CertificateException; +import java.security.cert.X509Certificate; +import java.util.ArrayList; +import java.util.Collections; +import java.util.HashMap; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.concurrent.TimeUnit; +import java.util.stream.Collectors; +import javax.net.ssl.HostnameVerifier; +import javax.net.ssl.SSLContext; +import javax.net.ssl.SSLSession; +import javax.net.ssl.SSLSocketFactory; +import javax.net.ssl.TrustManager; +import javax.net.ssl.X509TrustManager; + +/** + * For get es metadata by http/https. + **/ +public class EsRestClient { + + private static final Logger LOG = LogManager.getLogger(EsRestClient.class); + private static final OkHttpClient networkClient = new OkHttpClient + .Builder().readTimeout(10, TimeUnit.SECONDS).build(); + + private static OkHttpClient sslNetworkClient; + private final Request.Builder builder; + private final String[] nodes; + private String currentNode; + private int currentNodeIndex = 0; + private final boolean httpSslEnable; + + /** + * For EsTable. + **/ + public EsRestClient(String[] nodes, String authUser, String authPassword, boolean httpSslEnable) { + this.nodes = nodes; + this.builder = new Request.Builder(); + if (!Strings.isEmpty(authUser) && !Strings.isEmpty(authPassword)) { + this.builder.addHeader(HttpHeaders.AUTHORIZATION, Credentials.basic(authUser, authPassword)); + } + this.currentNode = nodes[currentNodeIndex]; + this.httpSslEnable = httpSslEnable; + } + + public OkHttpClient getClient() { + if (httpSslEnable) { + return getOrCreateSslNetworkClient(); + } + return networkClient; + } + + private void selectNextNode() { + currentNodeIndex++; + // reroute, because the previously failed node may have already been restored + if (currentNodeIndex >= nodes.length) { + currentNodeIndex = 0; + } + currentNode = nodes[currentNodeIndex]; + } + + /** + * Get http nodes. + **/ + public Map getHttpNodes() throws DorisEsException { + Map> nodesData = get("_nodes/http", "nodes"); + if (nodesData == null) { + return Collections.emptyMap(); + } + Map nodesMap = new HashMap<>(); + for (Map.Entry> entry : nodesData.entrySet()) { + EsNodeInfo node = new EsNodeInfo(entry.getKey(), entry.getValue(), httpSslEnable); + if (node.hasHttp()) { + nodesMap.put(node.getId(), node); + } + } + return nodesMap; + } + + /** + * Get mapping for indexName. + */ + public String getMapping(String indexName) throws DorisEsException { + String path = indexName + "/_mapping"; + String indexMapping = execute(path); + if (indexMapping == null) { + throw new DorisEsException("index[" + indexName + "] not found"); + } + return indexMapping; + } + + /** + * Search specific index + */ + public String searchIndex(String indexName, String body) throws DorisEsException { + String path = indexName + "/_search"; + RequestBody requestBody = null; + if (Strings.isNotEmpty(body)) { + requestBody = RequestBody.create( + body, + MediaType.get("application/json") + ); + } + return executeWithRequestBody(path, requestBody); + } + + /** + * Check whether index exist. + **/ + public boolean existIndex(OkHttpClient httpClient, String indexName) { + String path = indexName + "/_mapping"; + try (Response response = executeResponse(httpClient, path, null)) { + if (response.isSuccessful()) { + return true; + } + } catch (IOException e) { + LOG.warn("existIndex error", e); + return false; + } + return false; + } + + /** + * Get all index. + **/ + public List getIndices(boolean includeHiddenIndex) { + String indexes = execute("_cat/indices?h=index&format=json&s=index:asc"); + if (indexes == null) { + throw new DorisEsException("get es indexes error"); + } + List ret = new ArrayList<>(); + ArrayNode jsonNodes = JsonUtil.parseArray(indexes); + jsonNodes.forEach(json -> { + // es 7.17 has .geoip_databases, but _mapping response 400. + String index = json.get("index").asText(); + if (includeHiddenIndex) { + ret.add(index); + } else { + if (!index.startsWith(".")) { + ret.add(index); + } + } + + }); + return ret; + } + + /** + * Get all alias. + **/ + public Map> getAliases() { + String res = execute("_aliases"); + Map> ret = new HashMap<>(); + JsonNode root = JsonUtil.readTree(res); + if (root == null) { + return ret; + } + Iterator> elements = root.fields(); + while (elements.hasNext()) { + Map.Entry element = elements.next(); + JsonNode aliases = element.getValue().get("aliases"); + Iterator aliasNames = aliases.fieldNames(); + if (aliasNames.hasNext()) { + ret.put(element.getKey(), ImmutableList.copyOf(aliasNames)); + } + } + return ret; + } + + /** + * Returns the merge of index and alias + **/ + public List listTable(boolean includeHiddenIndex) { + List indices = getIndices(includeHiddenIndex).stream().distinct().collect(Collectors.toList()); + getAliases().entrySet().stream().filter(e -> indices.contains(e.getKey())).flatMap(e -> e.getValue().stream()) + .distinct().forEach(indices::add); + return indices; + } + + /** + * Get Shard location. + **/ + public EsShardPartitions searchShards(String indexName) throws DorisEsException { + String path = indexName + "/_search_shards"; + String searchShards = execute(path); + if (searchShards == null) { + throw new DorisEsException("request index [" + indexName + "] search_shards failure"); + } + return EsShardPartitions.findShardPartitions(indexName, searchShards); + } + + public boolean health() { + String res = execute(""); + return res != null; + } + + /** + * init ssl networkClient use lazy way + **/ + private synchronized OkHttpClient getOrCreateSslNetworkClient() { + if (sslNetworkClient == null) { + sslNetworkClient = new OkHttpClient.Builder().readTimeout(10, TimeUnit.SECONDS) + .sslSocketFactory(createSSLSocketFactory(), new TrustAllCerts()) + .hostnameVerifier(new TrustAllHostnameVerifier()).build(); + } + return sslNetworkClient; + } + + private Response executeResponse(OkHttpClient httpClient, String path, RequestBody requestBody) throws IOException { + currentNode = currentNode.trim(); + if (!(currentNode.startsWith("http://") || currentNode.startsWith("https://"))) { + currentNode = "http://" + currentNode; + } + if (!currentNode.endsWith("/")) { + currentNode = currentNode + "/"; + } + String url = currentNode + path; + try { + SecurityChecker.getInstance().startSSRFChecking(url); + Request request; + if (requestBody != null) { + request = builder.post(requestBody).url(currentNode + path).build(); + } else { + request = builder.get().url(currentNode + path).build(); + } + if (LOG.isInfoEnabled()) { + LOG.info("es rest client request URL: {}", request.url().toString()); + } + return httpClient.newCall(request).execute(); + } catch (Exception e) { + throw new IOException(e); + } finally { + SecurityChecker.getInstance().stopSSRFChecking(); + } + } + + private String execute(String path) throws DorisEsException { + return executeWithRequestBody(path, null); + } + + /** + * execute request for specific path,it will try again nodes.length times if it fails + * + * @param path the path must not leading with '/' + * @return response + */ + private String executeWithRequestBody(String path, RequestBody requestBody) throws DorisEsException { + // try 3 times for every node + int retrySize = nodes.length * 3; + DorisEsException scratchExceptionForThrow = null; + OkHttpClient httpClient; + if (httpSslEnable) { + httpClient = getOrCreateSslNetworkClient(); + } else { + httpClient = networkClient; + } + for (int i = 0; i < retrySize; i++) { + // maybe should add HTTP schema to the address + // actually, at this time we can only process http protocol + // NOTE. currentNode may have some spaces. + // User may set a config like described below: + // hosts: "http://192.168.0.1:8200, http://192.168.0.2:8200" + // then currentNode will be "http://192.168.0.1:8200", " http://192.168.0.2:8200" + if (LOG.isTraceEnabled()) { + LOG.trace("es rest client request URL: {}", currentNode + "/" + path); + } + try (Response response = executeResponse(httpClient, path, requestBody)) { + if (response.isSuccessful()) { + return response.body().string(); + } else { + LOG.warn("request response code: {}, body: {}", response.code(), response.message()); + scratchExceptionForThrow = new DorisEsException(response.message()); + } + } catch (IOException e) { + LOG.warn("request node [{}] [{}] failures {}, try next nodes", currentNode, path, e); + scratchExceptionForThrow = new DorisEsException(e.getMessage()); + } + selectNextNode(); + } + LOG.warn("try all nodes [{}], no other nodes left", nodes); + if (scratchExceptionForThrow != null) { + throw scratchExceptionForThrow; + } + return null; + } + + public T get(String q, String key) throws DorisEsException { + return parseContent(execute(q), key); + } + + @SuppressWarnings("unchecked") + private T parseContent(String response, String key) { + Map map; + try { + map = JsonUtil.readValue(response, Map.class); + } catch (IOException ex) { + LOG.error("parse es response failure: [{}]", response); + throw new DorisEsException(ex.getMessage()); + } + return (T) (key != null ? map.get(key) : map); + } + + /** + * support https + **/ + private static class TrustAllCerts implements X509TrustManager { + public void checkClientTrusted(X509Certificate[] chain, String authType) throws CertificateException { + } + + public void checkServerTrusted(X509Certificate[] chain, String authType) throws CertificateException { + } + + public X509Certificate[] getAcceptedIssuers() { + return new X509Certificate[0]; + } + } + + private static class TrustAllHostnameVerifier implements HostnameVerifier { + public boolean verify(String hostname, SSLSession session) { + return true; + } + } + + private static SSLSocketFactory createSSLSocketFactory() { + SSLSocketFactory ssfFactory; + try { + SSLContext sc = SSLContext.getInstance("TLS"); + sc.init(null, new TrustManager[] {new TrustAllCerts()}, new SecureRandom()); + ssfFactory = sc.getSocketFactory(); + } catch (Exception e) { + throw new DorisEsException("Errors happens when create ssl socket"); + } + return ssfFactory; + } +} diff --git a/fe/fe-connectors/connector-es/src/main/java/org/apache/doris/datasource/es/EsScanNodeProvider.java b/fe/fe-connectors/connector-es/src/main/java/org/apache/doris/datasource/es/EsScanNodeProvider.java new file mode 100644 index 00000000000000..d4799bf8efebc0 --- /dev/null +++ b/fe/fe-connectors/connector-es/src/main/java/org/apache/doris/datasource/es/EsScanNodeProvider.java @@ -0,0 +1,40 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +package org.apache.doris.datasource.es; + +import org.apache.doris.analysis.TupleDescriptor; +import org.apache.doris.datasource.ExternalTable; +import org.apache.doris.datasource.es.source.EsScanNode; +import org.apache.doris.planner.PlanNodeId; +import org.apache.doris.planner.ScanContext; +import org.apache.doris.planner.ScanNode; +import org.apache.doris.datasource.spi.ScanNodeProvider; + +/** + * ScanNodeProvider implementation for Elasticsearch. + * + *

Creates {@link EsScanNode} instances for ES table scan operations.

+ */ +public class EsScanNodeProvider implements ScanNodeProvider { + + @Override + public ScanNode createScanNode(PlanNodeId id, TupleDescriptor desc, + ExternalTable table, ScanContext scanContext) { + return new EsScanNode(id, desc, table instanceof EsExternalTable, scanContext); + } +} diff --git a/fe/fe-connectors/connector-es/src/main/java/org/apache/doris/datasource/es/EsSchemaCacheValue.java b/fe/fe-connectors/connector-es/src/main/java/org/apache/doris/datasource/es/EsSchemaCacheValue.java new file mode 100644 index 00000000000000..e53eae676e8107 --- /dev/null +++ b/fe/fe-connectors/connector-es/src/main/java/org/apache/doris/datasource/es/EsSchemaCacheValue.java @@ -0,0 +1,37 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +package org.apache.doris.datasource.es; + +import org.apache.doris.catalog.Column; +import org.apache.doris.datasource.SchemaCacheValue; + +import java.util.List; +import java.util.Map; + +public class EsSchemaCacheValue extends SchemaCacheValue { + public Map column2typeMap; + + public EsSchemaCacheValue(List columns, Map column2typeMap) { + super(columns); + this.column2typeMap = column2typeMap; + } + + public Map getColumn2typeMap() { + return column2typeMap; + } +} diff --git a/fe/fe-connectors/connector-es/src/main/java/org/apache/doris/datasource/es/EsShardPartitions.java b/fe/fe-connectors/connector-es/src/main/java/org/apache/doris/datasource/es/EsShardPartitions.java new file mode 100644 index 00000000000000..ee32b1103fc627 --- /dev/null +++ b/fe/fe-connectors/connector-es/src/main/java/org/apache/doris/datasource/es/EsShardPartitions.java @@ -0,0 +1,155 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +package org.apache.doris.datasource.es; + +import org.apache.doris.analysis.SinglePartitionDesc; +import org.apache.doris.catalog.PartitionKey; +import org.apache.doris.thrift.TNetworkAddress; + +import com.google.common.collect.Lists; +import com.google.common.collect.Maps; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; +import org.json.simple.JSONArray; +import org.json.simple.JSONObject; +import org.json.simple.JSONValue; + +import java.security.SecureRandom; +import java.util.List; +import java.util.Map; + +public class EsShardPartitions { + + private static final Logger LOG = LogManager.getLogger(EsShardPartitions.class); + + private final String indexName; + // shardid -> host1, host2, host3 + private Map> shardRoutings; + private SinglePartitionDesc partitionDesc; + private PartitionKey partitionKey; + private long partitionId = -1; + + public EsShardPartitions(String indexName) { + this.indexName = indexName; + this.shardRoutings = Maps.newHashMap(); + this.partitionDesc = null; + this.partitionKey = null; + } + + /** + * Parse shardRoutings from the json + * + * @param indexName indexName(alias or really name) + * @param searchShards the return value of _search_shards + * @return shardRoutings is used for searching + */ + public static EsShardPartitions findShardPartitions(String indexName, String searchShards) throws DorisEsException { + EsShardPartitions partitions = new EsShardPartitions(indexName); + JSONObject jsonObject = (JSONObject) JSONValue.parse(searchShards); + JSONArray shards = (JSONArray) jsonObject.get("shards"); + int size = shards.size(); + for (int i = 0; i < size; i++) { + List singleShardRouting = Lists.newArrayList(); + JSONArray shardsArray = (JSONArray) shards.get(i); + for (Object o : shardsArray) { + JSONObject indexShard = (JSONObject) o; + String shardState = (String) indexShard.get("state"); + if ("STARTED".equalsIgnoreCase(shardState) || "RELOCATING".equalsIgnoreCase(shardState)) { + try { + singleShardRouting.add(new EsShardRouting((String) indexShard.get("index"), + ((Long) indexShard.get("shard")).intValue(), (Boolean) indexShard.get("primary"), + (String) indexShard.get("node"))); + } catch (Exception e) { + LOG.error("fetch index [{}] shard partitions failure", indexName, e); + throw new DorisEsException( + "fetch [" + indexName + "] shard partitions failure [" + e.getMessage() + "]"); + } + } + } + if (singleShardRouting.isEmpty()) { + LOG.error("could not find a healthy allocation for [{}][{}]", indexName, i); + continue; + } + partitions.addShardRouting(i, singleShardRouting); + } + return partitions; + } + + public void addHttpAddress(Map nodesInfo) { + for (Map.Entry> entry : shardRoutings.entrySet()) { + List shardRoutings = entry.getValue(); + for (EsShardRouting shardRouting : shardRoutings) { + String nodeId = shardRouting.getNodeId(); + if (nodesInfo.containsKey(nodeId)) { + shardRouting.setHttpAddress(nodesInfo.get(nodeId).getPublishAddress()); + } else { + shardRouting.setHttpAddress(randomAddress(nodesInfo)); + } + } + } + } + + public TNetworkAddress randomAddress(Map nodesInfo) { + // return a random value between 0 and 32767 : [0, 32767) + int seed = new SecureRandom().nextInt(Short.MAX_VALUE) % nodesInfo.size(); + EsNodeInfo[] nodeInfos = nodesInfo.values().toArray(new EsNodeInfo[0]); + return nodeInfos[seed].getPublishAddress(); + } + + public void addShardRouting(int shardId, List singleShardRouting) { + shardRoutings.put(shardId, singleShardRouting); + } + + public String getIndexName() { + return indexName; + } + + public Map> getShardRoutings() { + return shardRoutings; + } + + public SinglePartitionDesc getPartitionDesc() { + return partitionDesc; + } + + public void setPartitionDesc(SinglePartitionDesc partitionDesc) { + this.partitionDesc = partitionDesc; + } + + public PartitionKey getPartitionKey() { + return partitionKey; + } + + public void setPartitionKey(PartitionKey partitionKey) { + this.partitionKey = partitionKey; + } + + public long getPartitionId() { + return partitionId; + } + + public void setPartitionId(long partitionId) { + this.partitionId = partitionId; + } + + @Override + public String toString() { + return "EsIndexState [indexName=" + indexName + ", partitionDesc=" + partitionDesc + ", partitionKey=" + + partitionKey + "]"; + } +} diff --git a/fe/fe-connectors/connector-es/src/main/java/org/apache/doris/datasource/es/EsShardRouting.java b/fe/fe-connectors/connector-es/src/main/java/org/apache/doris/datasource/es/EsShardRouting.java new file mode 100644 index 00000000000000..0ea626d3a7c14a --- /dev/null +++ b/fe/fe-connectors/connector-es/src/main/java/org/apache/doris/datasource/es/EsShardRouting.java @@ -0,0 +1,66 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +package org.apache.doris.datasource.es; + +import org.apache.doris.thrift.TNetworkAddress; + +public class EsShardRouting { + + private final String indexName; + private final int shardId; + private final boolean isPrimary; + private TNetworkAddress httpAddress; + private final String nodeId; + + public EsShardRouting(String indexName, int shardId, boolean isPrimary, String nodeId) { + this.indexName = indexName; + this.shardId = shardId; + this.isPrimary = isPrimary; + this.nodeId = nodeId; + } + + public int getShardId() { + return shardId; + } + + public boolean isPrimary() { + return isPrimary; + } + + public String getIndexName() { + return indexName; + } + + public TNetworkAddress getHttpAddress() { + return httpAddress; + } + + public void setHttpAddress(TNetworkAddress httpAddress) { + this.httpAddress = httpAddress; + } + + public String getNodeId() { + return nodeId; + } + + @Override + public String toString() { + return "EsShardRouting{" + "indexName='" + indexName + '\'' + ", shardId=" + shardId + ", isPrimary=" + + isPrimary + ", httpAddress=" + httpAddress + ", nodeId='" + nodeId + '\'' + '}'; + } +} diff --git a/fe/fe-connectors/connector-es/src/main/java/org/apache/doris/datasource/es/EsTablePartitions.java b/fe/fe-connectors/connector-es/src/main/java/org/apache/doris/datasource/es/EsTablePartitions.java new file mode 100644 index 00000000000000..b00264faf3bab8 --- /dev/null +++ b/fe/fe-connectors/connector-es/src/main/java/org/apache/doris/datasource/es/EsTablePartitions.java @@ -0,0 +1,162 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +package org.apache.doris.datasource.es; + +import org.apache.doris.catalog.Column; +import org.apache.doris.catalog.EsTable; +import org.apache.doris.catalog.PartitionInfo; +import org.apache.doris.catalog.PartitionItem; +import org.apache.doris.catalog.RangePartitionInfo; +import org.apache.doris.catalog.SinglePartitionInfo; +import org.apache.doris.common.DdlException; + +import com.google.common.collect.Maps; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; + +import java.util.ArrayList; +import java.util.Comparator; +import java.util.List; +import java.util.Map; + +/** + * save the dynamic info parsed from es cluster state such as shard routing, partition info + */ +public class EsTablePartitions { + + private static final Logger LOG = LogManager.getLogger(EsTablePartitions.class); + + private PartitionInfo partitionInfo; + private Map partitionIdToIndices; + private Map partitionedIndexStates; + private Map unPartitionedIndexStates; + + public EsTablePartitions() { + partitionInfo = null; + partitionIdToIndices = Maps.newHashMap(); + partitionedIndexStates = Maps.newHashMap(); + unPartitionedIndexStates = Maps.newHashMap(); + } + + public static EsTablePartitions fromShardPartitions(EsTable esTable, EsShardPartitions shardPartitions) + throws DorisEsException, DdlException { + EsTablePartitions esTablePartitions = new EsTablePartitions(); + RangePartitionInfo partitionInfo = null; + if (esTable.getPartitionInfo() != null) { + if (esTable.getPartitionInfo() instanceof RangePartitionInfo) { + RangePartitionInfo rangePartitionInfo = (RangePartitionInfo) esTable.getPartitionInfo(); + partitionInfo = new RangePartitionInfo(rangePartitionInfo.getPartitionColumns()); + esTablePartitions.setPartitionInfo(partitionInfo); + if (LOG.isDebugEnabled()) { + StringBuilder sb = new StringBuilder(); + int idx = 0; + for (Column column : rangePartitionInfo.getPartitionColumns()) { + if (idx != 0) { + sb.append(", "); + } + sb.append("`").append(column.getName()).append("`"); + idx++; + } + sb.append(")"); + if (LOG.isDebugEnabled()) { + LOG.debug("begin to parse es table [{}] state from search shards," + + " with partition info [{}]", esTable.getName(), sb.toString()); + } + } + } else if (esTable.getPartitionInfo() instanceof SinglePartitionInfo) { + if (LOG.isDebugEnabled()) { + LOG.debug("begin to parse es table [{}] state from search shards, " + + "with no partition info", esTable.getName()); + } + } else { + throw new DorisEsException("es table only support range partition, " + + "but current partition type is " + + esTable.getPartitionInfo().getType()); + } + } + esTablePartitions.addIndexState(esTable.getIndexName(), shardPartitions); + if (LOG.isDebugEnabled()) { + LOG.debug("add index {} to es table {}", shardPartitions, esTable.getName()); + } + if (partitionInfo != null) { + // sort the index state according to partition key and then add to range map + List esShardPartitionsList = new ArrayList<>( + esTablePartitions.getPartitionedIndexStates().values()); + esShardPartitionsList.sort(Comparator.comparing(EsShardPartitions::getPartitionKey)); + long partitionId = 0; + for (EsShardPartitions esShardPartitions : esShardPartitionsList) { + PartitionItem item = partitionInfo.handleNewSinglePartitionDesc( + esShardPartitions.getPartitionDesc(), partitionId, false); + esTablePartitions.addPartition(esShardPartitions.getIndexName(), partitionId); + esShardPartitions.setPartitionId(partitionId); + ++partitionId; + if (LOG.isDebugEnabled()) { + LOG.debug("add partition to es table [{}] with range [{}]", esTable.getName(), + item.getItems()); + } + } + } + return esTablePartitions; + } + + public PartitionInfo getPartitionInfo() { + return partitionInfo; + } + + public void setPartitionInfo(PartitionInfo partitionInfo) { + this.partitionInfo = partitionInfo; + } + + public Map getPartitionIdToIndices() { + return partitionIdToIndices; + } + + public void addPartition(String indexName, long partitionId) { + partitionIdToIndices.put(partitionId, indexName); + } + + public void addIndexState(String indexName, EsShardPartitions indexState) { + if (indexState.getPartitionDesc() != null) { + partitionedIndexStates.put(indexName, indexState); + } else { + unPartitionedIndexStates.put(indexName, indexState); + } + } + + public Map getPartitionedIndexStates() { + return partitionedIndexStates; + } + + public Map getUnPartitionedIndexStates() { + return unPartitionedIndexStates; + } + + public EsShardPartitions getEsShardPartitions(long partitionId) { + if (partitionIdToIndices.containsKey(partitionId)) { + return partitionedIndexStates.get(partitionIdToIndices.get(partitionId)); + } + return null; + } + + public EsShardPartitions getEsShardPartitions(String indexName) { + if (partitionedIndexStates.containsKey(indexName)) { + return partitionedIndexStates.get(indexName); + } + return unPartitionedIndexStates.get(indexName); + } +} diff --git a/fe/fe-connectors/connector-es/src/main/java/org/apache/doris/datasource/es/EsUrls.java b/fe/fe-connectors/connector-es/src/main/java/org/apache/doris/datasource/es/EsUrls.java new file mode 100644 index 00000000000000..193035740a964f --- /dev/null +++ b/fe/fe-connectors/connector-es/src/main/java/org/apache/doris/datasource/es/EsUrls.java @@ -0,0 +1,37 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +package org.apache.doris.datasource.es; + +import lombok.AllArgsConstructor; +import lombok.Data; +import lombok.NoArgsConstructor; + +/** + * Pack url. + **/ +@Data +@NoArgsConstructor +@AllArgsConstructor +public class EsUrls { + + private String searchUrl; + + private String initScrollUrl; + + private String nextScrollUrl; +} diff --git a/fe/fe-connectors/connector-es/src/main/java/org/apache/doris/datasource/es/EsUtil.java b/fe/fe-connectors/connector-es/src/main/java/org/apache/doris/datasource/es/EsUtil.java new file mode 100644 index 00000000000000..f1e907bfab9095 --- /dev/null +++ b/fe/fe-connectors/connector-es/src/main/java/org/apache/doris/datasource/es/EsUtil.java @@ -0,0 +1,377 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +package org.apache.doris.datasource.es; + +import org.apache.doris.analysis.DistributionDesc; +import org.apache.doris.analysis.PartitionDesc; +import org.apache.doris.analysis.RangePartitionDesc; +import org.apache.doris.catalog.ArrayType; +import org.apache.doris.catalog.Column; +import org.apache.doris.catalog.ScalarType; +import org.apache.doris.catalog.Type; +import org.apache.doris.common.AnalysisException; +import org.apache.doris.common.DdlException; +import org.apache.doris.common.util.JsonUtil; + +import com.fasterxml.jackson.databind.JsonNode; +import com.fasterxml.jackson.databind.node.ObjectNode; +import com.google.common.annotations.VisibleForTesting; +import com.google.common.collect.Lists; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; + +import java.util.ArrayList; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.Spliterator; +import java.util.Spliterators; +import java.util.stream.StreamSupport; + +/** + * Util for ES, some static method. + **/ +public class EsUtil { + + private static final Logger LOG = LogManager.getLogger(EsUtil.class); + + /** + * Analyze partition and distributionDesc. + **/ + public static void analyzePartitionAndDistributionDesc(PartitionDesc partitionDesc, + DistributionDesc distributionDesc) throws AnalysisException { + if (partitionDesc == null && distributionDesc == null) { + return; + } + + if (partitionDesc != null) { + if (!(partitionDesc instanceof RangePartitionDesc)) { + throw new AnalysisException("Elasticsearch table only permit range partition"); + } + + RangePartitionDesc rangePartitionDesc = (RangePartitionDesc) partitionDesc; + analyzePartitionDesc(rangePartitionDesc); + } + + if (distributionDesc != null) { + throw new AnalysisException("could not support distribution clause"); + } + } + + private static void analyzePartitionDesc(RangePartitionDesc partDesc) throws AnalysisException { + if (partDesc.getPartitionColNames() == null || partDesc.getPartitionColNames().isEmpty()) { + throw new AnalysisException("No partition columns."); + } + + if (partDesc.getPartitionColNames().size() > 1) { + throw new AnalysisException("Elasticsearch table's partition column could only be a single column"); + } + } + + /** + * Get boolean throw DdlException when parse error + **/ + public static boolean getBoolean(Map properties, String name) throws DdlException { + String property = properties.get(name).trim(); + try { + return Boolean.parseBoolean(property); + } catch (Exception e) { + throw new DdlException(String.format("fail to parse %s, %s = %s, `%s` should be like 'true' or 'false', " + + "value should be double quotation marks", name, name, property, name)); + } + } + + @VisibleForTesting + public static ObjectNode getMapping(String indexMapping) { + ObjectNode jsonNodes = JsonUtil.parseObject(indexMapping); + // If the indexName use alias takes the first mapping + return (ObjectNode) jsonNodes.iterator().next().get("mappings"); + } + + @VisibleForTesting + public static ObjectNode getRootSchema(ObjectNode mappings, String mappingType, List arrayFields) { + // Type is null in the following three cases + // 1. Equal 6.8.x and after + // 2. Multi-catalog auto infer + // 3. Equal 6.8.x and before user not passed + if (mappingType == null) { + // remove dynamic templates, for ES 7.x and 8.x + checkNonPropertiesFields(mappings, arrayFields); + String firstType = mappings.fieldNames().next(); + //The first parameter may not be properties, so we need to first determine whether it is 7.x or above. + if (StreamSupport.stream(Spliterators + .spliteratorUnknownSize(mappings.fieldNames(), Spliterator.ORDERED), false) + .anyMatch(s -> s.contains("properties"))) { + // Equal 7.x and after + return mappings; + } else { + ObjectNode firstData = (ObjectNode) mappings.get(firstType); + // check for ES 6.x and before + checkNonPropertiesFields(firstData, arrayFields); + return firstData; + } + } else { + if (mappings.has(mappingType)) { + ObjectNode jsonData = (ObjectNode) mappings.get(mappingType); + // check for ES 6.x and before + checkNonPropertiesFields(jsonData, arrayFields); + return jsonData; + } + // Compatible type error + return getRootSchema(mappings, null, arrayFields); + } + } + + /** + * Check non properties fields + * + * @param mappings + */ + private static void checkNonPropertiesFields(ObjectNode mappings, List arrayFields) { + // remove `_meta` field and parse array_fields + JsonNode metaNode = mappings.remove("_meta"); + if (metaNode != null) { + JsonNode dorisMeta = metaNode.get("doris"); + if (dorisMeta != null) { + JsonNode arrayNode = dorisMeta.get("array_fields"); + if (arrayNode != null) { + for (JsonNode jsonNode : arrayNode) { + arrayFields.add(jsonNode.asText()); + } + } + } + } + // remove `dynamic_templates` field + mappings.remove("dynamic_templates"); + // remove `dynamic` field + mappings.remove("dynamic"); + // remove `_default` field, we do not parse `_default_` mapping, only explicit mapping. + // `_default` _mapping type is deprecated in 7.0 and removed in 8.0 + // https://www.elastic.co/guide/en/elasticsearch/reference/7.17/removal-of-types.html + mappings.remove("_default_"); + // check explicit mapping + if (mappings.isEmpty()) { + throw new DorisEsException("Do not support index without explicit mapping."); + } + } + + /** + * Get mapping properties transform to ObjectNode. + **/ + public static ObjectNode getMappingProps(String sourceIndex, String indexMapping, String mappingType) { + ObjectNode mappings = getMapping(indexMapping); + ObjectNode rootSchema = getRootSchema(mappings, mappingType, new ArrayList<>()); + ObjectNode properties = (ObjectNode) rootSchema.get("properties"); + if (properties == null) { + throw new DorisEsException( + "index[" + sourceIndex + "] type[" + mappingType + "] mapping not found for the ES Cluster"); + } + return properties; + } + + /** + * Generate columns from ES Cluster. + * Add mappingEsId config in es external catalog. + **/ + public static List genColumnsFromEs(EsRestClient client, String indexName, String mappingType, + boolean mappingEsId, Map column2typeMap) { + String mapping = client.getMapping(indexName); + ObjectNode mappings = getMapping(mapping); + // Get array_fields while removing _meta property. + List arrayFields = new ArrayList<>(); + ObjectNode rootSchema = getRootSchema(mappings, mappingType, arrayFields); + return genColumnsFromEs(indexName, mappingType, rootSchema, mappingEsId, arrayFields, column2typeMap); + } + + @VisibleForTesting + public static List genColumnsFromEs(String indexName, String mappingType, ObjectNode rootSchema, + boolean mappingEsId, List arrayFields, Map column2typeMap) { + List columns = new ArrayList<>(); + if (mappingEsId) { + Column column = new Column(); + column.setName("_id"); + column.setIsKey(true); + column.setType(ScalarType.createVarcharType(255)); + column.setIsAllowNull(true); + column.setUniqueId(-1); + columns.add(column); + } + ObjectNode mappingProps = (ObjectNode) rootSchema.get("properties"); + if (mappingProps == null) { + throw new DorisEsException( + "index[" + indexName + "] type[" + mappingType + "] mapping not found for the ES Cluster"); + } + Iterator iterator = mappingProps.fieldNames(); + while (iterator.hasNext()) { + String fieldName = iterator.next(); + ObjectNode fieldValue = (ObjectNode) mappingProps.get(fieldName); + Column column = parseEsField(fieldName, replaceFieldAlias(mappingProps, fieldValue), arrayFields, + column2typeMap); + columns.add(column); + } + return columns; + } + + private static ObjectNode replaceFieldAlias(ObjectNode mappingProps, ObjectNode fieldValue) { + if (!fieldValue.has("type")) { + return fieldValue; + } + String typeStr = fieldValue.get("type").asText(); + if ("alias".equals(typeStr)) { + String path = fieldValue.get("path").asText(); + if ("_id".equals(path)) { + // _id is not in mappingProps, use keyword type. + fieldValue.put("type", "keyword"); + } else { + if (mappingProps.has(path)) { + return (ObjectNode) mappingProps.get(path); + } + } + } + return fieldValue; + } + + private static Column parseEsField(String fieldName, ObjectNode fieldValue, List arrayFields, + Map column2typeMap) { + Column column = new Column(); + column.setName(fieldName); + column.setIsKey(true); + column.setIsAllowNull(true); + column.setUniqueId(-1); + Type type; + // Complex types are treating as String types for now. + if (fieldValue.has("type")) { + String typeStr = fieldValue.get("type").asText(); + column.setComment("Elasticsearch type is " + typeStr); + column2typeMap.put(fieldName, typeStr); + // reference https://www.elastic.co/guide/en/elasticsearch/reference/8.3/sql-data-types.html + switch (typeStr) { + case "null": + type = Type.NULL; + break; + case "boolean": + type = Type.BOOLEAN; + break; + case "byte": + type = Type.TINYINT; + break; + case "short": + type = Type.SMALLINT; + break; + case "integer": + type = Type.INT; + break; + case "long": + type = Type.BIGINT; + break; + case "unsigned_long": + type = Type.LARGEINT; + break; + case "float": + case "half_float": + type = Type.FLOAT; + break; + case "double": + case "scaled_float": + type = Type.DOUBLE; + break; + case "date": + type = parseEsDateType(column, fieldValue); + break; + case "keyword": + case "text": + case "ip": + case "wildcard": + case "constant_keyword": + type = ScalarType.createStringType(); + break; + // When ES table is empty, object fields still have explicit "type": "object" in mapping + case "object": + case "nested": + case "flattened": + type = Type.JSONB; + break; + default: + type = Type.UNSUPPORTED; + } + } else { + // When there is no explicit type in mapping, it indicates this type is an `object` in Elasticsearch. + // reference: https://www.elastic.co/guide/en/elasticsearch/reference/current/object.html + type = Type.JSONB; + column.setComment("Elasticsearch type is object"); + column2typeMap.put(fieldName, "object"); + } + if (arrayFields.contains(fieldName)) { + column.setType(ArrayType.create(type, true)); + } else { + column.setType(type); + } + return column; + } + + private static final List ALLOW_DATE_FORMATS = Lists.newArrayList("yyyy-MM-dd HH:mm:ss", "yyyy-MM-dd", + "epoch_millis"); + + /** + * Parse es date to doris type by format + **/ + private static Type parseEsDateType(Column column, ObjectNode field) { + if (!field.has("format")) { + // default format + column.setComment("Elasticsearch type is date, no format"); + return ScalarType.createDatetimeV2Type(0); + } else { + String originFormat = field.get("format").asText(); + String[] formats = originFormat.split("\\|\\|"); + boolean dateTimeFlag = false; + boolean dateFlag = false; + boolean bigIntFlag = false; + for (String format : formats) { + // pre-check format + String trimFormat = format.trim(); + if (!ALLOW_DATE_FORMATS.contains(trimFormat)) { + column.setComment( + "Elasticsearch type is date, format is " + trimFormat + " not support, use String type"); + return ScalarType.createStringType(); + } + switch (trimFormat) { + case "yyyy-MM-dd HH:mm:ss": + dateTimeFlag = true; + break; + case "yyyy-MM-dd": + dateFlag = true; + break; + case "epoch_millis": + default: + bigIntFlag = true; + } + } + column.setComment("Elasticsearch type is date, format is " + originFormat); + if (dateTimeFlag) { + return ScalarType.createDatetimeV2Type(0); + } + if (dateFlag) { + return ScalarType.createDateV2Type(); + } + if (bigIntFlag) { + return Type.BIGINT; + } + return ScalarType.createStringType(); + } + } +} diff --git a/fe/fe-connectors/connector-es/src/main/java/org/apache/doris/datasource/es/MappingPhase.java b/fe/fe-connectors/connector-es/src/main/java/org/apache/doris/datasource/es/MappingPhase.java new file mode 100644 index 00000000000000..9e14f2721269f3 --- /dev/null +++ b/fe/fe-connectors/connector-es/src/main/java/org/apache/doris/datasource/es/MappingPhase.java @@ -0,0 +1,179 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +package org.apache.doris.datasource.es; + +import org.apache.doris.catalog.Column; +import org.apache.doris.catalog.EsTable; + +import com.fasterxml.jackson.databind.JsonNode; +import com.fasterxml.jackson.databind.node.ObjectNode; +import org.apache.commons.lang3.StringUtils; + +import java.util.Iterator; + +/** + * Get index mapping from remote ES Cluster, and resolved `keyword` and `doc_values` field + * Later we can use it to parse all relevant indexes + */ +public class MappingPhase implements SearchPhase { + + private EsRestClient client; + + // json response for `{index}/_mapping` API + private String jsonMapping; + + public MappingPhase(EsRestClient client) { + this.client = client; + } + + @Override + public void execute(SearchContext context) throws DorisEsException { + jsonMapping = client.getMapping(context.sourceIndex()); + } + + @Override + public void postProcess(SearchContext context) { + resolveFields(context, jsonMapping); + } + + /** + * Parse the required field information from the json. + * + * @param searchContext the current associated column searchContext + * @param indexMapping the return value of _mapping + */ + public static void resolveFields(SearchContext searchContext, String indexMapping) throws DorisEsException { + ObjectNode properties = EsUtil.getMappingProps(searchContext.sourceIndex(), indexMapping, searchContext.type()); + for (Column col : searchContext.columns()) { + String colName = col.getName(); + // _id not exist mapping, but be can query it. + if (!"_id".equals(colName)) { + if (!properties.has(colName)) { + throw new DorisEsException( + "index[" + searchContext.sourceIndex() + "] mapping[" + indexMapping + "] not found " + + "column " + colName + " for the ES Cluster"); + } + ObjectNode fieldObject = (ObjectNode) properties.get(colName); + if (!fieldObject.has("type")) { + continue; + } + String fieldType = fieldObject.get("type").asText(); + resolveDateFields(searchContext, fieldObject, colName, fieldType); + resolveKeywordFields(searchContext, fieldObject, colName, fieldType); + resolveDocValuesFields(searchContext, fieldObject, colName, fieldType); + } + } + } + + private static void resolveDateFields(SearchContext searchContext, ObjectNode fieldObject, String colName, + String fieldType) { + // Compat use default/strict_date_optional_time format date type, need transform datetime to + if ("date".equals(fieldType)) { + if (!fieldObject.has("format") || "strict_date_optional_time".equals(fieldObject.get("format").asText())) { + searchContext.needCompatDateFields().add(colName); + } + } + } + + + // get a field of keyword type in the fields + private static void resolveKeywordFields(SearchContext searchContext, ObjectNode fieldObject, String colName, + String fieldType) { + // string-type field used keyword type to generate predicate + // if text field type seen, we should use the `field` keyword type? + if ("text".equals(fieldType)) { + JsonNode fieldsObject = fieldObject.get("fields"); + if (fieldsObject != null) { + Iterator fieldNames = fieldsObject.fieldNames(); + while (fieldNames.hasNext()) { + String fieldName = fieldNames.next(); + ObjectNode innerTypeObject = (ObjectNode) fieldsObject.get(fieldName); + // just for text type + if ("keyword".equals(innerTypeObject.get("type").asText())) { + searchContext.fetchFieldsContext().put(colName, colName + "." + fieldName); + } + } + } + } + } + + private static void resolveDocValuesFields(SearchContext searchContext, ObjectNode fieldObject, String colName, + String fieldType) { + String docValueField = null; + if (EsTable.DEFAULT_DOCVALUE_DISABLED_FIELDS.contains(fieldType)) { + JsonNode fieldsObject = fieldObject.get("fields"); + if (fieldsObject != null) { + Iterator fieldNames = fieldsObject.fieldNames(); + while (fieldNames.hasNext()) { + String fieldName = fieldNames.next(); + ObjectNode innerTypeObject = (ObjectNode) fieldsObject.get(fieldName); + if (EsTable.DEFAULT_DOCVALUE_DISABLED_FIELDS.contains(innerTypeObject.get("type").asText())) { + continue; + } + if (innerTypeObject.has("doc_values")) { + boolean docValue = innerTypeObject.get("doc_values").asBoolean(); + if (docValue) { + docValueField = colName; + } + } else if (innerTypeObject.has("ignore_above")) { + // reference: + // https://www.elastic.co/guide/en/elasticsearch/reference/current/keyword.html#keyword-params + // > ignore_above + // > Do not index any string longer than this value. Defaults to 2147483647 so that all values + // > would be accepted. Please however note that default dynamic mapping rules create a sub + // > keyword field that overrides this default by setting ignore_above: 256. + // this field has `ignore_above` param + // Strings longer than the ignore_above setting will not be indexed or stored + // so we cannot rely on its doc_values + } else { + // a : {c : {}} -> a -> a.c + docValueField = colName + "." + fieldName; + } + } + } + } else { + // set doc_value = false manually + if (fieldObject.has("doc_values")) { + boolean docValue = fieldObject.get("doc_values").asBoolean(); + if (!docValue) { + return; + } + } else if (fieldType == null || "nested".equals(fieldType)) { + // The object field has no type, and nested not support doc value. + return; + } else if (fieldObject.has("ignore_above")) { + // reference: + // https://www.elastic.co/guide/en/elasticsearch/reference/current/keyword.html#keyword-params + // > ignore_above + // > Do not index any string longer than this value. Defaults to 2147483647 so that all values + // > would be accepted. Please however note that default dynamic mapping rules create a sub + // > keyword field that overrides this default by setting ignore_above: 256. + // this field has `ignore_above` param + // Strings longer than the ignore_above setting will not be indexed or stored + // so we cannot rely on its doc_values + return; + } + docValueField = colName; + } + // docValueField Cannot be null + if (StringUtils.isNotEmpty(docValueField)) { + searchContext.docValueFieldsContext().put(colName, docValueField); + } + } + +} diff --git a/fe/fe-connectors/connector-es/src/main/java/org/apache/doris/datasource/es/PartitionPhase.java b/fe/fe-connectors/connector-es/src/main/java/org/apache/doris/datasource/es/PartitionPhase.java new file mode 100644 index 00000000000000..d1118fdab66a37 --- /dev/null +++ b/fe/fe-connectors/connector-es/src/main/java/org/apache/doris/datasource/es/PartitionPhase.java @@ -0,0 +1,56 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +package org.apache.doris.datasource.es; + +import java.util.HashMap; +import java.util.Map; + +/** + * Fetch resolved indices's search shards from remote ES Cluster + */ +public class PartitionPhase implements SearchPhase { + + private EsRestClient client; + private EsShardPartitions shardPartitions; + private Map nodesInfo; + + public PartitionPhase(EsRestClient client) { + this.client = client; + } + + @Override + public void execute(SearchContext context) throws DorisEsException { + shardPartitions = client.searchShards(context.sourceIndex()); + if (context.nodesDiscovery()) { + nodesInfo = client.getHttpNodes(); + } else { + nodesInfo = new HashMap<>(); + String[] seeds = context.esTable().getSeeds(); + for (int i = 0; i < seeds.length; i++) { + nodesInfo.put(String.valueOf(i), new EsNodeInfo(String.valueOf(i), seeds[i])); + } + } + } + + + @Override + public void postProcess(SearchContext context) throws DorisEsException { + context.partitions(shardPartitions); + context.partitions().addHttpAddress(nodesInfo); + } +} diff --git a/fe/fe-connectors/connector-es/src/main/java/org/apache/doris/datasource/es/QueryBuilders.java b/fe/fe-connectors/connector-es/src/main/java/org/apache/doris/datasource/es/QueryBuilders.java new file mode 100644 index 00000000000000..0d78511a7dc0b6 --- /dev/null +++ b/fe/fe-connectors/connector-es/src/main/java/org/apache/doris/datasource/es/QueryBuilders.java @@ -0,0 +1,880 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +package org.apache.doris.datasource.es; + +import org.apache.doris.analysis.BinaryPredicate; +import org.apache.doris.analysis.BoolLiteral; +import org.apache.doris.analysis.CastExpr; +import org.apache.doris.analysis.CompoundPredicate; +import org.apache.doris.analysis.DecimalLiteral; +import org.apache.doris.analysis.Expr; +import org.apache.doris.analysis.FloatLiteral; +import org.apache.doris.analysis.FunctionCallExpr; +import org.apache.doris.analysis.InPredicate; +import org.apache.doris.analysis.IntLiteral; +import org.apache.doris.analysis.IsNullPredicate; +import org.apache.doris.analysis.LargeIntLiteral; +import org.apache.doris.analysis.LikePredicate; +import org.apache.doris.analysis.LikePredicate.Operator; +import org.apache.doris.analysis.LiteralExpr; +import org.apache.doris.analysis.SlotRef; +import org.apache.doris.catalog.EsResource; + +import com.fasterxml.jackson.core.JsonGenerator; +import com.fasterxml.jackson.databind.JsonNode; +import com.fasterxml.jackson.databind.ObjectMapper; +import lombok.Builder; +import lombok.Data; +import org.apache.commons.lang3.StringUtils; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; +import org.joda.time.format.DateTimeFormat; +import org.joda.time.format.DateTimeFormatter; +import org.joda.time.format.ISODateTimeFormat; + +import java.io.IOException; +import java.io.StringWriter; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.Map.Entry; +import java.util.Objects; +import java.util.stream.Collectors; + + +/** + * Utility class to generate elastic search queries. + * Some query builders and static helper method have been copied from Elasticsearch + */ +public final class QueryBuilders { + + /** + * Generate dsl from compound expr. + **/ + private static QueryBuilder toCompoundEsDsl(Expr expr, List notPushDownList, + Map fieldsContext, BuilderOptions builderOptions, Map column2typeMap) { + CompoundPredicate compoundPredicate = (CompoundPredicate) expr; + switch (compoundPredicate.getOp()) { + case AND: { + QueryBuilder left = toEsDsl(compoundPredicate.getChild(0), notPushDownList, fieldsContext, + builderOptions, column2typeMap); + QueryBuilder right = toEsDsl(compoundPredicate.getChild(1), notPushDownList, fieldsContext, + builderOptions, column2typeMap); + if (left != null && right != null) { + return QueryBuilders.boolQuery().must(left).must(right); + } + return null; + } + case OR: { + int beforeSize = notPushDownList.size(); + QueryBuilder left = toEsDsl(compoundPredicate.getChild(0), notPushDownList, fieldsContext, + builderOptions, column2typeMap); + QueryBuilder right = toEsDsl(compoundPredicate.getChild(1), notPushDownList, fieldsContext, + builderOptions, column2typeMap); + int afterSize = notPushDownList.size(); + if (left != null && right != null) { + return QueryBuilders.boolQuery().should(left).should(right); + } + // One 'or' association cannot be pushed down and the other cannot be pushed down + if (afterSize > beforeSize) { + notPushDownList.add(compoundPredicate); + } + return null; + } + case NOT: { + QueryBuilder child = toEsDsl(compoundPredicate.getChild(0), notPushDownList, fieldsContext, + builderOptions, column2typeMap); + if (child != null) { + return QueryBuilders.boolQuery().mustNot(child); + } + return null; + } + default: + return null; + } + } + + /** + * Get the expr inside the cast. + **/ + private static Expr exprWithoutCast(Expr expr) { + if (expr instanceof CastExpr) { + return exprWithoutCast(expr.getChild(0)); + } + return expr; + } + + public static QueryBuilder toEsDsl(Expr expr, Map column2typeMap) { + return toEsDsl(expr, new ArrayList<>(), new HashMap<>(), + BuilderOptions.builder().likePushDown(Boolean.parseBoolean(EsResource.LIKE_PUSH_DOWN_DEFAULT_VALUE)) + .build(), column2typeMap); + } + + private static BinaryPredicate.Operator flipOpCode(BinaryPredicate.Operator opCode) { + switch (opCode) { + case GE: + return BinaryPredicate.Operator.LE; + case GT: + return BinaryPredicate.Operator.LT; + case LE: + return BinaryPredicate.Operator.GE; + case LT: + return BinaryPredicate.Operator.GT; + default: + return opCode; + } + } + + private static QueryBuilder parseBinaryPredicate(LiteralExpr expr, BinaryPredicate.Operator opCode, String column, + boolean needDateCompat) { + Object value = toDorisLiteral(expr); + if (needDateCompat) { + value = compatDefaultDate(value); + } + switch (opCode) { + case EQ: + case EQ_FOR_NULL: + return QueryBuilders.termQuery(column, value); + case NE: + // col != '' means col.length() > 0 in SQL syntax. + // The `NULL` value should not present in results. + // It equals + // '{"bool":{"must":{"bool":{"must_not":{"term":{"col":""}},"must":{"exists":{"field":"col"}}}}}}' + // in Elasticsearch + if (value instanceof String && StringUtils.isEmpty((String) value)) { + return QueryBuilders.boolQuery().mustNot(QueryBuilders.termQuery(column, value)) + .must(QueryBuilders.existsQuery(column)); + } + return QueryBuilders.boolQuery().mustNot(QueryBuilders.termQuery(column, value)); + case GE: + return QueryBuilders.rangeQuery(column).gte(value); + case GT: + return QueryBuilders.rangeQuery(column).gt(value); + case LE: + return QueryBuilders.rangeQuery(column).lte(value); + case LT: + return QueryBuilders.rangeQuery(column).lt(value); + default: + return null; + } + } + + private static QueryBuilder parseIsNullPredicate(Expr expr, String column) { + IsNullPredicate isNullPredicate = (IsNullPredicate) expr; + if (isNullPredicate.isNotNull()) { + return QueryBuilders.existsQuery(column); + } + return QueryBuilders.boolQuery().mustNot(QueryBuilders.existsQuery(column)); + } + + private static QueryBuilder parseLikeExpression(Expr expr, String column) { + String pattern; + if (expr instanceof LikePredicate) { + LikePredicate likePredicate = (LikePredicate) expr; + if (!likePredicate.getOp().equals(Operator.LIKE)) { + return QueryBuilders.wildcardQuery(column, likePredicate.getChild(1).getStringValue()); + } + pattern = likePredicate.getChild(1).getStringValue(); + } else if (expr instanceof FunctionCallExpr) { + FunctionCallExpr functionCallExpr = (FunctionCallExpr) expr; + String fnName = functionCallExpr.getFnName().getFunction(); + if (!fnName.equalsIgnoreCase("like")) { + return QueryBuilders.wildcardQuery(column, functionCallExpr.getChild(1).getStringValue()); + } + pattern = functionCallExpr.getChild(1).getStringValue(); + } else { + throw new IllegalArgumentException("Unsupported expression type"); + } + char[] chars = pattern.toCharArray(); + // example of translation : + // abc_123 ===> abc?123 + // abc%ykz ===> abc*123 + // %abc123 ===> *abc123 + // _abc123 ===> ?abc123 + // \\_abc1 ===> \\_abc1 + // abc\\_123 ===> abc\\_123 + // abc\\%123 ===> abc\\%123 + // NOTE. user must input sql like 'abc\\_123' or 'abc\\%ykz' + for (int i = 0; i < chars.length; i++) { + if (chars[i] == '_' || chars[i] == '%') { + if (i == 0) { + chars[i] = (chars[i] == '_') ? '?' : '*'; + } else if (chars[i - 1] != '\\') { + chars[i] = (chars[i] == '_') ? '?' : '*'; + } + } + } + return QueryBuilders.wildcardQuery(column, new String(chars)); + } + + private static QueryBuilder parseInPredicate(Expr expr, String column, boolean needDateCompat) { + InPredicate inPredicate = (InPredicate) expr; + List values = inPredicate.getListChildren().stream().map(v -> { + if (needDateCompat) { + return compatDefaultDate(v); + } + return toDorisLiteral(v); + }).collect(Collectors.toList()); + if (inPredicate.isNotIn()) { + return QueryBuilders.boolQuery().mustNot(QueryBuilders.termsQuery(column, values)); + } + return QueryBuilders.termsQuery(column, values); + } + + private static QueryBuilder parseFunctionCallExpr(Expr expr) { + // esquery(k1, '{ + // "match_phrase": { + // "k1": "doris on es" + // } + // }'); + // The first child k1 compatible with expr syntax + FunctionCallExpr functionCallExpr = (FunctionCallExpr) expr; + String stringValue = functionCallExpr.getChild(1).getStringValue(); + return new QueryBuilders.EsQueryBuilder(stringValue); + } + + private static String getColumnFromExpr(Expr expr) { + // Type transformed cast can not pushdown + if (expr instanceof CastExpr) { + Expr withoutCastExpr = exprWithoutCast(expr); + if (withoutCastExpr.getType().equals(expr.getType()) + || (withoutCastExpr.getType().isFloatingPointType() && expr.getType().isFloatingPointType())) { + return ((SlotRef) withoutCastExpr).getColumnName(); + } + } else if (expr instanceof SlotRef) { + return ((SlotRef) expr).getColumnName(); + } + return null; + } + + /** + * Doris expr to es dsl. + **/ + public static QueryBuilder toEsDsl(Expr expr, List notPushDownList, Map fieldsContext, + BuilderOptions builderOptions, Map column2typeMap) { + if (expr == null) { + return null; + } + // esquery functionCallExpr will be rewritten to castExpr in where clause rewriter, + // so we get the functionCallExpr here. + if (expr instanceof CastExpr) { + return toEsDsl(expr.getChild(0), notPushDownList, fieldsContext, builderOptions, column2typeMap); + } + // CompoundPredicate, `between` also converted to CompoundPredicate. + if (expr instanceof CompoundPredicate) { + return toCompoundEsDsl(expr, notPushDownList, fieldsContext, builderOptions, column2typeMap); + } + boolean isFlip = false; + Expr leftExpr = expr.getChild(0); + String column = getColumnFromExpr(leftExpr); + + if (StringUtils.isEmpty(column)) { + Expr rightExpr = expr.getChild(1); + column = getColumnFromExpr(rightExpr); + isFlip = true; + } + + if (StringUtils.isEmpty(column)) { + notPushDownList.add(expr); + return null; + } + + String type = column2typeMap.get(column); + // Check whether the date type need compat, it must before keyword replace. + List needCompatDateFields = builderOptions.getNeedCompatDateFields(); + boolean needDateCompat = needCompatDateFields != null && needCompatDateFields.contains(column); + // Replace col with col.keyword if mapping exist. + column = fieldsContext.getOrDefault(column, column); + if (expr instanceof BinaryPredicate) { + BinaryPredicate binaryPredicate = (BinaryPredicate) expr; + BinaryPredicate.Operator opCode = binaryPredicate.getOp(); + if (isFlip) { + opCode = flipOpCode(opCode); + } + Expr value; + if (isFlip) { + value = binaryPredicate.getChild(0); + } else { + value = binaryPredicate.getChild(1); + } + // only push down literal expr to ES + if (value instanceof LiteralExpr) { + LiteralExpr literalExpr = (LiteralExpr) value; + return parseBinaryPredicate(literalExpr, opCode, column, needDateCompat); + } else { + notPushDownList.add(expr); + return null; + } + } + if (expr instanceof IsNullPredicate) { + return parseIsNullPredicate(expr, column); + } + if (expr instanceof LikePredicate) { + if (builderOptions.isLikePushDown() && "keyword".equals(type)) { + // only keyword can apply wildcard query + return parseLikeExpression(expr, column); + } else { + notPushDownList.add(expr); + return null; + } + } + if (expr instanceof InPredicate) { + return parseInPredicate(expr, column, needDateCompat); + } + if (expr instanceof FunctionCallExpr) { + // current only esquery and like applied in keyword functionCallExpr can be push down to ES + String fnName = ((FunctionCallExpr) expr).getFnName().getFunction(); + if ("esquery".equals(fnName)) { + return parseFunctionCallExpr(expr); + } else if (builderOptions.isLikePushDown() && "like".equalsIgnoreCase(fnName) && "keyword".equals(type)) { + return parseLikeExpression(expr, column); + } else if (builderOptions.isLikePushDown() && "regexp".equalsIgnoreCase(fnName)) { + return parseLikeExpression(expr, column); + } else { + notPushDownList.add(expr); + return null; + } + } + return null; + } + + private static final DateTimeFormatter dorisFmt = DateTimeFormat.forPattern("yyyy-MM-dd HH:mm:ss"); + private static final DateTimeFormatter esFmt = ISODateTimeFormat.dateTime(); + + private static Object compatDefaultDate(Object value) { + if (value == null) { + return null; + } + return dorisFmt.parseDateTime(value.toString()).toString(esFmt); + } + + /** + * Expr trans to doris literal. + **/ + private static Object toDorisLiteral(Expr expr) { + if (!expr.isLiteral()) { + return null; + } + if (expr instanceof BoolLiteral) { + BoolLiteral boolLiteral = (BoolLiteral) expr; + return boolLiteral.getValue(); + } else if (expr instanceof DecimalLiteral) { + DecimalLiteral decimalLiteral = (DecimalLiteral) expr; + return decimalLiteral.getValue(); + } else if (expr instanceof FloatLiteral) { + FloatLiteral floatLiteral = (FloatLiteral) expr; + return floatLiteral.getValue(); + } else if (expr instanceof IntLiteral) { + IntLiteral intLiteral = (IntLiteral) expr; + return intLiteral.getValue(); + } else if (expr instanceof LargeIntLiteral) { + LargeIntLiteral largeIntLiteral = (LargeIntLiteral) expr; + return largeIntLiteral.getLongValue(); + } + return expr.getStringValue(); + } + + /** + * A query that matches on all documents. + */ + public static MatchAllQueryBuilder matchAllQuery() { + return new MatchAllQueryBuilder(); + } + + /** + * A Query that matches documents containing a term. + * + * @param name The name of the field + * @param value The value of the term + */ + public static TermQueryBuilder termQuery(String name, String value) { + return new TermQueryBuilder(name, value); + } + + /** + * A Query that matches documents containing a term. + * + * @param name The name of the field + * @param value The value of the term + */ + public static TermQueryBuilder termQuery(String name, int value) { + return new TermQueryBuilder(name, value); + } + + /** + * A Query that matches documents containing a term. + * + * @param name The name of the field + * @param value The value of the term + */ + public static TermQueryBuilder termQuery(String name, long value) { + return new TermQueryBuilder(name, value); + } + + /** + * A Query that matches documents containing a term. + * + * @param name The name of the field + * @param value The value of the term + */ + public static TermQueryBuilder termQuery(String name, float value) { + return new TermQueryBuilder(name, value); + } + + /** + * A Query that matches documents containing a term. + * + * @param name The name of the field + * @param value The value of the term + */ + public static TermQueryBuilder termQuery(String name, double value) { + return new TermQueryBuilder(name, value); + } + + /** + * A Query that matches documents containing a term. + * + * @param name The name of the field + * @param value The value of the term + */ + public static TermQueryBuilder termQuery(String name, boolean value) { + return new TermQueryBuilder(name, value); + } + + /** + * A Query that matches documents containing a term. + * + * @param name The name of the field + * @param value The value of the term + */ + public static TermQueryBuilder termQuery(String name, Object value) { + return new TermQueryBuilder(name, value); + } + + /** + * Implements the wildcard search query. Supported wildcards are {@code *}, which + * matches any character sequence (including the empty one), and {@code ?}, + * which matches any single character. Note this query can be slow, as it + * needs to iterate over many terms. In order to prevent extremely slow WildcardQueries, + * a Wildcard term should not start with one of the wildcards {@code *} or + * {@code ?}. + * + * @param name The field name + * @param query The wildcard query string + */ + public static WildcardQueryBuilder wildcardQuery(String name, String query) { + return new WildcardQueryBuilder(name, query); + } + + /** + * A Query that matches documents matching boolean combinations of other queries. + */ + public static BoolQueryBuilder boolQuery() { + return new BoolQueryBuilder(); + } + + + /** + * A filter for a field based on several terms matching on any of them. + * + * @param name The field name + * @param values The terms + */ + public static TermsQueryBuilder termsQuery(String name, Iterable values) { + return new TermsQueryBuilder(name, values); + } + + /** + * A filter to filter only documents where a field exists in them. + * + * @param name The name of the field + */ + public static ExistsQueryBuilder existsQuery(String name) { + return new ExistsQueryBuilder(name); + } + + /** + * A Query that matches documents within an range of terms. + * + * @param name The field name + */ + public static RangeQueryBuilder rangeQuery(String name) { + return new RangeQueryBuilder(name); + } + + /** + * Used to pass some parameters to generate the dsl + **/ + @Builder + @Data + public static class BuilderOptions { + + private boolean likePushDown; + + private List needCompatDateFields; + } + + + /** + * Base class to build various ES queries + */ + public abstract static class QueryBuilder { + + private static final Logger LOG = LogManager.getLogger(QueryBuilder.class); + + final ObjectMapper mapper = new ObjectMapper(); + + /** + * Convert query to JSON format + * + * @param out used to generate JSON elements + * @throws IOException if IO error occurred + */ + public abstract void toJson(JsonGenerator out) throws IOException; + + /** + * Convert query to JSON format and catch error. + **/ + public String toJson() { + StringWriter writer = new StringWriter(); + try { + JsonGenerator gen = mapper.getFactory().createGenerator(writer); + this.toJson(gen); + gen.flush(); + gen.close(); + } catch (IOException e) { + LOG.warn("QueryBuilder toJson error", e); + return null; + } + return writer.toString(); + } + } + + /** + * Use for esquery, directly save value. + **/ + public static class EsQueryBuilder extends QueryBuilder { + + private final String value; + + public EsQueryBuilder(String value) { + this.value = value; + } + + @Override + public void toJson(JsonGenerator out) throws IOException { + JsonNode jsonNode = mapper.readTree(value); + out.writeStartObject(); + Iterator> values = jsonNode.fields(); + while (values.hasNext()) { + Entry value = values.next(); + out.writeFieldName(value.getKey()); + out.writeObject(value.getValue()); + } + out.writeEndObject(); + } + } + + /** + * A Query that matches documents matching boolean combinations of other queries. + */ + public static class BoolQueryBuilder extends QueryBuilder { + + private final List mustClauses = new ArrayList<>(); + private final List mustNotClauses = new ArrayList<>(); + private final List filterClauses = new ArrayList<>(); + private final List shouldClauses = new ArrayList<>(); + + /** + * Use for EsScanNode generate dsl. + **/ + public BoolQueryBuilder must(QueryBuilder queryBuilder) { + Objects.requireNonNull(queryBuilder); + mustClauses.add(queryBuilder); + return this; + } + + public BoolQueryBuilder filter(QueryBuilder queryBuilder) { + Objects.requireNonNull(queryBuilder); + filterClauses.add(queryBuilder); + return this; + } + + public BoolQueryBuilder mustNot(QueryBuilder queryBuilder) { + Objects.requireNonNull(queryBuilder); + mustNotClauses.add(queryBuilder); + return this; + } + + public BoolQueryBuilder should(QueryBuilder queryBuilder) { + Objects.requireNonNull(queryBuilder); + shouldClauses.add(queryBuilder); + return this; + } + + @Override + public void toJson(JsonGenerator out) throws IOException { + out.writeStartObject(); + out.writeFieldName("bool"); + out.writeStartObject(); + writeJsonArray("must", mustClauses, out); + writeJsonArray("filter", filterClauses, out); + writeJsonArray("must_not", mustNotClauses, out); + writeJsonArray("should", shouldClauses, out); + out.writeEndObject(); + out.writeEndObject(); + } + + private void writeJsonArray(String field, List clauses, JsonGenerator out) throws IOException { + if (clauses.isEmpty()) { + return; + } + + if (clauses.size() == 1) { + out.writeFieldName(field); + clauses.get(0).toJson(out); + } else { + out.writeArrayFieldStart(field); + for (QueryBuilder clause : clauses) { + clause.toJson(out); + } + out.writeEndArray(); + } + } + } + + /** + * A Query that matches documents containing a term + */ + static class TermQueryBuilder extends QueryBuilder { + private final String fieldName; + private final Object value; + + private TermQueryBuilder(final String fieldName, final Object value) { + this.fieldName = Objects.requireNonNull(fieldName, "fieldName"); + this.value = Objects.requireNonNull(value, "value"); + } + + @Override + public void toJson(final JsonGenerator out) throws IOException { + out.writeStartObject(); + out.writeFieldName("term"); + out.writeStartObject(); + out.writeFieldName(fieldName); + writeObject(out, value); + out.writeEndObject(); + out.writeEndObject(); + } + } + + /** + * A filter for a field based on several terms matching on any of them. + */ + static class TermsQueryBuilder extends QueryBuilder { + private final String fieldName; + private final Iterable values; + + private TermsQueryBuilder(final String fieldName, final Iterable values) { + this.fieldName = Objects.requireNonNull(fieldName, "fieldName"); + this.values = Objects.requireNonNull(values, "values"); + } + + @Override + public void toJson(final JsonGenerator out) throws IOException { + out.writeStartObject(); + out.writeFieldName("terms"); + out.writeStartObject(); + out.writeFieldName(fieldName); + out.writeStartArray(); + for (Object value : values) { + writeObject(out, value); + } + out.writeEndArray(); + out.writeEndObject(); + out.writeEndObject(); + } + } + + /** + * A Query that matches documents within an range of terms + */ + public static class RangeQueryBuilder extends QueryBuilder { + + private final String field; + + private Object lt; + private boolean lte; + private Object gt; + private boolean gte; + + private String format; + + private RangeQueryBuilder(final String field) { + this.field = Objects.requireNonNull(field, "fieldName"); + } + + private RangeQueryBuilder to(Object value, boolean lte) { + this.lt = Objects.requireNonNull(value, "value"); + this.lte = lte; + return this; + } + + private RangeQueryBuilder from(Object value, boolean gte) { + this.gt = Objects.requireNonNull(value, "value"); + this.gte = gte; + return this; + } + + public RangeQueryBuilder lt(Object value) { + return to(value, false); + } + + public RangeQueryBuilder lte(Object value) { + return to(value, true); + } + + public RangeQueryBuilder gt(Object value) { + return from(value, false); + } + + public RangeQueryBuilder gte(Object value) { + return from(value, true); + } + + public RangeQueryBuilder format(String format) { + this.format = format; + return this; + } + + @Override + public void toJson(final JsonGenerator out) throws IOException { + if (lt == null && gt == null) { + throw new IllegalStateException("Either lower or upper bound should be provided"); + } + + out.writeStartObject(); + out.writeFieldName("range"); + out.writeStartObject(); + out.writeFieldName(field); + out.writeStartObject(); + + if (gt != null) { + final String op = gte ? "gte" : "gt"; + out.writeFieldName(op); + writeObject(out, gt); + } + + if (lt != null) { + final String op = lte ? "lte" : "lt"; + out.writeFieldName(op); + writeObject(out, lt); + } + + if (format != null) { + out.writeStringField("format", format); + } + + out.writeEndObject(); + out.writeEndObject(); + out.writeEndObject(); + } + } + + + /** + * Supported wildcards are {@code *}, which + * matches any character sequence (including the empty one), and {@code ?}, + * which matches any single character + */ + static class WildcardQueryBuilder extends QueryBuilder { + + private final String fieldName; + private final String value; + + + public WildcardQueryBuilder(String fieldName, String value) { + this.fieldName = Objects.requireNonNull(fieldName, "fieldName"); + this.value = Objects.requireNonNull(value, "value"); + } + + @Override + public void toJson(JsonGenerator out) throws IOException { + out.writeStartObject(); + out.writeFieldName("wildcard"); + out.writeStartObject(); + out.writeFieldName(fieldName); + out.writeString(value); + out.writeEndObject(); + out.writeEndObject(); + } + } + + /** + * Query that only match on documents that the fieldName has a value in them + */ + static class ExistsQueryBuilder extends QueryBuilder { + + private final String fieldName; + + ExistsQueryBuilder(final String fieldName) { + this.fieldName = Objects.requireNonNull(fieldName, "fieldName"); + } + + @Override + public void toJson(JsonGenerator out) throws IOException { + out.writeStartObject(); + out.writeFieldName("exists"); + out.writeStartObject(); + out.writeStringField("field", fieldName); + out.writeEndObject(); + out.writeEndObject(); + + } + } + + /** + * A query that matches on all documents + */ + static class MatchAllQueryBuilder extends QueryBuilder { + + private MatchAllQueryBuilder() { + } + + @Override + public void toJson(final JsonGenerator out) throws IOException { + out.writeStartObject(); + out.writeFieldName("match_all"); + out.writeStartObject(); + out.writeEndObject(); + out.writeEndObject(); + } + } + + /** + * Write (scalar) value (string, number, boolean or null) to json format + * + * @param out source target + * @param value value to write + * @throws IOException if error + */ + private static void writeObject(JsonGenerator out, Object value) throws IOException { + out.writeObject(value); + } +} diff --git a/fe/fe-connectors/connector-es/src/main/java/org/apache/doris/datasource/es/SearchContext.java b/fe/fe-connectors/connector-es/src/main/java/org/apache/doris/datasource/es/SearchContext.java new file mode 100644 index 00000000000000..857d14551ae904 --- /dev/null +++ b/fe/fe-connectors/connector-es/src/main/java/org/apache/doris/datasource/es/SearchContext.java @@ -0,0 +1,159 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +package org.apache.doris.datasource.es; + +import org.apache.doris.catalog.Column; +import org.apache.doris.catalog.EsTable; + +import com.google.common.collect.Lists; +import com.google.common.collect.Maps; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; + +import java.util.Collections; +import java.util.List; +import java.util.Map; + +/** + * This class encapsulates the state needed to execute a query on ES table such as fields、doc_values、resolved index、 + * search shards etc. + * Since then, we would add more state or runtime information to this class such as + * query builder、slice scroll context、aggregation info etc. + **/ +public class SearchContext { + + private static final Logger LOG = LogManager.getLogger(SearchContext.class); + + // fetch string field value from not analyzed fields : userId => userId.keyword + // this is activated when `enable_keyword_sniff = true` + private Map fetchFieldsContext = Maps.newHashMap(); + // used to indicate which fields can get from ES docavalue + // because elasticsearch can have "fields" feature, field can have + // two or more types, the first type maybe have not docvalue but other + // can have, such as (text field not have docvalue, but keyword can have): + // "properties": { + // "city": { + // "type": "text", + // "fields": { + // "raw": { + // "type": "keyword" + // } + // } + // } + // } + // then the docvalue context provided the mapping between the select field and real request field : + // {"city": "city.raw"} + // use select city from table, if enable the docvalue, we will fetch the `city` field value from `city.raw` + // fetch field value from doc_values, this is activated when `enable_docvalue_scan= true` + private Map docValueFieldsContext = Maps.newHashMap(); + + private List needCompatDateFields = Lists.newArrayList(); + + // sourceIndex is the name of index when creating ES external table + private final String sourceIndex; + + // when the `sourceIndex` is `alias` or `wildcard` matched index, this maybe involved two or more indices + // `resolvedIndices` would return the matched underlying indices + private List resolvedIndices = Collections.emptyList(); + + // `type` of the `sourceIndex` + private final String type; + + + private EsTable table; + + // all columns which user created for ES external table + private final List fullSchema; + + // represent `resolvedIndices`'s searchable shards + private EsShardPartitions shardPartitions; + + // the ES cluster version + private EsMajorVersion version; + + // whether the nodes needs to be discovered + private boolean nodesDiscovery; + + + public SearchContext(EsTable table) { + this.table = table; + fullSchema = table.getFullSchema(); + sourceIndex = table.getIndexName(); + type = table.getMappingType(); + nodesDiscovery = table.isNodesDiscovery(); + } + + + public String sourceIndex() { + return sourceIndex; + } + + public List resolvedIndices() { + return resolvedIndices; + } + + + public String type() { + return type; + } + + public List columns() { + return fullSchema; + } + + public EsTable esTable() { + return table; + } + + public Map fetchFieldsContext() { + return fetchFieldsContext; + } + + public Map docValueFieldsContext() { + return docValueFieldsContext; + } + + public List needCompatDateFields() { + return needCompatDateFields; + } + + public void version(EsMajorVersion version) { + this.version = version; + } + + public EsMajorVersion version() { + return version; + } + + public void partitions(EsShardPartitions shardPartitions) { + this.shardPartitions = shardPartitions; + } + + public EsShardPartitions partitions() { + return shardPartitions; + } + + // this will be refactor soon + public EsTablePartitions tablePartitions() throws Exception { + return EsTablePartitions.fromShardPartitions(table, shardPartitions); + } + + public boolean nodesDiscovery() { + return nodesDiscovery; + } +} diff --git a/fe/fe-connectors/connector-es/src/main/java/org/apache/doris/datasource/es/SearchPhase.java b/fe/fe-connectors/connector-es/src/main/java/org/apache/doris/datasource/es/SearchPhase.java new file mode 100644 index 00000000000000..ebcd78b0468347 --- /dev/null +++ b/fe/fe-connectors/connector-es/src/main/java/org/apache/doris/datasource/es/SearchPhase.java @@ -0,0 +1,42 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +package org.apache.doris.datasource.es; + + +/** + * Represents a phase of a ES fetch index metadata request e.g. get mapping, get shard location etc through network + */ +public interface SearchPhase { + + /** + * Performs pre processing of the search context before the execute. + */ + default void preProcess(SearchContext context) { + } + + /** + * Executes the search phase + */ + void execute(SearchContext context); + + /** + * Performs post processing of the search context before the execute. + */ + default void postProcess(SearchContext context) { + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/es/source/EsScanNode.java b/fe/fe-connectors/connector-es/src/main/java/org/apache/doris/datasource/es/source/EsScanNode.java similarity index 100% rename from fe/fe-core/src/main/java/org/apache/doris/datasource/es/source/EsScanNode.java rename to fe/fe-connectors/connector-es/src/main/java/org/apache/doris/datasource/es/source/EsScanNode.java diff --git a/fe/fe-connectors/connector-es/src/main/resources/META-INF/services/org.apache.doris.datasource.spi.CatalogProvider b/fe/fe-connectors/connector-es/src/main/resources/META-INF/services/org.apache.doris.datasource.spi.CatalogProvider new file mode 100644 index 00000000000000..ba9d1c525a5205 --- /dev/null +++ b/fe/fe-connectors/connector-es/src/main/resources/META-INF/services/org.apache.doris.datasource.spi.CatalogProvider @@ -0,0 +1,18 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, +# software distributed under the License is distributed on an +# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +# KIND, either express or implied. See the License for the +# specific language governing permissions and limitations +# under the License. + +org.apache.doris.datasource.es.EsCatalogProvider diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/Env.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/Env.java index d6aadff5dcc005..b685fb7db118fd 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/catalog/Env.java +++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/Env.java @@ -94,7 +94,6 @@ import org.apache.doris.datasource.ExternalMetaIdMgr; import org.apache.doris.datasource.InternalCatalog; import org.apache.doris.datasource.SplitSourceManager; -import org.apache.doris.datasource.es.EsExternalCatalog; import org.apache.doris.datasource.es.EsRepository; import org.apache.doris.datasource.hive.HiveTransactionMgr; import org.apache.doris.datasource.hive.event.MetastoreEventsProcessor; @@ -102,6 +101,7 @@ import org.apache.doris.datasource.jdbc.JdbcExternalTable; import org.apache.doris.datasource.paimon.PaimonExternalTable; import org.apache.doris.datasource.paimon.PaimonSysExternalTable; +import org.apache.doris.datasource.spi.CatalogPluginLoader; import org.apache.doris.deploy.DeployManager; import org.apache.doris.deploy.impl.LocalFileDeployManager; import org.apache.doris.dictionary.DictionaryManager; @@ -1200,7 +1200,12 @@ public void initialize(String[] args) throws Exception { selfNode.getPort(), false /* new style */); } - // 3. Load image first and replay edits + // 3. Load catalog plugins before image replay + // This ensures all CatalogProvider SPIs are registered so that + // persisted external catalogs can be properly initialized when accessed. + CatalogPluginLoader.loadPlugins(); + + // 4. Load image first and replay edits this.editLog = new EditLog(nodeName); loadImage(this.imageDir); // load image file migrateConstraintsFromTables(); // migrate old table-based constraints @@ -1212,16 +1217,16 @@ public void initialize(String[] args) throws Exception { replayJournalsAndExit(); } - // 4. create load and export job label cleaner thread + // 5. create load and export job label cleaner thread createLabelCleaner(); - // 5. create txn cleaner thread + // 6. create txn cleaner thread createTxnCleaner(); - // 6. start state listener thread + // 7. start state listener thread startStateListener(); - // 7. create fe disk updater + // 8. create fe disk updater createFeDiskUpdater(); if (!Config.edit_log_type.equalsIgnoreCase("bdb")) { @@ -6439,8 +6444,8 @@ public void changeCatalog(ConnectContext ctx, String catalogName) throws DdlExce if (StringUtils.isNotEmpty(lastDb)) { ctx.setDatabase(lastDb); } - if (catalogIf instanceof EsExternalCatalog) { - ctx.setDatabase(EsExternalCatalog.DEFAULT_DB); + if ("es".equalsIgnoreCase(catalogIf.getType())) { + ctx.setDatabase("default_db"); } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/CatalogFactory.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/CatalogFactory.java index 8ff1db71771787..f3b99cbc4c06ba 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/CatalogFactory.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/CatalogFactory.java @@ -22,12 +22,13 @@ import org.apache.doris.common.DdlException; import org.apache.doris.common.FeConstants; import org.apache.doris.datasource.doris.RemoteDorisExternalCatalog; -import org.apache.doris.datasource.es.EsExternalCatalog; import org.apache.doris.datasource.hive.HMSExternalCatalog; import org.apache.doris.datasource.iceberg.IcebergExternalCatalogFactory; import org.apache.doris.datasource.jdbc.JdbcExternalCatalog; import org.apache.doris.datasource.maxcompute.MaxComputeExternalCatalog; import org.apache.doris.datasource.paimon.PaimonExternalCatalogFactory; +import org.apache.doris.datasource.spi.CatalogProvider; +import org.apache.doris.datasource.spi.CatalogProviderRegistry; import org.apache.doris.datasource.test.TestExternalCatalog; import org.apache.doris.datasource.trinoconnector.TrinoConnectorExternalCatalogFactory; import org.apache.doris.nereids.trees.plans.commands.CreateCatalogCommand; @@ -85,41 +86,46 @@ private static CatalogIf createCatalog(long catalogId, String name, String resou // create catalog ExternalCatalog catalog; - switch (catalogType) { - case "hms": - catalog = new HMSExternalCatalog(catalogId, name, resource, props, comment); - break; - case "es": - catalog = new EsExternalCatalog(catalogId, name, resource, props, comment); - break; - case "jdbc": - catalog = new JdbcExternalCatalog(catalogId, name, resource, props, comment); - break; - case "iceberg": - catalog = IcebergExternalCatalogFactory.createCatalog(catalogId, name, resource, props, comment); - break; - case "paimon": - catalog = PaimonExternalCatalogFactory.createCatalog(catalogId, name, resource, props, comment); - break; - case "trino-connector": - catalog = TrinoConnectorExternalCatalogFactory.createCatalog(catalogId, name, resource, props, comment); - break; - case "max_compute": - catalog = new MaxComputeExternalCatalog(catalogId, name, resource, props, comment); - break; - case "lakesoul": - throw new DdlException("Lakesoul catalog is no longer supported"); - case "doris": - catalog = new RemoteDorisExternalCatalog(catalogId, name, resource, props, comment); - break; - case "test": - if (!FeConstants.runningUnitTest) { - throw new DdlException("test catalog is only for FE unit test"); - } - catalog = new TestExternalCatalog(catalogId, name, resource, props, comment); - break; - default: - throw new DdlException("Unknown catalog type: " + catalogType); + // Try SPI-registered CatalogProvider first + CatalogProvider provider = CatalogProviderRegistry.getProvider(catalogType); + if (provider != null) { + catalog = provider.createCatalog(catalogId, name, resource, props, comment); + } else { + // Fallback to hardcoded switch-case for datasources not yet migrated to SPI + switch (catalogType) { + case "hms": + catalog = new HMSExternalCatalog(catalogId, name, resource, props, comment); + break; + case "jdbc": + catalog = new JdbcExternalCatalog(catalogId, name, resource, props, comment); + break; + case "iceberg": + catalog = IcebergExternalCatalogFactory.createCatalog(catalogId, name, resource, props, comment); + break; + case "paimon": + catalog = PaimonExternalCatalogFactory.createCatalog(catalogId, name, resource, props, comment); + break; + case "trino-connector": + catalog = TrinoConnectorExternalCatalogFactory.createCatalog( + catalogId, name, resource, props, comment); + break; + case "max_compute": + catalog = new MaxComputeExternalCatalog(catalogId, name, resource, props, comment); + break; + case "lakesoul": + throw new DdlException("Lakesoul catalog is no longer supported"); + case "doris": + catalog = new RemoteDorisExternalCatalog(catalogId, name, resource, props, comment); + break; + case "test": + if (!FeConstants.runningUnitTest) { + throw new DdlException("test catalog is only for FE unit test"); + } + catalog = new TestExternalCatalog(catalogId, name, resource, props, comment); + break; + default: + throw new DdlException("Unknown catalog type: " + catalogType); + } } // set some default properties if missing when creating catalog. diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/ExternalCatalog.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/ExternalCatalog.java index 7d188ff42d70a0..1197d84d5e1156 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/ExternalCatalog.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/ExternalCatalog.java @@ -40,7 +40,6 @@ import org.apache.doris.common.util.Util; import org.apache.doris.datasource.connectivity.CatalogConnectivityTestCoordinator; import org.apache.doris.datasource.doris.RemoteDorisExternalDatabase; -import org.apache.doris.datasource.es.EsExternalDatabase; import org.apache.doris.datasource.hive.HMSExternalCatalog; import org.apache.doris.datasource.hive.HMSExternalDatabase; import org.apache.doris.datasource.iceberg.IcebergExternalDatabase; @@ -52,6 +51,7 @@ import org.apache.doris.datasource.metacache.MetaCache; import org.apache.doris.datasource.operations.ExternalMetadataOps; import org.apache.doris.datasource.paimon.PaimonExternalDatabase; +import org.apache.doris.datasource.spi.ConnectorMetadata; import org.apache.doris.datasource.test.TestExternalCatalog; import org.apache.doris.datasource.test.TestExternalDatabase; import org.apache.doris.datasource.trinoconnector.TrinoConnectorExternalDatabase; @@ -174,6 +174,8 @@ public abstract class ExternalCatalog // db name does not contains "default_cluster" protected Map dbNameToId = Maps.newConcurrentMap(); private boolean objectCreated = false; + // ConnectorMetadata from SPI connector plugin (transient, not persisted) + protected transient ConnectorMetadata connectorMetadata; protected ExternalMetadataOps metadataOps; protected TransactionManager transactionManager; protected MetaCache> metaCache; @@ -237,11 +239,12 @@ private Configuration buildConf() { * @return list of database names in this catalog */ protected List listDatabaseNames() { - if (metadataOps == null) { - throw new UnsupportedOperationException("List databases is not supported for catalog: " + getName()); - } else { + if (metadataOps != null) { return metadataOps.listDatabaseNames(); + } else if (connectorMetadata != null) { + return connectorMetadata.listSchemaNames(); } + throw new UnsupportedOperationException("List databases is not supported for catalog: " + getName()); } public ExternalMetadataOps getMetadataOps() { @@ -249,6 +252,11 @@ public ExternalMetadataOps getMetadataOps() { return metadataOps; } + public ConnectorMetadata getConnectorMetadata() { + makeSureInitialized(); + return connectorMetadata; + } + // Will be called when creating catalog(so when as replaying) // to add some default properties if missing. public void setDefaultPropsIfMissing(boolean isReplay) { @@ -316,7 +324,14 @@ public final List listTableNames(SessionContext ctx, String dbName) { * @param dbName database name * @return names of tables in the specified database from the remote source */ - protected abstract List listTableNamesFromRemote(SessionContext ctx, String dbName); + protected List listTableNamesFromRemote(SessionContext ctx, String dbName) { + if (connectorMetadata != null) { + return connectorMetadata.listTableNames(dbName); + } + throw new UnsupportedOperationException( + "listTableNamesFromRemote is not supported for catalog: " + getName() + + ". No ConnectorMetadata registered for type: " + getType()); + } /** * check if the specified table exist. @@ -325,13 +340,25 @@ public final List listTableNames(SessionContext ctx, String dbName) { * @param tblName * @return true if table exists, false otherwise */ - public abstract boolean tableExist(SessionContext ctx, String dbName, String tblName); + public boolean tableExist(SessionContext ctx, String dbName, String tblName) { + if (connectorMetadata != null) { + return connectorMetadata.tableExists(dbName, tblName); + } + throw new UnsupportedOperationException( + "tableExist is not supported for catalog: " + getName() + + ". No ConnectorMetadata registered for type: " + getType()); + } /** * init some local objects such as: * hms client, read properties from hive-site.xml, es client */ - protected abstract void initLocalObjectsImpl(); + protected void initLocalObjectsImpl() { + // Subclasses should override this to: + // 1. Initialize data source clients (e.g., EsRestClient, HiveMetaStoreClient) + // 2. Create and assign this.connectorMetadata (for SPI-migrated connectors) + // or this.metadataOps (for legacy connectors) + } /** * check if the specified table exist in doris. @@ -780,6 +807,10 @@ public void onClose() { if (null != transactionManager) { transactionManager = null; } + if (null != connectorMetadata) { + connectorMetadata.close(); + connectorMetadata = null; + } } private void removeAccessController() { @@ -898,11 +929,15 @@ protected ExternalDatabase buildDbForInit(String remote if (localDbName.equalsIgnoreCase(MysqlDb.DATABASE_NAME)) { return new ExternalMysqlDatabase(this, dbId); } + // Try ConnectorMetadata first (SPI-migrated connectors) + if (connectorMetadata != null) { + return connectorMetadata.createDatabase(this, dbId, localDbName, remoteDbName); + } + + // Fallback to hardcoded switch-case for datasources not yet migrated to SPI switch (logType) { case HMS: return new HMSExternalDatabase(this, dbId, localDbName, remoteDbName); - case ES: - return new EsExternalDatabase(this, dbId, localDbName, remoteDbName); case JDBC: return new JdbcExternalDatabase(this, dbId, localDbName, remoteDbName); case ICEBERG: diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/spi/CatalogPluginLoader.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/spi/CatalogPluginLoader.java new file mode 100644 index 00000000000000..8eca9cd4f61de0 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/spi/CatalogPluginLoader.java @@ -0,0 +1,160 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +package org.apache.doris.datasource.spi; + +import org.apache.doris.DorisFE; + +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; + +import java.io.File; +import java.io.FilenameFilter; +import java.net.URL; +import java.net.URLClassLoader; +import java.util.ArrayList; +import java.util.List; +import java.util.Map; +import java.util.ServiceLoader; +import java.util.concurrent.ConcurrentHashMap; + +/** + * Loads catalog plugins from the {@code connectors/} directory under Doris FE home. + * + *

Each subdirectory under {@code connectors/} represents one catalog plugin. + * All JAR files in that subdirectory are loaded with an isolated {@link URLClassLoader}, + * providing ClassLoader-level isolation between different catalog plugins to prevent + * dependency conflicts.

+ * + *

This loader must be invoked before {@code Env.loadImage()} to ensure + * all catalog types are registered before EditLog replay.

+ * + *

Directory Layout

+ *
+ * output/fe/
+ * ├── lib/                    # fe-core JARs
+ * └── lib/
+ *     ├── doris-fe.jar           # fe-core
+ *     └── connectors/            # Connector plugins
+ *         ├── es/
+ *         │   └── doris-connector-es.jar
+ *         ├── iceberg/
+ *         │   └── doris-connector-iceberg.jar
+ *         └── ...
+ * 
+ */ +public class CatalogPluginLoader { + private static final Logger LOG = LogManager.getLogger(CatalogPluginLoader.class); + + private static final String CONNECTORS_DIR = "lib/connectors"; + + /** ClassLoaders for each loaded plugin, keyed by catalog type */ + private static final Map PLUGIN_CLASSLOADERS = new ConcurrentHashMap<>(); + + /** + * Scan the catalogs/ directory and load all catalog plugins. + * Each subdirectory is treated as a separate plugin with its own ClassLoader. + */ + public static void loadPlugins() { + String dorisHome = DorisFE.DORIS_HOME_DIR; + if (dorisHome == null || dorisHome.isEmpty()) { + LOG.warn("DORIS_HOME is not set. Skipping catalog plugin loading."); + return; + } + + File pluginDir = new File(dorisHome, CONNECTORS_DIR); + if (!pluginDir.exists() || !pluginDir.isDirectory()) { + LOG.info("No catalog plugin directory found at {}. " + + "External catalog plugins will not be loaded.", pluginDir.getAbsolutePath()); + return; + } + + LOG.info("Loading catalog plugins from: {}", pluginDir.getAbsolutePath()); + + File[] pluginDirs = pluginDir.listFiles(File::isDirectory); + if (pluginDirs == null || pluginDirs.length == 0) { + LOG.info("No catalog plugins found in {}", pluginDir.getAbsolutePath()); + return; + } + + for (File dir : pluginDirs) { + try { + loadPlugin(dir); + } catch (Exception e) { + LOG.error("Failed to load catalog plugin from {}: {}", dir.getName(), e.getMessage(), e); + } + } + + LOG.info("Catalog plugin loading complete. Registered types: {}", + CatalogProviderRegistry.getAllProviders().keySet()); + } + + private static void loadPlugin(File pluginDir) throws Exception { + URL[] jarUrls = findJars(pluginDir); + if (jarUrls.length == 0) { + LOG.warn("No JAR files found in plugin directory: {}", pluginDir.getName()); + return; + } + + LOG.info("Loading catalog plugin from '{}' with {} JAR(s)", pluginDir.getName(), jarUrls.length); + + // Create isolated ClassLoader with fe-core as parent + URLClassLoader pluginClassLoader = new URLClassLoader( + jarUrls, CatalogPluginLoader.class.getClassLoader()); + + ServiceLoader providers = ServiceLoader.load( + CatalogProvider.class, pluginClassLoader); + + int count = 0; + for (CatalogProvider provider : providers) { + CatalogProviderRegistry.register(provider); + PLUGIN_CLASSLOADERS.put(provider.getType(), pluginClassLoader); + count++; + } + + if (count == 0) { + LOG.warn("Plugin directory '{}' contains JAR files but no CatalogProvider implementation. " + + "Ensure META-INF/services/{} is correctly configured.", + pluginDir.getName(), CatalogProvider.class.getName()); + pluginClassLoader.close(); + } + } + + private static URL[] findJars(File dir) throws Exception { + FilenameFilter jarFilter = (d, name) -> name.endsWith(".jar"); + File[] jarFiles = dir.listFiles(jarFilter); + if (jarFiles == null) { + return new URL[0]; + } + + List urls = new ArrayList<>(jarFiles.length); + for (File jar : jarFiles) { + urls.add(jar.toURI().toURL()); + } + return urls.toArray(new URL[0]); + } + + /** + * Get the ClassLoader for a specific catalog type. + * + * @param type catalog type + * @return the plugin ClassLoader, or null if not loaded via plugin + */ + public static ClassLoader getPluginClassLoader(String type) { + return PLUGIN_CLASSLOADERS.get(type); + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/spi/CatalogProvider.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/spi/CatalogProvider.java new file mode 100644 index 00000000000000..6fb9bb80cc1d29 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/spi/CatalogProvider.java @@ -0,0 +1,64 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +package org.apache.doris.datasource.spi; + +import org.apache.doris.datasource.ExternalCatalog; + +import java.util.Map; + +/** + * Service Provider Interface for external data source catalogs. + * + *

Each external data source (ES, Iceberg, Paimon, etc.) implements this interface + * and registers it via Java {@link java.util.ServiceLoader}. This enables Doris to + * discover and use data source plugins without hard-coded dependencies in fe-core.

+ * + *

{@code CatalogProvider} acts as a factory only. It is responsible for:

+ *
    + *
  • Identifying the catalog type via {@link #getType()}
  • + *
  • Creating an {@link ExternalCatalog} instance via {@link #createCatalog}
  • + *
+ * + *

All metadata access, object construction, scan node creation, and DDL operations + * are delegated to the {@link ConnectorMetadata} interface, which is created by the + * {@link ExternalCatalog} subclass during initialization.

+ */ +public interface CatalogProvider { + + /** + * Returns the catalog type identifier. + * This must match the "type" property used in CREATE CATALOG statements. + * For example: "es", "hms", "iceberg", "paimon", "jdbc", etc. + */ + String getType(); + + /** + * Create a new ExternalCatalog instance for this data source type. + * Called by {@code CatalogFactory} when creating a new catalog via CREATE CATALOG + * or when replaying from EditLog. + * + * @param catalogId the catalog id + * @param name the catalog name + * @param resource the resource name (nullable) + * @param props catalog properties + * @param comment catalog comment + * @return a new ExternalCatalog instance + */ + ExternalCatalog createCatalog(long catalogId, String name, String resource, + Map props, String comment); +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/spi/CatalogProviderRegistry.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/spi/CatalogProviderRegistry.java new file mode 100644 index 00000000000000..d0bbd2cc193e1f --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/spi/CatalogProviderRegistry.java @@ -0,0 +1,79 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +package org.apache.doris.datasource.spi; + +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; + +import java.util.Collections; +import java.util.Map; +import java.util.concurrent.ConcurrentHashMap; + +/** + * Registry for catalog providers discovered via SPI. + * + *

This registry maintains a mapping from catalog type string (e.g., "es", "iceberg") + * to the corresponding {@link CatalogProvider} implementation. Providers are registered + * during FE startup by {@link CatalogPluginLoader} before any EditLog replay occurs.

+ */ +public class CatalogProviderRegistry { + private static final Logger LOG = LogManager.getLogger(CatalogProviderRegistry.class); + + private static final Map PROVIDERS = new ConcurrentHashMap<>(); + + /** + * Register a catalog provider. + * + * @param provider the provider to register + * @throws IllegalArgumentException if a provider with the same type is already registered + */ + public static void register(CatalogProvider provider) { + String type = provider.getType(); + CatalogProvider existing = PROVIDERS.putIfAbsent(type, provider); + if (existing != null) { + throw new IllegalArgumentException( + "Duplicate CatalogProvider for type '" + type + "': " + + existing.getClass().getName() + " vs " + provider.getClass().getName()); + } + LOG.info("Registered CatalogProvider for type '{}': {}", type, provider.getClass().getName()); + } + + /** + * Get the catalog provider for the given type. + * + * @param type catalog type string + * @return the provider, or null if no provider is registered for this type + */ + public static CatalogProvider getProvider(String type) { + return PROVIDERS.get(type); + } + + /** + * Check if a provider is registered for the given type. + */ + public static boolean hasProvider(String type) { + return PROVIDERS.containsKey(type); + } + + /** + * Get all registered providers (read-only view). + */ + public static Map getAllProviders() { + return Collections.unmodifiableMap(PROVIDERS); + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/spi/ConnectorMetadata.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/spi/ConnectorMetadata.java new file mode 100644 index 00000000000000..d7256599a9344f --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/spi/ConnectorMetadata.java @@ -0,0 +1,128 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +package org.apache.doris.datasource.spi; + +import org.apache.doris.datasource.ExternalCatalog; +import org.apache.doris.datasource.ExternalDatabase; +import org.apache.doris.datasource.ExternalTable; + +import java.util.List; + +/** + * Unified interface for all operations on an external data source. + * + *

This interface covers read-only metadata queries, Doris internal object construction, + * scan node creation, and (in future) DDL operations. It replaces the former + * {@code Connector} + {@code ConnectorMetadata} two-layer design with a single, + * flat interface that {@link org.apache.doris.datasource.ExternalCatalog} delegates to.

+ * + *

Each external data source (ES, HMS, Iceberg, etc.) provides an implementation + * of this interface. The implementation is created by the {@code ExternalCatalog} subclass + * during {@code initLocalObjectsImpl()} and stored as a transient field.

+ * + *

Inspired by Trino's {@code ConnectorMetadata}, adapted for Doris's architecture.

+ */ +public interface ConnectorMetadata { + + // ====== Read-only metadata queries ====== + + /** + * Returns the names of all databases (schemas) in this data source. + */ + List listSchemaNames(); + + /** + * Returns the names of all tables in the specified database. + * + * @param schemaName the database name + * @return list of table names + */ + List listTableNames(String schemaName); + + /** + * Checks if a table exists in the specified database. + * + *

The default implementation delegates to {@link #listTableNames(String)}, + * but connectors should override this for better performance.

+ * + * @param schemaName the database name + * @param tableName the table name + * @return true if the table exists + */ + default boolean tableExists(String schemaName, String tableName) { + return listTableNames(schemaName).contains(tableName); + } + + // ====== Doris internal object construction ====== + + /** + * Creates an {@link ExternalDatabase} instance for the given database. + * + *

This is a Doris-specific method (Trino uses opaque handles instead). + * The connector is responsible for creating the correct database subclass + * (e.g., {@code EsExternalDatabase}, {@code IcebergExternalDatabase}).

+ * + * @param catalog the parent catalog + * @param dbId the database id assigned by Doris + * @param localDbName the local (possibly case-mapped) database name + * @param remoteDbName the original remote database name + * @return a new ExternalDatabase instance + */ + ExternalDatabase createDatabase(ExternalCatalog catalog, + long dbId, String localDbName, String remoteDbName); + + /** + * Creates an {@link ExternalTable} instance for the given table. + * + *

This is a Doris-specific method. The connector is responsible for creating + * the correct table subclass.

+ * + * @param catalog the parent catalog + * @param db the parent database + * @param tblId the table id assigned by Doris + * @param localName the local (possibly case-mapped) table name + * @param remoteName the original remote table name + * @return a new ExternalTable instance + */ + ExternalTable createTable(ExternalCatalog catalog, + ExternalDatabase db, + long tblId, String localName, String remoteName); + + // ====== Query execution ====== + + /** + * Returns the scan node provider for query plan generation. + * + *

Override this method if the connector supports query execution. + * The default implementation throws {@link UnsupportedOperationException}.

+ */ + default ScanNodeProvider getScanNodeProvider() { + throw new UnsupportedOperationException( + "This connector does not support scan node creation"); + } + + // ====== Lifecycle ====== + + /** + * Release resources held by this connector metadata (e.g., client connections). + * + *

Called when the catalog is being closed or reset. + * The default implementation is a no-op.

+ */ + default void close() {} +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/spi/ScanNodeProvider.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/spi/ScanNodeProvider.java new file mode 100644 index 00000000000000..ff6b8cc464ac37 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/spi/ScanNodeProvider.java @@ -0,0 +1,48 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +package org.apache.doris.datasource.spi; + +import org.apache.doris.analysis.TupleDescriptor; +import org.apache.doris.datasource.ExternalTable; +import org.apache.doris.planner.PlanNodeId; +import org.apache.doris.planner.ScanContext; +import org.apache.doris.planner.ScanNode; + +/** + * Interface for creating scan nodes during query plan generation. + * + *

This separates the execution plan layer from the metadata layer, + * inspired by Trino's {@code ConnectorPageSourceProvider}.

+ * + *

Each connector that supports query execution should provide an implementation + * via {@link Connector#getScanNodeProvider()}.

+ */ +public interface ScanNodeProvider { + + /** + * Creates a {@link ScanNode} for reading data from the given external table. + * + * @param id the plan node id + * @param desc the tuple descriptor for scan output + * @param table the table being scanned + * @param scanContext the scan context + * @return a new ScanNode instance specific to this data source + */ + ScanNode createScanNode(PlanNodeId id, TupleDescriptor desc, + ExternalTable table, ScanContext scanContext); +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/httpv2/restv2/ESCatalogAction.java b/fe/fe-core/src/main/java/org/apache/doris/httpv2/restv2/ESCatalogAction.java deleted file mode 100644 index 6ede29fdd0bd41..00000000000000 --- a/fe/fe-core/src/main/java/org/apache/doris/httpv2/restv2/ESCatalogAction.java +++ /dev/null @@ -1,104 +0,0 @@ -// Licensed to the Apache Software Foundation (ASF) under one -// or more contributor license agreements. See the NOTICE file -// distributed with this work for additional information -// regarding copyright ownership. The ASF licenses this file -// to you under the Apache License, Version 2.0 (the -// "License"); you may not use this file except in compliance -// with the License. You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, -// software distributed under the License is distributed on an -// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -// KIND, either express or implied. See the License for the -// specific language governing permissions and limitations -// under the License. - -package org.apache.doris.httpv2.restv2; - -import org.apache.doris.catalog.Env; -import org.apache.doris.common.Config; -import org.apache.doris.common.util.JsonUtil; -import org.apache.doris.datasource.CatalogIf; -import org.apache.doris.datasource.es.EsExternalCatalog; -import org.apache.doris.httpv2.entity.ResponseEntityBuilder; -import org.apache.doris.httpv2.rest.RestBaseController; - -import com.fasterxml.jackson.databind.node.ObjectNode; -import com.google.common.collect.Maps; -import jakarta.servlet.http.HttpServletRequest; -import jakarta.servlet.http.HttpServletResponse; -import org.apache.logging.log4j.LogManager; -import org.apache.logging.log4j.Logger; -import org.springframework.web.bind.annotation.RequestMapping; -import org.springframework.web.bind.annotation.RequestMethod; -import org.springframework.web.bind.annotation.RestController; - -import java.io.BufferedReader; -import java.io.IOException; -import java.util.Map; -import java.util.function.BiFunction; -import java.util.stream.Collectors; - -@RestController -@RequestMapping("/rest/v2/api/es_catalog") -public class ESCatalogAction extends RestBaseController { - - private static final Logger LOG = LogManager.getLogger(ESCatalogAction.class); - private static final String CATALOG = "catalog"; - private static final String TABLE = "table"; - - private Object handleRequest(HttpServletRequest request, HttpServletResponse response, - BiFunction action) { - if (Config.enable_all_http_auth) { - executeCheckPassword(request, response); - } - - if (needRedirect(request.getScheme())) { - return redirectToHttps(request); - } - - Map resultMap = Maps.newHashMap(); - Env env = Env.getCurrentEnv(); - String catalogName = request.getParameter(CATALOG); - String tableName = request.getParameter(TABLE); - CatalogIf catalog = env.getCatalogMgr().getCatalog(catalogName); - if (!(catalog instanceof EsExternalCatalog)) { - return ResponseEntityBuilder.badRequest("unknown ES Catalog: " + catalogName); - } - EsExternalCatalog esExternalCatalog = (EsExternalCatalog) catalog; - esExternalCatalog.makeSureInitialized(); - String result = action.apply(esExternalCatalog, tableName); - ObjectNode jsonResult = JsonUtil.parseObject(result); - - resultMap.put("catalog", catalogName); - resultMap.put("table", tableName); - resultMap.put("result", jsonResult); - - return ResponseEntityBuilder.ok(resultMap); - } - - @RequestMapping(path = "/get_mapping", method = RequestMethod.GET) - public Object getMapping(HttpServletRequest request, HttpServletResponse response) { - return handleRequest(request, response, (esExternalCatalog, tableName) -> - esExternalCatalog.getEsRestClient().getMapping(tableName)); - } - - @RequestMapping(path = "/search", method = RequestMethod.POST) - public Object search(HttpServletRequest request, HttpServletResponse response) { - String body; - try { - body = getRequestBody(request); - } catch (IOException e) { - return ResponseEntityBuilder.okWithCommonError(e.getMessage()); - } - return handleRequest(request, response, (esExternalCatalog, tableName) -> - esExternalCatalog.getEsRestClient().searchIndex(tableName, body)); - } - - private String getRequestBody(HttpServletRequest request) throws IOException { - BufferedReader reader = request.getReader(); - return reader.lines().collect(Collectors.joining(System.lineSeparator())); - } -} diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/glue/translator/PhysicalPlanTranslator.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/glue/translator/PhysicalPlanTranslator.java index 4100fab5c23327..24beb9259a5a45 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/glue/translator/PhysicalPlanTranslator.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/glue/translator/PhysicalPlanTranslator.java @@ -48,8 +48,6 @@ import org.apache.doris.datasource.doris.RemoteDorisExternalTable; import org.apache.doris.datasource.doris.RemoteOlapTable; import org.apache.doris.datasource.doris.source.RemoteDorisScanNode; -import org.apache.doris.datasource.es.EsExternalTable; -import org.apache.doris.datasource.es.source.EsScanNode; import org.apache.doris.datasource.hive.HMSExternalTable; import org.apache.doris.datasource.hive.HMSExternalTable.DLAType; import org.apache.doris.datasource.hive.source.HiveScanNode; @@ -65,6 +63,9 @@ import org.apache.doris.datasource.maxcompute.source.MaxComputeScanNode; import org.apache.doris.datasource.odbc.source.OdbcScanNode; import org.apache.doris.datasource.paimon.source.PaimonScanNode; +import org.apache.doris.datasource.ExternalCatalog; +import org.apache.doris.datasource.spi.CatalogPluginLoader; +import org.apache.doris.datasource.spi.ConnectorMetadata; import org.apache.doris.datasource.trinoconnector.TrinoConnectorExternalTable; import org.apache.doris.datasource.trinoconnector.source.TrinoConnectorScanNode; import org.apache.doris.fs.DirectoryLister; @@ -212,12 +213,14 @@ import org.apache.doris.planner.PartitionSortNode; import org.apache.doris.planner.PlanFragment; import org.apache.doris.planner.PlanNode; +import org.apache.doris.planner.PlanNodeId; import org.apache.doris.planner.RecursiveCteNode; import org.apache.doris.planner.RecursiveCteScanNode; import org.apache.doris.planner.RemoteOlapTableSink; import org.apache.doris.planner.RepeatNode; import org.apache.doris.planner.ResultFileSink; import org.apache.doris.planner.ResultSink; +import org.apache.doris.planner.ScanContext; import org.apache.doris.planner.ScanNode; import org.apache.doris.planner.SchemaScanNode; import org.apache.doris.planner.SelectNode; @@ -755,15 +758,45 @@ public PlanFragment visitPhysicalEsScan(PhysicalEsScan esScan, PlanTranslatorCon List slots = esScan.getOutput(); TableIf table = esScan.getTable(); TupleDescriptor tupleDescriptor = generateTupleDesc(slots, table, context); - EsScanNode esScanNode = new EsScanNode(context.nextPlanNodeId(), tupleDescriptor, - table instanceof EsExternalTable, context.getScanContext()); - esScanNode.setNereidsId(esScan.getId()); - context.getNereidsIdToPlanNodeIdMap().put(esScan.getId(), esScanNode.getId()); - Utils.execWithUncheckedException(esScanNode::init); - context.addScanNode(esScanNode, esScan); + // Use ConnectorMetadata to create ES ScanNode + ScanNode scanNode; + if (table instanceof ExternalTable) { + ExternalTable extTable = (ExternalTable) table; + ExternalCatalog catalog = (ExternalCatalog) extTable.getCatalog(); + catalog.makeSureInitialized(); + ConnectorMetadata metadata = catalog.getConnectorMetadata(); + if (metadata != null) { + scanNode = metadata.getScanNodeProvider().createScanNode( + context.nextPlanNodeId(), tupleDescriptor, extTable, context.getScanContext()); + } else { + throw new RuntimeException("ConnectorMetadata not initialized for catalog: " + catalog.getName()); + } + } else { + // Temporary: for internal EsTable, load EsScanNode from connector plugin via reflection. + // TODO: remove this when internal EsTable support is dropped. + ClassLoader cl = CatalogPluginLoader.getPluginClassLoader("es"); + if (cl == null) { + throw new RuntimeException("ES connector plugin is not loaded. " + + "Please ensure the ES connector JAR is deployed in lib/connectors/es/"); + } + try { + Class clazz = cl.loadClass("org.apache.doris.datasource.es.source.EsScanNode"); + PlanNodeId nodeId = context.nextPlanNodeId(); + ScanContext scanCtx = context.getScanContext(); + scanNode = (ScanNode) clazz.getConstructor( + PlanNodeId.class, TupleDescriptor.class, boolean.class, ScanContext.class) + .newInstance(nodeId, tupleDescriptor, false, scanCtx); + } catch (Exception e) { + throw new RuntimeException("Failed to create EsScanNode via reflection", e); + } + } + scanNode.setNereidsId(esScan.getId()); + context.getNereidsIdToPlanNodeIdMap().put(esScan.getId(), scanNode.getId()); + Utils.execWithUncheckedException(scanNode::init); + context.addScanNode(scanNode, esScan); context.getRuntimeTranslator().ifPresent( runtimeFilterGenerator -> runtimeFilterGenerator.getContext().getTargetListByScan(esScan).forEach( - expr -> runtimeFilterGenerator.translateRuntimeFilterTarget(expr, esScanNode, context) + expr -> runtimeFilterGenerator.translateRuntimeFilterTarget(expr, scanNode, context) ) ); // translate rf v2 target @@ -771,12 +804,12 @@ public PlanFragment visitPhysicalEsScan(PhysicalEsScan esScan, PlanTranslatorCon .getRuntimeFilterV2ByTargetPlan(esScan); for (RuntimeFilterV2 rfV2 : rfV2s) { Expr targetExpr = rfV2.getTargetExpression().accept(ExpressionTranslator.INSTANCE, context); - rfV2.setLegacyTargetNode(esScanNode); + rfV2.setLegacyTargetNode(scanNode); rfV2.setLegacyTargetExpr(targetExpr); } - context.getTopnFilterContext().translateTarget(esScan, esScanNode, context); + context.getTopnFilterContext().translateTarget(esScan, scanNode, context); DataPartition dataPartition = DataPartition.RANDOM; - PlanFragment planFragment = new PlanFragment(context.nextFragmentId(), esScanNode, dataPartition); + PlanFragment planFragment = new PlanFragment(context.nextFragmentId(), scanNode, dataPartition); context.addPlanFragment(planFragment); updateLegacyPlanIdToPhysicalPlan(planFragment.getPlanRoot(), esScan); return planFragment; diff --git a/fe/fe-core/src/main/java/org/apache/doris/persist/gson/GsonUtils.java b/fe/fe-core/src/main/java/org/apache/doris/persist/gson/GsonUtils.java index 15250aecc8a471..97aa2322fa596a 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/persist/gson/GsonUtils.java +++ b/fe/fe-core/src/main/java/org/apache/doris/persist/gson/GsonUtils.java @@ -130,13 +130,11 @@ import org.apache.doris.common.io.Text; import org.apache.doris.common.util.RangeUtils; import org.apache.doris.datasource.CatalogIf; +import org.apache.doris.datasource.ExternalCatalog; import org.apache.doris.datasource.ExternalDatabase; import org.apache.doris.datasource.ExternalTable; import org.apache.doris.datasource.InternalCatalog; import org.apache.doris.datasource.doris.RemoteDorisExternalCatalog; -import org.apache.doris.datasource.es.EsExternalCatalog; -import org.apache.doris.datasource.es.EsExternalDatabase; -import org.apache.doris.datasource.es.EsExternalTable; import org.apache.doris.datasource.hive.HMSExternalCatalog; import org.apache.doris.datasource.hive.HMSExternalDatabase; import org.apache.doris.datasource.hive.HMSExternalTable; @@ -407,7 +405,9 @@ public class GsonUtils { dsTypeAdapterFactory = RuntimeTypeAdapterFactory.of(CatalogIf.class, "clazz") .registerSubtype(CloudInternalCatalog.class, CloudInternalCatalog.class.getSimpleName()) .registerSubtype(HMSExternalCatalog.class, HMSExternalCatalog.class.getSimpleName()) - .registerSubtype(EsExternalCatalog.class, EsExternalCatalog.class.getSimpleName()) + .registerSubtype(ExternalCatalog.class, ExternalCatalog.class.getSimpleName()) + // ES: old EditLog "EsExternalCatalog" maps to ExternalCatalog (ES migrated to connector plugin) + .registerCompatibleSubtype(ExternalCatalog.class, "EsExternalCatalog") .registerSubtype(JdbcExternalCatalog.class, JdbcExternalCatalog.class.getSimpleName()) .registerSubtype(IcebergExternalCatalog.class, IcebergExternalCatalog.class.getSimpleName()) .registerSubtype(IcebergHMSExternalCatalog.class, IcebergHMSExternalCatalog.class.getSimpleName()) @@ -461,7 +461,8 @@ public class GsonUtils { private static RuntimeTypeAdapterFactory dbTypeAdapterFactory = RuntimeTypeAdapterFactory.of( DatabaseIf.class, "clazz") .registerSubtype(ExternalDatabase.class, ExternalDatabase.class.getSimpleName()) - .registerSubtype(EsExternalDatabase.class, EsExternalDatabase.class.getSimpleName()) + // ES: old EditLog "EsExternalDatabase" maps to ExternalDatabase (ES migrated to connector plugin) + .registerCompatibleSubtype(ExternalDatabase.class, "EsExternalDatabase") .registerSubtype(HMSExternalDatabase.class, HMSExternalDatabase.class.getSimpleName()) .registerSubtype(JdbcExternalDatabase.class, JdbcExternalDatabase.class.getSimpleName()) .registerSubtype(IcebergExternalDatabase.class, IcebergExternalDatabase.class.getSimpleName()) @@ -475,7 +476,8 @@ public class GsonUtils { private static RuntimeTypeAdapterFactory tblTypeAdapterFactory = RuntimeTypeAdapterFactory.of( TableIf.class, "clazz").registerSubtype(ExternalTable.class, ExternalTable.class.getSimpleName()) - .registerSubtype(EsExternalTable.class, EsExternalTable.class.getSimpleName()) + // ES: old EditLog "EsExternalTable" maps to ExternalTable (ES migrated to connector plugin) + .registerCompatibleSubtype(ExternalTable.class, "EsExternalTable") .registerSubtype(OlapTable.class, OlapTable.class.getSimpleName()) .registerSubtype(HMSExternalTable.class, HMSExternalTable.class.getSimpleName()) .registerSubtype(JdbcExternalTable.class, JdbcExternalTable.class.getSimpleName()) diff --git a/fe/pom.xml b/fe/pom.xml index eed81db5955626..cd05f832aaf09c 100644 --- a/fe/pom.xml +++ b/fe/pom.xml @@ -257,6 +257,7 @@ under the License. hive-udf be-java-extensions fe-authentication + fe-connectors/connector-es fe-thrift fe-type diff --git a/regression-test/suites/external_table_p0/es/test_es_flatten_type.groovy b/regression-test/suites/external_table_p0/es/test_es_flatten_type.groovy index 073f34f34293e3..2dca30a442497d 100644 --- a/regression-test/suites/external_table_p0/es/test_es_flatten_type.groovy +++ b/regression-test/suites/external_table_p0/es/test_es_flatten_type.groovy @@ -21,7 +21,8 @@ suite("test_es_flatten_type", "p0,external") { String externalEnvIp = context.config.otherConfigs.get("externalEnvIp") String es_7_port = context.config.otherConfigs.get("es_7_port") - sql """drop catalog if exists test_es7_flatten_type;""" + sql """drop catalog if exists test_es_query_es7_false;""" + sql """drop catalog if exists test_es_query_es7_true;""" sql """create catalog test_es_query_es7_false properties( "type"="es",