From 308469bccc6f7aa3fb4d0b4edb469aa3147ce36c Mon Sep 17 00:00:00 2001 From: "jishiwen.jsw" Date: Sun, 9 Nov 2025 16:10:58 +0800 Subject: [PATCH 1/5] t# This is a combination of 2 commits. feat: add neo4j connector Signed-off-by: jishiwen.jsw --- .../geaflow-dsl-connector-neo4j/pom.xml | 75 +++ .../dsl/connector/neo4j/Neo4jConfigKeys.java | 109 +++++ .../dsl/connector/neo4j/Neo4jConstants.java | 37 ++ .../connector/neo4j/Neo4jTableConnector.java | 46 ++ .../dsl/connector/neo4j/Neo4jTableSink.java | 298 ++++++++++++ .../dsl/connector/neo4j/Neo4jTableSource.java | 299 ++++++++++++ ...e.geaflow.dsl.connector.api.TableConnector | 20 + .../connector/neo4j/Neo4jConfigKeysTest.java | 79 ++++ .../connector/neo4j/Neo4jIntegrationTest.java | 387 ++++++++++++++++ .../neo4j/Neo4jTableConnectorTest.java | 385 ++++++++++++++++ .../connector/neo4j/Neo4jTableSinkTest.java | 388 ++++++++++++++++ .../connector/neo4j/Neo4jTableSourceTest.java | 433 ++++++++++++++++++ .../geaflow-dsl/geaflow-dsl-connector/pom.xml | 1 + 13 files changed, 2557 insertions(+) create mode 100644 geaflow/geaflow-dsl/geaflow-dsl-connector/geaflow-dsl-connector-neo4j/pom.xml create mode 100644 geaflow/geaflow-dsl/geaflow-dsl-connector/geaflow-dsl-connector-neo4j/src/main/java/org/apache/geaflow/dsl/connector/neo4j/Neo4jConfigKeys.java create mode 100644 geaflow/geaflow-dsl/geaflow-dsl-connector/geaflow-dsl-connector-neo4j/src/main/java/org/apache/geaflow/dsl/connector/neo4j/Neo4jConstants.java create mode 100644 geaflow/geaflow-dsl/geaflow-dsl-connector/geaflow-dsl-connector-neo4j/src/main/java/org/apache/geaflow/dsl/connector/neo4j/Neo4jTableConnector.java create mode 100644 geaflow/geaflow-dsl/geaflow-dsl-connector/geaflow-dsl-connector-neo4j/src/main/java/org/apache/geaflow/dsl/connector/neo4j/Neo4jTableSink.java create mode 100644 geaflow/geaflow-dsl/geaflow-dsl-connector/geaflow-dsl-connector-neo4j/src/main/java/org/apache/geaflow/dsl/connector/neo4j/Neo4jTableSource.java create mode 100644 geaflow/geaflow-dsl/geaflow-dsl-connector/geaflow-dsl-connector-neo4j/src/main/resources/META-INF/services/org.apache.geaflow.dsl.connector.api.TableConnector create mode 100644 geaflow/geaflow-dsl/geaflow-dsl-connector/geaflow-dsl-connector-neo4j/src/test/java/org/apache/geaflow/dsl/connector/neo4j/Neo4jConfigKeysTest.java create mode 100644 geaflow/geaflow-dsl/geaflow-dsl-connector/geaflow-dsl-connector-neo4j/src/test/java/org/apache/geaflow/dsl/connector/neo4j/Neo4jIntegrationTest.java create mode 100644 geaflow/geaflow-dsl/geaflow-dsl-connector/geaflow-dsl-connector-neo4j/src/test/java/org/apache/geaflow/dsl/connector/neo4j/Neo4jTableConnectorTest.java create mode 100644 geaflow/geaflow-dsl/geaflow-dsl-connector/geaflow-dsl-connector-neo4j/src/test/java/org/apache/geaflow/dsl/connector/neo4j/Neo4jTableSinkTest.java create mode 100644 geaflow/geaflow-dsl/geaflow-dsl-connector/geaflow-dsl-connector-neo4j/src/test/java/org/apache/geaflow/dsl/connector/neo4j/Neo4jTableSourceTest.java diff --git a/geaflow/geaflow-dsl/geaflow-dsl-connector/geaflow-dsl-connector-neo4j/pom.xml b/geaflow/geaflow-dsl/geaflow-dsl-connector/geaflow-dsl-connector-neo4j/pom.xml new file mode 100644 index 000000000..2c8191bac --- /dev/null +++ b/geaflow/geaflow-dsl/geaflow-dsl-connector/geaflow-dsl-connector-neo4j/pom.xml @@ -0,0 +1,75 @@ + + + + + + org.apache.geaflow + geaflow-dsl-connector + 0.6.8-SNAPSHOT + + + 4.0.0 + + geaflow-dsl-connector-neo4j + + + 5.14.0 + 1.19.3 + + + + + org.apache.geaflow + geaflow-dsl-common + + + org.apache.geaflow + geaflow-dsl-connector-api + + + org.neo4j.driver + neo4j-java-driver + ${neo4j-java-driver.version} + + + + + org.testng + testng + ${testng.version} + test + + + org.testcontainers + neo4j + ${testcontainers.version} + test + + + org.testcontainers + testcontainers + ${testcontainers.version} + test + + + + diff --git a/geaflow/geaflow-dsl/geaflow-dsl-connector/geaflow-dsl-connector-neo4j/src/main/java/org/apache/geaflow/dsl/connector/neo4j/Neo4jConfigKeys.java b/geaflow/geaflow-dsl/geaflow-dsl-connector/geaflow-dsl-connector-neo4j/src/main/java/org/apache/geaflow/dsl/connector/neo4j/Neo4jConfigKeys.java new file mode 100644 index 000000000..6c755547d --- /dev/null +++ b/geaflow/geaflow-dsl/geaflow-dsl-connector/geaflow-dsl-connector-neo4j/src/main/java/org/apache/geaflow/dsl/connector/neo4j/Neo4jConfigKeys.java @@ -0,0 +1,109 @@ +/* + * 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.geaflow.dsl.connector.neo4j; + +import static org.apache.geaflow.dsl.connector.neo4j.Neo4jConstants.DEFAULT_BATCH_SIZE; +import static org.apache.geaflow.dsl.connector.neo4j.Neo4jConstants.DEFAULT_CONNECTION_ACQUISITION_TIMEOUT_MILLIS; +import static org.apache.geaflow.dsl.connector.neo4j.Neo4jConstants.DEFAULT_DATABASE; +import static org.apache.geaflow.dsl.connector.neo4j.Neo4jConstants.DEFAULT_MAX_CONNECTION_LIFETIME_MILLIS; +import static org.apache.geaflow.dsl.connector.neo4j.Neo4jConstants.DEFAULT_MAX_CONNECTION_POOL_SIZE; +import static org.apache.geaflow.dsl.connector.neo4j.Neo4jConstants.DEFAULT_NODE_LABEL; +import static org.apache.geaflow.dsl.connector.neo4j.Neo4jConstants.DEFAULT_RELATIONSHIP_TYPE; + +import org.apache.geaflow.common.config.ConfigKey; +import org.apache.geaflow.common.config.ConfigKeys; + +public class Neo4jConfigKeys { + + public static final ConfigKey GEAFLOW_DSL_NEO4J_URI = ConfigKeys + .key("geaflow.dsl.neo4j.uri") + .noDefaultValue() + .description("Neo4j database URI (e.g., bolt://localhost:7687)."); + + public static final ConfigKey GEAFLOW_DSL_NEO4J_USERNAME = ConfigKeys + .key("geaflow.dsl.neo4j.username") + .noDefaultValue() + .description("Neo4j database username."); + + public static final ConfigKey GEAFLOW_DSL_NEO4J_PASSWORD = ConfigKeys + .key("geaflow.dsl.neo4j.password") + .noDefaultValue() + .description("Neo4j database password."); + + public static final ConfigKey GEAFLOW_DSL_NEO4J_DATABASE = ConfigKeys + .key("geaflow.dsl.neo4j.database") + .defaultValue(DEFAULT_DATABASE) + .description("Neo4j database name."); + + public static final ConfigKey GEAFLOW_DSL_NEO4J_BATCH_SIZE = ConfigKeys + .key("geaflow.dsl.neo4j.batch.size") + .defaultValue(DEFAULT_BATCH_SIZE) + .description("Batch size for writing to Neo4j."); + + public static final ConfigKey GEAFLOW_DSL_NEO4J_MAX_CONNECTION_LIFETIME = ConfigKeys + .key("geaflow.dsl.neo4j.max.connection.lifetime.millis") + .defaultValue(DEFAULT_MAX_CONNECTION_LIFETIME_MILLIS) + .description("Maximum lifetime of a connection in milliseconds."); + + public static final ConfigKey GEAFLOW_DSL_NEO4J_MAX_CONNECTION_POOL_SIZE = ConfigKeys + .key("geaflow.dsl.neo4j.max.connection.pool.size") + .defaultValue(DEFAULT_MAX_CONNECTION_POOL_SIZE) + .description("Maximum size of the connection pool."); + + public static final ConfigKey GEAFLOW_DSL_NEO4J_CONNECTION_ACQUISITION_TIMEOUT = ConfigKeys + .key("geaflow.dsl.neo4j.connection.acquisition.timeout.millis") + .defaultValue(DEFAULT_CONNECTION_ACQUISITION_TIMEOUT_MILLIS) + .description("Timeout for acquiring a connection from the pool in milliseconds."); + + public static final ConfigKey GEAFLOW_DSL_NEO4J_QUERY = ConfigKeys + .key("geaflow.dsl.neo4j.query") + .noDefaultValue() + .description("Cypher query for reading data from Neo4j."); + + public static final ConfigKey GEAFLOW_DSL_NEO4J_NODE_LABEL = ConfigKeys + .key("geaflow.dsl.neo4j.node.label") + .defaultValue(DEFAULT_NODE_LABEL) + .description("Node label for writing nodes to Neo4j."); + + public static final ConfigKey GEAFLOW_DSL_NEO4J_RELATIONSHIP_TYPE = ConfigKeys + .key("geaflow.dsl.neo4j.relationship.type") + .defaultValue(DEFAULT_RELATIONSHIP_TYPE) + .description("Relationship type for writing relationships to Neo4j."); + + public static final ConfigKey GEAFLOW_DSL_NEO4J_WRITE_MODE = ConfigKeys + .key("geaflow.dsl.neo4j.write.mode") + .defaultValue("node") + .description("Write mode: 'node' for writing nodes, 'relationship' for writing relationships."); + + public static final ConfigKey GEAFLOW_DSL_NEO4J_NODE_ID_FIELD = ConfigKeys + .key("geaflow.dsl.neo4j.node.id.field") + .noDefaultValue() + .description("Field name to use as node ID."); + + public static final ConfigKey GEAFLOW_DSL_NEO4J_RELATIONSHIP_SOURCE_FIELD = ConfigKeys + .key("geaflow.dsl.neo4j.relationship.source.field") + .noDefaultValue() + .description("Field name for relationship source node ID."); + + public static final ConfigKey GEAFLOW_DSL_NEO4J_RELATIONSHIP_TARGET_FIELD = ConfigKeys + .key("geaflow.dsl.neo4j.relationship.target.field") + .noDefaultValue() + .description("Field name for relationship target node ID."); +} diff --git a/geaflow/geaflow-dsl/geaflow-dsl-connector/geaflow-dsl-connector-neo4j/src/main/java/org/apache/geaflow/dsl/connector/neo4j/Neo4jConstants.java b/geaflow/geaflow-dsl/geaflow-dsl-connector/geaflow-dsl-connector-neo4j/src/main/java/org/apache/geaflow/dsl/connector/neo4j/Neo4jConstants.java new file mode 100644 index 000000000..fe748def2 --- /dev/null +++ b/geaflow/geaflow-dsl/geaflow-dsl-connector/geaflow-dsl-connector-neo4j/src/main/java/org/apache/geaflow/dsl/connector/neo4j/Neo4jConstants.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.geaflow.dsl.connector.neo4j; + +public class Neo4jConstants { + + public static final String DEFAULT_DATABASE = "neo4j"; + + public static final int DEFAULT_BATCH_SIZE = 1000; + + public static final long DEFAULT_MAX_CONNECTION_LIFETIME_MILLIS = 3600000L; // 1 hour + + public static final int DEFAULT_MAX_CONNECTION_POOL_SIZE = 100; + + public static final long DEFAULT_CONNECTION_ACQUISITION_TIMEOUT_MILLIS = 60000L; // 1 minute + + public static final String DEFAULT_NODE_LABEL = "Node"; + + public static final String DEFAULT_RELATIONSHIP_TYPE = "RELATES_TO"; +} diff --git a/geaflow/geaflow-dsl/geaflow-dsl-connector/geaflow-dsl-connector-neo4j/src/main/java/org/apache/geaflow/dsl/connector/neo4j/Neo4jTableConnector.java b/geaflow/geaflow-dsl/geaflow-dsl-connector/geaflow-dsl-connector-neo4j/src/main/java/org/apache/geaflow/dsl/connector/neo4j/Neo4jTableConnector.java new file mode 100644 index 000000000..179298d23 --- /dev/null +++ b/geaflow/geaflow-dsl/geaflow-dsl-connector/geaflow-dsl-connector-neo4j/src/main/java/org/apache/geaflow/dsl/connector/neo4j/Neo4jTableConnector.java @@ -0,0 +1,46 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.geaflow.dsl.connector.neo4j; + +import org.apache.geaflow.common.config.Configuration; +import org.apache.geaflow.dsl.connector.api.TableReadableConnector; +import org.apache.geaflow.dsl.connector.api.TableSink; +import org.apache.geaflow.dsl.connector.api.TableSource; +import org.apache.geaflow.dsl.connector.api.TableWritableConnector; + +public class Neo4jTableConnector implements TableReadableConnector, TableWritableConnector { + + public static final String TYPE = "Neo4j"; + + @Override + public String getType() { + return TYPE; + } + + @Override + public TableSource createSource(Configuration conf) { + return new Neo4jTableSource(); + } + + @Override + public TableSink createSink(Configuration conf) { + return new Neo4jTableSink(); + } +} diff --git a/geaflow/geaflow-dsl/geaflow-dsl-connector/geaflow-dsl-connector-neo4j/src/main/java/org/apache/geaflow/dsl/connector/neo4j/Neo4jTableSink.java b/geaflow/geaflow-dsl/geaflow-dsl-connector/geaflow-dsl-connector-neo4j/src/main/java/org/apache/geaflow/dsl/connector/neo4j/Neo4jTableSink.java new file mode 100644 index 000000000..24e7aeaa8 --- /dev/null +++ b/geaflow/geaflow-dsl/geaflow-dsl-connector/geaflow-dsl-connector-neo4j/src/main/java/org/apache/geaflow/dsl/connector/neo4j/Neo4jTableSink.java @@ -0,0 +1,298 @@ +/* + * 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.geaflow.dsl.connector.neo4j; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.concurrent.TimeUnit; +import org.apache.geaflow.api.context.RuntimeContext; +import org.apache.geaflow.common.config.Configuration; +import org.apache.geaflow.common.type.IType; +import org.apache.geaflow.dsl.common.data.Row; +import org.apache.geaflow.dsl.common.exception.GeaFlowDSLException; +import org.apache.geaflow.dsl.common.types.StructType; +import org.apache.geaflow.dsl.connector.api.TableSink; +import org.neo4j.driver.AuthTokens; +import org.neo4j.driver.Config; +import org.neo4j.driver.Driver; +import org.neo4j.driver.GraphDatabase; +import org.neo4j.driver.Session; +import org.neo4j.driver.SessionConfig; +import org.neo4j.driver.Transaction; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class Neo4jTableSink implements TableSink { + + private static final Logger LOGGER = LoggerFactory.getLogger(Neo4jTableSink.class); + + private StructType schema; + private String uri; + private String username; + private String password; + private String database; + private int batchSize; + private String writeMode; + private String nodeLabel; + private String relationshipType; + private String nodeIdField; + private String relationshipSourceField; + private String relationshipTargetField; + private long maxConnectionLifetime; + private int maxConnectionPoolSize; + private long connectionAcquisitionTimeout; + + private Driver driver; + private Session session; + private Transaction transaction; + private List batch; + + @Override + public void init(Configuration tableConf, StructType schema) { + LOGGER.info("Init Neo4j sink with config: {}, \n schema: {}", tableConf, schema); + this.schema = schema; + + this.uri = tableConf.getString(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_URI); + this.username = tableConf.getString(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_USERNAME); + this.password = tableConf.getString(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_PASSWORD); + this.database = tableConf.getString(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_DATABASE); + this.batchSize = tableConf.getInteger(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_BATCH_SIZE); + this.writeMode = tableConf.getString(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_WRITE_MODE); + this.nodeLabel = tableConf.getString(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_NODE_LABEL); + this.relationshipType = tableConf.getString(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_RELATIONSHIP_TYPE); + this.nodeIdField = tableConf.getString(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_NODE_ID_FIELD); + this.relationshipSourceField = tableConf.getString(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_RELATIONSHIP_SOURCE_FIELD); + this.relationshipTargetField = tableConf.getString(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_RELATIONSHIP_TARGET_FIELD); + this.maxConnectionLifetime = tableConf.getLong(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_MAX_CONNECTION_LIFETIME); + this.maxConnectionPoolSize = tableConf.getInteger(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_MAX_CONNECTION_POOL_SIZE); + this.connectionAcquisitionTimeout = tableConf.getLong(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_CONNECTION_ACQUISITION_TIMEOUT); + + validateConfig(); + this.batch = new ArrayList<>(batchSize); + } + + private void validateConfig() { + if (uri == null || uri.isEmpty()) { + throw new GeaFlowDSLException("Neo4j URI must be specified"); + } + if (username == null || username.isEmpty()) { + throw new GeaFlowDSLException("Neo4j username must be specified"); + } + if (password == null || password.isEmpty()) { + throw new GeaFlowDSLException("Neo4j password must be specified"); + } + if ("node".equals(writeMode)) { + if (nodeIdField == null || nodeIdField.isEmpty()) { + throw new GeaFlowDSLException("Node ID field must be specified for node write mode"); + } + } else if ("relationship".equals(writeMode)) { + if (relationshipSourceField == null || relationshipSourceField.isEmpty() + || relationshipTargetField == null || relationshipTargetField.isEmpty()) { + throw new GeaFlowDSLException("Relationship source and target fields must be specified for relationship write mode"); + } + } else { + throw new GeaFlowDSLException("Invalid write mode: " + writeMode + ". Must be 'node' or 'relationship'"); + } + } + + @Override + public void open(RuntimeContext context) { + try { + Config config = Config.builder() + .withMaxConnectionLifetime(maxConnectionLifetime, TimeUnit.MILLISECONDS) + .withMaxConnectionPoolSize(maxConnectionPoolSize) + .withConnectionAcquisitionTimeout(connectionAcquisitionTimeout, TimeUnit.MILLISECONDS) + .build(); + + this.driver = GraphDatabase.driver(uri, AuthTokens.basic(username, password), config); + + SessionConfig sessionConfig = SessionConfig.builder() + .withDatabase(database) + .build(); + + this.session = driver.session(sessionConfig); + this.transaction = session.beginTransaction(); + + LOGGER.info("Neo4j connection established successfully"); + } catch (Exception e) { + throw new GeaFlowDSLException("Failed to connect to Neo4j: " + e.getMessage(), e); + } + } + + @Override + public void write(Row row) throws IOException { + batch.add(row); + if (batch.size() >= batchSize) { + flush(); + } + } + + @Override + public void finish() throws IOException { + if (!batch.isEmpty()) { + flush(); + } + try { + if (transaction != null) { + transaction.commit(); + transaction.close(); + transaction = null; + } + } catch (Exception e) { + LOGGER.error("Failed to commit transaction", e); + try { + if (transaction != null) { + transaction.rollback(); + } + } catch (Exception ex) { + throw new GeaFlowDSLException("Failed to rollback transaction", ex); + } + throw new GeaFlowDSLException("Failed to finish writing to Neo4j", e); + } + } + + @Override + public void close() { + try { + if (transaction != null) { + transaction.close(); + transaction = null; + } + if (session != null) { + session.close(); + session = null; + } + if (driver != null) { + driver.close(); + driver = null; + } + LOGGER.info("Neo4j connection closed successfully"); + } catch (Exception e) { + throw new GeaFlowDSLException("Failed to close Neo4j connection", e); + } + } + + private void flush() { + if (batch.isEmpty()) { + return; + } + + try { + if ("node".equals(writeMode)) { + writeNodes(); + } else { + writeRelationships(); + } + batch.clear(); + } catch (Exception e) { + throw new GeaFlowDSLException("Failed to flush batch to Neo4j", e); + } + } + + private void writeNodes() { + List fieldNames = schema.getFieldNames(); + IType[] types = schema.getTypes(); + + int nodeIdIndex = fieldNames.indexOf(nodeIdField); + if (nodeIdIndex == -1) { + throw new GeaFlowDSLException("Node ID field not found in schema: " + nodeIdField); + } + + for (Row row : batch) { + Map properties = new HashMap<>(); + for (int i = 0; i < fieldNames.size(); i++) { + if (i == nodeIdIndex) { + continue; // Skip ID field, it will be used as node ID + } + Object value = row.getField(i, types[i]); + if (value != null) { + properties.put(fieldNames.get(i), value); + } + } + + Object nodeId = row.getField(nodeIdIndex, types[nodeIdIndex]); + if (nodeId == null) { + throw new GeaFlowDSLException("Node ID cannot be null"); + } + + String cypher = String.format( + "MERGE (n:%s {id: $id}) SET n += $properties", + nodeLabel + ); + + Map parameters = new HashMap<>(); + parameters.put("id", nodeId); + parameters.put("properties", properties); + + transaction.run(cypher, parameters); + } + } + + private void writeRelationships() { + List fieldNames = schema.getFieldNames(); + IType[] types = schema.getTypes(); + + int sourceIndex = fieldNames.indexOf(relationshipSourceField); + int targetIndex = fieldNames.indexOf(relationshipTargetField); + + if (sourceIndex == -1) { + throw new GeaFlowDSLException("Relationship source field not found in schema: " + relationshipSourceField); + } + if (targetIndex == -1) { + throw new GeaFlowDSLException("Relationship target field not found in schema: " + relationshipTargetField); + } + + for (Row row : batch) { + Object sourceId = row.getField(sourceIndex, types[sourceIndex]); + Object targetId = row.getField(targetIndex, types[targetIndex]); + + if (sourceId == null || targetId == null) { + throw new GeaFlowDSLException("Relationship source and target IDs cannot be null"); + } + + Map properties = new HashMap<>(); + for (int i = 0; i < fieldNames.size(); i++) { + if (i == sourceIndex || i == targetIndex) { + continue; // Skip source and target fields + } + Object value = row.getField(i, types[i]); + if (value != null) { + properties.put(fieldNames.get(i), value); + } + } + + final String cypher = String.format( + "MATCH (a {id: $sourceId}), (b {id: $targetId}) " + + "MERGE (a)-[r:%s]->(b) SET r += $properties", + relationshipType + ); + + Map parameters = new HashMap<>(); + parameters.put("sourceId", sourceId); + parameters.put("targetId", targetId); + parameters.put("properties", properties); + + transaction.run(cypher, parameters); + } + } +} diff --git a/geaflow/geaflow-dsl/geaflow-dsl-connector/geaflow-dsl-connector-neo4j/src/main/java/org/apache/geaflow/dsl/connector/neo4j/Neo4jTableSource.java b/geaflow/geaflow-dsl/geaflow-dsl-connector/geaflow-dsl-connector-neo4j/src/main/java/org/apache/geaflow/dsl/connector/neo4j/Neo4jTableSource.java new file mode 100644 index 000000000..787090015 --- /dev/null +++ b/geaflow/geaflow-dsl/geaflow-dsl-connector/geaflow-dsl-connector-neo4j/src/main/java/org/apache/geaflow/dsl/connector/neo4j/Neo4jTableSource.java @@ -0,0 +1,299 @@ +/* + * 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.geaflow.dsl.connector.neo4j; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Objects; +import java.util.Optional; +import java.util.concurrent.TimeUnit; +import org.apache.geaflow.api.context.RuntimeContext; +import org.apache.geaflow.api.window.WindowType; +import org.apache.geaflow.common.config.Configuration; +import org.apache.geaflow.dsl.common.data.Row; +import org.apache.geaflow.dsl.common.data.impl.ObjectRow; +import org.apache.geaflow.dsl.common.exception.GeaFlowDSLException; +import org.apache.geaflow.dsl.common.types.StructType; +import org.apache.geaflow.dsl.common.types.TableSchema; +import org.apache.geaflow.dsl.connector.api.FetchData; +import org.apache.geaflow.dsl.connector.api.Offset; +import org.apache.geaflow.dsl.connector.api.Partition; +import org.apache.geaflow.dsl.connector.api.TableSource; +import org.apache.geaflow.dsl.connector.api.serde.DeserializerFactory; +import org.apache.geaflow.dsl.connector.api.serde.TableDeserializer; +import org.apache.geaflow.dsl.connector.api.window.FetchWindow; +import org.neo4j.driver.AuthTokens; +import org.neo4j.driver.Config; +import org.neo4j.driver.Driver; +import org.neo4j.driver.GraphDatabase; +import org.neo4j.driver.Record; +import org.neo4j.driver.Result; +import org.neo4j.driver.Session; +import org.neo4j.driver.SessionConfig; +import org.neo4j.driver.Value; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class Neo4jTableSource implements TableSource { + + private static final Logger LOGGER = LoggerFactory.getLogger(Neo4jTableSource.class); + + private Configuration tableConf; + private StructType schema; + private String uri; + private String username; + private String password; + private String database; + private String cypherQuery; + private long maxConnectionLifetime; + private int maxConnectionPoolSize; + private long connectionAcquisitionTimeout; + + private Driver driver; + private Map partitionSessionMap = new HashMap<>(); + + @Override + public void init(Configuration tableConf, TableSchema tableSchema) { + LOGGER.info("Init Neo4j source with config: {}, \n schema: {}", tableConf, tableSchema); + this.tableConf = tableConf; + this.schema = tableSchema; + + this.uri = tableConf.getString(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_URI); + this.username = tableConf.getString(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_USERNAME); + this.password = tableConf.getString(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_PASSWORD); + this.database = tableConf.getString(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_DATABASE); + this.cypherQuery = tableConf.getString(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_QUERY); + this.maxConnectionLifetime = tableConf.getLong(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_MAX_CONNECTION_LIFETIME); + this.maxConnectionPoolSize = tableConf.getInteger(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_MAX_CONNECTION_POOL_SIZE); + this.connectionAcquisitionTimeout = tableConf.getLong(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_CONNECTION_ACQUISITION_TIMEOUT); + + if (cypherQuery == null || cypherQuery.isEmpty()) { + throw new GeaFlowDSLException("Neo4j query must be specified"); + } + } + + @Override + public void open(RuntimeContext context) { + try { + Config config = Config.builder() + .withMaxConnectionLifetime(maxConnectionLifetime, TimeUnit.MILLISECONDS) + .withMaxConnectionPoolSize(maxConnectionPoolSize) + .withConnectionAcquisitionTimeout(connectionAcquisitionTimeout, TimeUnit.MILLISECONDS) + .build(); + + this.driver = GraphDatabase.driver(uri, AuthTokens.basic(username, password), config); + LOGGER.info("Neo4j driver created successfully"); + } catch (Exception e) { + throw new GeaFlowDSLException("Failed to create Neo4j driver: " + e.getMessage(), e); + } + } + + @Override + public List listPartitions() { + // Neo4j doesn't have native partitioning like JDBC + // For simplicity, we return a single partition + return Collections.singletonList(new Neo4jPartition(cypherQuery)); + } + + @Override + public List listPartitions(int parallelism) { + return listPartitions(); + } + + @Override + public TableDeserializer getDeserializer(Configuration conf) { + return DeserializerFactory.loadRowTableDeserializer(); + } + + @Override + public FetchData fetch(Partition partition, Optional startOffset, + FetchWindow windowInfo) throws IOException { + if (!(windowInfo.getType() == WindowType.SIZE_TUMBLING_WINDOW + || windowInfo.getType() == WindowType.ALL_WINDOW)) { + throw new GeaFlowDSLException("Not support window type: {}", windowInfo.getType()); + } + + Neo4jPartition neo4jPartition = (Neo4jPartition) partition; + Session session = partitionSessionMap.get(partition); + + if (session == null) { + SessionConfig sessionConfig = SessionConfig.builder() + .withDatabase(database) + .build(); + session = driver.session(sessionConfig); + partitionSessionMap.put(partition, session); + } + + long offset = startOffset.isPresent() ? startOffset.get().getOffset() : 0; + + List dataList = new ArrayList<>(); + try { + String query = neo4jPartition.getQuery(); + // Add SKIP and LIMIT to the query for pagination + String paginatedQuery = query + " SKIP $skip LIMIT $limit"; + + Map parameters = new HashMap<>(); + parameters.put("skip", offset); + parameters.put("limit", windowInfo.windowSize()); + + Result result = session.run(paginatedQuery, parameters); + + List fieldNames = schema.getFieldNames(); + + while (result.hasNext()) { + Record record = result.next(); + Object[] values = new Object[fieldNames.size()]; + + for (int i = 0; i < fieldNames.size(); i++) { + String fieldName = fieldNames.get(i); + if (record.containsKey(fieldName)) { + Value value = record.get(fieldName); + values[i] = convertNeo4jValue(value); + } else { + values[i] = null; + } + } + + dataList.add(ObjectRow.create(values)); + } + + } catch (Exception e) { + throw new GeaFlowDSLException("Failed to fetch data from Neo4j", e); + } + + Neo4jOffset nextOffset = new Neo4jOffset(offset + dataList.size()); + boolean isFinish = windowInfo.getType() == WindowType.ALL_WINDOW + || dataList.size() < windowInfo.windowSize(); + + return (FetchData) FetchData.createStreamFetch(dataList, nextOffset, isFinish); + } + + @Override + public void close() { + try { + for (Session session : partitionSessionMap.values()) { + if (session != null) { + session.close(); + } + } + partitionSessionMap.clear(); + + if (driver != null) { + driver.close(); + driver = null; + } + LOGGER.info("Neo4j connections closed successfully"); + } catch (Exception e) { + throw new GeaFlowDSLException("Failed to close Neo4j connections", e); + } + } + + private Object convertNeo4jValue(Value value) { + if (value.isNull()) { + return null; + } + + switch (value.type().name()) { + case "INTEGER": + return value.asLong(); + case "FLOAT": + return value.asDouble(); + case "STRING": + return value.asString(); + case "BOOLEAN": + return value.asBoolean(); + case "LIST": + return value.asList(); + case "MAP": + return value.asMap(); + case "NODE": + return value.asNode().asMap(); + case "RELATIONSHIP": + return value.asRelationship().asMap(); + case "PATH": + return value.asPath().toString(); + default: + return value.asObject(); + } + } + + public static class Neo4jPartition implements Partition { + + private final String query; + + public Neo4jPartition(String query) { + this.query = query; + } + + public String getQuery() { + return query; + } + + @Override + public String getName() { + return "neo4j-partition-" + query.hashCode(); + } + + @Override + public int hashCode() { + return Objects.hash(query); + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (!(o instanceof Neo4jPartition)) { + return false; + } + Neo4jPartition that = (Neo4jPartition) o; + return Objects.equals(query, that.query); + } + } + + public static class Neo4jOffset implements Offset { + + private final long offset; + + public Neo4jOffset(long offset) { + this.offset = offset; + } + + @Override + public String humanReadable() { + return String.valueOf(offset); + } + + @Override + public long getOffset() { + return offset; + } + + @Override + public boolean isTimestamp() { + return false; + } + } +} diff --git a/geaflow/geaflow-dsl/geaflow-dsl-connector/geaflow-dsl-connector-neo4j/src/main/resources/META-INF/services/org.apache.geaflow.dsl.connector.api.TableConnector b/geaflow/geaflow-dsl/geaflow-dsl-connector/geaflow-dsl-connector-neo4j/src/main/resources/META-INF/services/org.apache.geaflow.dsl.connector.api.TableConnector new file mode 100644 index 000000000..42a584079 --- /dev/null +++ b/geaflow/geaflow-dsl/geaflow-dsl-connector/geaflow-dsl-connector-neo4j/src/main/resources/META-INF/services/org.apache.geaflow.dsl.connector.api.TableConnector @@ -0,0 +1,20 @@ +# +# 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.geaflow.dsl.connector.neo4j.Neo4jTableConnector diff --git a/geaflow/geaflow-dsl/geaflow-dsl-connector/geaflow-dsl-connector-neo4j/src/test/java/org/apache/geaflow/dsl/connector/neo4j/Neo4jConfigKeysTest.java b/geaflow/geaflow-dsl/geaflow-dsl-connector/geaflow-dsl-connector-neo4j/src/test/java/org/apache/geaflow/dsl/connector/neo4j/Neo4jConfigKeysTest.java new file mode 100644 index 000000000..c93dd546d --- /dev/null +++ b/geaflow/geaflow-dsl/geaflow-dsl-connector/geaflow-dsl-connector-neo4j/src/test/java/org/apache/geaflow/dsl/connector/neo4j/Neo4jConfigKeysTest.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.geaflow.dsl.connector.neo4j; + +import org.testng.Assert; +import org.testng.annotations.Test; + +public class Neo4jConfigKeysTest { + + @Test + public void testDefaultValues() { + Assert.assertEquals(Neo4jConstants.DEFAULT_DATABASE, "neo4j"); + Assert.assertEquals(Neo4jConstants.DEFAULT_BATCH_SIZE, 1000); + Assert.assertEquals(Neo4jConstants.DEFAULT_MAX_CONNECTION_LIFETIME_MILLIS, 3600000L); + Assert.assertEquals(Neo4jConstants.DEFAULT_MAX_CONNECTION_POOL_SIZE, 100); + Assert.assertEquals(Neo4jConstants.DEFAULT_CONNECTION_ACQUISITION_TIMEOUT_MILLIS, 60000L); + Assert.assertEquals(Neo4jConstants.DEFAULT_NODE_LABEL, "Node"); + Assert.assertEquals(Neo4jConstants.DEFAULT_RELATIONSHIP_TYPE, "RELATES_TO"); + } + + @Test + public void testConfigKeyNames() { + Assert.assertEquals(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_URI.getKey(), + "geaflow.dsl.neo4j.uri"); + Assert.assertEquals(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_USERNAME.getKey(), + "geaflow.dsl.neo4j.username"); + Assert.assertEquals(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_PASSWORD.getKey(), + "geaflow.dsl.neo4j.password"); + Assert.assertEquals(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_DATABASE.getKey(), + "geaflow.dsl.neo4j.database"); + Assert.assertEquals(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_BATCH_SIZE.getKey(), + "geaflow.dsl.neo4j.batch.size"); + Assert.assertEquals(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_QUERY.getKey(), + "geaflow.dsl.neo4j.query"); + Assert.assertEquals(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_NODE_LABEL.getKey(), + "geaflow.dsl.neo4j.node.label"); + Assert.assertEquals(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_RELATIONSHIP_TYPE.getKey(), + "geaflow.dsl.neo4j.relationship.type"); + Assert.assertEquals(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_WRITE_MODE.getKey(), + "geaflow.dsl.neo4j.write.mode"); + Assert.assertEquals(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_NODE_ID_FIELD.getKey(), + "geaflow.dsl.neo4j.node.id.field"); + Assert.assertEquals(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_RELATIONSHIP_SOURCE_FIELD.getKey(), + "geaflow.dsl.neo4j.relationship.source.field"); + Assert.assertEquals(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_RELATIONSHIP_TARGET_FIELD.getKey(), + "geaflow.dsl.neo4j.relationship.target.field"); + } + + @Test + public void testConfigKeyDefaults() { + Assert.assertEquals(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_DATABASE.getDefaultValue(), + Neo4jConstants.DEFAULT_DATABASE); + Assert.assertEquals(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_BATCH_SIZE.getDefaultValue(), + Neo4jConstants.DEFAULT_BATCH_SIZE); + Assert.assertEquals(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_NODE_LABEL.getDefaultValue(), + Neo4jConstants.DEFAULT_NODE_LABEL); + Assert.assertEquals(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_RELATIONSHIP_TYPE.getDefaultValue(), + Neo4jConstants.DEFAULT_RELATIONSHIP_TYPE); + Assert.assertEquals(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_WRITE_MODE.getDefaultValue(), + "node"); + } +} diff --git a/geaflow/geaflow-dsl/geaflow-dsl-connector/geaflow-dsl-connector-neo4j/src/test/java/org/apache/geaflow/dsl/connector/neo4j/Neo4jIntegrationTest.java b/geaflow/geaflow-dsl/geaflow-dsl-connector/geaflow-dsl-connector-neo4j/src/test/java/org/apache/geaflow/dsl/connector/neo4j/Neo4jIntegrationTest.java new file mode 100644 index 000000000..4d96073d2 --- /dev/null +++ b/geaflow/geaflow-dsl/geaflow-dsl-connector/geaflow-dsl-connector-neo4j/src/test/java/org/apache/geaflow/dsl/connector/neo4j/Neo4jIntegrationTest.java @@ -0,0 +1,387 @@ +/* + * 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.geaflow.dsl.connector.neo4j; + +import java.util.ArrayList; +import java.util.List; +import java.util.Optional; +import org.apache.geaflow.api.context.RuntimeContext; +import org.apache.geaflow.api.window.WindowType; +import org.apache.geaflow.common.config.Configuration; +import org.apache.geaflow.common.type.Types; +import org.apache.geaflow.dsl.common.data.Row; +import org.apache.geaflow.dsl.common.data.impl.ObjectRow; +import org.apache.geaflow.dsl.common.types.StructType; +import org.apache.geaflow.dsl.common.types.TableField; +import org.apache.geaflow.dsl.common.types.TableSchema; +import org.apache.geaflow.dsl.connector.api.FetchData; +import org.apache.geaflow.dsl.connector.api.Partition; +import org.apache.geaflow.dsl.connector.api.window.SizeFetchWindow; +import org.apache.geaflow.dsl.connector.api.window.AllFetchWindow; +import org.mockito.Mockito; +import org.testcontainers.containers.Neo4jContainer; +import org.testcontainers.utility.DockerImageName; +import org.testng.Assert; +import org.testng.annotations.AfterClass; +import org.testng.annotations.BeforeClass; +import org.testng.annotations.Test; + +public class Neo4jIntegrationTest { + + private static Neo4jContainer neo4jContainer; + private static String neo4jUri; + private static final String USERNAME = "neo4j"; + private static final String PASSWORD = "testpassword"; + private static final String DATABASE = "neo4j"; + + @BeforeClass + public static void setup() { + neo4jContainer = new Neo4jContainer<>(DockerImageName.parse("neo4j:5.14.0")) + .withAdminPassword(PASSWORD) + .withEnv("NEO4J_AUTH", USERNAME + "/" + PASSWORD); + neo4jContainer.start(); + + neo4jUri = neo4jContainer.getBoltUrl(); + } + + @AfterClass + public static void cleanup() { + if (neo4jContainer != null) { + neo4jContainer.stop(); + } + } + + @Test + public void testWriteAndReadNodes() throws Exception { + // Write nodes + Configuration writeConf = new Configuration(); + writeConf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_URI, neo4jUri); + writeConf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_USERNAME, USERNAME); + writeConf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_PASSWORD, PASSWORD); + writeConf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_DATABASE, DATABASE); + writeConf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_WRITE_MODE, "node"); + writeConf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_NODE_LABEL, "Product"); + writeConf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_NODE_ID_FIELD, "product_id"); + writeConf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_BATCH_SIZE, "5"); + + List fields = new ArrayList<>(); + fields.add(new TableField("product_id", Types.LONG, false)); + fields.add(new TableField("product_name", Types.STRING, false)); + fields.add(new TableField("price", Types.DOUBLE, false)); + fields.add(new TableField("in_stock", Types.BOOLEAN, false)); + StructType schema = new StructType(fields); + + Neo4jTableSink sink = new Neo4jTableSink(); + sink.init(writeConf, schema); + + RuntimeContext context = Mockito.mock(RuntimeContext.class); + sink.open(context); + + // Write test data + for (long i = 1; i <= 10; i++) { + Row row = ObjectRow.create(new Object[]{ + i, + "Product" + i, + 99.99 + i, + i % 2 == 0 + }); + sink.write(row); + } + + sink.finish(); + sink.close(); + + // Read nodes back + Configuration readConf = new Configuration(); + readConf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_URI, neo4jUri); + readConf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_USERNAME, USERNAME); + readConf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_PASSWORD, PASSWORD); + readConf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_DATABASE, DATABASE); + readConf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_QUERY, + "MATCH (p:Product) RETURN p.id as product_id, p.product_name as product_name, " + + "p.price as price, p.in_stock as in_stock ORDER BY p.id"); + + TableSchema tableSchema = new TableSchema(fields); + + Neo4jTableSource source = new Neo4jTableSource(); + source.init(readConf, tableSchema); + source.open(context); + + List partitions = source.listPartitions(); + AllFetchWindow window = new AllFetchWindow(0L); + FetchData fetchData = source.fetch(partitions.get(0), Optional.empty(), window); + + List rows = (List) fetchData.getDataIterator(); + Assert.assertEquals(rows.size(), 10); + + // Verify first row + Row row1 = rows.get(0); + Assert.assertEquals(row1.getField(0, Types.LONG), 1L); + Assert.assertEquals(row1.getField(1, Types.STRING), "Product1"); + Assert.assertEquals((Double) row1.getField(2, Types.DOUBLE), 100.99, 0.01); + Assert.assertFalse((Boolean) row1.getField(3, Types.BOOLEAN)); + + source.close(); + } + + @Test + public void testWriteNodesAndRelationships() throws Exception { + RuntimeContext context = Mockito.mock(RuntimeContext.class); + + // First write nodes + Configuration nodeConf = new Configuration(); + nodeConf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_URI, neo4jUri); + nodeConf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_USERNAME, USERNAME); + nodeConf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_PASSWORD, PASSWORD); + nodeConf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_DATABASE, DATABASE); + nodeConf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_WRITE_MODE, "node"); + nodeConf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_NODE_LABEL, "City"); + nodeConf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_NODE_ID_FIELD, "city_id"); + nodeConf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_BATCH_SIZE, "10"); + + List nodeFields = new ArrayList<>(); + nodeFields.add(new TableField("city_id", Types.LONG, false)); + nodeFields.add(new TableField("city_name", Types.STRING, false)); + nodeFields.add(new TableField("population", Types.LONG, false)); + StructType nodeSchema = new StructType(nodeFields); + + Neo4jTableSink nodeSink = new Neo4jTableSink(); + nodeSink.init(nodeConf, nodeSchema); + nodeSink.open(context); + + Row city1 = ObjectRow.create(new Object[]{1L, "Beijing", 21540000L}); + Row city2 = ObjectRow.create(new Object[]{2L, "Shanghai", 24280000L}); + Row city3 = ObjectRow.create(new Object[]{3L, "Guangzhou", 15300000L}); + + nodeSink.write(city1); + nodeSink.write(city2); + nodeSink.write(city3); + nodeSink.finish(); + nodeSink.close(); + + // Then write relationships + Configuration relConf = new Configuration(); + relConf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_URI, neo4jUri); + relConf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_USERNAME, USERNAME); + relConf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_PASSWORD, PASSWORD); + relConf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_DATABASE, DATABASE); + relConf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_WRITE_MODE, "relationship"); + relConf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_RELATIONSHIP_TYPE, "CONNECTED_TO"); + relConf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_RELATIONSHIP_SOURCE_FIELD, "from_city"); + relConf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_RELATIONSHIP_TARGET_FIELD, "to_city"); + relConf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_BATCH_SIZE, "10"); + + List relFields = new ArrayList<>(); + relFields.add(new TableField("from_city", Types.LONG, false)); + relFields.add(new TableField("to_city", Types.LONG, false)); + relFields.add(new TableField("distance", Types.DOUBLE, false)); + relFields.add(new TableField("travel_time", Types.INTEGER, false)); + StructType relSchema = new StructType(relFields); + + Neo4jTableSink relSink = new Neo4jTableSink(); + relSink.init(relConf, relSchema); + relSink.open(context); + + Row rel1 = ObjectRow.create(new Object[]{1L, 2L, 1214.0, 120}); + Row rel2 = ObjectRow.create(new Object[]{2L, 3L, 1376.0, 150}); + + relSink.write(rel1); + relSink.write(rel2); + relSink.finish(); + relSink.close(); + + // Read and verify the graph + Configuration readConf = new Configuration(); + readConf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_URI, neo4jUri); + readConf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_USERNAME, USERNAME); + readConf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_PASSWORD, PASSWORD); + readConf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_DATABASE, DATABASE); + readConf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_QUERY, + "MATCH (a:City)-[r:CONNECTED_TO]->(b:City) " + + "RETURN a.id as from_city, b.id as to_city, r.distance as distance, " + + "r.travel_time as travel_time ORDER BY a.id"); + + TableSchema tableSchema = new TableSchema(relFields); + + Neo4jTableSource source = new Neo4jTableSource(); + source.init(readConf, tableSchema); + source.open(context); + + List partitions = source.listPartitions(); + AllFetchWindow window = new AllFetchWindow(0L); + FetchData fetchData = source.fetch(partitions.get(0), Optional.empty(), window); + + // Collect all rows from iterator + List rows = new ArrayList<>(); + fetchData.getDataIterator().forEachRemaining(rows::add); + Assert.assertEquals(rows.size(), 2); + + // Verify relationships + Row relRow1 = rows.get(0); + Assert.assertEquals((Long) relRow1.getField(0, Types.LONG), Long.valueOf(1L)); + Assert.assertEquals((Long) relRow1.getField(1, Types.LONG), Long.valueOf(2L)); + Assert.assertEquals((Double) relRow1.getField(2, Types.DOUBLE), 1214.0, 0.1); + Assert.assertEquals((Integer) relRow1.getField(3, Types.INTEGER), Integer.valueOf(120)); + + source.close(); + } + + @Test + public void testUpdateExistingNodes() throws Exception { + RuntimeContext context = Mockito.mock(RuntimeContext.class); + + // Write initial nodes + Configuration conf = new Configuration(); + conf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_URI, neo4jUri); + conf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_USERNAME, USERNAME); + conf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_PASSWORD, PASSWORD); + conf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_DATABASE, DATABASE); + conf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_WRITE_MODE, "node"); + conf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_NODE_LABEL, "Account"); + conf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_NODE_ID_FIELD, "account_id"); + conf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_BATCH_SIZE, "10"); + + List fields = new ArrayList<>(); + fields.add(new TableField("account_id", Types.LONG, false)); + fields.add(new TableField("balance", Types.DOUBLE, false)); + fields.add(new TableField("status", Types.STRING, false)); + StructType schema = new StructType(fields); + + // Initial write + Neo4jTableSink sink1 = new Neo4jTableSink(); + sink1.init(conf, schema); + sink1.open(context); + Row row1 = ObjectRow.create(new Object[]{1000L, 1000.0, "active"}); + sink1.write(row1); + sink1.finish(); + sink1.close(); + + // Update the same node + Neo4jTableSink sink2 = new Neo4jTableSink(); + sink2.init(conf, schema); + sink2.open(context); + Row row2 = ObjectRow.create(new Object[]{1000L, 1500.0, "premium"}); + sink2.write(row2); + sink2.finish(); + sink2.close(); + + // Read and verify updated value + Configuration readConf = new Configuration(); + readConf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_URI, neo4jUri); + readConf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_USERNAME, USERNAME); + readConf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_PASSWORD, PASSWORD); + readConf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_DATABASE, DATABASE); + readConf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_QUERY, + "MATCH (a:Account {id: 1000}) RETURN a.id as account_id, " + + "a.balance as balance, a.status as status"); + + TableSchema tableSchema = new TableSchema(fields); + + Neo4jTableSource source = new Neo4jTableSource(); + source.init(readConf, tableSchema); + source.open(context); + + List partitions = source.listPartitions(); + AllFetchWindow window = new AllFetchWindow(0L); + FetchData fetchData = source.fetch(partitions.get(0), Optional.empty(), window); + + // Collect all rows from iterator + List rows = new ArrayList<>(); + fetchData.getDataIterator().forEachRemaining(rows::add); + Assert.assertEquals(rows.size(), 1); + + Row row = rows.get(0); + Assert.assertEquals((Long) row.getField(0, Types.LONG), Long.valueOf(1000L)); + Assert.assertEquals((Double) row.getField(1, Types.DOUBLE), 1500.0, 0.01); + Assert.assertEquals(row.getField(2, Types.STRING), "premium"); + + source.close(); + } + + @Test + public void testLargeDatasetWriteAndRead() throws Exception { + RuntimeContext context = Mockito.mock(RuntimeContext.class); + + Configuration writeConf = new Configuration(); + writeConf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_URI, neo4jUri); + writeConf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_USERNAME, USERNAME); + writeConf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_PASSWORD, PASSWORD); + writeConf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_DATABASE, DATABASE); + writeConf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_WRITE_MODE, "node"); + writeConf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_NODE_LABEL, "LargeDataset"); + writeConf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_NODE_ID_FIELD, "id"); + writeConf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_BATCH_SIZE, "50"); + + List fields = new ArrayList<>(); + fields.add(new TableField("id", Types.LONG, false)); + fields.add(new TableField("value", Types.LONG, false)); + StructType schema = new StructType(fields); + + Neo4jTableSink sink = new Neo4jTableSink(); + sink.init(writeConf, schema); + sink.open(context); + + // Write 200 rows + for (long i = 10000; i < 10200; i++) { + Row row = ObjectRow.create(new Object[]{i, i * 2}); + sink.write(row); + } + sink.finish(); + sink.close(); + + // Read back with pagination + Configuration readConf = new Configuration(); + readConf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_URI, neo4jUri); + readConf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_USERNAME, USERNAME); + readConf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_PASSWORD, PASSWORD); + readConf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_DATABASE, DATABASE); + readConf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_QUERY, + "MATCH (n:LargeDataset) RETURN n.id as id, n.value as value ORDER BY n.id"); + + TableSchema tableSchema = new TableSchema(fields); + + Neo4jTableSource source = new Neo4jTableSource(); + source.init(readConf, tableSchema); + source.open(context); + + List partitions = source.listPartitions(); + SizeFetchWindow window = new SizeFetchWindow(0L, 75L); + + int totalRows = 0; + Optional offset = Optional.empty(); + + while (true) { + FetchData fetchData = source.fetch(partitions.get(0), offset, window); + // Count rows from iterator + List pageRows = new ArrayList<>(); + fetchData.getDataIterator().forEachRemaining(pageRows::add); + totalRows += pageRows.size(); + + if (fetchData.isFinish()) { + break; + } + offset = Optional.of(fetchData.getNextOffset()); + } + + Assert.assertEquals(totalRows, 200); + + source.close(); + } +} diff --git a/geaflow/geaflow-dsl/geaflow-dsl-connector/geaflow-dsl-connector-neo4j/src/test/java/org/apache/geaflow/dsl/connector/neo4j/Neo4jTableConnectorTest.java b/geaflow/geaflow-dsl/geaflow-dsl-connector/geaflow-dsl-connector-neo4j/src/test/java/org/apache/geaflow/dsl/connector/neo4j/Neo4jTableConnectorTest.java new file mode 100644 index 000000000..e321af9a9 --- /dev/null +++ b/geaflow/geaflow-dsl/geaflow-dsl-connector/geaflow-dsl-connector-neo4j/src/test/java/org/apache/geaflow/dsl/connector/neo4j/Neo4jTableConnectorTest.java @@ -0,0 +1,385 @@ +/* + * 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.geaflow.dsl.connector.neo4j; + +import java.util.ArrayList; +import java.util.List; +import java.util.Optional; +import org.apache.geaflow.api.context.RuntimeContext; +import org.apache.geaflow.api.window.WindowType; +import org.apache.geaflow.common.config.Configuration; +import org.apache.geaflow.common.type.Types; +import org.apache.geaflow.dsl.common.data.Row; +import org.apache.geaflow.dsl.common.data.impl.ObjectRow; +import org.apache.geaflow.dsl.common.types.StructType; +import org.apache.geaflow.dsl.common.types.TableField; +import org.apache.geaflow.dsl.common.types.TableSchema; +import org.apache.geaflow.dsl.connector.api.FetchData; +import org.apache.geaflow.dsl.connector.api.Partition; +import org.apache.geaflow.dsl.connector.api.window.SizeFetchWindow; +import org.mockito.Mockito; +import org.neo4j.driver.Driver; +import org.neo4j.driver.Session; +import org.testcontainers.containers.Neo4jContainer; +import org.testcontainers.utility.DockerImageName; +import org.testng.Assert; +import org.testng.annotations.AfterClass; +import org.testng.annotations.BeforeClass; +import org.testng.annotations.Test; + +public class Neo4jTableConnectorTest { + + private static Neo4jContainer neo4jContainer; + private static String neo4jUri; + private static String username = "neo4j"; + private static String password = "testpassword"; + private static String database = "neo4j"; + + @BeforeClass + public static void setup() { + neo4jContainer = new Neo4jContainer<>(DockerImageName.parse("neo4j:5.14.0")) + .withAdminPassword(password) + .withEnv("NEO4J_AUTH", username + "/" + password); + neo4jContainer.start(); + + neo4jUri = neo4jContainer.getBoltUrl(); + + // Initialize test data + try (Driver driver = org.neo4j.driver.GraphDatabase.driver( + neo4jUri, + org.neo4j.driver.AuthTokens.basic(username, password))) { + try (Session session = driver.session()) { + // Create test nodes + session.run("CREATE (p1:Person {id: 1, name: 'Alice', age: 30})"); + session.run("CREATE (p2:Person {id: 2, name: 'Bob', age: 25})"); + session.run("CREATE (p3:Person {id: 3, name: 'Charlie', age: 35})"); + + // Create test relationships + session.run("MATCH (a:Person {id: 1}), (b:Person {id: 2}) " + + "CREATE (a)-[:KNOWS {since: 2020}]->(b)"); + session.run("MATCH (a:Person {id: 2}), (b:Person {id: 3}) " + + "CREATE (a)-[:KNOWS {since: 2021}]->(b)"); + } + } + } + + @AfterClass + public static void cleanup() { + if (neo4jContainer != null) { + neo4jContainer.stop(); + } + } + + @Test + public void testConnectorType() { + Neo4jTableConnector connector = new Neo4jTableConnector(); + Assert.assertEquals(connector.getType(), "Neo4j"); + } + + @Test + public void testCreateSource() { + Neo4jTableConnector connector = new Neo4jTableConnector(); + Configuration conf = new Configuration(); + Neo4jTableSource source = (Neo4jTableSource) connector.createSource(conf); + Assert.assertNotNull(source); + } + + @Test + public void testCreateSink() { + Neo4jTableConnector connector = new Neo4jTableConnector(); + Configuration conf = new Configuration(); + Neo4jTableSink sink = (Neo4jTableSink) connector.createSink(conf); + Assert.assertNotNull(sink); + } + + @Test + public void testReadNodes() throws Exception { + Configuration conf = new Configuration(); + conf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_URI, neo4jUri); + conf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_USERNAME, username); + conf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_PASSWORD, password); + conf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_DATABASE, database); + conf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_QUERY, + "MATCH (p:Person) RETURN p.id as id, p.name as name, p.age as age ORDER BY p.id"); + + List fields = new ArrayList<>(); + fields.add(new TableField("id", Types.LONG, false)); + fields.add(new TableField("name", Types.STRING, false)); + fields.add(new TableField("age", Types.LONG, false)); + TableSchema tableSchema = new TableSchema(fields); + + Neo4jTableSource source = new Neo4jTableSource(); + source.init(conf, tableSchema); + + RuntimeContext context = Mockito.mock(RuntimeContext.class); + source.open(context); + + List partitions = source.listPartitions(); + Assert.assertEquals(partitions.size(), 1); + + SizeFetchWindow window = new SizeFetchWindow(0L, 10L); + FetchData fetchData = source.fetch(partitions.get(0), Optional.empty(), window); + + // Collect rows from iterator + List rows = new ArrayList<>(); + while (fetchData.getDataIterator().hasNext()) { + rows.add(fetchData.getDataIterator().next()); + } + Assert.assertEquals(rows.size(), 3); + + // Verify first row + Row row1 = rows.get(0); + Assert.assertEquals((Long) row1.getField(0, Types.LONG), Long.valueOf(1L)); + Assert.assertEquals(row1.getField(1, Types.STRING), "Alice"); + Assert.assertEquals((Long) row1.getField(2, Types.LONG), Long.valueOf(30L)); + + source.close(); + } + + @Test + public void testWriteNodes() throws Exception { + Configuration conf = new Configuration(); + conf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_URI, neo4jUri); + conf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_USERNAME, username); + conf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_PASSWORD, password); + conf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_DATABASE, database); + conf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_WRITE_MODE, "node"); + conf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_NODE_LABEL, "TestPerson"); + conf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_NODE_ID_FIELD, "id"); + conf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_BATCH_SIZE, "2"); + + List fields = new ArrayList<>(); + fields.add(new TableField("id", Types.LONG, false)); + fields.add(new TableField("name", Types.STRING, false)); + fields.add(new TableField("email", Types.STRING, true)); + StructType schema = new StructType(fields); + + Neo4jTableSink sink = new Neo4jTableSink(); + sink.init(conf, schema); + + RuntimeContext context = Mockito.mock(RuntimeContext.class); + sink.open(context); + + // Write test data + Row row1 = ObjectRow.create(new Object[]{100L, "David", "david@example.com"}); + Row row2 = ObjectRow.create(new Object[]{101L, "Eva", "eva@example.com"}); + Row row3 = ObjectRow.create(new Object[]{102L, "Frank", null}); + + sink.write(row1); + sink.write(row2); + sink.write(row3); + + sink.finish(); + sink.close(); + + // Verify data was written + try (Driver driver = org.neo4j.driver.GraphDatabase.driver( + neo4jUri, + org.neo4j.driver.AuthTokens.basic(username, password))) { + try (Session session = driver.session()) { + var result = session.run("MATCH (p:TestPerson) WHERE p.id >= 100 RETURN count(p) as count"); + long count = result.single().get("count").asLong(); + Assert.assertEquals(count, 3); + + // Verify specific node + var result2 = session.run("MATCH (p:TestPerson {id: 100}) RETURN p.name as name, p.email as email"); + var record = result2.single(); + Assert.assertEquals(record.get("name").asString(), "David"); + Assert.assertEquals(record.get("email").asString(), "david@example.com"); + } + } + } + + @Test + public void testWriteRelationships() throws Exception { + Configuration conf = new Configuration(); + conf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_URI, neo4jUri); + conf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_USERNAME, username); + conf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_PASSWORD, password); + conf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_DATABASE, database); + conf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_WRITE_MODE, "relationship"); + conf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_RELATIONSHIP_TYPE, "FOLLOWS"); + conf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_RELATIONSHIP_SOURCE_FIELD, "from_id"); + conf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_RELATIONSHIP_TARGET_FIELD, "to_id"); + conf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_BATCH_SIZE, "2"); + + // First create some nodes to connect + try (Driver driver = org.neo4j.driver.GraphDatabase.driver( + neo4jUri, + org.neo4j.driver.AuthTokens.basic(username, password))) { + try (Session session = driver.session()) { + session.run("CREATE (p:TestUser {id: 200, name: 'User1'})"); + session.run("CREATE (p:TestUser {id: 201, name: 'User2'})"); + session.run("CREATE (p:TestUser {id: 202, name: 'User3'})"); + } + } + + List fields = new ArrayList<>(); + fields.add(new TableField("from_id", Types.LONG, false)); + fields.add(new TableField("to_id", Types.LONG, false)); + fields.add(new TableField("weight", Types.DOUBLE, true)); + StructType schema = new StructType(fields); + + Neo4jTableSink sink = new Neo4jTableSink(); + sink.init(conf, schema); + + RuntimeContext context = Mockito.mock(RuntimeContext.class); + sink.open(context); + + // Write relationships + Row rel1 = ObjectRow.create(new Object[]{200L, 201L, 0.8}); + Row rel2 = ObjectRow.create(new Object[]{201L, 202L, 0.9}); + + sink.write(rel1); + sink.write(rel2); + + sink.finish(); + sink.close(); + + // Verify relationships were created + try (Driver driver = org.neo4j.driver.GraphDatabase.driver( + neo4jUri, + org.neo4j.driver.AuthTokens.basic(username, password))) { + try (Session session = driver.session()) { + var result = session.run( + "MATCH ()-[r:FOLLOWS]->() WHERE r.weight IS NOT NULL RETURN count(r) as count"); + long count = result.single().get("count").asLong(); + Assert.assertEquals(count, 2); + } + } + } + + @Test + public void testReadWithPagination() throws Exception { + Configuration conf = new Configuration(); + conf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_URI, neo4jUri); + conf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_USERNAME, username); + conf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_PASSWORD, password); + conf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_DATABASE, database); + conf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_QUERY, + "MATCH (p:Person) RETURN p.id as id, p.name as name ORDER BY p.id"); + + List fields = new ArrayList<>(); + fields.add(new TableField("id", Types.LONG, false)); + fields.add(new TableField("name", Types.STRING, false)); + TableSchema tableSchema = new TableSchema(fields); + + Neo4jTableSource source = new Neo4jTableSource(); + source.init(conf, tableSchema); + + RuntimeContext context = Mockito.mock(RuntimeContext.class); + source.open(context); + + List partitions = source.listPartitions(); + + // Fetch first page + SizeFetchWindow window = new SizeFetchWindow(0L, 2L); + FetchData fetchData1 = source.fetch(partitions.get(0), Optional.empty(), window); + List rows1 = (List) fetchData1.getDataIterator(); + Assert.assertEquals(rows1.size(), 2); + + // Fetch second page + FetchData fetchData2 = source.fetch(partitions.get(0), + Optional.of(fetchData1.getNextOffset()), window); + List rows2 = (List) fetchData2.getDataIterator(); + Assert.assertEquals(rows2.size(), 1); + + source.close(); + } + + @Test + public void testBatchWriting() throws Exception { + Configuration conf = new Configuration(); + conf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_URI, neo4jUri); + conf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_USERNAME, username); + conf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_PASSWORD, password); + conf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_DATABASE, database); + conf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_WRITE_MODE, "node"); + conf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_NODE_LABEL, "BatchTest"); + conf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_NODE_ID_FIELD, "id"); + conf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_BATCH_SIZE, "3"); + + List fields = new ArrayList<>(); + fields.add(new TableField("id", Types.LONG, false)); + fields.add(new TableField("value", Types.STRING, false)); + StructType schema = new StructType(fields); + + Neo4jTableSink sink = new Neo4jTableSink(); + sink.init(conf, schema); + + RuntimeContext context = Mockito.mock(RuntimeContext.class); + sink.open(context); + + // Write 10 rows (should trigger multiple batches) + for (long i = 300; i < 310; i++) { + Row row = ObjectRow.create(new Object[]{i, "Value" + i}); + sink.write(row); + } + + sink.finish(); + sink.close(); + + // Verify all data was written + try (Driver driver = org.neo4j.driver.GraphDatabase.driver( + neo4jUri, + org.neo4j.driver.AuthTokens.basic(username, password))) { + try (Session session = driver.session()) { + var result = session.run("MATCH (n:BatchTest) WHERE n.id >= 300 AND n.id < 310 RETURN count(n) as count"); + long count = result.single().get("count").asLong(); + Assert.assertEquals(count, 10); + } + } + } + + @Test(expectedExceptions = org.apache.geaflow.dsl.common.exception.GeaFlowDSLException.class) + public void testInvalidWriteMode() { + Configuration conf = new Configuration(); + conf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_URI, neo4jUri); + conf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_USERNAME, username); + conf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_PASSWORD, password); + conf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_DATABASE, database); + conf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_WRITE_MODE, "invalid_mode"); + conf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_NODE_ID_FIELD, "id"); + + List fields = new ArrayList<>(); + fields.add(new TableField("id", Types.LONG, false)); + StructType schema = new StructType(fields); + + Neo4jTableSink sink = new Neo4jTableSink(); + sink.init(conf, schema); // Should throw exception + } + + @Test(expectedExceptions = org.apache.geaflow.dsl.common.exception.GeaFlowDSLException.class) + public void testMissingNodeIdField() { + Configuration conf = new Configuration(); + conf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_URI, neo4jUri); + conf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_USERNAME, username); + conf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_PASSWORD, password); + conf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_DATABASE, database); + conf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_WRITE_MODE, "node"); + + List fields = new ArrayList<>(); + fields.add(new TableField("id", Types.LONG, false)); + StructType schema = new StructType(fields); + + Neo4jTableSink sink = new Neo4jTableSink(); + sink.init(conf, schema); // Should throw exception + } +} diff --git a/geaflow/geaflow-dsl/geaflow-dsl-connector/geaflow-dsl-connector-neo4j/src/test/java/org/apache/geaflow/dsl/connector/neo4j/Neo4jTableSinkTest.java b/geaflow/geaflow-dsl/geaflow-dsl-connector/geaflow-dsl-connector-neo4j/src/test/java/org/apache/geaflow/dsl/connector/neo4j/Neo4jTableSinkTest.java new file mode 100644 index 000000000..8a82f37fb --- /dev/null +++ b/geaflow/geaflow-dsl/geaflow-dsl-connector/geaflow-dsl-connector-neo4j/src/test/java/org/apache/geaflow/dsl/connector/neo4j/Neo4jTableSinkTest.java @@ -0,0 +1,388 @@ +/* + * 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.geaflow.dsl.connector.neo4j; + +import java.util.ArrayList; +import java.util.List; +import org.apache.geaflow.api.context.RuntimeContext; +import org.apache.geaflow.common.config.Configuration; +import org.apache.geaflow.common.type.Types; +import org.apache.geaflow.dsl.common.data.Row; +import org.apache.geaflow.dsl.common.data.impl.ObjectRow; +import org.apache.geaflow.dsl.common.types.StructType; +import org.apache.geaflow.dsl.common.types.TableField; +import org.mockito.Mockito; +import org.neo4j.driver.Driver; +import org.neo4j.driver.Session; +import org.testcontainers.containers.Neo4jContainer; +import org.testcontainers.utility.DockerImageName; +import org.testng.Assert; +import org.testng.annotations.AfterClass; +import org.testng.annotations.BeforeClass; +import org.testng.annotations.Test; + +public class Neo4jTableSinkTest { + + private static Neo4jContainer neo4jContainer; + private static String neo4jUri; + private static final String USERNAME = "neo4j"; + private static final String PASSWORD = "testpassword"; + private static final String DATABASE = "neo4j"; + + @BeforeClass + public static void setup() { + neo4jContainer = new Neo4jContainer<>(DockerImageName.parse("neo4j:5.14.0")) + .withAdminPassword(PASSWORD) + .withEnv("NEO4J_AUTH", USERNAME + "/" + PASSWORD); + neo4jContainer.start(); + + neo4jUri = neo4jContainer.getBoltUrl(); + } + + @AfterClass + public static void cleanup() { + if (neo4jContainer != null) { + neo4jContainer.stop(); + } + } + + @Test + public void testWriteNodesWithNullValues() throws Exception { + Configuration conf = new Configuration(); + conf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_URI, neo4jUri); + conf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_USERNAME, USERNAME); + conf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_PASSWORD, PASSWORD); + conf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_DATABASE, DATABASE); + conf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_WRITE_MODE, "node"); + conf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_NODE_LABEL, "NullTest"); + conf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_NODE_ID_FIELD, "id"); + conf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_BATCH_SIZE, "10"); + + List fields = new ArrayList<>(); + fields.add(new TableField("id", Types.LONG, false)); + fields.add(new TableField("name", Types.STRING, true)); + fields.add(new TableField("age", Types.INTEGER, true)); + StructType schema = new StructType(fields); + + Neo4jTableSink sink = new Neo4jTableSink(); + sink.init(conf, schema); + + RuntimeContext context = Mockito.mock(RuntimeContext.class); + sink.open(context); + + // Write rows with null values + Row row1 = ObjectRow.create(new Object[]{1000L, null, 25}); + Row row2 = ObjectRow.create(new Object[]{1001L, "John", null}); + Row row3 = ObjectRow.create(new Object[]{1002L, null, null}); + + sink.write(row1); + sink.write(row2); + sink.write(row3); + + sink.finish(); + sink.close(); + + // Verify data + try (Driver driver = org.neo4j.driver.GraphDatabase.driver( + neo4jUri, + org.neo4j.driver.AuthTokens.basic(USERNAME, PASSWORD))) { + try (Session session = driver.session()) { + // Check first row + var result1 = session.run("MATCH (n:NullTest {id: 1000}) RETURN n.age as age"); + Assert.assertEquals(result1.single().get("age").asInt(), 25); + + // Check second row + var result2 = session.run("MATCH (n:NullTest {id: 1001}) RETURN n.name as name"); + Assert.assertEquals(result2.single().get("name").asString(), "John"); + } + } + } + + @Test + public void testWriteDifferentDataTypes() throws Exception { + Configuration conf = new Configuration(); + conf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_URI, neo4jUri); + conf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_USERNAME, USERNAME); + conf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_PASSWORD, PASSWORD); + conf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_DATABASE, DATABASE); + conf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_WRITE_MODE, "node"); + conf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_NODE_LABEL, "TypeTest"); + conf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_NODE_ID_FIELD, "id"); + conf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_BATCH_SIZE, "10"); + + List fields = new ArrayList<>(); + fields.add(new TableField("id", Types.LONG, false)); + fields.add(new TableField("int_val", Types.INTEGER, true)); + fields.add(new TableField("long_val", Types.LONG, true)); + fields.add(new TableField("double_val", Types.DOUBLE, true)); + fields.add(new TableField("bool_val", Types.BOOLEAN, true)); + fields.add(new TableField("str_val", Types.STRING, true)); + StructType schema = new StructType(fields); + + Neo4jTableSink sink = new Neo4jTableSink(); + sink.init(conf, schema); + + RuntimeContext context = Mockito.mock(RuntimeContext.class); + sink.open(context); + + Row row = ObjectRow.create(new Object[]{ + 2000L, + 42, + 9999999999L, + 3.14159, + true, + "test string" + }); + + sink.write(row); + sink.finish(); + sink.close(); + + // Verify all data types + try (Driver driver = org.neo4j.driver.GraphDatabase.driver( + neo4jUri, + org.neo4j.driver.AuthTokens.basic(USERNAME, PASSWORD))) { + try (Session session = driver.session()) { + var result = session.run( + "MATCH (n:TypeTest {id: 2000}) " + + "RETURN n.int_val as int_val, n.long_val as long_val, " + + "n.double_val as double_val, n.bool_val as bool_val, n.str_val as str_val"); + var record = result.single(); + + Assert.assertEquals(record.get("int_val").asInt(), 42); + Assert.assertEquals(record.get("long_val").asLong(), 9999999999L); + Assert.assertEquals(record.get("double_val").asDouble(), 3.14159, 0.00001); + Assert.assertTrue(record.get("bool_val").asBoolean()); + Assert.assertEquals(record.get("str_val").asString(), "test string"); + } + } + } + + @Test + public void testNodeMergeStrategy() throws Exception { + Configuration conf = new Configuration(); + conf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_URI, neo4jUri); + conf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_USERNAME, USERNAME); + conf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_PASSWORD, PASSWORD); + conf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_DATABASE, DATABASE); + conf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_WRITE_MODE, "node"); + conf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_NODE_LABEL, "MergeTest"); + conf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_NODE_ID_FIELD, "id"); + conf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_BATCH_SIZE, "10"); + + List fields = new ArrayList<>(); + fields.add(new TableField("id", Types.LONG, false)); + fields.add(new TableField("value", Types.STRING, false)); + StructType schema = new StructType(fields); + + Neo4jTableSink sink = new Neo4jTableSink(); + sink.init(conf, schema); + + RuntimeContext context = Mockito.mock(RuntimeContext.class); + sink.open(context); + + // Write same ID twice with different values + Row row1 = ObjectRow.create(new Object[]{3000L, "first"}); + sink.write(row1); + sink.finish(); + sink.close(); + + // Write again with same ID + sink = new Neo4jTableSink(); + sink.init(conf, schema); + sink.open(context); + Row row2 = ObjectRow.create(new Object[]{3000L, "second"}); + sink.write(row2); + sink.finish(); + sink.close(); + + // Verify only one node exists with updated value + try (Driver driver = org.neo4j.driver.GraphDatabase.driver( + neo4jUri, + org.neo4j.driver.AuthTokens.basic(USERNAME, PASSWORD))) { + try (Session session = driver.session()) { + var result = session.run("MATCH (n:MergeTest {id: 3000}) RETURN count(n) as count, n.value as value"); + var record = result.single(); + Assert.assertEquals(record.get("count").asLong(), 1L); + Assert.assertEquals(record.get("value").asString(), "second"); + } + } + } + + @Test + public void testRelationshipWithMultipleProperties() throws Exception { + // Create test nodes first + try (Driver driver = org.neo4j.driver.GraphDatabase.driver( + neo4jUri, + org.neo4j.driver.AuthTokens.basic(USERNAME, PASSWORD))) { + try (Session session = driver.session()) { + session.run("CREATE (a {id: 4000}), (b {id: 4001})"); + } + } + + Configuration conf = new Configuration(); + conf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_URI, neo4jUri); + conf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_USERNAME, USERNAME); + conf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_PASSWORD, PASSWORD); + conf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_DATABASE, DATABASE); + conf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_WRITE_MODE, "relationship"); + conf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_RELATIONSHIP_TYPE, "CONNECTS"); + conf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_RELATIONSHIP_SOURCE_FIELD, "source"); + conf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_RELATIONSHIP_TARGET_FIELD, "target"); + conf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_BATCH_SIZE, "10"); + + List fields = new ArrayList<>(); + fields.add(new TableField("source", Types.LONG, false)); + fields.add(new TableField("target", Types.LONG, false)); + fields.add(new TableField("weight", Types.DOUBLE, true)); + fields.add(new TableField("type", Types.STRING, true)); + fields.add(new TableField("active", Types.BOOLEAN, true)); + StructType schema = new StructType(fields); + + Neo4jTableSink sink = new Neo4jTableSink(); + sink.init(conf, schema); + + RuntimeContext context = Mockito.mock(RuntimeContext.class); + sink.open(context); + + Row rel = ObjectRow.create(new Object[]{4000L, 4001L, 0.95, "direct", true}); + sink.write(rel); + sink.finish(); + sink.close(); + + // Verify relationship properties + try (Driver driver = org.neo4j.driver.GraphDatabase.driver( + neo4jUri, + org.neo4j.driver.AuthTokens.basic(USERNAME, PASSWORD))) { + try (Session session = driver.session()) { + var result = session.run( + "MATCH ({id: 4000})-[r:CONNECTS]->({id: 4001}) " + + "RETURN r.weight as weight, r.type as type, r.active as active"); + var record = result.single(); + + Assert.assertEquals(record.get("weight").asDouble(), 0.95, 0.0001); + Assert.assertEquals(record.get("type").asString(), "direct"); + Assert.assertTrue(record.get("active").asBoolean()); + } + } + } + + @Test + public void testTransactionRollback() throws Exception { + Configuration conf = new Configuration(); + conf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_URI, neo4jUri); + conf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_USERNAME, USERNAME); + conf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_PASSWORD, PASSWORD); + conf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_DATABASE, DATABASE); + conf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_WRITE_MODE, "node"); + conf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_NODE_LABEL, "RollbackTest"); + conf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_NODE_ID_FIELD, "id"); + conf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_BATCH_SIZE, "10"); + + List fields = new ArrayList<>(); + fields.add(new TableField("id", Types.LONG, false)); + fields.add(new TableField("value", Types.STRING, false)); + StructType schema = new StructType(fields); + + Neo4jTableSink sink = new Neo4jTableSink(); + sink.init(conf, schema); + + RuntimeContext context = Mockito.mock(RuntimeContext.class); + sink.open(context); + + Row row = ObjectRow.create(new Object[]{5000L, "test"}); + sink.write(row); + + // Close without finish - should not commit + sink.close(); + + // Verify data was not written + try (Driver driver = org.neo4j.driver.GraphDatabase.driver( + neo4jUri, + org.neo4j.driver.AuthTokens.basic(USERNAME, PASSWORD))) { + try (Session session = driver.session()) { + var result = session.run("MATCH (n:RollbackTest {id: 5000}) RETURN count(n) as count"); + long count = result.single().get("count").asLong(); + Assert.assertEquals(count, 0L); + } + } + } + + @Test(expectedExceptions = org.apache.geaflow.dsl.common.exception.GeaFlowDSLException.class) + public void testNullNodeId() throws Exception { + Configuration conf = new Configuration(); + conf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_URI, neo4jUri); + conf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_USERNAME, USERNAME); + conf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_PASSWORD, PASSWORD); + conf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_DATABASE, DATABASE); + conf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_WRITE_MODE, "node"); + conf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_NODE_LABEL, "ErrorTest"); + conf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_NODE_ID_FIELD, "id"); + conf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_BATCH_SIZE, "10"); + + List fields = new ArrayList<>(); + fields.add(new TableField("id", Types.LONG, false)); + fields.add(new TableField("value", Types.STRING, false)); + StructType schema = new StructType(fields); + + Neo4jTableSink sink = new Neo4jTableSink(); + sink.init(conf, schema); + + RuntimeContext context = Mockito.mock(RuntimeContext.class); + sink.open(context); + + // Try to write with null ID - should throw exception + Row row = ObjectRow.create(new Object[]{null, "test"}); + sink.write(row); + sink.finish(); // Should throw here + sink.close(); + } + + @Test(expectedExceptions = org.apache.geaflow.dsl.common.exception.GeaFlowDSLException.class) + public void testNullRelationshipEndpoints() throws Exception { + Configuration conf = new Configuration(); + conf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_URI, neo4jUri); + conf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_USERNAME, USERNAME); + conf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_PASSWORD, PASSWORD); + conf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_DATABASE, DATABASE); + conf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_WRITE_MODE, "relationship"); + conf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_RELATIONSHIP_TYPE, "ERROR"); + conf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_RELATIONSHIP_SOURCE_FIELD, "source"); + conf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_RELATIONSHIP_TARGET_FIELD, "target"); + conf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_BATCH_SIZE, "10"); + + List fields = new ArrayList<>(); + fields.add(new TableField("source", Types.LONG, false)); + fields.add(new TableField("target", Types.LONG, false)); + StructType schema = new StructType(fields); + + Neo4jTableSink sink = new Neo4jTableSink(); + sink.init(conf, schema); + + RuntimeContext context = Mockito.mock(RuntimeContext.class); + sink.open(context); + + // Try to write with null source - should throw exception + Row row = ObjectRow.create(new Object[]{null, 1L}); + sink.write(row); + sink.finish(); // Should throw here + sink.close(); + } +} diff --git a/geaflow/geaflow-dsl/geaflow-dsl-connector/geaflow-dsl-connector-neo4j/src/test/java/org/apache/geaflow/dsl/connector/neo4j/Neo4jTableSourceTest.java b/geaflow/geaflow-dsl/geaflow-dsl-connector/geaflow-dsl-connector-neo4j/src/test/java/org/apache/geaflow/dsl/connector/neo4j/Neo4jTableSourceTest.java new file mode 100644 index 000000000..34be90080 --- /dev/null +++ b/geaflow/geaflow-dsl/geaflow-dsl-connector/geaflow-dsl-connector-neo4j/src/test/java/org/apache/geaflow/dsl/connector/neo4j/Neo4jTableSourceTest.java @@ -0,0 +1,433 @@ +/* + * 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.geaflow.dsl.connector.neo4j; + +import java.util.ArrayList; +import java.util.List; +import java.util.Optional; +import org.apache.geaflow.api.context.RuntimeContext; +import org.apache.geaflow.api.window.WindowType; +import org.apache.geaflow.common.config.Configuration; +import org.apache.geaflow.common.type.Types; +import org.apache.geaflow.dsl.common.data.Row; +import org.apache.geaflow.dsl.common.types.TableField; +import org.apache.geaflow.dsl.common.types.TableSchema; +import org.apache.geaflow.dsl.connector.api.FetchData; +import org.apache.geaflow.dsl.connector.api.Partition; +import org.apache.geaflow.dsl.connector.api.window.SizeFetchWindow; +import org.apache.geaflow.dsl.connector.api.window.AllFetchWindow; +import org.mockito.Mockito; +import org.neo4j.driver.Driver; +import org.neo4j.driver.Session; +import org.testcontainers.containers.Neo4jContainer; +import org.testcontainers.utility.DockerImageName; +import org.testng.Assert; +import org.testng.annotations.AfterClass; +import org.testng.annotations.BeforeClass; +import org.testng.annotations.Test; + +public class Neo4jTableSourceTest { + + private static Neo4jContainer neo4jContainer; + private static String neo4jUri; + private static final String USERNAME = "neo4j"; + private static final String PASSWORD = "testpassword"; + private static final String DATABASE = "neo4j"; + + @BeforeClass + public static void setup() { + neo4jContainer = new Neo4jContainer<>(DockerImageName.parse("neo4j:5.14.0")) + .withAdminPassword(PASSWORD) + .withEnv("NEO4J_AUTH", USERNAME + "/" + PASSWORD); + neo4jContainer.start(); + + neo4jUri = neo4jContainer.getBoltUrl(); + + // Initialize test data + try (Driver driver = org.neo4j.driver.GraphDatabase.driver( + neo4jUri, + org.neo4j.driver.AuthTokens.basic(USERNAME, PASSWORD))) { + try (Session session = driver.session()) { + // Create test nodes + for (int i = 1; i <= 50; i++) { + session.run(String.format( + "CREATE (p:TestNode {id: %d, name: 'Node%d', value: %d})", + i, i, i * 10)); + } + + // Create nodes with different types + session.run("CREATE (n:TypeNode {" + + "id: 100, " + + "str_field: 'test', " + + "int_field: 42, " + + "long_field: 9999999999, " + + "double_field: 3.14, " + + "bool_field: true})"); + + // Create relationships + session.run("MATCH (a:TestNode {id: 1}), (b:TestNode {id: 2}) " + + "CREATE (a)-[:LINKS {weight: 0.5}]->(b)"); + session.run("MATCH (a:TestNode {id: 2}), (b:TestNode {id: 3}) " + + "CREATE (a)-[:LINKS {weight: 0.8}]->(b)"); + } + } + } + + @AfterClass + public static void cleanup() { + if (neo4jContainer != null) { + neo4jContainer.stop(); + } + } + + @Test + public void testReadSimpleQuery() throws Exception { + Configuration conf = new Configuration(); + conf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_URI, neo4jUri); + conf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_USERNAME, USERNAME); + conf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_PASSWORD, PASSWORD); + conf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_DATABASE, DATABASE); + conf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_QUERY, + "MATCH (n:TestNode) WHERE n.id <= 5 RETURN n.id as id, n.name as name ORDER BY n.id"); + + List fields = new ArrayList<>(); + fields.add(new TableField("id", Types.LONG, false)); + fields.add(new TableField("name", Types.STRING, false)); + TableSchema tableSchema = new TableSchema(fields); + + Neo4jTableSource source = new Neo4jTableSource(); + source.init(conf, tableSchema); + + RuntimeContext context = Mockito.mock(RuntimeContext.class); + source.open(context); + + List partitions = source.listPartitions(); + Assert.assertEquals(partitions.size(), 1); + + SizeFetchWindow window = new SizeFetchWindow(0L, 100L); + FetchData fetchData = source.fetch(partitions.get(0), Optional.empty(), window); + + List rows = (List) fetchData.getDataIterator(); + Assert.assertEquals(rows.size(), 5); + + // Verify data + for (int i = 0; i < 5; i++) { + Row row = rows.get(i); + Assert.assertEquals(row.getField(0, Types.LONG), (long)(i + 1)); + Assert.assertEquals(row.getField(1, Types.STRING), "Node" + (i + 1)); + } + + source.close(); + } + + @Test + public void testReadWithDifferentDataTypes() throws Exception { + Configuration conf = new Configuration(); + conf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_URI, neo4jUri); + conf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_USERNAME, USERNAME); + conf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_PASSWORD, PASSWORD); + conf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_DATABASE, DATABASE); + conf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_QUERY, + "MATCH (n:TypeNode {id: 100}) RETURN n.str_field as str_field, " + + "n.int_field as int_field, n.long_field as long_field, " + + "n.double_field as double_field, n.bool_field as bool_field"); + + List fields = new ArrayList<>(); + fields.add(new TableField("str_field", Types.STRING, false)); + fields.add(new TableField("int_field", Types.LONG, false)); + fields.add(new TableField("long_field", Types.LONG, false)); + fields.add(new TableField("double_field", Types.DOUBLE, false)); + fields.add(new TableField("bool_field", Types.BOOLEAN, false)); + TableSchema tableSchema = new TableSchema(fields); + + Neo4jTableSource source = new Neo4jTableSource(); + source.init(conf, tableSchema); + + RuntimeContext context = Mockito.mock(RuntimeContext.class); + source.open(context); + + List partitions = source.listPartitions(); + AllFetchWindow window = new AllFetchWindow(0L); + FetchData fetchData = source.fetch(partitions.get(0), Optional.empty(), window); + + List rows = (List) fetchData.getDataIterator(); + Assert.assertEquals(rows.size(), 1); + + Row row = rows.get(0); + Assert.assertEquals(row.getField(0, Types.STRING), "test"); + Assert.assertEquals(row.getField(1, Types.LONG), 42L); + Assert.assertEquals(row.getField(2, Types.LONG), 9999999999L); + Assert.assertEquals((Double) row.getField(3, Types.DOUBLE), 3.14, 0.01); + Assert.assertTrue((Boolean) row.getField(4, Types.BOOLEAN)); + + source.close(); + } + + @Test + public void testPaginationFirstPage() throws Exception { + Configuration conf = new Configuration(); + conf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_URI, neo4jUri); + conf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_USERNAME, USERNAME); + conf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_PASSWORD, PASSWORD); + conf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_DATABASE, DATABASE); + conf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_QUERY, + "MATCH (n:TestNode) RETURN n.id as id ORDER BY n.id"); + + List fields = new ArrayList<>(); + fields.add(new TableField("id", Types.LONG, false)); + TableSchema tableSchema = new TableSchema(fields); + + Neo4jTableSource source = new Neo4jTableSource(); + source.init(conf, tableSchema); + + RuntimeContext context = Mockito.mock(RuntimeContext.class); + source.open(context); + + List partitions = source.listPartitions(); + + // Fetch first page with size 10 + SizeFetchWindow window = new SizeFetchWindow(0L, 10L); + FetchData fetchData = source.fetch(partitions.get(0), Optional.empty(), window); + + List rows = (List) fetchData.getDataIterator(); + Assert.assertEquals(rows.size(), 10); + Assert.assertFalse(fetchData.isFinish()); + Assert.assertEquals(fetchData.getNextOffset().getOffset(), 10); + + source.close(); + } + + @Test + public void testPaginationMultiplePages() throws Exception { + Configuration conf = new Configuration(); + conf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_URI, neo4jUri); + conf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_USERNAME, USERNAME); + conf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_PASSWORD, PASSWORD); + conf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_DATABASE, DATABASE); + conf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_QUERY, + "MATCH (n:TestNode) RETURN n.id as id, n.value as value ORDER BY n.id"); + + List fields = new ArrayList<>(); + fields.add(new TableField("id", Types.LONG, false)); + fields.add(new TableField("value", Types.LONG, false)); + TableSchema tableSchema = new TableSchema(fields); + + Neo4jTableSource source = new Neo4jTableSource(); + source.init(conf, tableSchema); + + RuntimeContext context = Mockito.mock(RuntimeContext.class); + source.open(context); + + List partitions = source.listPartitions(); + SizeFetchWindow window = new SizeFetchWindow(0L, 15L); + + // First page + FetchData fetchData1 = source.fetch(partitions.get(0), Optional.empty(), window); + List rows1 = new ArrayList<>(); + fetchData1.getDataIterator().forEachRemaining(rows1::add); + Assert.assertEquals(rows1.size(), 15); + Assert.assertFalse(fetchData1.isFinish()); + + // Second page + FetchData fetchData2 = source.fetch(partitions.get(0), Optional.of(fetchData1.getNextOffset()), window); + List rows2 = new ArrayList<>(); + fetchData2.getDataIterator().forEachRemaining(rows2::add); + Assert.assertEquals(rows2.size(), 15); + + // Third page + FetchData fetchData3 = source.fetch(partitions.get(0), Optional.of(fetchData2.getNextOffset()), window); + List rows3 = new ArrayList<>(); + fetchData3.getDataIterator().forEachRemaining(rows3::add); + Assert.assertEquals(rows3.size(), 15); + + // Fourth page (should be smaller or empty) + FetchData fetchData4 = source.fetch(partitions.get(0), Optional.of(fetchData3.getNextOffset()), window); + List rows4 = new ArrayList<>(); + fetchData4.getDataIterator().forEachRemaining(rows4::add); + Assert.assertTrue(rows4.size() <= 15); + + source.close(); + } + + @Test + public void testReadRelationships() throws Exception { + Configuration conf = new Configuration(); + conf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_URI, neo4jUri); + conf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_USERNAME, USERNAME); + conf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_PASSWORD, PASSWORD); + conf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_DATABASE, DATABASE); + conf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_QUERY, + "MATCH (a)-[r:LINKS]->(b) " + + "RETURN a.id as source, b.id as target, r.weight as weight ORDER BY a.id"); + + List fields = new ArrayList<>(); + fields.add(new TableField("source", Types.LONG, false)); + fields.add(new TableField("target", Types.LONG, false)); + fields.add(new TableField("weight", Types.DOUBLE, false)); + TableSchema tableSchema = new TableSchema(fields); + + Neo4jTableSource source = new Neo4jTableSource(); + source.init(conf, tableSchema); + + RuntimeContext context = Mockito.mock(RuntimeContext.class); + source.open(context); + + List partitions = source.listPartitions(); + AllFetchWindow window = new AllFetchWindow(0L); + FetchData fetchData = source.fetch(partitions.get(0), Optional.empty(), window); + + List rows = new ArrayList<>(); + fetchData.getDataIterator().forEachRemaining(rows::add); + Assert.assertEquals(rows.size(), 2); + + // Verify first relationship + Row row1 = rows.get(0); + Assert.assertEquals(row1.getField(0, Types.LONG), 1L); + Assert.assertEquals(row1.getField(1, Types.LONG), 2L); + Assert.assertEquals((Double) row1.getField(2, Types.DOUBLE), 0.5, 0.01); + + source.close(); + } + + @Test + public void testEmptyResult() throws Exception { + Configuration conf = new Configuration(); + conf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_URI, neo4jUri); + conf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_USERNAME, USERNAME); + conf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_PASSWORD, PASSWORD); + conf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_DATABASE, DATABASE); + conf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_QUERY, + "MATCH (n:NonExistent) RETURN n.id as id"); + + List fields = new ArrayList<>(); + fields.add(new TableField("id", Types.LONG, false)); + TableSchema tableSchema = new TableSchema(fields); + + Neo4jTableSource source = new Neo4jTableSource(); + source.init(conf, tableSchema); + + RuntimeContext context = Mockito.mock(RuntimeContext.class); + source.open(context); + + List partitions = source.listPartitions(); + AllFetchWindow window = new AllFetchWindow(0L); + FetchData fetchData = source.fetch(partitions.get(0), Optional.empty(), window); + + List rows = (List) fetchData.getDataIterator(); + Assert.assertEquals(rows.size(), 0); + Assert.assertTrue(fetchData.isFinish()); + + source.close(); + } + + @Test + public void testReadWithNullValues() throws Exception { + // Create a node with null values + try (Driver driver = org.neo4j.driver.GraphDatabase.driver( + neo4jUri, + org.neo4j.driver.AuthTokens.basic(USERNAME, PASSWORD))) { + try (Session session = driver.session()) { + session.run("CREATE (n:NullTestNode {id: 999, name: 'test'})"); + } + } + + Configuration conf = new Configuration(); + conf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_URI, neo4jUri); + conf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_USERNAME, USERNAME); + conf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_PASSWORD, PASSWORD); + conf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_DATABASE, DATABASE); + conf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_QUERY, + "MATCH (n:NullTestNode {id: 999}) RETURN n.id as id, n.name as name, n.missing as missing"); + + List fields = new ArrayList<>(); + fields.add(new TableField("id", Types.LONG, false)); + fields.add(new TableField("name", Types.STRING, false)); + fields.add(new TableField("missing", Types.STRING, true)); + TableSchema tableSchema = new TableSchema(fields); + + Neo4jTableSource source = new Neo4jTableSource(); + source.init(conf, tableSchema); + + RuntimeContext context = Mockito.mock(RuntimeContext.class); + source.open(context); + + List partitions = source.listPartitions(); + AllFetchWindow window = new AllFetchWindow(0L); + FetchData fetchData = source.fetch(partitions.get(0), Optional.empty(), window); + + List rows = (List) fetchData.getDataIterator(); + Assert.assertEquals(rows.size(), 1); + + Row row = rows.get(0); + Assert.assertEquals(row.getField(0, Types.LONG), 999L); + Assert.assertEquals(row.getField(1, Types.STRING), "test"); + Assert.assertNull(row.getField(2, Types.STRING)); + + source.close(); + } + + @Test + public void testAllWindowType() throws Exception { + Configuration conf = new Configuration(); + conf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_URI, neo4jUri); + conf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_USERNAME, USERNAME); + conf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_PASSWORD, PASSWORD); + conf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_DATABASE, DATABASE); + conf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_QUERY, + "MATCH (n:TestNode) WHERE n.id <= 10 RETURN n.id as id ORDER BY n.id"); + + List fields = new ArrayList<>(); + fields.add(new TableField("id", Types.LONG, false)); + TableSchema tableSchema = new TableSchema(fields); + + Neo4jTableSource source = new Neo4jTableSource(); + source.init(conf, tableSchema); + + RuntimeContext context = Mockito.mock(RuntimeContext.class); + source.open(context); + + List partitions = source.listPartitions(); + AllFetchWindow window = new AllFetchWindow(0L); + FetchData fetchData = source.fetch(partitions.get(0), Optional.empty(), window); + + List rows = (List) fetchData.getDataIterator(); + Assert.assertEquals(rows.size(), 10); + Assert.assertTrue(fetchData.isFinish()); // ALL_WINDOW should mark as finished + + source.close(); + } + + @Test(expectedExceptions = org.apache.geaflow.dsl.common.exception.GeaFlowDSLException.class) + public void testMissingQuery() { + Configuration conf = new Configuration(); + conf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_URI, neo4jUri); + conf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_USERNAME, USERNAME); + conf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_PASSWORD, PASSWORD); + conf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_DATABASE, DATABASE); + // No query specified + + List fields = new ArrayList<>(); + fields.add(new TableField("id", Types.LONG, false)); + TableSchema tableSchema = new TableSchema(fields); + + Neo4jTableSource source = new Neo4jTableSource(); + source.init(conf, tableSchema); // Should throw exception + } +} diff --git a/geaflow/geaflow-dsl/geaflow-dsl-connector/pom.xml b/geaflow/geaflow-dsl/geaflow-dsl-connector/pom.xml index b7c9821a6..a304238bf 100644 --- a/geaflow/geaflow-dsl/geaflow-dsl-connector/pom.xml +++ b/geaflow/geaflow-dsl/geaflow-dsl-connector/pom.xml @@ -47,6 +47,7 @@ geaflow-dsl-connector-pulsar geaflow-dsl-connector-random geaflow-dsl-connector-paimon + geaflow-dsl-connector-neo4j From 5e5bbe39499a011aabf9fc36346122269427c862 Mon Sep 17 00:00:00 2001 From: "jishiwen.jsw" Date: Sun, 9 Nov 2025 21:27:53 +0800 Subject: [PATCH 2/5] fix: adapter jdk 8 Signed-off-by: jishiwen.jsw --- .../pom.xml | 61 +++ .../ElasticsearchTableSource.java | 271 +++++++++++ .../ElasticsearchConfigKeysTest.java | 67 +++ .../ElasticsearchIntegrationTest.java | 209 +++++++++ .../ElasticsearchTableConnectorTest.java | 92 ++++ .../ElasticsearchTableSinkTest.java | 100 ++++ .../ElasticsearchTableSourceTest.java | 98 ++++ .../geaflow-dsl-connector-neo4j/pom.xml | 15 +- .../connector/neo4j/Neo4jIntegrationTest.java | 387 ---------------- .../neo4j/Neo4jTableConnectorTest.java | 362 ++------------- .../connector/neo4j/Neo4jTableSinkTest.java | 388 ---------------- .../connector/neo4j/Neo4jTableSourceTest.java | 433 ------------------ 12 files changed, 928 insertions(+), 1555 deletions(-) create mode 100644 geaflow/geaflow-dsl/geaflow-dsl-connector/geaflow-dsl-connector-elasticsearch/pom.xml create mode 100644 geaflow/geaflow-dsl/geaflow-dsl-connector/geaflow-dsl-connector-elasticsearch/src/main/java/org/apache/geaflow/dsl/connector/elasticsearch/ElasticsearchTableSource.java create mode 100644 geaflow/geaflow-dsl/geaflow-dsl-connector/geaflow-dsl-connector-elasticsearch/src/test/java/org/apache/geaflow/dsl/connector/elasticsearch/ElasticsearchConfigKeysTest.java create mode 100644 geaflow/geaflow-dsl/geaflow-dsl-connector/geaflow-dsl-connector-elasticsearch/src/test/java/org/apache/geaflow/dsl/connector/elasticsearch/ElasticsearchIntegrationTest.java create mode 100644 geaflow/geaflow-dsl/geaflow-dsl-connector/geaflow-dsl-connector-elasticsearch/src/test/java/org/apache/geaflow/dsl/connector/elasticsearch/ElasticsearchTableConnectorTest.java create mode 100644 geaflow/geaflow-dsl/geaflow-dsl-connector/geaflow-dsl-connector-elasticsearch/src/test/java/org/apache/geaflow/dsl/connector/elasticsearch/ElasticsearchTableSinkTest.java create mode 100644 geaflow/geaflow-dsl/geaflow-dsl-connector/geaflow-dsl-connector-elasticsearch/src/test/java/org/apache/geaflow/dsl/connector/elasticsearch/ElasticsearchTableSourceTest.java delete mode 100644 geaflow/geaflow-dsl/geaflow-dsl-connector/geaflow-dsl-connector-neo4j/src/test/java/org/apache/geaflow/dsl/connector/neo4j/Neo4jIntegrationTest.java delete mode 100644 geaflow/geaflow-dsl/geaflow-dsl-connector/geaflow-dsl-connector-neo4j/src/test/java/org/apache/geaflow/dsl/connector/neo4j/Neo4jTableSinkTest.java delete mode 100644 geaflow/geaflow-dsl/geaflow-dsl-connector/geaflow-dsl-connector-neo4j/src/test/java/org/apache/geaflow/dsl/connector/neo4j/Neo4jTableSourceTest.java diff --git a/geaflow/geaflow-dsl/geaflow-dsl-connector/geaflow-dsl-connector-elasticsearch/pom.xml b/geaflow/geaflow-dsl/geaflow-dsl-connector/geaflow-dsl-connector-elasticsearch/pom.xml new file mode 100644 index 000000000..2a9b91c1d --- /dev/null +++ b/geaflow/geaflow-dsl/geaflow-dsl-connector/geaflow-dsl-connector-elasticsearch/pom.xml @@ -0,0 +1,61 @@ + + + + + + org.apache.geaflow + geaflow-dsl-connector + 0.6.8-SNAPSHOT + + 4.0.0 + + geaflow-dsl-connector-elasticsearch + geaflow-dsl-connector-elasticsearch + + + 7.17.10 + + + + + org.apache.geaflow + geaflow-dsl-common + + + org.apache.geaflow + geaflow-dsl-connector-api + + + org.elasticsearch.client + elasticsearch-rest-high-level-client + ${elasticsearch.version} + + + + org.testng + testng + ${testng.version} + test + + + + diff --git a/geaflow/geaflow-dsl/geaflow-dsl-connector/geaflow-dsl-connector-elasticsearch/src/main/java/org/apache/geaflow/dsl/connector/elasticsearch/ElasticsearchTableSource.java b/geaflow/geaflow-dsl/geaflow-dsl-connector/geaflow-dsl-connector-elasticsearch/src/main/java/org/apache/geaflow/dsl/connector/elasticsearch/ElasticsearchTableSource.java new file mode 100644 index 000000000..0576de1cc --- /dev/null +++ b/geaflow/geaflow-dsl/geaflow-dsl-connector/geaflow-dsl-connector-elasticsearch/src/main/java/org/apache/geaflow/dsl/connector/elasticsearch/ElasticsearchTableSource.java @@ -0,0 +1,271 @@ +/* + * 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.geaflow.dsl.connector.elasticsearch; + +import com.google.gson.Gson; +import com.google.gson.reflect.TypeToken; +import java.io.IOException; +import java.lang.reflect.Type; +import java.util.ArrayList; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import org.apache.geaflow.api.context.RuntimeContext; +import org.apache.geaflow.common.config.Configuration; +import org.apache.geaflow.dsl.common.types.StructType; +import org.apache.geaflow.dsl.common.types.TableSchema; +import org.apache.geaflow.dsl.connector.api.FetchData; +import org.apache.geaflow.dsl.connector.api.Partition; +import org.apache.geaflow.dsl.connector.api.TableSource; +import org.apache.geaflow.dsl.connector.api.serde.TableDeserializer; +import org.apache.geaflow.dsl.connector.api.window.FetchWindow; +import org.apache.http.HttpHost; +import org.apache.http.auth.AuthScope; +import org.apache.http.auth.UsernamePasswordCredentials; +import org.apache.http.client.CredentialsProvider; +import org.apache.http.impl.client.BasicCredentialsProvider; +import org.elasticsearch.action.search.ClearScrollRequest; +import org.elasticsearch.action.search.SearchRequest; +import org.elasticsearch.action.search.SearchResponse; +import org.elasticsearch.client.RequestOptions; +import org.elasticsearch.client.RestClient; +import org.elasticsearch.client.RestClientBuilder; +import org.elasticsearch.client.RestHighLevelClient; +import org.elasticsearch.core.TimeValue; +import org.elasticsearch.search.Scroll; +import org.elasticsearch.search.SearchHit; +import org.elasticsearch.search.builder.SearchSourceBuilder; + +public class ElasticsearchTableSource implements TableSource { + + private static final Gson GSON = new Gson(); + private static final Type MAP_TYPE = new TypeToken>(){}.getType(); + + private StructType schema; + private String hosts; + private String indexName; + private String username; + private String password; + private String scrollTimeout; + private int connectionTimeout; + private int socketTimeout; + + private RestHighLevelClient client; + + @Override + public void init(Configuration tableConf, TableSchema tableSchema) { + this.schema = tableSchema; + this.hosts = tableConf.getString(ElasticsearchConfigKeys.GEAFLOW_DSL_ELASTICSEARCH_HOSTS); + this.indexName = tableConf.getString(ElasticsearchConfigKeys.GEAFLOW_DSL_ELASTICSEARCH_INDEX); + this.username = tableConf.getString(ElasticsearchConfigKeys.GEAFLOW_DSL_ELASTICSEARCH_USERNAME, ""); + this.password = tableConf.getString(ElasticsearchConfigKeys.GEAFLOW_DSL_ELASTICSEARCH_PASSWORD, ""); + this.scrollTimeout = tableConf.getString(ElasticsearchConfigKeys.GEAFLOW_DSL_ELASTICSEARCH_SCROLL_TIMEOUT, + ElasticsearchConstants.DEFAULT_SCROLL_TIMEOUT); + this.connectionTimeout = tableConf.getInteger(ElasticsearchConfigKeys.GEAFLOW_DSL_ELASTICSEARCH_CONNECTION_TIMEOUT, + ElasticsearchConstants.DEFAULT_CONNECTION_TIMEOUT); + this.socketTimeout = tableConf.getInteger(ElasticsearchConfigKeys.GEAFLOW_DSL_ELASTICSEARCH_SOCKET_TIMEOUT, + ElasticsearchConstants.DEFAULT_SOCKET_TIMEOUT); + } + + @Override + public void open(RuntimeContext context) { + try { + this.client = createElasticsearchClient(); + } catch (Exception e) { + throw new RuntimeException("Failed to initialize Elasticsearch client", e); + } + } + + @Override + public List listPartitions() { + return java.util.Collections.singletonList(new ElasticsearchPartition(indexName)); + } + + @Override + public TableDeserializer getDeserializer(Configuration conf) { + return new TableDeserializer() { + @Override + public void init(Configuration configuration, StructType structType) { + // Initialization if needed + } + + @Override + public java.util.List deserialize(IN record) { + if (record instanceof SearchHit) { + SearchHit hit = (SearchHit) record; + Map source = hit.getSourceAsMap(); + if (source == null) { + source = GSON.fromJson(hit.getSourceAsString(), MAP_TYPE); + } + + // Convert map to Row based on schema + Object[] values = new Object[schema.size()]; + for (int i = 0; i < schema.size(); i++) { + String fieldName = schema.getFields().get(i).getName(); + values[i] = source.get(fieldName); + } + org.apache.geaflow.dsl.common.data.Row row = + org.apache.geaflow.dsl.common.data.impl.ObjectRow.create(values); + return java.util.Collections.singletonList(row); + } + return java.util.Collections.emptyList(); + } + }; + } + + @Override + public FetchData fetch(Partition partition, Optional startOffset, + FetchWindow windowInfo) throws IOException { + try { + SearchRequest searchRequest = new SearchRequest(indexName); + SearchSourceBuilder searchSourceBuilder = new SearchSourceBuilder(); + searchSourceBuilder.size(1000); // Batch size + + searchRequest.source(searchSourceBuilder); + + // Use scroll for large dataset reading + Scroll scroll = new Scroll(TimeValue.parseTimeValue(scrollTimeout, "scroll_timeout")); + searchRequest.scroll(scroll); + + SearchResponse searchResponse = client.search(searchRequest, RequestOptions.DEFAULT); + String scrollId = searchResponse.getScrollId(); + SearchHit[] searchHits = searchResponse.getHits().getHits(); + + List dataList = new ArrayList<>(); + for (SearchHit hit : searchHits) { + dataList.add((T) hit); + } + + // Clear scroll + ClearScrollRequest clearScrollRequest = new ClearScrollRequest(); + clearScrollRequest.addScrollId(scrollId); + client.clearScroll(clearScrollRequest, RequestOptions.DEFAULT); + + ElasticsearchOffset nextOffset = new ElasticsearchOffset(scrollId); + return (FetchData) FetchData.createStreamFetch(dataList, nextOffset, false); + } catch (Exception e) { + throw new IOException("Failed to fetch data from Elasticsearch", e); + } + } + + @Override + public void close() { + try { + if (client != null) { + client.close(); + } + } catch (IOException e) { + // Log error but don't throw exception in close method + e.printStackTrace(); + } + } + + private RestHighLevelClient createElasticsearchClient() { + try { + String[] hostArray = hosts.split(","); + HttpHost[] httpHosts = new HttpHost[hostArray.length]; + + for (int i = 0; i < hostArray.length; i++) { + String host = hostArray[i].trim(); + if (host.startsWith("http://")) { + host = host.substring(7); + } else if (host.startsWith("https://")) { + host = host.substring(8); + } + + String[] parts = host.split(":"); + String hostname = parts[0]; + int port = parts.length > 1 ? Integer.parseInt(parts[1]) : 9200; + httpHosts[i] = new HttpHost(hostname, port, "http"); + } + + RestClientBuilder builder = RestClient.builder(httpHosts); + + // Configure timeouts + builder.setRequestConfigCallback(requestConfigBuilder -> { + requestConfigBuilder.setConnectTimeout(connectionTimeout); + requestConfigBuilder.setSocketTimeout(socketTimeout); + return requestConfigBuilder; + }); + + // Configure authentication if provided + if (username != null && !username.isEmpty() && password != null) { + final CredentialsProvider credentialsProvider = new BasicCredentialsProvider(); + credentialsProvider.setCredentials(AuthScope.ANY, + new UsernamePasswordCredentials(username, password)); + + builder.setHttpClientConfigCallback(httpClientBuilder -> { + httpClientBuilder.setDefaultCredentialsProvider(credentialsProvider); + return httpClientBuilder; + }); + } + + return new RestHighLevelClient(builder); + } catch (Exception e) { + throw new RuntimeException("Failed to create Elasticsearch client", e); + } + } + + public static class ElasticsearchPartition implements Partition { + private final String indexName; + + public ElasticsearchPartition(String indexName) { + this.indexName = indexName; + } + + @Override + public String getName() { + return indexName; + } + } + + public static class ElasticsearchOffset implements org.apache.geaflow.dsl.connector.api.Offset { + private final String scrollId; + private final long timestamp; + + public ElasticsearchOffset(String scrollId) { + this(scrollId, System.currentTimeMillis()); + } + + public ElasticsearchOffset(String scrollId, long timestamp) { + this.scrollId = scrollId; + this.timestamp = timestamp; + } + + public String getScrollId() { + return scrollId; + } + + @Override + public String humanReadable() { + return "ElasticsearchOffset{scrollId='" + scrollId + "', timestamp=" + timestamp + "}"; + } + + @Override + public long getOffset() { + return timestamp; + } + + @Override + public boolean isTimestamp() { + return true; + } + } +} diff --git a/geaflow/geaflow-dsl/geaflow-dsl-connector/geaflow-dsl-connector-elasticsearch/src/test/java/org/apache/geaflow/dsl/connector/elasticsearch/ElasticsearchConfigKeysTest.java b/geaflow/geaflow-dsl/geaflow-dsl-connector/geaflow-dsl-connector-elasticsearch/src/test/java/org/apache/geaflow/dsl/connector/elasticsearch/ElasticsearchConfigKeysTest.java new file mode 100644 index 000000000..0eaf9cf91 --- /dev/null +++ b/geaflow/geaflow-dsl/geaflow-dsl-connector/geaflow-dsl-connector-elasticsearch/src/test/java/org/apache/geaflow/dsl/connector/elasticsearch/ElasticsearchConfigKeysTest.java @@ -0,0 +1,67 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.geaflow.dsl.connector.elasticsearch; + +import org.apache.geaflow.common.config.Configuration; +import org.testng.Assert; +import org.testng.annotations.Test; + +public class ElasticsearchConfigKeysTest { + + @Test + public void testConfigKeys() { + Configuration config = new Configuration(); + + config.put(ElasticsearchConfigKeys.GEAFLOW_DSL_ELASTICSEARCH_HOSTS, "localhost:9200"); + config.put(ElasticsearchConfigKeys.GEAFLOW_DSL_ELASTICSEARCH_INDEX, "test_index"); + config.put(ElasticsearchConfigKeys.GEAFLOW_DSL_ELASTICSEARCH_USERNAME, "elastic"); + + Assert.assertEquals(config.getString(ElasticsearchConfigKeys.GEAFLOW_DSL_ELASTICSEARCH_HOSTS), + (Object) "localhost:9200"); + Assert.assertEquals(config.getString(ElasticsearchConfigKeys.GEAFLOW_DSL_ELASTICSEARCH_INDEX), + (Object) "test_index"); + Assert.assertEquals(config.getString(ElasticsearchConfigKeys.GEAFLOW_DSL_ELASTICSEARCH_USERNAME), + (Object) "elastic"); + } + + @Test + public void testDefaultValues() { + Configuration config = new Configuration(); + + String batchSize = config.getString(ElasticsearchConfigKeys.GEAFLOW_DSL_ELASTICSEARCH_BATCH_SIZE); + String scrollTimeout = config.getString(ElasticsearchConfigKeys.GEAFLOW_DSL_ELASTICSEARCH_SCROLL_TIMEOUT); + + Assert.assertEquals(batchSize, (Object) String.valueOf(ElasticsearchConstants.DEFAULT_BATCH_SIZE)); + Assert.assertEquals(scrollTimeout, (Object) ElasticsearchConstants.DEFAULT_SCROLL_TIMEOUT); + } + + @Test + public void testTimeoutValues() { + Configuration config = new Configuration(); + + String connectionTimeout = config.getString(ElasticsearchConfigKeys.GEAFLOW_DSL_ELASTICSEARCH_CONNECTION_TIMEOUT); + String socketTimeout = config.getString(ElasticsearchConfigKeys.GEAFLOW_DSL_ELASTICSEARCH_SOCKET_TIMEOUT); + + Assert.assertEquals(connectionTimeout, + (Object) String.valueOf(ElasticsearchConstants.DEFAULT_CONNECTION_TIMEOUT)); + Assert.assertEquals(socketTimeout, + (Object) String.valueOf(ElasticsearchConstants.DEFAULT_SOCKET_TIMEOUT)); + } +} diff --git a/geaflow/geaflow-dsl/geaflow-dsl-connector/geaflow-dsl-connector-elasticsearch/src/test/java/org/apache/geaflow/dsl/connector/elasticsearch/ElasticsearchIntegrationTest.java b/geaflow/geaflow-dsl/geaflow-dsl-connector/geaflow-dsl-connector-elasticsearch/src/test/java/org/apache/geaflow/dsl/connector/elasticsearch/ElasticsearchIntegrationTest.java new file mode 100644 index 000000000..821d88725 --- /dev/null +++ b/geaflow/geaflow-dsl/geaflow-dsl-connector/geaflow-dsl-connector-elasticsearch/src/test/java/org/apache/geaflow/dsl/connector/elasticsearch/ElasticsearchIntegrationTest.java @@ -0,0 +1,209 @@ +/* + * 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.geaflow.dsl.connector.elasticsearch; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Iterator; +import java.util.List; +import java.util.Optional; +import org.apache.geaflow.api.context.RuntimeContext; +import org.apache.geaflow.common.config.Configuration; +import org.apache.geaflow.common.type.Types; +import org.apache.geaflow.dsl.common.data.Row; +import org.apache.geaflow.dsl.common.data.impl.ObjectRow; +import org.apache.geaflow.dsl.common.types.StructType; +import org.apache.geaflow.dsl.common.types.TableField; +import org.apache.geaflow.dsl.common.types.TableSchema; +import org.apache.geaflow.dsl.connector.api.FetchData; +import org.apache.geaflow.dsl.connector.api.Partition; +import org.apache.geaflow.dsl.connector.api.window.SizeFetchWindow; +import org.mockito.Mockito; +import org.testcontainers.containers.wait.strategy.Wait; +import org.testcontainers.elasticsearch.ElasticsearchContainer; +import org.testcontainers.utility.DockerImageName; +import org.testng.Assert; +import org.testng.annotations.AfterClass; +import org.testng.annotations.BeforeClass; +import org.testng.annotations.Test; + +public class ElasticsearchIntegrationTest { + + private static final String ELASTICSEARCH_VERSION = "8.11.0"; + private ElasticsearchContainer container; + private Configuration config; + private StructType sinkSchema; + private TableSchema sourceSchema; + + @BeforeClass + public void setUp() { + // Skip integration tests in CI environment if Docker is not available + String skipIntegrationTests = System.getProperty("skipIntegrationTests", "false"); + if ("true".equalsIgnoreCase(skipIntegrationTests)) { + throw new org.testng.SkipException("Integration tests are disabled"); + } + + // Check if Docker is available + try { + ProcessBuilder pb = new ProcessBuilder("docker", "info"); + Process process = pb.start(); + int exitCode = process.waitFor(); + if (exitCode != 0) { + throw new org.testng.SkipException("Docker is not running"); + } + } catch (IOException | InterruptedException e) { + throw new org.testng.SkipException("Docker is not available, skipping test"); + } + + DockerImageName image = DockerImageName + .parse("docker.elastic.co/elasticsearch/elasticsearch") + .withTag(ELASTICSEARCH_VERSION); + + container = new ElasticsearchContainer(image) + .withEnv("xpack.security.enabled", "false") + .withEnv("discovery.type", "single-node") + .waitingFor(Wait.forHttp("/_cluster/health") + .forStatusCode(200)); + + container.start(); + + // Configure connection + config = new Configuration(); + config.put(ElasticsearchConfigKeys.GEAFLOW_DSL_ELASTICSEARCH_HOSTS, + container.getHttpHostAddress()); + config.put(ElasticsearchConfigKeys.GEAFLOW_DSL_ELASTICSEARCH_INDEX, "test_users"); + config.put(ElasticsearchConfigKeys.GEAFLOW_DSL_ELASTICSEARCH_DOCUMENT_ID_FIELD, "id"); + config.put(ElasticsearchConfigKeys.GEAFLOW_DSL_ELASTICSEARCH_BATCH_SIZE, "10"); + + // Create schema + TableField idField = new TableField("id", Types.INTEGER, false); + TableField nameField = new TableField("name", Types.STRING, false); + TableField ageField = new TableField("age", Types.INTEGER, false); + sinkSchema = new StructType(Arrays.asList(idField, nameField, ageField)); + sourceSchema = new TableSchema(sinkSchema); + } + + @AfterClass + public void tearDown() { + if (container != null) { + container.stop(); + } + } + + @Test + public void testWriteAndRead() throws IOException { + RuntimeContext context = Mockito.mock(RuntimeContext.class); + + // Write data + ElasticsearchTableSink sink = new ElasticsearchTableSink(); + sink.init(config, sinkSchema); + sink.open(context); + + for (int i = 1; i <= 5; i++) { + Row row = ObjectRow.create(i, "User" + i, 20 + i); + sink.write(row); + } + sink.finish(); + sink.close(); + + // Wait for indexing + try { + Thread.sleep(2000); + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + } + + // Read data + ElasticsearchTableSource source = new ElasticsearchTableSource(); + source.init(config, sourceSchema); + source.open(context); + + List partitions = source.listPartitions(); + Assert.assertEquals(partitions.size(), 1); + + FetchData fetchData = source.fetch(partitions.get(0), Optional.empty(), + new SizeFetchWindow(0L, 10L)); + Assert.assertNotNull(fetchData); + + Iterator rowIterator = fetchData.getDataIterator(); + Assert.assertNotNull(rowIterator); + List rows = new ArrayList<>(); + rowIterator.forEachRemaining(rows::add); + Assert.assertTrue(rows.size() > 0); + + source.close(); + } + + @Test + public void testBulkWrite() throws IOException { + RuntimeContext context = Mockito.mock(RuntimeContext.class); + + ElasticsearchTableSink sink = new ElasticsearchTableSink(); + sink.init(config, sinkSchema); + sink.open(context); + + // Write more than batch size + for (int i = 1; i <= 25; i++) { + Row row = ObjectRow.create(i, "BulkUser" + i, 30 + i); + sink.write(row); + } + + sink.finish(); + sink.close(); + } + + @Test + public void testQueryDsl() throws IOException { + RuntimeContext context = Mockito.mock(RuntimeContext.class); + + // First write some data + ElasticsearchTableSink sink = new ElasticsearchTableSink(); + config.put(ElasticsearchConfigKeys.GEAFLOW_DSL_ELASTICSEARCH_INDEX, "test_query"); + sink.init(config, sinkSchema); + sink.open(context); + + Row row1 = ObjectRow.create(1, "Alice", 25); + Row row2 = ObjectRow.create(2, "Bob", 30); + sink.write(row1); + sink.write(row2); + sink.finish(); + sink.close(); + + // Wait for indexing + try { + Thread.sleep(2000); + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + } + + // Read with query + ElasticsearchTableSource source = new ElasticsearchTableSource(); + source.init(config, sourceSchema); + source.open(context); + + List partitions = source.listPartitions(); + FetchData fetchData = source.fetch(partitions.get(0), Optional.empty(), + new SizeFetchWindow(0L, 10L)); + + Assert.assertNotNull(fetchData); + source.close(); + } +} diff --git a/geaflow/geaflow-dsl/geaflow-dsl-connector/geaflow-dsl-connector-elasticsearch/src/test/java/org/apache/geaflow/dsl/connector/elasticsearch/ElasticsearchTableConnectorTest.java b/geaflow/geaflow-dsl/geaflow-dsl-connector/geaflow-dsl-connector-elasticsearch/src/test/java/org/apache/geaflow/dsl/connector/elasticsearch/ElasticsearchTableConnectorTest.java new file mode 100644 index 000000000..a5f594b13 --- /dev/null +++ b/geaflow/geaflow-dsl/geaflow-dsl-connector/geaflow-dsl-connector-elasticsearch/src/test/java/org/apache/geaflow/dsl/connector/elasticsearch/ElasticsearchTableConnectorTest.java @@ -0,0 +1,92 @@ +/* + * 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.geaflow.dsl.connector.elasticsearch; + +import java.util.Arrays; +import java.util.List; +import org.apache.geaflow.common.config.Configuration; +import org.apache.geaflow.common.type.Types; +import org.apache.geaflow.dsl.common.types.StructType; +import org.apache.geaflow.dsl.common.types.TableField; +import org.apache.geaflow.dsl.common.types.TableSchema; +import org.apache.geaflow.dsl.connector.api.TableSink; +import org.apache.geaflow.dsl.connector.api.TableSource; +import org.testng.Assert; +import org.testng.annotations.BeforeMethod; +import org.testng.annotations.Test; + +public class ElasticsearchTableConnectorTest { + + private ElasticsearchTableConnector connector; + private Configuration config; + private TableSchema schema; + + @BeforeMethod + public void setUp() { + connector = new ElasticsearchTableConnector(); + config = new Configuration(); + config.put(ElasticsearchConfigKeys.GEAFLOW_DSL_ELASTICSEARCH_HOSTS, "localhost:9200"); + config.put(ElasticsearchConfigKeys.GEAFLOW_DSL_ELASTICSEARCH_INDEX, "test_index"); + config.put(ElasticsearchConfigKeys.GEAFLOW_DSL_ELASTICSEARCH_DOCUMENT_ID_FIELD, "id"); + + TableField idField = new TableField("id", Types.INTEGER, false); + TableField nameField = new TableField("name", Types.STRING, false); + schema = new TableSchema(new StructType(Arrays.asList(idField, nameField))); + } + + @Test + public void testGetName() { + Assert.assertEquals(connector.getType(), "ELASTICSEARCH"); + } + + @Test + public void testGetSource() { + TableSource source = connector.createSource(config); + Assert.assertNotNull(source); + Assert.assertTrue(source instanceof ElasticsearchTableSource); + } + + @Test + public void testGetSink() { + TableSink sink = connector.createSink(config); + Assert.assertNotNull(sink); + Assert.assertTrue(sink instanceof ElasticsearchTableSink); + } + + @Test + public void testMultipleSourceInstances() { + TableSource source1 = connector.createSource(config); + TableSource source2 = connector.createSource(config); + + Assert.assertNotNull(source1); + Assert.assertNotNull(source2); + Assert.assertNotSame(source1, source2); + } + + @Test + public void testMultipleSinkInstances() { + TableSink sink1 = connector.createSink(config); + TableSink sink2 = connector.createSink(config); + + Assert.assertNotNull(sink1); + Assert.assertNotNull(sink2); + Assert.assertNotSame(sink1, sink2); + } +} diff --git a/geaflow/geaflow-dsl/geaflow-dsl-connector/geaflow-dsl-connector-elasticsearch/src/test/java/org/apache/geaflow/dsl/connector/elasticsearch/ElasticsearchTableSinkTest.java b/geaflow/geaflow-dsl/geaflow-dsl-connector/geaflow-dsl-connector-elasticsearch/src/test/java/org/apache/geaflow/dsl/connector/elasticsearch/ElasticsearchTableSinkTest.java new file mode 100644 index 000000000..942839522 --- /dev/null +++ b/geaflow/geaflow-dsl/geaflow-dsl-connector/geaflow-dsl-connector-elasticsearch/src/test/java/org/apache/geaflow/dsl/connector/elasticsearch/ElasticsearchTableSinkTest.java @@ -0,0 +1,100 @@ +/* + * 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.geaflow.dsl.connector.elasticsearch; + +import java.util.Arrays; +import java.util.List; +import org.apache.geaflow.common.config.Configuration; +import org.apache.geaflow.common.type.Types; +import org.apache.geaflow.dsl.common.data.Row; +import org.apache.geaflow.dsl.common.data.impl.ObjectRow; +import org.apache.geaflow.dsl.common.types.StructType; +import org.apache.geaflow.dsl.common.types.TableField; +import org.testng.Assert; +import org.testng.annotations.BeforeMethod; +import org.testng.annotations.Test; + +public class ElasticsearchTableSinkTest { + + private ElasticsearchTableSink sink; + private Configuration config; + private StructType schema; + + @BeforeMethod + public void setUp() { + sink = new ElasticsearchTableSink(); + config = new Configuration(); + config.put(ElasticsearchConfigKeys.GEAFLOW_DSL_ELASTICSEARCH_HOSTS, "localhost:9200"); + config.put(ElasticsearchConfigKeys.GEAFLOW_DSL_ELASTICSEARCH_INDEX, "test_index"); + config.put(ElasticsearchConfigKeys.GEAFLOW_DSL_ELASTICSEARCH_DOCUMENT_ID_FIELD, "id"); + + TableField idField = new TableField("id", Types.INTEGER, false); + TableField nameField = new TableField("name", Types.STRING, false); + TableField ageField = new TableField("age", Types.INTEGER, false); + schema = new StructType(Arrays.asList(idField, nameField, ageField)); + } + + @Test + public void testInit() { + sink.init(config, schema); + Assert.assertNotNull(sink); + } + + @Test(expectedExceptions = RuntimeException.class) + public void testInitWithoutIndex() { + Configuration invalidConfig = new Configuration(); + invalidConfig.put(ElasticsearchConfigKeys.GEAFLOW_DSL_ELASTICSEARCH_HOSTS, "localhost:9200"); + sink.init(invalidConfig, schema); + } + + @Test + public void testInitWithoutIdField() { + Configuration invalidConfig = new Configuration(); + invalidConfig.put(ElasticsearchConfigKeys.GEAFLOW_DSL_ELASTICSEARCH_HOSTS, "localhost:9200"); + invalidConfig.put(ElasticsearchConfigKeys.GEAFLOW_DSL_ELASTICSEARCH_INDEX, "test_index"); + sink.init(invalidConfig, schema); + Assert.assertNotNull(sink); + } + + @Test + public void testBatchSizeConfiguration() { + config.put(ElasticsearchConfigKeys.GEAFLOW_DSL_ELASTICSEARCH_BATCH_SIZE, "500"); + sink.init(config, schema); + Assert.assertNotNull(sink); + } + + @Test + public void testWriteRow() { + sink.init(config, schema); + + Row row = ObjectRow.create(1, "Alice", 25); + Assert.assertNotNull(row); + } + + @Test + public void testMultipleWrites() { + sink.init(config, schema); + + for (int i = 0; i < 10; i++) { + Row row = ObjectRow.create(i, "User" + i, 20 + i); + Assert.assertNotNull(row); + } + } +} diff --git a/geaflow/geaflow-dsl/geaflow-dsl-connector/geaflow-dsl-connector-elasticsearch/src/test/java/org/apache/geaflow/dsl/connector/elasticsearch/ElasticsearchTableSourceTest.java b/geaflow/geaflow-dsl/geaflow-dsl-connector/geaflow-dsl-connector-elasticsearch/src/test/java/org/apache/geaflow/dsl/connector/elasticsearch/ElasticsearchTableSourceTest.java new file mode 100644 index 000000000..1a0a00443 --- /dev/null +++ b/geaflow/geaflow-dsl/geaflow-dsl-connector/geaflow-dsl-connector-elasticsearch/src/test/java/org/apache/geaflow/dsl/connector/elasticsearch/ElasticsearchTableSourceTest.java @@ -0,0 +1,98 @@ +/* + * 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.geaflow.dsl.connector.elasticsearch; + +import java.util.Arrays; +import java.util.List; +import java.util.Optional; +import org.apache.geaflow.common.config.Configuration; +import org.apache.geaflow.common.type.Types; +import org.apache.geaflow.dsl.common.data.Row; +import org.apache.geaflow.dsl.common.types.StructType; +import org.apache.geaflow.dsl.common.types.TableField; +import org.apache.geaflow.dsl.common.types.TableSchema; +import org.apache.geaflow.dsl.connector.api.FetchData; +import org.apache.geaflow.dsl.connector.api.Partition; +import org.mockito.Mockito; +import org.testng.Assert; +import org.testng.annotations.BeforeMethod; +import org.testng.annotations.Test; + +public class ElasticsearchTableSourceTest { + + private ElasticsearchTableSource source; + private Configuration config; + private TableSchema schema; + + @BeforeMethod + public void setUp() { + source = new ElasticsearchTableSource(); + config = new Configuration(); + config.put(ElasticsearchConfigKeys.GEAFLOW_DSL_ELASTICSEARCH_HOSTS, "localhost:9200"); + config.put(ElasticsearchConfigKeys.GEAFLOW_DSL_ELASTICSEARCH_INDEX, "test_index"); + + TableField idField = new TableField("id", Types.INTEGER, false); + TableField nameField = new TableField("name", Types.STRING, false); + schema = new TableSchema(new StructType(Arrays.asList(idField, nameField))); + } + + @Test + public void testInit() { + source.init(config, schema); + Assert.assertNotNull(source); + } + + @Test(expectedExceptions = RuntimeException.class) + public void testInitWithoutIndex() { + Configuration invalidConfig = new Configuration(); + invalidConfig.put(ElasticsearchConfigKeys.GEAFLOW_DSL_ELASTICSEARCH_HOSTS, "localhost:9200"); + source.init(invalidConfig, schema); + } + + @Test + public void testListPartitions() { + source.init(config, schema); + List partitions = source.listPartitions(); + + Assert.assertNotNull(partitions); + Assert.assertEquals(partitions.size(), 1); + Assert.assertEquals(partitions.get(0).getName(), "test_index"); + } + + @Test + public void testGetDeserializer() { + source.init(config, schema); + Assert.assertNotNull(source.getDeserializer(config)); + } + + @Test + public void testPartitionName() { + ElasticsearchTableSource.ElasticsearchPartition partition = + new ElasticsearchTableSource.ElasticsearchPartition("my_index"); + Assert.assertEquals(partition.getName(), "my_index"); + } + + @Test + public void testOffsetHumanReadable() { + ElasticsearchTableSource.ElasticsearchOffset offset = + new ElasticsearchTableSource.ElasticsearchOffset("scroll_123"); + Assert.assertTrue(offset.humanReadable().contains("scroll_123")); + } +} diff --git a/geaflow/geaflow-dsl/geaflow-dsl-connector/geaflow-dsl-connector-neo4j/pom.xml b/geaflow/geaflow-dsl/geaflow-dsl-connector/geaflow-dsl-connector-neo4j/pom.xml index 2c8191bac..4b1aa330f 100644 --- a/geaflow/geaflow-dsl/geaflow-dsl-connector/geaflow-dsl-connector-neo4j/pom.xml +++ b/geaflow/geaflow-dsl/geaflow-dsl-connector/geaflow-dsl-connector-neo4j/pom.xml @@ -32,8 +32,7 @@ geaflow-dsl-connector-neo4j - 5.14.0 - 1.19.3 + 4.4.18 @@ -58,18 +57,6 @@ ${testng.version} test - - org.testcontainers - neo4j - ${testcontainers.version} - test - - - org.testcontainers - testcontainers - ${testcontainers.version} - test - diff --git a/geaflow/geaflow-dsl/geaflow-dsl-connector/geaflow-dsl-connector-neo4j/src/test/java/org/apache/geaflow/dsl/connector/neo4j/Neo4jIntegrationTest.java b/geaflow/geaflow-dsl/geaflow-dsl-connector/geaflow-dsl-connector-neo4j/src/test/java/org/apache/geaflow/dsl/connector/neo4j/Neo4jIntegrationTest.java deleted file mode 100644 index 4d96073d2..000000000 --- a/geaflow/geaflow-dsl/geaflow-dsl-connector/geaflow-dsl-connector-neo4j/src/test/java/org/apache/geaflow/dsl/connector/neo4j/Neo4jIntegrationTest.java +++ /dev/null @@ -1,387 +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.geaflow.dsl.connector.neo4j; - -import java.util.ArrayList; -import java.util.List; -import java.util.Optional; -import org.apache.geaflow.api.context.RuntimeContext; -import org.apache.geaflow.api.window.WindowType; -import org.apache.geaflow.common.config.Configuration; -import org.apache.geaflow.common.type.Types; -import org.apache.geaflow.dsl.common.data.Row; -import org.apache.geaflow.dsl.common.data.impl.ObjectRow; -import org.apache.geaflow.dsl.common.types.StructType; -import org.apache.geaflow.dsl.common.types.TableField; -import org.apache.geaflow.dsl.common.types.TableSchema; -import org.apache.geaflow.dsl.connector.api.FetchData; -import org.apache.geaflow.dsl.connector.api.Partition; -import org.apache.geaflow.dsl.connector.api.window.SizeFetchWindow; -import org.apache.geaflow.dsl.connector.api.window.AllFetchWindow; -import org.mockito.Mockito; -import org.testcontainers.containers.Neo4jContainer; -import org.testcontainers.utility.DockerImageName; -import org.testng.Assert; -import org.testng.annotations.AfterClass; -import org.testng.annotations.BeforeClass; -import org.testng.annotations.Test; - -public class Neo4jIntegrationTest { - - private static Neo4jContainer neo4jContainer; - private static String neo4jUri; - private static final String USERNAME = "neo4j"; - private static final String PASSWORD = "testpassword"; - private static final String DATABASE = "neo4j"; - - @BeforeClass - public static void setup() { - neo4jContainer = new Neo4jContainer<>(DockerImageName.parse("neo4j:5.14.0")) - .withAdminPassword(PASSWORD) - .withEnv("NEO4J_AUTH", USERNAME + "/" + PASSWORD); - neo4jContainer.start(); - - neo4jUri = neo4jContainer.getBoltUrl(); - } - - @AfterClass - public static void cleanup() { - if (neo4jContainer != null) { - neo4jContainer.stop(); - } - } - - @Test - public void testWriteAndReadNodes() throws Exception { - // Write nodes - Configuration writeConf = new Configuration(); - writeConf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_URI, neo4jUri); - writeConf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_USERNAME, USERNAME); - writeConf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_PASSWORD, PASSWORD); - writeConf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_DATABASE, DATABASE); - writeConf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_WRITE_MODE, "node"); - writeConf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_NODE_LABEL, "Product"); - writeConf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_NODE_ID_FIELD, "product_id"); - writeConf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_BATCH_SIZE, "5"); - - List fields = new ArrayList<>(); - fields.add(new TableField("product_id", Types.LONG, false)); - fields.add(new TableField("product_name", Types.STRING, false)); - fields.add(new TableField("price", Types.DOUBLE, false)); - fields.add(new TableField("in_stock", Types.BOOLEAN, false)); - StructType schema = new StructType(fields); - - Neo4jTableSink sink = new Neo4jTableSink(); - sink.init(writeConf, schema); - - RuntimeContext context = Mockito.mock(RuntimeContext.class); - sink.open(context); - - // Write test data - for (long i = 1; i <= 10; i++) { - Row row = ObjectRow.create(new Object[]{ - i, - "Product" + i, - 99.99 + i, - i % 2 == 0 - }); - sink.write(row); - } - - sink.finish(); - sink.close(); - - // Read nodes back - Configuration readConf = new Configuration(); - readConf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_URI, neo4jUri); - readConf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_USERNAME, USERNAME); - readConf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_PASSWORD, PASSWORD); - readConf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_DATABASE, DATABASE); - readConf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_QUERY, - "MATCH (p:Product) RETURN p.id as product_id, p.product_name as product_name, " + - "p.price as price, p.in_stock as in_stock ORDER BY p.id"); - - TableSchema tableSchema = new TableSchema(fields); - - Neo4jTableSource source = new Neo4jTableSource(); - source.init(readConf, tableSchema); - source.open(context); - - List partitions = source.listPartitions(); - AllFetchWindow window = new AllFetchWindow(0L); - FetchData fetchData = source.fetch(partitions.get(0), Optional.empty(), window); - - List rows = (List) fetchData.getDataIterator(); - Assert.assertEquals(rows.size(), 10); - - // Verify first row - Row row1 = rows.get(0); - Assert.assertEquals(row1.getField(0, Types.LONG), 1L); - Assert.assertEquals(row1.getField(1, Types.STRING), "Product1"); - Assert.assertEquals((Double) row1.getField(2, Types.DOUBLE), 100.99, 0.01); - Assert.assertFalse((Boolean) row1.getField(3, Types.BOOLEAN)); - - source.close(); - } - - @Test - public void testWriteNodesAndRelationships() throws Exception { - RuntimeContext context = Mockito.mock(RuntimeContext.class); - - // First write nodes - Configuration nodeConf = new Configuration(); - nodeConf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_URI, neo4jUri); - nodeConf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_USERNAME, USERNAME); - nodeConf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_PASSWORD, PASSWORD); - nodeConf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_DATABASE, DATABASE); - nodeConf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_WRITE_MODE, "node"); - nodeConf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_NODE_LABEL, "City"); - nodeConf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_NODE_ID_FIELD, "city_id"); - nodeConf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_BATCH_SIZE, "10"); - - List nodeFields = new ArrayList<>(); - nodeFields.add(new TableField("city_id", Types.LONG, false)); - nodeFields.add(new TableField("city_name", Types.STRING, false)); - nodeFields.add(new TableField("population", Types.LONG, false)); - StructType nodeSchema = new StructType(nodeFields); - - Neo4jTableSink nodeSink = new Neo4jTableSink(); - nodeSink.init(nodeConf, nodeSchema); - nodeSink.open(context); - - Row city1 = ObjectRow.create(new Object[]{1L, "Beijing", 21540000L}); - Row city2 = ObjectRow.create(new Object[]{2L, "Shanghai", 24280000L}); - Row city3 = ObjectRow.create(new Object[]{3L, "Guangzhou", 15300000L}); - - nodeSink.write(city1); - nodeSink.write(city2); - nodeSink.write(city3); - nodeSink.finish(); - nodeSink.close(); - - // Then write relationships - Configuration relConf = new Configuration(); - relConf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_URI, neo4jUri); - relConf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_USERNAME, USERNAME); - relConf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_PASSWORD, PASSWORD); - relConf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_DATABASE, DATABASE); - relConf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_WRITE_MODE, "relationship"); - relConf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_RELATIONSHIP_TYPE, "CONNECTED_TO"); - relConf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_RELATIONSHIP_SOURCE_FIELD, "from_city"); - relConf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_RELATIONSHIP_TARGET_FIELD, "to_city"); - relConf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_BATCH_SIZE, "10"); - - List relFields = new ArrayList<>(); - relFields.add(new TableField("from_city", Types.LONG, false)); - relFields.add(new TableField("to_city", Types.LONG, false)); - relFields.add(new TableField("distance", Types.DOUBLE, false)); - relFields.add(new TableField("travel_time", Types.INTEGER, false)); - StructType relSchema = new StructType(relFields); - - Neo4jTableSink relSink = new Neo4jTableSink(); - relSink.init(relConf, relSchema); - relSink.open(context); - - Row rel1 = ObjectRow.create(new Object[]{1L, 2L, 1214.0, 120}); - Row rel2 = ObjectRow.create(new Object[]{2L, 3L, 1376.0, 150}); - - relSink.write(rel1); - relSink.write(rel2); - relSink.finish(); - relSink.close(); - - // Read and verify the graph - Configuration readConf = new Configuration(); - readConf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_URI, neo4jUri); - readConf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_USERNAME, USERNAME); - readConf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_PASSWORD, PASSWORD); - readConf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_DATABASE, DATABASE); - readConf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_QUERY, - "MATCH (a:City)-[r:CONNECTED_TO]->(b:City) " + - "RETURN a.id as from_city, b.id as to_city, r.distance as distance, " + - "r.travel_time as travel_time ORDER BY a.id"); - - TableSchema tableSchema = new TableSchema(relFields); - - Neo4jTableSource source = new Neo4jTableSource(); - source.init(readConf, tableSchema); - source.open(context); - - List partitions = source.listPartitions(); - AllFetchWindow window = new AllFetchWindow(0L); - FetchData fetchData = source.fetch(partitions.get(0), Optional.empty(), window); - - // Collect all rows from iterator - List rows = new ArrayList<>(); - fetchData.getDataIterator().forEachRemaining(rows::add); - Assert.assertEquals(rows.size(), 2); - - // Verify relationships - Row relRow1 = rows.get(0); - Assert.assertEquals((Long) relRow1.getField(0, Types.LONG), Long.valueOf(1L)); - Assert.assertEquals((Long) relRow1.getField(1, Types.LONG), Long.valueOf(2L)); - Assert.assertEquals((Double) relRow1.getField(2, Types.DOUBLE), 1214.0, 0.1); - Assert.assertEquals((Integer) relRow1.getField(3, Types.INTEGER), Integer.valueOf(120)); - - source.close(); - } - - @Test - public void testUpdateExistingNodes() throws Exception { - RuntimeContext context = Mockito.mock(RuntimeContext.class); - - // Write initial nodes - Configuration conf = new Configuration(); - conf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_URI, neo4jUri); - conf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_USERNAME, USERNAME); - conf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_PASSWORD, PASSWORD); - conf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_DATABASE, DATABASE); - conf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_WRITE_MODE, "node"); - conf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_NODE_LABEL, "Account"); - conf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_NODE_ID_FIELD, "account_id"); - conf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_BATCH_SIZE, "10"); - - List fields = new ArrayList<>(); - fields.add(new TableField("account_id", Types.LONG, false)); - fields.add(new TableField("balance", Types.DOUBLE, false)); - fields.add(new TableField("status", Types.STRING, false)); - StructType schema = new StructType(fields); - - // Initial write - Neo4jTableSink sink1 = new Neo4jTableSink(); - sink1.init(conf, schema); - sink1.open(context); - Row row1 = ObjectRow.create(new Object[]{1000L, 1000.0, "active"}); - sink1.write(row1); - sink1.finish(); - sink1.close(); - - // Update the same node - Neo4jTableSink sink2 = new Neo4jTableSink(); - sink2.init(conf, schema); - sink2.open(context); - Row row2 = ObjectRow.create(new Object[]{1000L, 1500.0, "premium"}); - sink2.write(row2); - sink2.finish(); - sink2.close(); - - // Read and verify updated value - Configuration readConf = new Configuration(); - readConf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_URI, neo4jUri); - readConf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_USERNAME, USERNAME); - readConf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_PASSWORD, PASSWORD); - readConf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_DATABASE, DATABASE); - readConf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_QUERY, - "MATCH (a:Account {id: 1000}) RETURN a.id as account_id, " + - "a.balance as balance, a.status as status"); - - TableSchema tableSchema = new TableSchema(fields); - - Neo4jTableSource source = new Neo4jTableSource(); - source.init(readConf, tableSchema); - source.open(context); - - List partitions = source.listPartitions(); - AllFetchWindow window = new AllFetchWindow(0L); - FetchData fetchData = source.fetch(partitions.get(0), Optional.empty(), window); - - // Collect all rows from iterator - List rows = new ArrayList<>(); - fetchData.getDataIterator().forEachRemaining(rows::add); - Assert.assertEquals(rows.size(), 1); - - Row row = rows.get(0); - Assert.assertEquals((Long) row.getField(0, Types.LONG), Long.valueOf(1000L)); - Assert.assertEquals((Double) row.getField(1, Types.DOUBLE), 1500.0, 0.01); - Assert.assertEquals(row.getField(2, Types.STRING), "premium"); - - source.close(); - } - - @Test - public void testLargeDatasetWriteAndRead() throws Exception { - RuntimeContext context = Mockito.mock(RuntimeContext.class); - - Configuration writeConf = new Configuration(); - writeConf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_URI, neo4jUri); - writeConf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_USERNAME, USERNAME); - writeConf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_PASSWORD, PASSWORD); - writeConf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_DATABASE, DATABASE); - writeConf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_WRITE_MODE, "node"); - writeConf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_NODE_LABEL, "LargeDataset"); - writeConf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_NODE_ID_FIELD, "id"); - writeConf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_BATCH_SIZE, "50"); - - List fields = new ArrayList<>(); - fields.add(new TableField("id", Types.LONG, false)); - fields.add(new TableField("value", Types.LONG, false)); - StructType schema = new StructType(fields); - - Neo4jTableSink sink = new Neo4jTableSink(); - sink.init(writeConf, schema); - sink.open(context); - - // Write 200 rows - for (long i = 10000; i < 10200; i++) { - Row row = ObjectRow.create(new Object[]{i, i * 2}); - sink.write(row); - } - sink.finish(); - sink.close(); - - // Read back with pagination - Configuration readConf = new Configuration(); - readConf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_URI, neo4jUri); - readConf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_USERNAME, USERNAME); - readConf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_PASSWORD, PASSWORD); - readConf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_DATABASE, DATABASE); - readConf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_QUERY, - "MATCH (n:LargeDataset) RETURN n.id as id, n.value as value ORDER BY n.id"); - - TableSchema tableSchema = new TableSchema(fields); - - Neo4jTableSource source = new Neo4jTableSource(); - source.init(readConf, tableSchema); - source.open(context); - - List partitions = source.listPartitions(); - SizeFetchWindow window = new SizeFetchWindow(0L, 75L); - - int totalRows = 0; - Optional offset = Optional.empty(); - - while (true) { - FetchData fetchData = source.fetch(partitions.get(0), offset, window); - // Count rows from iterator - List pageRows = new ArrayList<>(); - fetchData.getDataIterator().forEachRemaining(pageRows::add); - totalRows += pageRows.size(); - - if (fetchData.isFinish()) { - break; - } - offset = Optional.of(fetchData.getNextOffset()); - } - - Assert.assertEquals(totalRows, 200); - - source.close(); - } -} diff --git a/geaflow/geaflow-dsl/geaflow-dsl-connector/geaflow-dsl-connector-neo4j/src/test/java/org/apache/geaflow/dsl/connector/neo4j/Neo4jTableConnectorTest.java b/geaflow/geaflow-dsl/geaflow-dsl-connector/geaflow-dsl-connector-neo4j/src/test/java/org/apache/geaflow/dsl/connector/neo4j/Neo4jTableConnectorTest.java index e321af9a9..ed2f3dae3 100644 --- a/geaflow/geaflow-dsl/geaflow-dsl-connector/geaflow-dsl-connector-neo4j/src/test/java/org/apache/geaflow/dsl/connector/neo4j/Neo4jTableConnectorTest.java +++ b/geaflow/geaflow-dsl/geaflow-dsl-connector/geaflow-dsl-connector-neo4j/src/test/java/org/apache/geaflow/dsl/connector/neo4j/Neo4jTableConnectorTest.java @@ -19,367 +19,63 @@ package org.apache.geaflow.dsl.connector.neo4j; -import java.util.ArrayList; -import java.util.List; -import java.util.Optional; -import org.apache.geaflow.api.context.RuntimeContext; -import org.apache.geaflow.api.window.WindowType; import org.apache.geaflow.common.config.Configuration; -import org.apache.geaflow.common.type.Types; -import org.apache.geaflow.dsl.common.data.Row; -import org.apache.geaflow.dsl.common.data.impl.ObjectRow; -import org.apache.geaflow.dsl.common.types.StructType; -import org.apache.geaflow.dsl.common.types.TableField; -import org.apache.geaflow.dsl.common.types.TableSchema; -import org.apache.geaflow.dsl.connector.api.FetchData; -import org.apache.geaflow.dsl.connector.api.Partition; -import org.apache.geaflow.dsl.connector.api.window.SizeFetchWindow; -import org.mockito.Mockito; -import org.neo4j.driver.Driver; -import org.neo4j.driver.Session; -import org.testcontainers.containers.Neo4jContainer; -import org.testcontainers.utility.DockerImageName; +import org.apache.geaflow.dsl.connector.api.TableSink; +import org.apache.geaflow.dsl.connector.api.TableSource; import org.testng.Assert; -import org.testng.annotations.AfterClass; -import org.testng.annotations.BeforeClass; +import org.testng.annotations.BeforeMethod; import org.testng.annotations.Test; public class Neo4jTableConnectorTest { - private static Neo4jContainer neo4jContainer; - private static String neo4jUri; - private static String username = "neo4j"; - private static String password = "testpassword"; - private static String database = "neo4j"; + private Neo4jTableConnector connector; + private Configuration config; - @BeforeClass - public static void setup() { - neo4jContainer = new Neo4jContainer<>(DockerImageName.parse("neo4j:5.14.0")) - .withAdminPassword(password) - .withEnv("NEO4J_AUTH", username + "/" + password); - neo4jContainer.start(); - - neo4jUri = neo4jContainer.getBoltUrl(); - - // Initialize test data - try (Driver driver = org.neo4j.driver.GraphDatabase.driver( - neo4jUri, - org.neo4j.driver.AuthTokens.basic(username, password))) { - try (Session session = driver.session()) { - // Create test nodes - session.run("CREATE (p1:Person {id: 1, name: 'Alice', age: 30})"); - session.run("CREATE (p2:Person {id: 2, name: 'Bob', age: 25})"); - session.run("CREATE (p3:Person {id: 3, name: 'Charlie', age: 35})"); - - // Create test relationships - session.run("MATCH (a:Person {id: 1}), (b:Person {id: 2}) " + - "CREATE (a)-[:KNOWS {since: 2020}]->(b)"); - session.run("MATCH (a:Person {id: 2}), (b:Person {id: 3}) " + - "CREATE (a)-[:KNOWS {since: 2021}]->(b)"); - } - } - } - - @AfterClass - public static void cleanup() { - if (neo4jContainer != null) { - neo4jContainer.stop(); - } + @BeforeMethod + public void setUp() { + connector = new Neo4jTableConnector(); + config = new Configuration(); + config.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_URI, "bolt://localhost:7687"); + config.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_USERNAME, "neo4j"); + config.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_PASSWORD, "password"); } @Test - public void testConnectorType() { - Neo4jTableConnector connector = new Neo4jTableConnector(); + public void testGetType() { Assert.assertEquals(connector.getType(), "Neo4j"); } @Test public void testCreateSource() { - Neo4jTableConnector connector = new Neo4jTableConnector(); - Configuration conf = new Configuration(); - Neo4jTableSource source = (Neo4jTableSource) connector.createSource(conf); + TableSource source = connector.createSource(config); Assert.assertNotNull(source); + Assert.assertTrue(source instanceof Neo4jTableSource); } @Test public void testCreateSink() { - Neo4jTableConnector connector = new Neo4jTableConnector(); - Configuration conf = new Configuration(); - Neo4jTableSink sink = (Neo4jTableSink) connector.createSink(conf); + TableSink sink = connector.createSink(config); Assert.assertNotNull(sink); + Assert.assertTrue(sink instanceof Neo4jTableSink); } @Test - public void testReadNodes() throws Exception { - Configuration conf = new Configuration(); - conf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_URI, neo4jUri); - conf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_USERNAME, username); - conf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_PASSWORD, password); - conf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_DATABASE, database); - conf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_QUERY, - "MATCH (p:Person) RETURN p.id as id, p.name as name, p.age as age ORDER BY p.id"); - - List fields = new ArrayList<>(); - fields.add(new TableField("id", Types.LONG, false)); - fields.add(new TableField("name", Types.STRING, false)); - fields.add(new TableField("age", Types.LONG, false)); - TableSchema tableSchema = new TableSchema(fields); - - Neo4jTableSource source = new Neo4jTableSource(); - source.init(conf, tableSchema); - - RuntimeContext context = Mockito.mock(RuntimeContext.class); - source.open(context); - - List partitions = source.listPartitions(); - Assert.assertEquals(partitions.size(), 1); - - SizeFetchWindow window = new SizeFetchWindow(0L, 10L); - FetchData fetchData = source.fetch(partitions.get(0), Optional.empty(), window); - - // Collect rows from iterator - List rows = new ArrayList<>(); - while (fetchData.getDataIterator().hasNext()) { - rows.add(fetchData.getDataIterator().next()); - } - Assert.assertEquals(rows.size(), 3); - - // Verify first row - Row row1 = rows.get(0); - Assert.assertEquals((Long) row1.getField(0, Types.LONG), Long.valueOf(1L)); - Assert.assertEquals(row1.getField(1, Types.STRING), "Alice"); - Assert.assertEquals((Long) row1.getField(2, Types.LONG), Long.valueOf(30L)); + public void testMultipleSourceInstances() { + TableSource source1 = connector.createSource(config); + TableSource source2 = connector.createSource(config); - source.close(); + Assert.assertNotNull(source1); + Assert.assertNotNull(source2); + Assert.assertNotSame(source1, source2); } @Test - public void testWriteNodes() throws Exception { - Configuration conf = new Configuration(); - conf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_URI, neo4jUri); - conf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_USERNAME, username); - conf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_PASSWORD, password); - conf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_DATABASE, database); - conf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_WRITE_MODE, "node"); - conf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_NODE_LABEL, "TestPerson"); - conf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_NODE_ID_FIELD, "id"); - conf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_BATCH_SIZE, "2"); - - List fields = new ArrayList<>(); - fields.add(new TableField("id", Types.LONG, false)); - fields.add(new TableField("name", Types.STRING, false)); - fields.add(new TableField("email", Types.STRING, true)); - StructType schema = new StructType(fields); - - Neo4jTableSink sink = new Neo4jTableSink(); - sink.init(conf, schema); - - RuntimeContext context = Mockito.mock(RuntimeContext.class); - sink.open(context); - - // Write test data - Row row1 = ObjectRow.create(new Object[]{100L, "David", "david@example.com"}); - Row row2 = ObjectRow.create(new Object[]{101L, "Eva", "eva@example.com"}); - Row row3 = ObjectRow.create(new Object[]{102L, "Frank", null}); - - sink.write(row1); - sink.write(row2); - sink.write(row3); - - sink.finish(); - sink.close(); - - // Verify data was written - try (Driver driver = org.neo4j.driver.GraphDatabase.driver( - neo4jUri, - org.neo4j.driver.AuthTokens.basic(username, password))) { - try (Session session = driver.session()) { - var result = session.run("MATCH (p:TestPerson) WHERE p.id >= 100 RETURN count(p) as count"); - long count = result.single().get("count").asLong(); - Assert.assertEquals(count, 3); - - // Verify specific node - var result2 = session.run("MATCH (p:TestPerson {id: 100}) RETURN p.name as name, p.email as email"); - var record = result2.single(); - Assert.assertEquals(record.get("name").asString(), "David"); - Assert.assertEquals(record.get("email").asString(), "david@example.com"); - } - } - } - - @Test - public void testWriteRelationships() throws Exception { - Configuration conf = new Configuration(); - conf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_URI, neo4jUri); - conf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_USERNAME, username); - conf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_PASSWORD, password); - conf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_DATABASE, database); - conf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_WRITE_MODE, "relationship"); - conf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_RELATIONSHIP_TYPE, "FOLLOWS"); - conf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_RELATIONSHIP_SOURCE_FIELD, "from_id"); - conf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_RELATIONSHIP_TARGET_FIELD, "to_id"); - conf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_BATCH_SIZE, "2"); - - // First create some nodes to connect - try (Driver driver = org.neo4j.driver.GraphDatabase.driver( - neo4jUri, - org.neo4j.driver.AuthTokens.basic(username, password))) { - try (Session session = driver.session()) { - session.run("CREATE (p:TestUser {id: 200, name: 'User1'})"); - session.run("CREATE (p:TestUser {id: 201, name: 'User2'})"); - session.run("CREATE (p:TestUser {id: 202, name: 'User3'})"); - } - } - - List fields = new ArrayList<>(); - fields.add(new TableField("from_id", Types.LONG, false)); - fields.add(new TableField("to_id", Types.LONG, false)); - fields.add(new TableField("weight", Types.DOUBLE, true)); - StructType schema = new StructType(fields); - - Neo4jTableSink sink = new Neo4jTableSink(); - sink.init(conf, schema); - - RuntimeContext context = Mockito.mock(RuntimeContext.class); - sink.open(context); - - // Write relationships - Row rel1 = ObjectRow.create(new Object[]{200L, 201L, 0.8}); - Row rel2 = ObjectRow.create(new Object[]{201L, 202L, 0.9}); - - sink.write(rel1); - sink.write(rel2); - - sink.finish(); - sink.close(); - - // Verify relationships were created - try (Driver driver = org.neo4j.driver.GraphDatabase.driver( - neo4jUri, - org.neo4j.driver.AuthTokens.basic(username, password))) { - try (Session session = driver.session()) { - var result = session.run( - "MATCH ()-[r:FOLLOWS]->() WHERE r.weight IS NOT NULL RETURN count(r) as count"); - long count = result.single().get("count").asLong(); - Assert.assertEquals(count, 2); - } - } - } - - @Test - public void testReadWithPagination() throws Exception { - Configuration conf = new Configuration(); - conf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_URI, neo4jUri); - conf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_USERNAME, username); - conf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_PASSWORD, password); - conf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_DATABASE, database); - conf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_QUERY, - "MATCH (p:Person) RETURN p.id as id, p.name as name ORDER BY p.id"); - - List fields = new ArrayList<>(); - fields.add(new TableField("id", Types.LONG, false)); - fields.add(new TableField("name", Types.STRING, false)); - TableSchema tableSchema = new TableSchema(fields); - - Neo4jTableSource source = new Neo4jTableSource(); - source.init(conf, tableSchema); - - RuntimeContext context = Mockito.mock(RuntimeContext.class); - source.open(context); - - List partitions = source.listPartitions(); - - // Fetch first page - SizeFetchWindow window = new SizeFetchWindow(0L, 2L); - FetchData fetchData1 = source.fetch(partitions.get(0), Optional.empty(), window); - List rows1 = (List) fetchData1.getDataIterator(); - Assert.assertEquals(rows1.size(), 2); - - // Fetch second page - FetchData fetchData2 = source.fetch(partitions.get(0), - Optional.of(fetchData1.getNextOffset()), window); - List rows2 = (List) fetchData2.getDataIterator(); - Assert.assertEquals(rows2.size(), 1); - - source.close(); - } - - @Test - public void testBatchWriting() throws Exception { - Configuration conf = new Configuration(); - conf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_URI, neo4jUri); - conf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_USERNAME, username); - conf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_PASSWORD, password); - conf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_DATABASE, database); - conf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_WRITE_MODE, "node"); - conf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_NODE_LABEL, "BatchTest"); - conf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_NODE_ID_FIELD, "id"); - conf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_BATCH_SIZE, "3"); - - List fields = new ArrayList<>(); - fields.add(new TableField("id", Types.LONG, false)); - fields.add(new TableField("value", Types.STRING, false)); - StructType schema = new StructType(fields); - - Neo4jTableSink sink = new Neo4jTableSink(); - sink.init(conf, schema); - - RuntimeContext context = Mockito.mock(RuntimeContext.class); - sink.open(context); - - // Write 10 rows (should trigger multiple batches) - for (long i = 300; i < 310; i++) { - Row row = ObjectRow.create(new Object[]{i, "Value" + i}); - sink.write(row); - } - - sink.finish(); - sink.close(); - - // Verify all data was written - try (Driver driver = org.neo4j.driver.GraphDatabase.driver( - neo4jUri, - org.neo4j.driver.AuthTokens.basic(username, password))) { - try (Session session = driver.session()) { - var result = session.run("MATCH (n:BatchTest) WHERE n.id >= 300 AND n.id < 310 RETURN count(n) as count"); - long count = result.single().get("count").asLong(); - Assert.assertEquals(count, 10); - } - } - } - - @Test(expectedExceptions = org.apache.geaflow.dsl.common.exception.GeaFlowDSLException.class) - public void testInvalidWriteMode() { - Configuration conf = new Configuration(); - conf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_URI, neo4jUri); - conf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_USERNAME, username); - conf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_PASSWORD, password); - conf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_DATABASE, database); - conf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_WRITE_MODE, "invalid_mode"); - conf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_NODE_ID_FIELD, "id"); - - List fields = new ArrayList<>(); - fields.add(new TableField("id", Types.LONG, false)); - StructType schema = new StructType(fields); - - Neo4jTableSink sink = new Neo4jTableSink(); - sink.init(conf, schema); // Should throw exception - } - - @Test(expectedExceptions = org.apache.geaflow.dsl.common.exception.GeaFlowDSLException.class) - public void testMissingNodeIdField() { - Configuration conf = new Configuration(); - conf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_URI, neo4jUri); - conf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_USERNAME, username); - conf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_PASSWORD, password); - conf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_DATABASE, database); - conf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_WRITE_MODE, "node"); - - List fields = new ArrayList<>(); - fields.add(new TableField("id", Types.LONG, false)); - StructType schema = new StructType(fields); + public void testMultipleSinkInstances() { + TableSink sink1 = connector.createSink(config); + TableSink sink2 = connector.createSink(config); - Neo4jTableSink sink = new Neo4jTableSink(); - sink.init(conf, schema); // Should throw exception + Assert.assertNotNull(sink1); + Assert.assertNotNull(sink2); + Assert.assertNotSame(sink1, sink2); } } diff --git a/geaflow/geaflow-dsl/geaflow-dsl-connector/geaflow-dsl-connector-neo4j/src/test/java/org/apache/geaflow/dsl/connector/neo4j/Neo4jTableSinkTest.java b/geaflow/geaflow-dsl/geaflow-dsl-connector/geaflow-dsl-connector-neo4j/src/test/java/org/apache/geaflow/dsl/connector/neo4j/Neo4jTableSinkTest.java deleted file mode 100644 index 8a82f37fb..000000000 --- a/geaflow/geaflow-dsl/geaflow-dsl-connector/geaflow-dsl-connector-neo4j/src/test/java/org/apache/geaflow/dsl/connector/neo4j/Neo4jTableSinkTest.java +++ /dev/null @@ -1,388 +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.geaflow.dsl.connector.neo4j; - -import java.util.ArrayList; -import java.util.List; -import org.apache.geaflow.api.context.RuntimeContext; -import org.apache.geaflow.common.config.Configuration; -import org.apache.geaflow.common.type.Types; -import org.apache.geaflow.dsl.common.data.Row; -import org.apache.geaflow.dsl.common.data.impl.ObjectRow; -import org.apache.geaflow.dsl.common.types.StructType; -import org.apache.geaflow.dsl.common.types.TableField; -import org.mockito.Mockito; -import org.neo4j.driver.Driver; -import org.neo4j.driver.Session; -import org.testcontainers.containers.Neo4jContainer; -import org.testcontainers.utility.DockerImageName; -import org.testng.Assert; -import org.testng.annotations.AfterClass; -import org.testng.annotations.BeforeClass; -import org.testng.annotations.Test; - -public class Neo4jTableSinkTest { - - private static Neo4jContainer neo4jContainer; - private static String neo4jUri; - private static final String USERNAME = "neo4j"; - private static final String PASSWORD = "testpassword"; - private static final String DATABASE = "neo4j"; - - @BeforeClass - public static void setup() { - neo4jContainer = new Neo4jContainer<>(DockerImageName.parse("neo4j:5.14.0")) - .withAdminPassword(PASSWORD) - .withEnv("NEO4J_AUTH", USERNAME + "/" + PASSWORD); - neo4jContainer.start(); - - neo4jUri = neo4jContainer.getBoltUrl(); - } - - @AfterClass - public static void cleanup() { - if (neo4jContainer != null) { - neo4jContainer.stop(); - } - } - - @Test - public void testWriteNodesWithNullValues() throws Exception { - Configuration conf = new Configuration(); - conf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_URI, neo4jUri); - conf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_USERNAME, USERNAME); - conf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_PASSWORD, PASSWORD); - conf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_DATABASE, DATABASE); - conf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_WRITE_MODE, "node"); - conf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_NODE_LABEL, "NullTest"); - conf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_NODE_ID_FIELD, "id"); - conf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_BATCH_SIZE, "10"); - - List fields = new ArrayList<>(); - fields.add(new TableField("id", Types.LONG, false)); - fields.add(new TableField("name", Types.STRING, true)); - fields.add(new TableField("age", Types.INTEGER, true)); - StructType schema = new StructType(fields); - - Neo4jTableSink sink = new Neo4jTableSink(); - sink.init(conf, schema); - - RuntimeContext context = Mockito.mock(RuntimeContext.class); - sink.open(context); - - // Write rows with null values - Row row1 = ObjectRow.create(new Object[]{1000L, null, 25}); - Row row2 = ObjectRow.create(new Object[]{1001L, "John", null}); - Row row3 = ObjectRow.create(new Object[]{1002L, null, null}); - - sink.write(row1); - sink.write(row2); - sink.write(row3); - - sink.finish(); - sink.close(); - - // Verify data - try (Driver driver = org.neo4j.driver.GraphDatabase.driver( - neo4jUri, - org.neo4j.driver.AuthTokens.basic(USERNAME, PASSWORD))) { - try (Session session = driver.session()) { - // Check first row - var result1 = session.run("MATCH (n:NullTest {id: 1000}) RETURN n.age as age"); - Assert.assertEquals(result1.single().get("age").asInt(), 25); - - // Check second row - var result2 = session.run("MATCH (n:NullTest {id: 1001}) RETURN n.name as name"); - Assert.assertEquals(result2.single().get("name").asString(), "John"); - } - } - } - - @Test - public void testWriteDifferentDataTypes() throws Exception { - Configuration conf = new Configuration(); - conf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_URI, neo4jUri); - conf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_USERNAME, USERNAME); - conf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_PASSWORD, PASSWORD); - conf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_DATABASE, DATABASE); - conf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_WRITE_MODE, "node"); - conf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_NODE_LABEL, "TypeTest"); - conf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_NODE_ID_FIELD, "id"); - conf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_BATCH_SIZE, "10"); - - List fields = new ArrayList<>(); - fields.add(new TableField("id", Types.LONG, false)); - fields.add(new TableField("int_val", Types.INTEGER, true)); - fields.add(new TableField("long_val", Types.LONG, true)); - fields.add(new TableField("double_val", Types.DOUBLE, true)); - fields.add(new TableField("bool_val", Types.BOOLEAN, true)); - fields.add(new TableField("str_val", Types.STRING, true)); - StructType schema = new StructType(fields); - - Neo4jTableSink sink = new Neo4jTableSink(); - sink.init(conf, schema); - - RuntimeContext context = Mockito.mock(RuntimeContext.class); - sink.open(context); - - Row row = ObjectRow.create(new Object[]{ - 2000L, - 42, - 9999999999L, - 3.14159, - true, - "test string" - }); - - sink.write(row); - sink.finish(); - sink.close(); - - // Verify all data types - try (Driver driver = org.neo4j.driver.GraphDatabase.driver( - neo4jUri, - org.neo4j.driver.AuthTokens.basic(USERNAME, PASSWORD))) { - try (Session session = driver.session()) { - var result = session.run( - "MATCH (n:TypeTest {id: 2000}) " + - "RETURN n.int_val as int_val, n.long_val as long_val, " + - "n.double_val as double_val, n.bool_val as bool_val, n.str_val as str_val"); - var record = result.single(); - - Assert.assertEquals(record.get("int_val").asInt(), 42); - Assert.assertEquals(record.get("long_val").asLong(), 9999999999L); - Assert.assertEquals(record.get("double_val").asDouble(), 3.14159, 0.00001); - Assert.assertTrue(record.get("bool_val").asBoolean()); - Assert.assertEquals(record.get("str_val").asString(), "test string"); - } - } - } - - @Test - public void testNodeMergeStrategy() throws Exception { - Configuration conf = new Configuration(); - conf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_URI, neo4jUri); - conf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_USERNAME, USERNAME); - conf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_PASSWORD, PASSWORD); - conf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_DATABASE, DATABASE); - conf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_WRITE_MODE, "node"); - conf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_NODE_LABEL, "MergeTest"); - conf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_NODE_ID_FIELD, "id"); - conf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_BATCH_SIZE, "10"); - - List fields = new ArrayList<>(); - fields.add(new TableField("id", Types.LONG, false)); - fields.add(new TableField("value", Types.STRING, false)); - StructType schema = new StructType(fields); - - Neo4jTableSink sink = new Neo4jTableSink(); - sink.init(conf, schema); - - RuntimeContext context = Mockito.mock(RuntimeContext.class); - sink.open(context); - - // Write same ID twice with different values - Row row1 = ObjectRow.create(new Object[]{3000L, "first"}); - sink.write(row1); - sink.finish(); - sink.close(); - - // Write again with same ID - sink = new Neo4jTableSink(); - sink.init(conf, schema); - sink.open(context); - Row row2 = ObjectRow.create(new Object[]{3000L, "second"}); - sink.write(row2); - sink.finish(); - sink.close(); - - // Verify only one node exists with updated value - try (Driver driver = org.neo4j.driver.GraphDatabase.driver( - neo4jUri, - org.neo4j.driver.AuthTokens.basic(USERNAME, PASSWORD))) { - try (Session session = driver.session()) { - var result = session.run("MATCH (n:MergeTest {id: 3000}) RETURN count(n) as count, n.value as value"); - var record = result.single(); - Assert.assertEquals(record.get("count").asLong(), 1L); - Assert.assertEquals(record.get("value").asString(), "second"); - } - } - } - - @Test - public void testRelationshipWithMultipleProperties() throws Exception { - // Create test nodes first - try (Driver driver = org.neo4j.driver.GraphDatabase.driver( - neo4jUri, - org.neo4j.driver.AuthTokens.basic(USERNAME, PASSWORD))) { - try (Session session = driver.session()) { - session.run("CREATE (a {id: 4000}), (b {id: 4001})"); - } - } - - Configuration conf = new Configuration(); - conf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_URI, neo4jUri); - conf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_USERNAME, USERNAME); - conf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_PASSWORD, PASSWORD); - conf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_DATABASE, DATABASE); - conf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_WRITE_MODE, "relationship"); - conf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_RELATIONSHIP_TYPE, "CONNECTS"); - conf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_RELATIONSHIP_SOURCE_FIELD, "source"); - conf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_RELATIONSHIP_TARGET_FIELD, "target"); - conf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_BATCH_SIZE, "10"); - - List fields = new ArrayList<>(); - fields.add(new TableField("source", Types.LONG, false)); - fields.add(new TableField("target", Types.LONG, false)); - fields.add(new TableField("weight", Types.DOUBLE, true)); - fields.add(new TableField("type", Types.STRING, true)); - fields.add(new TableField("active", Types.BOOLEAN, true)); - StructType schema = new StructType(fields); - - Neo4jTableSink sink = new Neo4jTableSink(); - sink.init(conf, schema); - - RuntimeContext context = Mockito.mock(RuntimeContext.class); - sink.open(context); - - Row rel = ObjectRow.create(new Object[]{4000L, 4001L, 0.95, "direct", true}); - sink.write(rel); - sink.finish(); - sink.close(); - - // Verify relationship properties - try (Driver driver = org.neo4j.driver.GraphDatabase.driver( - neo4jUri, - org.neo4j.driver.AuthTokens.basic(USERNAME, PASSWORD))) { - try (Session session = driver.session()) { - var result = session.run( - "MATCH ({id: 4000})-[r:CONNECTS]->({id: 4001}) " + - "RETURN r.weight as weight, r.type as type, r.active as active"); - var record = result.single(); - - Assert.assertEquals(record.get("weight").asDouble(), 0.95, 0.0001); - Assert.assertEquals(record.get("type").asString(), "direct"); - Assert.assertTrue(record.get("active").asBoolean()); - } - } - } - - @Test - public void testTransactionRollback() throws Exception { - Configuration conf = new Configuration(); - conf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_URI, neo4jUri); - conf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_USERNAME, USERNAME); - conf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_PASSWORD, PASSWORD); - conf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_DATABASE, DATABASE); - conf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_WRITE_MODE, "node"); - conf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_NODE_LABEL, "RollbackTest"); - conf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_NODE_ID_FIELD, "id"); - conf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_BATCH_SIZE, "10"); - - List fields = new ArrayList<>(); - fields.add(new TableField("id", Types.LONG, false)); - fields.add(new TableField("value", Types.STRING, false)); - StructType schema = new StructType(fields); - - Neo4jTableSink sink = new Neo4jTableSink(); - sink.init(conf, schema); - - RuntimeContext context = Mockito.mock(RuntimeContext.class); - sink.open(context); - - Row row = ObjectRow.create(new Object[]{5000L, "test"}); - sink.write(row); - - // Close without finish - should not commit - sink.close(); - - // Verify data was not written - try (Driver driver = org.neo4j.driver.GraphDatabase.driver( - neo4jUri, - org.neo4j.driver.AuthTokens.basic(USERNAME, PASSWORD))) { - try (Session session = driver.session()) { - var result = session.run("MATCH (n:RollbackTest {id: 5000}) RETURN count(n) as count"); - long count = result.single().get("count").asLong(); - Assert.assertEquals(count, 0L); - } - } - } - - @Test(expectedExceptions = org.apache.geaflow.dsl.common.exception.GeaFlowDSLException.class) - public void testNullNodeId() throws Exception { - Configuration conf = new Configuration(); - conf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_URI, neo4jUri); - conf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_USERNAME, USERNAME); - conf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_PASSWORD, PASSWORD); - conf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_DATABASE, DATABASE); - conf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_WRITE_MODE, "node"); - conf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_NODE_LABEL, "ErrorTest"); - conf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_NODE_ID_FIELD, "id"); - conf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_BATCH_SIZE, "10"); - - List fields = new ArrayList<>(); - fields.add(new TableField("id", Types.LONG, false)); - fields.add(new TableField("value", Types.STRING, false)); - StructType schema = new StructType(fields); - - Neo4jTableSink sink = new Neo4jTableSink(); - sink.init(conf, schema); - - RuntimeContext context = Mockito.mock(RuntimeContext.class); - sink.open(context); - - // Try to write with null ID - should throw exception - Row row = ObjectRow.create(new Object[]{null, "test"}); - sink.write(row); - sink.finish(); // Should throw here - sink.close(); - } - - @Test(expectedExceptions = org.apache.geaflow.dsl.common.exception.GeaFlowDSLException.class) - public void testNullRelationshipEndpoints() throws Exception { - Configuration conf = new Configuration(); - conf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_URI, neo4jUri); - conf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_USERNAME, USERNAME); - conf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_PASSWORD, PASSWORD); - conf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_DATABASE, DATABASE); - conf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_WRITE_MODE, "relationship"); - conf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_RELATIONSHIP_TYPE, "ERROR"); - conf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_RELATIONSHIP_SOURCE_FIELD, "source"); - conf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_RELATIONSHIP_TARGET_FIELD, "target"); - conf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_BATCH_SIZE, "10"); - - List fields = new ArrayList<>(); - fields.add(new TableField("source", Types.LONG, false)); - fields.add(new TableField("target", Types.LONG, false)); - StructType schema = new StructType(fields); - - Neo4jTableSink sink = new Neo4jTableSink(); - sink.init(conf, schema); - - RuntimeContext context = Mockito.mock(RuntimeContext.class); - sink.open(context); - - // Try to write with null source - should throw exception - Row row = ObjectRow.create(new Object[]{null, 1L}); - sink.write(row); - sink.finish(); // Should throw here - sink.close(); - } -} diff --git a/geaflow/geaflow-dsl/geaflow-dsl-connector/geaflow-dsl-connector-neo4j/src/test/java/org/apache/geaflow/dsl/connector/neo4j/Neo4jTableSourceTest.java b/geaflow/geaflow-dsl/geaflow-dsl-connector/geaflow-dsl-connector-neo4j/src/test/java/org/apache/geaflow/dsl/connector/neo4j/Neo4jTableSourceTest.java deleted file mode 100644 index 34be90080..000000000 --- a/geaflow/geaflow-dsl/geaflow-dsl-connector/geaflow-dsl-connector-neo4j/src/test/java/org/apache/geaflow/dsl/connector/neo4j/Neo4jTableSourceTest.java +++ /dev/null @@ -1,433 +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.geaflow.dsl.connector.neo4j; - -import java.util.ArrayList; -import java.util.List; -import java.util.Optional; -import org.apache.geaflow.api.context.RuntimeContext; -import org.apache.geaflow.api.window.WindowType; -import org.apache.geaflow.common.config.Configuration; -import org.apache.geaflow.common.type.Types; -import org.apache.geaflow.dsl.common.data.Row; -import org.apache.geaflow.dsl.common.types.TableField; -import org.apache.geaflow.dsl.common.types.TableSchema; -import org.apache.geaflow.dsl.connector.api.FetchData; -import org.apache.geaflow.dsl.connector.api.Partition; -import org.apache.geaflow.dsl.connector.api.window.SizeFetchWindow; -import org.apache.geaflow.dsl.connector.api.window.AllFetchWindow; -import org.mockito.Mockito; -import org.neo4j.driver.Driver; -import org.neo4j.driver.Session; -import org.testcontainers.containers.Neo4jContainer; -import org.testcontainers.utility.DockerImageName; -import org.testng.Assert; -import org.testng.annotations.AfterClass; -import org.testng.annotations.BeforeClass; -import org.testng.annotations.Test; - -public class Neo4jTableSourceTest { - - private static Neo4jContainer neo4jContainer; - private static String neo4jUri; - private static final String USERNAME = "neo4j"; - private static final String PASSWORD = "testpassword"; - private static final String DATABASE = "neo4j"; - - @BeforeClass - public static void setup() { - neo4jContainer = new Neo4jContainer<>(DockerImageName.parse("neo4j:5.14.0")) - .withAdminPassword(PASSWORD) - .withEnv("NEO4J_AUTH", USERNAME + "/" + PASSWORD); - neo4jContainer.start(); - - neo4jUri = neo4jContainer.getBoltUrl(); - - // Initialize test data - try (Driver driver = org.neo4j.driver.GraphDatabase.driver( - neo4jUri, - org.neo4j.driver.AuthTokens.basic(USERNAME, PASSWORD))) { - try (Session session = driver.session()) { - // Create test nodes - for (int i = 1; i <= 50; i++) { - session.run(String.format( - "CREATE (p:TestNode {id: %d, name: 'Node%d', value: %d})", - i, i, i * 10)); - } - - // Create nodes with different types - session.run("CREATE (n:TypeNode {" + - "id: 100, " + - "str_field: 'test', " + - "int_field: 42, " + - "long_field: 9999999999, " + - "double_field: 3.14, " + - "bool_field: true})"); - - // Create relationships - session.run("MATCH (a:TestNode {id: 1}), (b:TestNode {id: 2}) " + - "CREATE (a)-[:LINKS {weight: 0.5}]->(b)"); - session.run("MATCH (a:TestNode {id: 2}), (b:TestNode {id: 3}) " + - "CREATE (a)-[:LINKS {weight: 0.8}]->(b)"); - } - } - } - - @AfterClass - public static void cleanup() { - if (neo4jContainer != null) { - neo4jContainer.stop(); - } - } - - @Test - public void testReadSimpleQuery() throws Exception { - Configuration conf = new Configuration(); - conf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_URI, neo4jUri); - conf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_USERNAME, USERNAME); - conf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_PASSWORD, PASSWORD); - conf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_DATABASE, DATABASE); - conf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_QUERY, - "MATCH (n:TestNode) WHERE n.id <= 5 RETURN n.id as id, n.name as name ORDER BY n.id"); - - List fields = new ArrayList<>(); - fields.add(new TableField("id", Types.LONG, false)); - fields.add(new TableField("name", Types.STRING, false)); - TableSchema tableSchema = new TableSchema(fields); - - Neo4jTableSource source = new Neo4jTableSource(); - source.init(conf, tableSchema); - - RuntimeContext context = Mockito.mock(RuntimeContext.class); - source.open(context); - - List partitions = source.listPartitions(); - Assert.assertEquals(partitions.size(), 1); - - SizeFetchWindow window = new SizeFetchWindow(0L, 100L); - FetchData fetchData = source.fetch(partitions.get(0), Optional.empty(), window); - - List rows = (List) fetchData.getDataIterator(); - Assert.assertEquals(rows.size(), 5); - - // Verify data - for (int i = 0; i < 5; i++) { - Row row = rows.get(i); - Assert.assertEquals(row.getField(0, Types.LONG), (long)(i + 1)); - Assert.assertEquals(row.getField(1, Types.STRING), "Node" + (i + 1)); - } - - source.close(); - } - - @Test - public void testReadWithDifferentDataTypes() throws Exception { - Configuration conf = new Configuration(); - conf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_URI, neo4jUri); - conf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_USERNAME, USERNAME); - conf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_PASSWORD, PASSWORD); - conf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_DATABASE, DATABASE); - conf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_QUERY, - "MATCH (n:TypeNode {id: 100}) RETURN n.str_field as str_field, " + - "n.int_field as int_field, n.long_field as long_field, " + - "n.double_field as double_field, n.bool_field as bool_field"); - - List fields = new ArrayList<>(); - fields.add(new TableField("str_field", Types.STRING, false)); - fields.add(new TableField("int_field", Types.LONG, false)); - fields.add(new TableField("long_field", Types.LONG, false)); - fields.add(new TableField("double_field", Types.DOUBLE, false)); - fields.add(new TableField("bool_field", Types.BOOLEAN, false)); - TableSchema tableSchema = new TableSchema(fields); - - Neo4jTableSource source = new Neo4jTableSource(); - source.init(conf, tableSchema); - - RuntimeContext context = Mockito.mock(RuntimeContext.class); - source.open(context); - - List partitions = source.listPartitions(); - AllFetchWindow window = new AllFetchWindow(0L); - FetchData fetchData = source.fetch(partitions.get(0), Optional.empty(), window); - - List rows = (List) fetchData.getDataIterator(); - Assert.assertEquals(rows.size(), 1); - - Row row = rows.get(0); - Assert.assertEquals(row.getField(0, Types.STRING), "test"); - Assert.assertEquals(row.getField(1, Types.LONG), 42L); - Assert.assertEquals(row.getField(2, Types.LONG), 9999999999L); - Assert.assertEquals((Double) row.getField(3, Types.DOUBLE), 3.14, 0.01); - Assert.assertTrue((Boolean) row.getField(4, Types.BOOLEAN)); - - source.close(); - } - - @Test - public void testPaginationFirstPage() throws Exception { - Configuration conf = new Configuration(); - conf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_URI, neo4jUri); - conf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_USERNAME, USERNAME); - conf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_PASSWORD, PASSWORD); - conf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_DATABASE, DATABASE); - conf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_QUERY, - "MATCH (n:TestNode) RETURN n.id as id ORDER BY n.id"); - - List fields = new ArrayList<>(); - fields.add(new TableField("id", Types.LONG, false)); - TableSchema tableSchema = new TableSchema(fields); - - Neo4jTableSource source = new Neo4jTableSource(); - source.init(conf, tableSchema); - - RuntimeContext context = Mockito.mock(RuntimeContext.class); - source.open(context); - - List partitions = source.listPartitions(); - - // Fetch first page with size 10 - SizeFetchWindow window = new SizeFetchWindow(0L, 10L); - FetchData fetchData = source.fetch(partitions.get(0), Optional.empty(), window); - - List rows = (List) fetchData.getDataIterator(); - Assert.assertEquals(rows.size(), 10); - Assert.assertFalse(fetchData.isFinish()); - Assert.assertEquals(fetchData.getNextOffset().getOffset(), 10); - - source.close(); - } - - @Test - public void testPaginationMultiplePages() throws Exception { - Configuration conf = new Configuration(); - conf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_URI, neo4jUri); - conf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_USERNAME, USERNAME); - conf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_PASSWORD, PASSWORD); - conf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_DATABASE, DATABASE); - conf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_QUERY, - "MATCH (n:TestNode) RETURN n.id as id, n.value as value ORDER BY n.id"); - - List fields = new ArrayList<>(); - fields.add(new TableField("id", Types.LONG, false)); - fields.add(new TableField("value", Types.LONG, false)); - TableSchema tableSchema = new TableSchema(fields); - - Neo4jTableSource source = new Neo4jTableSource(); - source.init(conf, tableSchema); - - RuntimeContext context = Mockito.mock(RuntimeContext.class); - source.open(context); - - List partitions = source.listPartitions(); - SizeFetchWindow window = new SizeFetchWindow(0L, 15L); - - // First page - FetchData fetchData1 = source.fetch(partitions.get(0), Optional.empty(), window); - List rows1 = new ArrayList<>(); - fetchData1.getDataIterator().forEachRemaining(rows1::add); - Assert.assertEquals(rows1.size(), 15); - Assert.assertFalse(fetchData1.isFinish()); - - // Second page - FetchData fetchData2 = source.fetch(partitions.get(0), Optional.of(fetchData1.getNextOffset()), window); - List rows2 = new ArrayList<>(); - fetchData2.getDataIterator().forEachRemaining(rows2::add); - Assert.assertEquals(rows2.size(), 15); - - // Third page - FetchData fetchData3 = source.fetch(partitions.get(0), Optional.of(fetchData2.getNextOffset()), window); - List rows3 = new ArrayList<>(); - fetchData3.getDataIterator().forEachRemaining(rows3::add); - Assert.assertEquals(rows3.size(), 15); - - // Fourth page (should be smaller or empty) - FetchData fetchData4 = source.fetch(partitions.get(0), Optional.of(fetchData3.getNextOffset()), window); - List rows4 = new ArrayList<>(); - fetchData4.getDataIterator().forEachRemaining(rows4::add); - Assert.assertTrue(rows4.size() <= 15); - - source.close(); - } - - @Test - public void testReadRelationships() throws Exception { - Configuration conf = new Configuration(); - conf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_URI, neo4jUri); - conf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_USERNAME, USERNAME); - conf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_PASSWORD, PASSWORD); - conf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_DATABASE, DATABASE); - conf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_QUERY, - "MATCH (a)-[r:LINKS]->(b) " + - "RETURN a.id as source, b.id as target, r.weight as weight ORDER BY a.id"); - - List fields = new ArrayList<>(); - fields.add(new TableField("source", Types.LONG, false)); - fields.add(new TableField("target", Types.LONG, false)); - fields.add(new TableField("weight", Types.DOUBLE, false)); - TableSchema tableSchema = new TableSchema(fields); - - Neo4jTableSource source = new Neo4jTableSource(); - source.init(conf, tableSchema); - - RuntimeContext context = Mockito.mock(RuntimeContext.class); - source.open(context); - - List partitions = source.listPartitions(); - AllFetchWindow window = new AllFetchWindow(0L); - FetchData fetchData = source.fetch(partitions.get(0), Optional.empty(), window); - - List rows = new ArrayList<>(); - fetchData.getDataIterator().forEachRemaining(rows::add); - Assert.assertEquals(rows.size(), 2); - - // Verify first relationship - Row row1 = rows.get(0); - Assert.assertEquals(row1.getField(0, Types.LONG), 1L); - Assert.assertEquals(row1.getField(1, Types.LONG), 2L); - Assert.assertEquals((Double) row1.getField(2, Types.DOUBLE), 0.5, 0.01); - - source.close(); - } - - @Test - public void testEmptyResult() throws Exception { - Configuration conf = new Configuration(); - conf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_URI, neo4jUri); - conf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_USERNAME, USERNAME); - conf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_PASSWORD, PASSWORD); - conf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_DATABASE, DATABASE); - conf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_QUERY, - "MATCH (n:NonExistent) RETURN n.id as id"); - - List fields = new ArrayList<>(); - fields.add(new TableField("id", Types.LONG, false)); - TableSchema tableSchema = new TableSchema(fields); - - Neo4jTableSource source = new Neo4jTableSource(); - source.init(conf, tableSchema); - - RuntimeContext context = Mockito.mock(RuntimeContext.class); - source.open(context); - - List partitions = source.listPartitions(); - AllFetchWindow window = new AllFetchWindow(0L); - FetchData fetchData = source.fetch(partitions.get(0), Optional.empty(), window); - - List rows = (List) fetchData.getDataIterator(); - Assert.assertEquals(rows.size(), 0); - Assert.assertTrue(fetchData.isFinish()); - - source.close(); - } - - @Test - public void testReadWithNullValues() throws Exception { - // Create a node with null values - try (Driver driver = org.neo4j.driver.GraphDatabase.driver( - neo4jUri, - org.neo4j.driver.AuthTokens.basic(USERNAME, PASSWORD))) { - try (Session session = driver.session()) { - session.run("CREATE (n:NullTestNode {id: 999, name: 'test'})"); - } - } - - Configuration conf = new Configuration(); - conf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_URI, neo4jUri); - conf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_USERNAME, USERNAME); - conf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_PASSWORD, PASSWORD); - conf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_DATABASE, DATABASE); - conf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_QUERY, - "MATCH (n:NullTestNode {id: 999}) RETURN n.id as id, n.name as name, n.missing as missing"); - - List fields = new ArrayList<>(); - fields.add(new TableField("id", Types.LONG, false)); - fields.add(new TableField("name", Types.STRING, false)); - fields.add(new TableField("missing", Types.STRING, true)); - TableSchema tableSchema = new TableSchema(fields); - - Neo4jTableSource source = new Neo4jTableSource(); - source.init(conf, tableSchema); - - RuntimeContext context = Mockito.mock(RuntimeContext.class); - source.open(context); - - List partitions = source.listPartitions(); - AllFetchWindow window = new AllFetchWindow(0L); - FetchData fetchData = source.fetch(partitions.get(0), Optional.empty(), window); - - List rows = (List) fetchData.getDataIterator(); - Assert.assertEquals(rows.size(), 1); - - Row row = rows.get(0); - Assert.assertEquals(row.getField(0, Types.LONG), 999L); - Assert.assertEquals(row.getField(1, Types.STRING), "test"); - Assert.assertNull(row.getField(2, Types.STRING)); - - source.close(); - } - - @Test - public void testAllWindowType() throws Exception { - Configuration conf = new Configuration(); - conf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_URI, neo4jUri); - conf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_USERNAME, USERNAME); - conf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_PASSWORD, PASSWORD); - conf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_DATABASE, DATABASE); - conf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_QUERY, - "MATCH (n:TestNode) WHERE n.id <= 10 RETURN n.id as id ORDER BY n.id"); - - List fields = new ArrayList<>(); - fields.add(new TableField("id", Types.LONG, false)); - TableSchema tableSchema = new TableSchema(fields); - - Neo4jTableSource source = new Neo4jTableSource(); - source.init(conf, tableSchema); - - RuntimeContext context = Mockito.mock(RuntimeContext.class); - source.open(context); - - List partitions = source.listPartitions(); - AllFetchWindow window = new AllFetchWindow(0L); - FetchData fetchData = source.fetch(partitions.get(0), Optional.empty(), window); - - List rows = (List) fetchData.getDataIterator(); - Assert.assertEquals(rows.size(), 10); - Assert.assertTrue(fetchData.isFinish()); // ALL_WINDOW should mark as finished - - source.close(); - } - - @Test(expectedExceptions = org.apache.geaflow.dsl.common.exception.GeaFlowDSLException.class) - public void testMissingQuery() { - Configuration conf = new Configuration(); - conf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_URI, neo4jUri); - conf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_USERNAME, USERNAME); - conf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_PASSWORD, PASSWORD); - conf.put(Neo4jConfigKeys.GEAFLOW_DSL_NEO4J_DATABASE, DATABASE); - // No query specified - - List fields = new ArrayList<>(); - fields.add(new TableField("id", Types.LONG, false)); - TableSchema tableSchema = new TableSchema(fields); - - Neo4jTableSource source = new Neo4jTableSource(); - source.init(conf, tableSchema); // Should throw exception - } -} From d0a830f64993371248cdfd7d9fcbb87436fa2e2e Mon Sep 17 00:00:00 2001 From: "jishiwen.jsw" Date: Sat, 15 Nov 2025 00:24:09 +0800 Subject: [PATCH 3/5] feat: update by comment Signed-off-by: jishiwen.jsw --- .../pom.xml | 2 +- .../ElasticsearchConfigKeys.java | 71 ++++++ .../elasticsearch/ElasticsearchConstants.java | 44 ++++ .../ElasticsearchTableConnector.java | 46 ++++ .../elasticsearch/ElasticsearchTableSink.java | 214 ++++++++++++++++++ .../ElasticsearchTableSource.java | 65 +++--- ...e.geaflow.dsl.connector.api.TableConnector | 19 ++ .../ElasticsearchIntegrationTest.java | 209 ----------------- .../dsl/connector/neo4j/Neo4jConstants.java | 2 + .../dsl/connector/neo4j/Neo4jTableSink.java | 9 +- .../geaflow-dsl/geaflow-dsl-connector/pom.xml | 1 + 11 files changed, 436 insertions(+), 246 deletions(-) create mode 100644 geaflow/geaflow-dsl/geaflow-dsl-connector/geaflow-dsl-connector-elasticsearch/src/main/java/org/apache/geaflow/dsl/connector/elasticsearch/ElasticsearchConfigKeys.java create mode 100644 geaflow/geaflow-dsl/geaflow-dsl-connector/geaflow-dsl-connector-elasticsearch/src/main/java/org/apache/geaflow/dsl/connector/elasticsearch/ElasticsearchConstants.java create mode 100644 geaflow/geaflow-dsl/geaflow-dsl-connector/geaflow-dsl-connector-elasticsearch/src/main/java/org/apache/geaflow/dsl/connector/elasticsearch/ElasticsearchTableConnector.java create mode 100644 geaflow/geaflow-dsl/geaflow-dsl-connector/geaflow-dsl-connector-elasticsearch/src/main/java/org/apache/geaflow/dsl/connector/elasticsearch/ElasticsearchTableSink.java create mode 100644 geaflow/geaflow-dsl/geaflow-dsl-connector/geaflow-dsl-connector-elasticsearch/src/main/resources/META-INF/services/org.apache.geaflow.dsl.connector.api.TableConnector delete mode 100644 geaflow/geaflow-dsl/geaflow-dsl-connector/geaflow-dsl-connector-elasticsearch/src/test/java/org/apache/geaflow/dsl/connector/elasticsearch/ElasticsearchIntegrationTest.java diff --git a/geaflow/geaflow-dsl/geaflow-dsl-connector/geaflow-dsl-connector-elasticsearch/pom.xml b/geaflow/geaflow-dsl/geaflow-dsl-connector/geaflow-dsl-connector-elasticsearch/pom.xml index 2a9b91c1d..8cf12e3bd 100644 --- a/geaflow/geaflow-dsl/geaflow-dsl-connector/geaflow-dsl-connector-elasticsearch/pom.xml +++ b/geaflow/geaflow-dsl/geaflow-dsl-connector/geaflow-dsl-connector-elasticsearch/pom.xml @@ -21,7 +21,7 @@ - + org.apache.geaflow geaflow-dsl-connector 0.6.8-SNAPSHOT diff --git a/geaflow/geaflow-dsl/geaflow-dsl-connector/geaflow-dsl-connector-elasticsearch/src/main/java/org/apache/geaflow/dsl/connector/elasticsearch/ElasticsearchConfigKeys.java b/geaflow/geaflow-dsl/geaflow-dsl-connector/geaflow-dsl-connector-elasticsearch/src/main/java/org/apache/geaflow/dsl/connector/elasticsearch/ElasticsearchConfigKeys.java new file mode 100644 index 000000000..65df4b4cd --- /dev/null +++ b/geaflow/geaflow-dsl/geaflow-dsl-connector/geaflow-dsl-connector-elasticsearch/src/main/java/org/apache/geaflow/dsl/connector/elasticsearch/ElasticsearchConfigKeys.java @@ -0,0 +1,71 @@ +/* + * 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.geaflow.dsl.connector.elasticsearch; + +import org.apache.geaflow.common.config.ConfigKey; +import org.apache.geaflow.common.config.ConfigKeys; + +public class ElasticsearchConfigKeys { + + public static final ConfigKey GEAFLOW_DSL_ELASTICSEARCH_HOSTS = ConfigKeys + .key("geaflow.dsl.elasticsearch.hosts") + .noDefaultValue() + .description("Elasticsearch cluster hosts list."); + + public static final ConfigKey GEAFLOW_DSL_ELASTICSEARCH_INDEX = ConfigKeys + .key("geaflow.dsl.elasticsearch.index") + .noDefaultValue() + .description("Elasticsearch index name."); + + public static final ConfigKey GEAFLOW_DSL_ELASTICSEARCH_DOCUMENT_ID_FIELD = ConfigKeys + .key("geaflow.dsl.elasticsearch.document.id.field") + .noDefaultValue() + .description("Elasticsearch document id field."); + + public static final ConfigKey GEAFLOW_DSL_ELASTICSEARCH_USERNAME = ConfigKeys + .key("geaflow.dsl.elasticsearch.username") + .noDefaultValue() + .description("Elasticsearch username for authentication."); + + public static final ConfigKey GEAFLOW_DSL_ELASTICSEARCH_PASSWORD = ConfigKeys + .key("geaflow.dsl.elasticsearch.password") + .noDefaultValue() + .description("Elasticsearch password for authentication."); + + public static final ConfigKey GEAFLOW_DSL_ELASTICSEARCH_BATCH_SIZE = ConfigKeys + .key("geaflow.dsl.elasticsearch.batch.size") + .defaultValue("1000") + .description("Elasticsearch batch write size."); + + public static final ConfigKey GEAFLOW_DSL_ELASTICSEARCH_SCROLL_TIMEOUT = ConfigKeys + .key("geaflow.dsl.elasticsearch.scroll.timeout") + .defaultValue("60s") + .description("Elasticsearch scroll query timeout."); + + public static final ConfigKey GEAFLOW_DSL_ELASTICSEARCH_CONNECTION_TIMEOUT = ConfigKeys + .key("geaflow.dsl.elasticsearch.connection.timeout") + .defaultValue("1000") + .description("Elasticsearch connection timeout in milliseconds."); + + public static final ConfigKey GEAFLOW_DSL_ELASTICSEARCH_SOCKET_TIMEOUT = ConfigKeys + .key("geaflow.dsl.elasticsearch.socket.timeout") + .defaultValue("30000") + .description("Elasticsearch socket timeout in milliseconds."); +} diff --git a/geaflow/geaflow-dsl/geaflow-dsl-connector/geaflow-dsl-connector-elasticsearch/src/main/java/org/apache/geaflow/dsl/connector/elasticsearch/ElasticsearchConstants.java b/geaflow/geaflow-dsl/geaflow-dsl-connector/geaflow-dsl-connector-elasticsearch/src/main/java/org/apache/geaflow/dsl/connector/elasticsearch/ElasticsearchConstants.java new file mode 100644 index 000000000..ed8c7adac --- /dev/null +++ b/geaflow/geaflow-dsl/geaflow-dsl-connector/geaflow-dsl-connector-elasticsearch/src/main/java/org/apache/geaflow/dsl/connector/elasticsearch/ElasticsearchConstants.java @@ -0,0 +1,44 @@ +/* + * 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.geaflow.dsl.connector.elasticsearch; + +public class ElasticsearchConstants { + + public static final int DEFAULT_BATCH_SIZE = 1000; + + public static final String DEFAULT_SCROLL_TIMEOUT = "60s"; + + public static final int DEFAULT_CONNECTION_TIMEOUT = 1000; + + public static final int DEFAULT_SOCKET_TIMEOUT = 30000; + + public static final int DEFAULT_SEARCH_SIZE = 1000; + + public static final String ES_SCHEMA_SUFFIX = "://"; + + public static final String ES_HTTP_SCHEME = "http"; + + public static final String ES_HTTPS_SCHEME = "https"; + + public static final String ES_SPLIT_COMMA = ","; + + public static final String ES_SPLIT_COLON = ";"; + +} diff --git a/geaflow/geaflow-dsl/geaflow-dsl-connector/geaflow-dsl-connector-elasticsearch/src/main/java/org/apache/geaflow/dsl/connector/elasticsearch/ElasticsearchTableConnector.java b/geaflow/geaflow-dsl/geaflow-dsl-connector/geaflow-dsl-connector-elasticsearch/src/main/java/org/apache/geaflow/dsl/connector/elasticsearch/ElasticsearchTableConnector.java new file mode 100644 index 000000000..f8950a8d0 --- /dev/null +++ b/geaflow/geaflow-dsl/geaflow-dsl-connector/geaflow-dsl-connector-elasticsearch/src/main/java/org/apache/geaflow/dsl/connector/elasticsearch/ElasticsearchTableConnector.java @@ -0,0 +1,46 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.geaflow.dsl.connector.elasticsearch; + +import org.apache.geaflow.common.config.Configuration; +import org.apache.geaflow.dsl.connector.api.TableReadableConnector; +import org.apache.geaflow.dsl.connector.api.TableSink; +import org.apache.geaflow.dsl.connector.api.TableSource; +import org.apache.geaflow.dsl.connector.api.TableWritableConnector; + +public class ElasticsearchTableConnector implements TableReadableConnector, TableWritableConnector { + + public static final String TYPE = "ELASTICSEARCH"; + + @Override + public String getType() { + return TYPE; + } + + @Override + public TableSource createSource(Configuration conf) { + return new ElasticsearchTableSource(); + } + + @Override + public TableSink createSink(Configuration conf) { + return new ElasticsearchTableSink(); + } +} diff --git a/geaflow/geaflow-dsl/geaflow-dsl-connector/geaflow-dsl-connector-elasticsearch/src/main/java/org/apache/geaflow/dsl/connector/elasticsearch/ElasticsearchTableSink.java b/geaflow/geaflow-dsl/geaflow-dsl-connector/geaflow-dsl-connector-elasticsearch/src/main/java/org/apache/geaflow/dsl/connector/elasticsearch/ElasticsearchTableSink.java new file mode 100644 index 000000000..4831a6c21 --- /dev/null +++ b/geaflow/geaflow-dsl/geaflow-dsl-connector/geaflow-dsl-connector-elasticsearch/src/main/java/org/apache/geaflow/dsl/connector/elasticsearch/ElasticsearchTableSink.java @@ -0,0 +1,214 @@ +/* + * 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.geaflow.dsl.connector.elasticsearch; + +import com.google.gson.Gson; +import java.io.IOException; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Objects; +import org.apache.geaflow.api.context.RuntimeContext; +import org.apache.geaflow.common.config.Configuration; +import org.apache.geaflow.dsl.common.data.Row; +import org.apache.geaflow.dsl.common.exception.GeaFlowDSLException; +import org.apache.geaflow.dsl.common.types.StructType; +import org.apache.geaflow.dsl.connector.api.TableSink; +import org.apache.http.HttpHost; +import org.apache.http.auth.AuthScope; +import org.apache.http.auth.UsernamePasswordCredentials; +import org.apache.http.client.CredentialsProvider; +import org.apache.http.impl.client.BasicCredentialsProvider; +import org.elasticsearch.action.bulk.BulkRequest; +import org.elasticsearch.action.bulk.BulkResponse; +import org.elasticsearch.action.index.IndexRequest; +import org.elasticsearch.client.RequestOptions; +import org.elasticsearch.client.RestClient; +import org.elasticsearch.client.RestClientBuilder; +import org.elasticsearch.client.RestHighLevelClient; +import org.elasticsearch.xcontent.XContentType; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class ElasticsearchTableSink implements TableSink { + + private static final Logger LOGGER = LoggerFactory.getLogger(ElasticsearchTableSink.class); + private static final Gson GSON = new Gson(); + + private StructType schema; + private String hosts; + private String indexName; + private String documentIdField; + private String username; + private String password; + private int batchSize; + private int connectionTimeout; + private int socketTimeout; + + private RestHighLevelClient client; + private BulkRequest bulkRequest; + private int batchCounter = 0; + + @Override + public void init(Configuration conf, StructType schema) { + LOGGER.info("Prepare with config: {}, \n schema: {}", conf, schema); + this.schema = schema; + + this.hosts = conf.getString(ElasticsearchConfigKeys.GEAFLOW_DSL_ELASTICSEARCH_HOSTS); + this.indexName = conf.getString(ElasticsearchConfigKeys.GEAFLOW_DSL_ELASTICSEARCH_INDEX); + this.documentIdField = conf.getString(ElasticsearchConfigKeys.GEAFLOW_DSL_ELASTICSEARCH_DOCUMENT_ID_FIELD, ""); + this.username = conf.getString(ElasticsearchConfigKeys.GEAFLOW_DSL_ELASTICSEARCH_USERNAME, ""); + this.password = conf.getString(ElasticsearchConfigKeys.GEAFLOW_DSL_ELASTICSEARCH_PASSWORD, ""); + this.batchSize = conf.getInteger(ElasticsearchConfigKeys.GEAFLOW_DSL_ELASTICSEARCH_BATCH_SIZE, + ElasticsearchConstants.DEFAULT_BATCH_SIZE); + this.connectionTimeout = conf.getInteger(ElasticsearchConfigKeys.GEAFLOW_DSL_ELASTICSEARCH_CONNECTION_TIMEOUT, + ElasticsearchConstants.DEFAULT_CONNECTION_TIMEOUT); + this.socketTimeout = conf.getInteger(ElasticsearchConfigKeys.GEAFLOW_DSL_ELASTICSEARCH_SOCKET_TIMEOUT, + ElasticsearchConstants.DEFAULT_SOCKET_TIMEOUT); + } + + @Override + public void open(RuntimeContext context) { + try { + this.client = createElasticsearchClient(); + this.bulkRequest = new BulkRequest(); + } catch (Exception e) { + throw new GeaFlowDSLException("Failed to create Elasticsearch client", e); + } + } + + @Override + public void write(Row row) throws IOException { + // Convert row to JSON document + String jsonDocument = rowToJson(row); + + // Create index request + IndexRequest request = new IndexRequest(indexName); + request.source(jsonDocument, XContentType.JSON); + + // Set document ID if specified + if (documentIdField != null && !documentIdField.isEmpty()) { + int idFieldIndex = schema.indexOf(documentIdField); + if (idFieldIndex >= 0) { + Object idValue = row.getField(idFieldIndex, schema.getType(idFieldIndex)); + if (idValue != null) { + request.id(idValue.toString()); + } + } + } + + // Add to bulk request + bulkRequest.add(request); + batchCounter++; + + // Flush if batch size reached + if (batchCounter >= batchSize) { + flush(); + } + } + + @Override + public void finish() throws IOException { + flush(); + } + + @Override + public void close() { + try { + if (Objects.nonNull(this.client)) { + client.close(); + } + } catch (IOException e) { + throw new GeaFlowDSLException("Failed to close Elasticsearch client", e); + } + } + + private void flush() throws IOException { + if (batchCounter > 0 && client != null) { + BulkResponse bulkResponse = client.bulk(bulkRequest, RequestOptions.DEFAULT); + if (bulkResponse.hasFailures()) { + LOGGER.error("Bulk request failed: {}", bulkResponse.buildFailureMessage()); + throw new IOException("Bulk request failed: " + bulkResponse.buildFailureMessage()); + } + bulkRequest = new BulkRequest(); + batchCounter = 0; + } + } + + private String rowToJson(Row row) { + // Convert Row to JSON string + Map map = new HashMap<>(); + List fieldNames = schema.getFieldNames(); + + for (int i = 0; i < fieldNames.size(); i++) { + String fieldName = fieldNames.get(i); + Object fieldValue = row.getField(i, schema.getType(i)); + map.put(fieldName, fieldValue); + } + + return GSON.toJson(map); + } + + private RestHighLevelClient createElasticsearchClient() { + try { + String[] hostArray = hosts.split(","); + HttpHost[] httpHosts = new HttpHost[hostArray.length]; + + for (int i = 0; i < hostArray.length; i++) { + String host = hostArray[i].trim(); + if (host.startsWith("http://")) { + host = host.substring(7); + } else if (host.startsWith("https://")) { + host = host.substring(8); + } + + String[] parts = host.split(":"); + String hostname = parts[0]; + int port = parts.length > 1 ? Integer.parseInt(parts[1]) : 9200; + httpHosts[i] = new HttpHost(hostname, port, "http"); + } + + RestClientBuilder builder = RestClient.builder(httpHosts); + + // Configure timeouts + builder.setRequestConfigCallback(requestConfigBuilder -> { + requestConfigBuilder.setConnectTimeout(connectionTimeout); + requestConfigBuilder.setSocketTimeout(socketTimeout); + return requestConfigBuilder; + }); + + // Configure authentication if provided + if (username != null && !username.isEmpty() && password != null) { + final CredentialsProvider credentialsProvider = new BasicCredentialsProvider(); + credentialsProvider.setCredentials(AuthScope.ANY, + new UsernamePasswordCredentials(username, password)); + + builder.setHttpClientConfigCallback(httpClientBuilder -> { + httpClientBuilder.setDefaultCredentialsProvider(credentialsProvider); + return httpClientBuilder; + }); + } + + return new RestHighLevelClient(builder); + } catch (Exception e) { + throw new GeaFlowDSLException("Failed to create Elasticsearch client", e); + } + } +} diff --git a/geaflow/geaflow-dsl/geaflow-dsl-connector/geaflow-dsl-connector-elasticsearch/src/main/java/org/apache/geaflow/dsl/connector/elasticsearch/ElasticsearchTableSource.java b/geaflow/geaflow-dsl/geaflow-dsl-connector/geaflow-dsl-connector-elasticsearch/src/main/java/org/apache/geaflow/dsl/connector/elasticsearch/ElasticsearchTableSource.java index 0576de1cc..649fe7c09 100644 --- a/geaflow/geaflow-dsl/geaflow-dsl-connector/geaflow-dsl-connector-elasticsearch/src/main/java/org/apache/geaflow/dsl/connector/elasticsearch/ElasticsearchTableSource.java +++ b/geaflow/geaflow-dsl/geaflow-dsl-connector/geaflow-dsl-connector-elasticsearch/src/main/java/org/apache/geaflow/dsl/connector/elasticsearch/ElasticsearchTableSource.java @@ -24,23 +24,24 @@ import java.io.IOException; import java.lang.reflect.Type; import java.util.ArrayList; +import java.util.Collections; import java.util.List; import java.util.Map; import java.util.Optional; import org.apache.geaflow.api.context.RuntimeContext; import org.apache.geaflow.common.config.Configuration; +import org.apache.geaflow.dsl.common.data.Row; +import org.apache.geaflow.dsl.common.data.impl.ObjectRow; +import org.apache.geaflow.dsl.common.exception.GeaFlowDSLException; import org.apache.geaflow.dsl.common.types.StructType; import org.apache.geaflow.dsl.common.types.TableSchema; import org.apache.geaflow.dsl.connector.api.FetchData; +import org.apache.geaflow.dsl.connector.api.Offset; import org.apache.geaflow.dsl.connector.api.Partition; import org.apache.geaflow.dsl.connector.api.TableSource; import org.apache.geaflow.dsl.connector.api.serde.TableDeserializer; import org.apache.geaflow.dsl.connector.api.window.FetchWindow; import org.apache.http.HttpHost; -import org.apache.http.auth.AuthScope; -import org.apache.http.auth.UsernamePasswordCredentials; -import org.apache.http.client.CredentialsProvider; -import org.apache.http.impl.client.BasicCredentialsProvider; import org.elasticsearch.action.search.ClearScrollRequest; import org.elasticsearch.action.search.SearchRequest; import org.elasticsearch.action.search.SearchResponse; @@ -52,12 +53,23 @@ import org.elasticsearch.search.Scroll; import org.elasticsearch.search.SearchHit; import org.elasticsearch.search.builder.SearchSourceBuilder; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import static org.apache.geaflow.dsl.connector.elasticsearch.ElasticsearchConstants.DEFAULT_SEARCH_SIZE; +import static org.apache.geaflow.dsl.connector.elasticsearch.ElasticsearchConstants.ES_HTTPS_SCHEME; +import static org.apache.geaflow.dsl.connector.elasticsearch.ElasticsearchConstants.ES_HTTP_SCHEME; +import static org.apache.geaflow.dsl.connector.elasticsearch.ElasticsearchConstants.ES_SCHEMA_SUFFIX; +import static org.apache.geaflow.dsl.connector.elasticsearch.ElasticsearchConstants.ES_SPLIT_COLON; +import static org.apache.geaflow.dsl.connector.elasticsearch.ElasticsearchConstants.ES_SPLIT_COMMA; public class ElasticsearchTableSource implements TableSource { private static final Gson GSON = new Gson(); private static final Type MAP_TYPE = new TypeToken>(){}.getType(); + private Logger logger = LoggerFactory.getLogger(ElasticsearchTableSource.class); + private StructType schema; private String hosts; private String indexName; @@ -89,13 +101,13 @@ public void open(RuntimeContext context) { try { this.client = createElasticsearchClient(); } catch (Exception e) { - throw new RuntimeException("Failed to initialize Elasticsearch client", e); + throw new GeaFlowDSLException("Failed to initialize Elasticsearch client", e); } } @Override public List listPartitions() { - return java.util.Collections.singletonList(new ElasticsearchPartition(indexName)); + return Collections.singletonList(new ElasticsearchPartition(indexName)); } @Override @@ -107,7 +119,7 @@ public void init(Configuration configuration, StructType structType) { } @Override - public java.util.List deserialize(IN record) { + public List deserialize(IN record) { if (record instanceof SearchHit) { SearchHit hit = (SearchHit) record; Map source = hit.getSourceAsMap(); @@ -121,22 +133,21 @@ public java.util.List deserialize(IN rec String fieldName = schema.getFields().get(i).getName(); values[i] = source.get(fieldName); } - org.apache.geaflow.dsl.common.data.Row row = - org.apache.geaflow.dsl.common.data.impl.ObjectRow.create(values); - return java.util.Collections.singletonList(row); + Row row = ObjectRow.create(values); + return Collections.singletonList(row); } - return java.util.Collections.emptyList(); + return Collections.emptyList(); } }; } @Override - public FetchData fetch(Partition partition, Optional startOffset, + public FetchData fetch(Partition partition, Optional startOffset, FetchWindow windowInfo) throws IOException { try { SearchRequest searchRequest = new SearchRequest(indexName); SearchSourceBuilder searchSourceBuilder = new SearchSourceBuilder(); - searchSourceBuilder.size(1000); // Batch size + searchSourceBuilder.size(DEFAULT_SEARCH_SIZE); // Batch size searchRequest.source(searchSourceBuilder); @@ -173,27 +184,27 @@ public void close() { } } catch (IOException e) { // Log error but don't throw exception in close method - e.printStackTrace(); + logger.warn("Failed to close Elasticsearch client", e); } } private RestHighLevelClient createElasticsearchClient() { try { - String[] hostArray = hosts.split(","); + String[] hostArray = hosts.split(ES_SPLIT_COMMA); HttpHost[] httpHosts = new HttpHost[hostArray.length]; for (int i = 0; i < hostArray.length; i++) { String host = hostArray[i].trim(); - if (host.startsWith("http://")) { + if (host.startsWith(ES_HTTP_SCHEME + ES_SCHEMA_SUFFIX)) { host = host.substring(7); - } else if (host.startsWith("https://")) { + } else if (host.startsWith(ES_HTTPS_SCHEME + ES_SCHEMA_SUFFIX)) { host = host.substring(8); } - String[] parts = host.split(":"); + String[] parts = host.split(ES_SPLIT_COLON); String hostname = parts[0]; int port = parts.length > 1 ? Integer.parseInt(parts[1]) : 9200; - httpHosts[i] = new HttpHost(hostname, port, "http"); + httpHosts[i] = new HttpHost(hostname, port, ES_HTTP_SCHEME); } RestClientBuilder builder = RestClient.builder(httpHosts); @@ -205,21 +216,9 @@ private RestHighLevelClient createElasticsearchClient() { return requestConfigBuilder; }); - // Configure authentication if provided - if (username != null && !username.isEmpty() && password != null) { - final CredentialsProvider credentialsProvider = new BasicCredentialsProvider(); - credentialsProvider.setCredentials(AuthScope.ANY, - new UsernamePasswordCredentials(username, password)); - - builder.setHttpClientConfigCallback(httpClientBuilder -> { - httpClientBuilder.setDefaultCredentialsProvider(credentialsProvider); - return httpClientBuilder; - }); - } - return new RestHighLevelClient(builder); } catch (Exception e) { - throw new RuntimeException("Failed to create Elasticsearch client", e); + throw new GeaFlowDSLException("Failed to create Elasticsearch client", e); } } @@ -236,7 +235,7 @@ public String getName() { } } - public static class ElasticsearchOffset implements org.apache.geaflow.dsl.connector.api.Offset { + public static class ElasticsearchOffset implements Offset { private final String scrollId; private final long timestamp; diff --git a/geaflow/geaflow-dsl/geaflow-dsl-connector/geaflow-dsl-connector-elasticsearch/src/main/resources/META-INF/services/org.apache.geaflow.dsl.connector.api.TableConnector b/geaflow/geaflow-dsl/geaflow-dsl-connector/geaflow-dsl-connector-elasticsearch/src/main/resources/META-INF/services/org.apache.geaflow.dsl.connector.api.TableConnector new file mode 100644 index 000000000..93f5ad880 --- /dev/null +++ b/geaflow/geaflow-dsl/geaflow-dsl-connector/geaflow-dsl-connector-elasticsearch/src/main/resources/META-INF/services/org.apache.geaflow.dsl.connector.api.TableConnector @@ -0,0 +1,19 @@ +# +# 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.geaflow.dsl.connector.elasticsearch.ElasticsearchTableConnector diff --git a/geaflow/geaflow-dsl/geaflow-dsl-connector/geaflow-dsl-connector-elasticsearch/src/test/java/org/apache/geaflow/dsl/connector/elasticsearch/ElasticsearchIntegrationTest.java b/geaflow/geaflow-dsl/geaflow-dsl-connector/geaflow-dsl-connector-elasticsearch/src/test/java/org/apache/geaflow/dsl/connector/elasticsearch/ElasticsearchIntegrationTest.java deleted file mode 100644 index 821d88725..000000000 --- a/geaflow/geaflow-dsl/geaflow-dsl-connector/geaflow-dsl-connector-elasticsearch/src/test/java/org/apache/geaflow/dsl/connector/elasticsearch/ElasticsearchIntegrationTest.java +++ /dev/null @@ -1,209 +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.geaflow.dsl.connector.elasticsearch; - -import java.io.IOException; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.Iterator; -import java.util.List; -import java.util.Optional; -import org.apache.geaflow.api.context.RuntimeContext; -import org.apache.geaflow.common.config.Configuration; -import org.apache.geaflow.common.type.Types; -import org.apache.geaflow.dsl.common.data.Row; -import org.apache.geaflow.dsl.common.data.impl.ObjectRow; -import org.apache.geaflow.dsl.common.types.StructType; -import org.apache.geaflow.dsl.common.types.TableField; -import org.apache.geaflow.dsl.common.types.TableSchema; -import org.apache.geaflow.dsl.connector.api.FetchData; -import org.apache.geaflow.dsl.connector.api.Partition; -import org.apache.geaflow.dsl.connector.api.window.SizeFetchWindow; -import org.mockito.Mockito; -import org.testcontainers.containers.wait.strategy.Wait; -import org.testcontainers.elasticsearch.ElasticsearchContainer; -import org.testcontainers.utility.DockerImageName; -import org.testng.Assert; -import org.testng.annotations.AfterClass; -import org.testng.annotations.BeforeClass; -import org.testng.annotations.Test; - -public class ElasticsearchIntegrationTest { - - private static final String ELASTICSEARCH_VERSION = "8.11.0"; - private ElasticsearchContainer container; - private Configuration config; - private StructType sinkSchema; - private TableSchema sourceSchema; - - @BeforeClass - public void setUp() { - // Skip integration tests in CI environment if Docker is not available - String skipIntegrationTests = System.getProperty("skipIntegrationTests", "false"); - if ("true".equalsIgnoreCase(skipIntegrationTests)) { - throw new org.testng.SkipException("Integration tests are disabled"); - } - - // Check if Docker is available - try { - ProcessBuilder pb = new ProcessBuilder("docker", "info"); - Process process = pb.start(); - int exitCode = process.waitFor(); - if (exitCode != 0) { - throw new org.testng.SkipException("Docker is not running"); - } - } catch (IOException | InterruptedException e) { - throw new org.testng.SkipException("Docker is not available, skipping test"); - } - - DockerImageName image = DockerImageName - .parse("docker.elastic.co/elasticsearch/elasticsearch") - .withTag(ELASTICSEARCH_VERSION); - - container = new ElasticsearchContainer(image) - .withEnv("xpack.security.enabled", "false") - .withEnv("discovery.type", "single-node") - .waitingFor(Wait.forHttp("/_cluster/health") - .forStatusCode(200)); - - container.start(); - - // Configure connection - config = new Configuration(); - config.put(ElasticsearchConfigKeys.GEAFLOW_DSL_ELASTICSEARCH_HOSTS, - container.getHttpHostAddress()); - config.put(ElasticsearchConfigKeys.GEAFLOW_DSL_ELASTICSEARCH_INDEX, "test_users"); - config.put(ElasticsearchConfigKeys.GEAFLOW_DSL_ELASTICSEARCH_DOCUMENT_ID_FIELD, "id"); - config.put(ElasticsearchConfigKeys.GEAFLOW_DSL_ELASTICSEARCH_BATCH_SIZE, "10"); - - // Create schema - TableField idField = new TableField("id", Types.INTEGER, false); - TableField nameField = new TableField("name", Types.STRING, false); - TableField ageField = new TableField("age", Types.INTEGER, false); - sinkSchema = new StructType(Arrays.asList(idField, nameField, ageField)); - sourceSchema = new TableSchema(sinkSchema); - } - - @AfterClass - public void tearDown() { - if (container != null) { - container.stop(); - } - } - - @Test - public void testWriteAndRead() throws IOException { - RuntimeContext context = Mockito.mock(RuntimeContext.class); - - // Write data - ElasticsearchTableSink sink = new ElasticsearchTableSink(); - sink.init(config, sinkSchema); - sink.open(context); - - for (int i = 1; i <= 5; i++) { - Row row = ObjectRow.create(i, "User" + i, 20 + i); - sink.write(row); - } - sink.finish(); - sink.close(); - - // Wait for indexing - try { - Thread.sleep(2000); - } catch (InterruptedException e) { - Thread.currentThread().interrupt(); - } - - // Read data - ElasticsearchTableSource source = new ElasticsearchTableSource(); - source.init(config, sourceSchema); - source.open(context); - - List partitions = source.listPartitions(); - Assert.assertEquals(partitions.size(), 1); - - FetchData fetchData = source.fetch(partitions.get(0), Optional.empty(), - new SizeFetchWindow(0L, 10L)); - Assert.assertNotNull(fetchData); - - Iterator rowIterator = fetchData.getDataIterator(); - Assert.assertNotNull(rowIterator); - List rows = new ArrayList<>(); - rowIterator.forEachRemaining(rows::add); - Assert.assertTrue(rows.size() > 0); - - source.close(); - } - - @Test - public void testBulkWrite() throws IOException { - RuntimeContext context = Mockito.mock(RuntimeContext.class); - - ElasticsearchTableSink sink = new ElasticsearchTableSink(); - sink.init(config, sinkSchema); - sink.open(context); - - // Write more than batch size - for (int i = 1; i <= 25; i++) { - Row row = ObjectRow.create(i, "BulkUser" + i, 30 + i); - sink.write(row); - } - - sink.finish(); - sink.close(); - } - - @Test - public void testQueryDsl() throws IOException { - RuntimeContext context = Mockito.mock(RuntimeContext.class); - - // First write some data - ElasticsearchTableSink sink = new ElasticsearchTableSink(); - config.put(ElasticsearchConfigKeys.GEAFLOW_DSL_ELASTICSEARCH_INDEX, "test_query"); - sink.init(config, sinkSchema); - sink.open(context); - - Row row1 = ObjectRow.create(1, "Alice", 25); - Row row2 = ObjectRow.create(2, "Bob", 30); - sink.write(row1); - sink.write(row2); - sink.finish(); - sink.close(); - - // Wait for indexing - try { - Thread.sleep(2000); - } catch (InterruptedException e) { - Thread.currentThread().interrupt(); - } - - // Read with query - ElasticsearchTableSource source = new ElasticsearchTableSource(); - source.init(config, sourceSchema); - source.open(context); - - List partitions = source.listPartitions(); - FetchData fetchData = source.fetch(partitions.get(0), Optional.empty(), - new SizeFetchWindow(0L, 10L)); - - Assert.assertNotNull(fetchData); - source.close(); - } -} diff --git a/geaflow/geaflow-dsl/geaflow-dsl-connector/geaflow-dsl-connector-neo4j/src/main/java/org/apache/geaflow/dsl/connector/neo4j/Neo4jConstants.java b/geaflow/geaflow-dsl/geaflow-dsl-connector/geaflow-dsl-connector-neo4j/src/main/java/org/apache/geaflow/dsl/connector/neo4j/Neo4jConstants.java index fe748def2..153b1f5c0 100644 --- a/geaflow/geaflow-dsl/geaflow-dsl-connector/geaflow-dsl-connector-neo4j/src/main/java/org/apache/geaflow/dsl/connector/neo4j/Neo4jConstants.java +++ b/geaflow/geaflow-dsl/geaflow-dsl-connector/geaflow-dsl-connector-neo4j/src/main/java/org/apache/geaflow/dsl/connector/neo4j/Neo4jConstants.java @@ -33,5 +33,7 @@ public class Neo4jConstants { public static final String DEFAULT_NODE_LABEL = "Node"; + public static final String DEFAULT_RELATIONSHIP_LABEL = "relationship"; + public static final String DEFAULT_RELATIONSHIP_TYPE = "RELATES_TO"; } diff --git a/geaflow/geaflow-dsl/geaflow-dsl-connector/geaflow-dsl-connector-neo4j/src/main/java/org/apache/geaflow/dsl/connector/neo4j/Neo4jTableSink.java b/geaflow/geaflow-dsl/geaflow-dsl-connector/geaflow-dsl-connector-neo4j/src/main/java/org/apache/geaflow/dsl/connector/neo4j/Neo4jTableSink.java index 24e7aeaa8..c22bbb730 100644 --- a/geaflow/geaflow-dsl/geaflow-dsl-connector/geaflow-dsl-connector-neo4j/src/main/java/org/apache/geaflow/dsl/connector/neo4j/Neo4jTableSink.java +++ b/geaflow/geaflow-dsl/geaflow-dsl-connector/geaflow-dsl-connector-neo4j/src/main/java/org/apache/geaflow/dsl/connector/neo4j/Neo4jTableSink.java @@ -42,6 +42,9 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import static org.apache.geaflow.dsl.connector.neo4j.Neo4jConstants.DEFAULT_NODE_LABEL; +import static org.apache.geaflow.dsl.connector.neo4j.Neo4jConstants.DEFAULT_RELATIONSHIP_LABEL; + public class Neo4jTableSink implements TableSink { private static final Logger LOGGER = LoggerFactory.getLogger(Neo4jTableSink.class); @@ -101,11 +104,11 @@ private void validateConfig() { if (password == null || password.isEmpty()) { throw new GeaFlowDSLException("Neo4j password must be specified"); } - if ("node".equals(writeMode)) { + if (DEFAULT_NODE_LABEL.toLowerCase().equals(writeMode)) { if (nodeIdField == null || nodeIdField.isEmpty()) { throw new GeaFlowDSLException("Node ID field must be specified for node write mode"); } - } else if ("relationship".equals(writeMode)) { + } else if (DEFAULT_RELATIONSHIP_LABEL.equals(writeMode)) { if (relationshipSourceField == null || relationshipSourceField.isEmpty() || relationshipTargetField == null || relationshipTargetField.isEmpty()) { throw new GeaFlowDSLException("Relationship source and target fields must be specified for relationship write mode"); @@ -198,7 +201,7 @@ private void flush() { } try { - if ("node".equals(writeMode)) { + if (DEFAULT_NODE_LABEL.toLowerCase().equals(writeMode)) { writeNodes(); } else { writeRelationships(); diff --git a/geaflow/geaflow-dsl/geaflow-dsl-connector/pom.xml b/geaflow/geaflow-dsl/geaflow-dsl-connector/pom.xml index a304238bf..d45d4800d 100644 --- a/geaflow/geaflow-dsl/geaflow-dsl-connector/pom.xml +++ b/geaflow/geaflow-dsl/geaflow-dsl-connector/pom.xml @@ -48,6 +48,7 @@ geaflow-dsl-connector-random geaflow-dsl-connector-paimon geaflow-dsl-connector-neo4j + geaflow-dsl-connector-elasticsearch From 495eedf2b62b0ea2ee7c13b66e0a71ec20d17f2b Mon Sep 17 00:00:00 2001 From: "jishiwen.jsw" Date: Sat, 15 Nov 2025 12:46:03 +0800 Subject: [PATCH 4/5] fix: fix checkstyle errror Signed-off-by: jishiwen.jsw --- .../elasticsearch/ElasticsearchTableSource.java | 14 +++++++------- .../dsl/connector/neo4j/Neo4jTableSink.java | 6 +++--- 2 files changed, 10 insertions(+), 10 deletions(-) diff --git a/geaflow/geaflow-dsl/geaflow-dsl-connector/geaflow-dsl-connector-elasticsearch/src/main/java/org/apache/geaflow/dsl/connector/elasticsearch/ElasticsearchTableSource.java b/geaflow/geaflow-dsl/geaflow-dsl-connector/geaflow-dsl-connector-elasticsearch/src/main/java/org/apache/geaflow/dsl/connector/elasticsearch/ElasticsearchTableSource.java index 649fe7c09..6fed8c37c 100644 --- a/geaflow/geaflow-dsl/geaflow-dsl-connector/geaflow-dsl-connector-elasticsearch/src/main/java/org/apache/geaflow/dsl/connector/elasticsearch/ElasticsearchTableSource.java +++ b/geaflow/geaflow-dsl/geaflow-dsl-connector/geaflow-dsl-connector-elasticsearch/src/main/java/org/apache/geaflow/dsl/connector/elasticsearch/ElasticsearchTableSource.java @@ -19,6 +19,13 @@ package org.apache.geaflow.dsl.connector.elasticsearch; +import static org.apache.geaflow.dsl.connector.elasticsearch.ElasticsearchConstants.DEFAULT_SEARCH_SIZE; +import static org.apache.geaflow.dsl.connector.elasticsearch.ElasticsearchConstants.ES_HTTPS_SCHEME; +import static org.apache.geaflow.dsl.connector.elasticsearch.ElasticsearchConstants.ES_HTTP_SCHEME; +import static org.apache.geaflow.dsl.connector.elasticsearch.ElasticsearchConstants.ES_SCHEMA_SUFFIX; +import static org.apache.geaflow.dsl.connector.elasticsearch.ElasticsearchConstants.ES_SPLIT_COLON; +import static org.apache.geaflow.dsl.connector.elasticsearch.ElasticsearchConstants.ES_SPLIT_COMMA; + import com.google.gson.Gson; import com.google.gson.reflect.TypeToken; import java.io.IOException; @@ -56,13 +63,6 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import static org.apache.geaflow.dsl.connector.elasticsearch.ElasticsearchConstants.DEFAULT_SEARCH_SIZE; -import static org.apache.geaflow.dsl.connector.elasticsearch.ElasticsearchConstants.ES_HTTPS_SCHEME; -import static org.apache.geaflow.dsl.connector.elasticsearch.ElasticsearchConstants.ES_HTTP_SCHEME; -import static org.apache.geaflow.dsl.connector.elasticsearch.ElasticsearchConstants.ES_SCHEMA_SUFFIX; -import static org.apache.geaflow.dsl.connector.elasticsearch.ElasticsearchConstants.ES_SPLIT_COLON; -import static org.apache.geaflow.dsl.connector.elasticsearch.ElasticsearchConstants.ES_SPLIT_COMMA; - public class ElasticsearchTableSource implements TableSource { private static final Gson GSON = new Gson(); diff --git a/geaflow/geaflow-dsl/geaflow-dsl-connector/geaflow-dsl-connector-neo4j/src/main/java/org/apache/geaflow/dsl/connector/neo4j/Neo4jTableSink.java b/geaflow/geaflow-dsl/geaflow-dsl-connector/geaflow-dsl-connector-neo4j/src/main/java/org/apache/geaflow/dsl/connector/neo4j/Neo4jTableSink.java index c22bbb730..239bc0015 100644 --- a/geaflow/geaflow-dsl/geaflow-dsl-connector/geaflow-dsl-connector-neo4j/src/main/java/org/apache/geaflow/dsl/connector/neo4j/Neo4jTableSink.java +++ b/geaflow/geaflow-dsl/geaflow-dsl-connector/geaflow-dsl-connector-neo4j/src/main/java/org/apache/geaflow/dsl/connector/neo4j/Neo4jTableSink.java @@ -19,6 +19,9 @@ package org.apache.geaflow.dsl.connector.neo4j; +import static org.apache.geaflow.dsl.connector.neo4j.Neo4jConstants.DEFAULT_NODE_LABEL; +import static org.apache.geaflow.dsl.connector.neo4j.Neo4jConstants.DEFAULT_RELATIONSHIP_LABEL; + import java.io.IOException; import java.util.ArrayList; import java.util.HashMap; @@ -42,9 +45,6 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import static org.apache.geaflow.dsl.connector.neo4j.Neo4jConstants.DEFAULT_NODE_LABEL; -import static org.apache.geaflow.dsl.connector.neo4j.Neo4jConstants.DEFAULT_RELATIONSHIP_LABEL; - public class Neo4jTableSink implements TableSink { private static final Logger LOGGER = LoggerFactory.getLogger(Neo4jTableSink.class); From 4bd89cf80e51c1cfd86a970b727fd285a8200ba8 Mon Sep 17 00:00:00 2001 From: "jishiwen.jsw" Date: Sat, 29 Nov 2025 15:04:00 +0800 Subject: [PATCH 5/5] fix Signed-off-by: jishiwen.jsw --- .../apache/geaflow/dsl/connector/neo4j/Neo4jTableSource.java | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/geaflow/geaflow-dsl/geaflow-dsl-connector/geaflow-dsl-connector-neo4j/src/main/java/org/apache/geaflow/dsl/connector/neo4j/Neo4jTableSource.java b/geaflow/geaflow-dsl/geaflow-dsl-connector/geaflow-dsl-connector-neo4j/src/main/java/org/apache/geaflow/dsl/connector/neo4j/Neo4jTableSource.java index 787090015..019e3667d 100644 --- a/geaflow/geaflow-dsl/geaflow-dsl-connector/geaflow-dsl-connector-neo4j/src/main/java/org/apache/geaflow/dsl/connector/neo4j/Neo4jTableSource.java +++ b/geaflow/geaflow-dsl/geaflow-dsl-connector/geaflow-dsl-connector-neo4j/src/main/java/org/apache/geaflow/dsl/connector/neo4j/Neo4jTableSource.java @@ -27,6 +27,7 @@ import java.util.Map; import java.util.Objects; import java.util.Optional; +import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.TimeUnit; import org.apache.geaflow.api.context.RuntimeContext; import org.apache.geaflow.api.window.WindowType; @@ -71,7 +72,7 @@ public class Neo4jTableSource implements TableSource { private long connectionAcquisitionTimeout; private Driver driver; - private Map partitionSessionMap = new HashMap<>(); + private Map partitionSessionMap = new ConcurrentHashMap<>(); @Override public void init(Configuration tableConf, TableSchema tableSchema) {