├── .gitignore ├── .travis.yml ├── README.md ├── pom.xml └── src ├── main └── java │ └── org │ └── apache │ └── flink │ └── api │ └── java │ └── io │ └── neo4j │ ├── Neo4jFormatBase.java │ ├── Neo4jInputFormat.java │ └── Neo4jOutputFormat.java └── test ├── java └── org │ └── apache │ └── flink │ └── api │ └── java │ └── io │ └── neo4j │ ├── Neo4jFormatTest.java │ ├── Neo4jInputTest.java │ └── Neo4jOutputTest.java └── resources └── log4j-test.properties /.gitignore: -------------------------------------------------------------------------------- 1 | .idea 2 | target/ 3 | output/ 4 | *.iml 5 | -------------------------------------------------------------------------------- /.travis.yml: -------------------------------------------------------------------------------- 1 | language: java -------------------------------------------------------------------------------- /README.md: -------------------------------------------------------------------------------- 1 | [![Build Status](https://travis-ci.org/s1ck/flink-neo4j.svg?branch=master)](https://travis-ci.org/s1ck/flink-neo4j) 2 | 3 | 4 | ## flink-neo4j 5 | 6 | Contains [Apache Flink](https://flink.apache.org/) specific input and output formats to read [Cypher](http://neo4j.com/docs/stable/cypher-query-lang.html) results from [Neo4j](http://neo4j.com/) and write data back in parallel using Cypher batches. 7 | 8 | ## Examples 9 | 10 | * Read data from Neo4j into Flink datasets 11 | 12 | ```java 13 | ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); 14 | 15 | Neo4jInputFormat> input = Neo4jInputFormat 16 | .buildNeo4jInputFormat() 17 | .setRestURI("http://localhost:7475/db/data/") 18 | .setUsername("neo4j") 19 | .setPassword("password") 20 | .setCypherQuery("MATCH (n:User) RETURN id(n), n.name, n.born") 21 | .setConnectTimeout(10_000) 22 | .setReadTimeout(10_000) 23 | .finish(); 24 | 25 | DataSet> vertices = env.createInput(input, 26 | new TupleTypeInfo>( 27 | BasicTypeInfo.INT_TYPE_INFO, 28 | BasicTypeInfo.STRING_TYPE_INFO, 29 | BasicTypeInfo.INT_TYPE_INFO 30 | )); 31 | 32 | // do something 33 | ``` 34 | 35 | * Write data to Neo4j using CREATE statements 36 | 37 | ```java 38 | ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); 39 | 40 | Neo4jOutputFormat> outputFormat = Neo4jOutputFormat 41 | .buildNeo4jOutputFormat() 42 | .setRestURI("http://localhost:7475/db/data/") 43 | .setConnectTimeout(1_000) 44 | .setReadTimeout(1_000) 45 | .setCypherQuery("UNWIND {inserts} AS i CREATE (a:User {name:i.name, born:i.born})") 46 | .addParameterKey(0, "name") 47 | .addParameterKey(1, "born") 48 | .setTaskBatchSize(1000) 49 | .finish(); 50 | 51 | env.fromElements(new Tuple2<>("Alice", 1984),new Tuple2<>("Bob", 1976)).output(outputFormat); 52 | 53 | env.execute(); 54 | ``` 55 | 56 | ## Setup 57 | 58 | * Add repository and dependency to your maven project 59 | 60 | ``` 61 | 62 | 63 | dbleipzig 64 | Database Group Leipzig University 65 | https://wdiserv1.informatik.uni-leipzig.de:443/archiva/repository/dbleipzig/ 66 | 67 | true 68 | 69 | 70 | true 71 | 72 | 73 | 74 | 75 | 76 | org.s1ck 77 | flink-neo4j 78 | 0.1-SNAPSHOT 79 | 80 | ``` 81 | 82 | ## License 83 | 84 | Licensed under the [Apache License, Version 2.0](http://www.apache.org/licenses/LICENSE-2.0). 85 | -------------------------------------------------------------------------------- /pom.xml: -------------------------------------------------------------------------------- 1 | 2 | 5 | 4.0.0 6 | 7 | org.s1ck 8 | flink-neo4j 9 | 0.1-SNAPSHOT 10 | 11 | flink-neo4j 12 | Input format to read Cypher results from Neo4j into Flink tuples for further processing. 13 | http://github.com/s1ck/flink-neo4j 14 | 15 | 16 | 17 | Martin Junghanns 18 | m.junghanns@mailbox.org 19 | Leipzig University 20 | http://dbs.uni-leipzig.de/ 21 | s1ck 22 | 23 | 24 | 25 | 26 | scm:git:s1ck@github.com/s1ck/flink-neo4j.git 27 | scm:s1ck@github.com/s1ck/flink-neo4j.git 28 | http://github.com/s1ck/flink-neo4j.git 29 | 30 | 31 | 32 | 33 | dbleipzig 34 | https://wdiserv1.informatik.uni-leipzig.de:443/archiva/repository/dbleipzig/ 35 | 36 | 37 | 38 | 39 | 40 | apache.snapshots 41 | Apache Development Snapshot Repository 42 | https://repository.apache.org/content/repositories/snapshots/ 43 | 44 | false 45 | 46 | 47 | true 48 | 49 | 50 | 51 | 52 | 53 | UTF-8 54 | 3.0.0 55 | 1.7 56 | 57 | 3.0 58 | 2.19 59 | 60 | 1.1.3 61 | 1.19 62 | 4.12 63 | 2.3.7 64 | 65 | log4j-test.properties 66 | 67 | 68 | 69 | 70 | 71 | org.apache.maven.plugins 72 | maven-compiler-plugin 73 | ${plugin.maven-compiler.version} 74 | 75 | ${project.build.targetJdk} 76 | ${project.build.targetJdk} 77 | 78 | 79 | 80 | org.apache.maven.plugins 81 | maven-surefire-plugin 82 | ${plugin.maven-surefire.version} 83 | 84 | -Dlog4j.configuration=${log4j.properties} 85 | 86 | 87 | 88 | 89 | 90 | 91 | 92 | org.codehaus.jackson 93 | jackson-mapper-asl 94 | 1.9.13 95 | 96 | 97 | 98 | org.apache.flink 99 | flink-java 100 | ${dep.flink.version} 101 | 102 | 103 | org.codehaus.jackson 104 | jackson-core-asl 105 | 106 | 107 | org.codehaus.jackson 108 | jackson-mapper-asl 109 | 110 | 111 | com.twitter 112 | chill_2.10 113 | 114 | 115 | org.eclipse.jetty 116 | jetty-server 117 | 118 | 119 | org.eclipse.jetty 120 | jetty-servlet 121 | 122 | 123 | org.eclipse.jetty 124 | jetty-security 125 | 126 | 127 | 128 | 129 | 130 | com.twitter 131 | chill_2.11 132 | 0.5.2 133 | 134 | 135 | 136 | com.sun.jersey 137 | jersey-client 138 | ${dep.jersey.version} 139 | compile 140 | 141 | 142 | 143 | 144 | 145 | junit 146 | junit 147 | ${dep.junit.version} 148 | test 149 | 150 | 151 | 152 | org.neo4j.test 153 | neo4j-harness 154 | ${dep.neo4j.version} 155 | test 156 | 157 | 158 | 159 | org.apache.flink 160 | flink-clients_2.11 161 | ${dep.flink.version} 162 | test 163 | 164 | 165 | org.codehaus.jackson 166 | jackson-core-asl 167 | 168 | 169 | org.eclipse.jetty 170 | jetty-server 171 | 172 | 173 | org.eclipse.jetty 174 | jetty-servlet 175 | 176 | 177 | org.eclipse.jetty 178 | jetty-security 179 | 180 | 181 | com.sun.jersey 182 | jersey-client 183 | 184 | 185 | 186 | 187 | 188 | -------------------------------------------------------------------------------- /src/main/java/org/apache/flink/api/java/io/neo4j/Neo4jFormatBase.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | 19 | package org.apache.flink.api.java.io.neo4j; 20 | 21 | import com.sun.jersey.api.client.Client; 22 | import com.sun.jersey.api.client.filter.HTTPBasicAuthFilter; 23 | import org.apache.flink.hadoop.shaded.com.google.common.base.Strings; 24 | 25 | import java.io.Serializable; 26 | 27 | /** 28 | * Base class for {@link Neo4jInputFormat} and {@link Neo4jOutputFormat} that handles 29 | * connection related information. 30 | */ 31 | public abstract class Neo4jFormatBase implements Serializable { 32 | 33 | private static final long serialVersionUID = 1L; 34 | 35 | /** 36 | * Gets attached to the rest URI for transactional queries 37 | */ 38 | protected static final String TRANSACTION_URI = "transaction/commit"; 39 | 40 | /** 41 | * Payload template which is used to send cypher statements and payload to the endpoint. 42 | */ 43 | protected static final String PAYLOAD_TEMPLATE = 44 | "{\"statements\" : [ {\"statement\" : \"%s\", \"parameters\" : {%s} }]}"; 45 | 46 | protected String restURI; 47 | 48 | protected String query; 49 | 50 | protected String username; 51 | 52 | protected String password; 53 | 54 | protected int connectTimeout; 55 | 56 | protected int readTimeout; 57 | 58 | public Neo4jFormatBase(Builder builder) { 59 | this.restURI = builder.restURI; 60 | this.query = builder.query.replaceAll("\"", "\\\\\""); 61 | this.username = builder.username; 62 | this.password = builder.password; 63 | this.connectTimeout = builder.connectTimeout; 64 | this.readTimeout = builder.readTimeout; 65 | } 66 | 67 | public String getRestURI() { 68 | return restURI; 69 | } 70 | 71 | public String getCypherQuery() { 72 | return query; 73 | } 74 | 75 | public String getUsername() { 76 | return username; 77 | } 78 | 79 | public String getPassword() { 80 | return password; 81 | } 82 | 83 | public int getConnectTimeout() { 84 | return connectTimeout; 85 | } 86 | 87 | public int getReadTimeout() { 88 | return readTimeout; 89 | } 90 | 91 | /** 92 | * Create and configure the client for the REST call. 93 | * 94 | * @return Client 95 | */ 96 | protected Client createClient() { 97 | Client client = Client.create(); 98 | client.setConnectTimeout(getConnectTimeout()); 99 | client.setReadTimeout(getReadTimeout()); 100 | 101 | if (getUsername() != null && getPassword() != null) { 102 | client.addFilter(new HTTPBasicAuthFilter(getUsername(), getPassword())); 103 | } 104 | return client; 105 | } 106 | 107 | /** 108 | * Base builder class for building Neo4j in- and output formats. 109 | * 110 | * @param child builder instance 111 | */ 112 | public static abstract class Builder { 113 | 114 | protected String restURI; 115 | 116 | protected String query; 117 | 118 | protected String username; 119 | 120 | protected String password; 121 | 122 | /** 123 | * connect timeout in ms (default 1000) 124 | */ 125 | protected int connectTimeout = 1000; 126 | 127 | /** 128 | * read timeout in ms (default 1000) 129 | */ 130 | protected int readTimeout = 1000; 131 | 132 | /** 133 | * Set the Neo4j REST endpoint (e.g. "http://localhost:7475/db/data/") 134 | * 135 | * @param restURI Neo4j REST endpoint 136 | * @return builder 137 | */ 138 | public T setRestURI(String restURI) { 139 | this.restURI = restURI; 140 | return getThis(); 141 | } 142 | 143 | /** 144 | * Set Cypher query to execute against the endpoint. 145 | * 146 | * @param query cypher query 147 | * @return builder 148 | */ 149 | public T setCypherQuery(String query) { 150 | this.query = query; 151 | return getThis(); 152 | } 153 | 154 | /** 155 | * Set username if http authentication is enabled. 156 | * 157 | * @param userName Neo4j http auth username 158 | * @return builder 159 | */ 160 | public T setUsername(String userName) { 161 | this.username = userName; 162 | return getThis(); 163 | } 164 | 165 | /** 166 | * Set password if http authentication is enabled. 167 | * 168 | * @param password Neo4j http auth password 169 | * @return builder 170 | */ 171 | public T setPassword(String password) { 172 | this.password = password; 173 | return getThis(); 174 | } 175 | 176 | /** 177 | * Set connect timeout in milliseconds. 178 | * 179 | * @param connectTimeout connect timeout 180 | * @return builder 181 | */ 182 | public T setConnectTimeout(int connectTimeout) { 183 | this.connectTimeout = connectTimeout; 184 | return getThis(); 185 | } 186 | 187 | /** 188 | * Set read timeout in milliseconds. 189 | * 190 | * @param readTimeout read timeout 191 | * @return builder 192 | */ 193 | public T setReadTimeout(int readTimeout) { 194 | this.readTimeout = readTimeout; 195 | return getThis(); 196 | } 197 | 198 | /** 199 | * Validates mandatory arguments. 200 | */ 201 | protected void validate() { 202 | if (Strings.isNullOrEmpty(restURI)) { 203 | throw new IllegalArgumentException("No Rest URI was supplied."); 204 | } 205 | if (Strings.isNullOrEmpty(query)) { 206 | throw new IllegalArgumentException("No Cypher statement was supplied."); 207 | } 208 | } 209 | 210 | /** 211 | * Returns the concrete builder instance. 212 | * @return builder instance 213 | */ 214 | public abstract T getThis(); 215 | } 216 | } 217 | -------------------------------------------------------------------------------- /src/main/java/org/apache/flink/api/java/io/neo4j/Neo4jInputFormat.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | 19 | package org.apache.flink.api.java.io.neo4j; 20 | 21 | import com.sun.jersey.api.client.Client; 22 | import com.sun.jersey.api.client.ClientResponse; 23 | import org.apache.flink.api.common.io.DefaultInputSplitAssigner; 24 | import org.apache.flink.api.common.io.InputFormat; 25 | import org.apache.flink.api.common.io.NonParallelInput; 26 | import org.apache.flink.api.common.io.statistics.BaseStatistics; 27 | import org.apache.flink.api.java.tuple.Tuple; 28 | import org.apache.flink.configuration.Configuration; 29 | import org.apache.flink.core.io.GenericInputSplit; 30 | import org.apache.flink.core.io.InputSplit; 31 | import org.apache.flink.core.io.InputSplitAssigner; 32 | import org.apache.flink.types.NullValue; 33 | import org.codehaus.jackson.JsonNode; 34 | import org.codehaus.jackson.JsonParser; 35 | import org.codehaus.jackson.map.MappingJsonFactory; 36 | 37 | import javax.ws.rs.core.MediaType; 38 | import javax.ws.rs.core.Response; 39 | import java.io.IOException; 40 | 41 | public class Neo4jInputFormat extends Neo4jFormatBase 42 | implements InputFormat, NonParallelInput { 43 | 44 | private static final long serialVersionUID = 1L; 45 | 46 | private static final String ROW_FIELD = "row"; 47 | 48 | private transient ClientResponse response; 49 | 50 | private transient JsonParser jsonParser; 51 | 52 | public Neo4jInputFormat(Builder builder) { 53 | super(builder); 54 | } 55 | 56 | @Override 57 | public void open(InputSplit ignored) throws IOException { 58 | Client client = createClient(); 59 | 60 | String payload = String.format(PAYLOAD_TEMPLATE, query, ""); 61 | 62 | response = client 63 | .resource(restURI + TRANSACTION_URI) 64 | .accept(MediaType.APPLICATION_JSON) 65 | .header("Content-Type", "application/json;charset=UTF-8") 66 | .header("X-Stream", "true") 67 | .entity(payload) 68 | .post(ClientResponse.class); 69 | 70 | if (response.getStatus() == Response.Status.OK.getStatusCode()) { 71 | jsonParser = new MappingJsonFactory().createJsonParser(response.getEntityInputStream()); 72 | } else { 73 | close(); 74 | throw new IOException(String.format("Server returned status [%d]", response.getStatus())); 75 | } 76 | } 77 | 78 | /** 79 | * Moves JsonParser through the document until a "row" element is found or end of input. 80 | * 81 | * @return true, if there is another row element, false otherwise 82 | * @throws IOException 83 | */ 84 | @Override 85 | public boolean reachedEnd() throws IOException { 86 | boolean foundTuple = false; 87 | while(!foundTuple && jsonParser.nextToken() != null) { 88 | foundTuple = ROW_FIELD.equals(jsonParser.getCurrentName()); 89 | } 90 | return !foundTuple; 91 | } 92 | 93 | /** 94 | * Read "row" json node into tuple. 95 | * 96 | * @param reuse tuple for writing 97 | * @return tuple with row content 98 | * @throws IOException 99 | */ 100 | @Override 101 | public OUT nextRecord(OUT reuse) throws IOException { 102 | JsonNode node = jsonParser.readValueAsTree().get(ROW_FIELD); 103 | readFields(reuse, node); 104 | return reuse; 105 | } 106 | 107 | /** 108 | * Fills the given tuple fields with values according to their JSON type. 109 | * 110 | * @param reuse tuple for writing 111 | * @param fieldValues row values form query result 112 | * @throws IOException 113 | */ 114 | private void readFields(OUT reuse, JsonNode fieldValues) throws IOException { 115 | for (int i = 0; i < fieldValues.size(); i++) { 116 | JsonNode fieldValue = fieldValues.get(i); 117 | if (fieldValue.isNull()) { 118 | reuse.setField(NullValue.getInstance(), i); 119 | } else if (fieldValue.isBoolean()) { 120 | reuse.setField(fieldValue.getBooleanValue(), i); 121 | } else if (fieldValue.isInt()) { 122 | reuse.setField(fieldValue.getIntValue(), i); 123 | } else if (fieldValue.isLong()) { 124 | reuse.setField(fieldValue.getLongValue(), i); 125 | } else if (fieldValue.isDouble()) { 126 | reuse.setField(fieldValue.getDoubleValue(), i); 127 | } else if (fieldValue.isTextual()) { 128 | reuse.setField(fieldValue.getTextValue(), i); 129 | } else { 130 | close(); 131 | throw new IOException("Unsupported field type for value: " + fieldValue.getTextValue()); 132 | } 133 | } 134 | } 135 | 136 | @Override 137 | public void configure(Configuration configuration) { 138 | } 139 | 140 | /** 141 | * Close all used resources. 142 | * 143 | * @throws IOException 144 | */ 145 | @Override 146 | public void close() throws IOException { 147 | if (jsonParser != null) { 148 | jsonParser.close(); 149 | } 150 | if (response != null) { 151 | response.close(); 152 | } 153 | } 154 | 155 | @Override 156 | public BaseStatistics getStatistics(BaseStatistics baseStatistics) throws IOException { 157 | return baseStatistics; 158 | } 159 | 160 | @Override 161 | public InputSplit[] createInputSplits(int minNumSplits) throws IOException { 162 | return new GenericInputSplit[]{ 163 | new GenericInputSplit(0, 1) 164 | }; 165 | } 166 | 167 | @Override 168 | public InputSplitAssigner getInputSplitAssigner(InputSplit[] inputSplits) { 169 | return new DefaultInputSplitAssigner(inputSplits); 170 | } 171 | 172 | public static Builder buildNeo4jInputFormat() { 173 | return new Builder(); 174 | } 175 | 176 | /** 177 | * Used to build instances of {@link Neo4jInputFormat}. 178 | */ 179 | public static class Builder extends Neo4jFormatBase.Builder { 180 | 181 | @Override 182 | public Builder getThis() { 183 | return this; 184 | } 185 | 186 | public Neo4jInputFormat finish() { 187 | validate(); 188 | return new Neo4jInputFormat(this); 189 | } 190 | } 191 | } 192 | -------------------------------------------------------------------------------- /src/main/java/org/apache/flink/api/java/io/neo4j/Neo4jOutputFormat.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | 19 | package org.apache.flink.api.java.io.neo4j; 20 | 21 | import com.sun.jersey.api.client.Client; 22 | import com.sun.jersey.api.client.ClientResponse; 23 | import org.apache.flink.api.common.io.OutputFormat; 24 | import org.apache.flink.api.java.tuple.Tuple; 25 | import org.apache.flink.configuration.Configuration; 26 | import org.apache.flink.hadoop.shaded.com.google.common.base.Strings; 27 | import org.apache.flink.hadoop.shaded.com.google.common.collect.Lists; 28 | import org.codehaus.jackson.node.JsonNodeFactory; 29 | import org.codehaus.jackson.node.ObjectNode; 30 | 31 | import javax.ws.rs.core.MediaType; 32 | import javax.ws.rs.core.Response; 33 | import java.io.IOException; 34 | import java.util.List; 35 | import java.util.regex.Matcher; 36 | import java.util.regex.Pattern; 37 | 38 | import static org.apache.flink.util.Preconditions.checkArgument; 39 | 40 | public class Neo4jOutputFormat 41 | extends Neo4jFormatBase implements OutputFormat { 42 | 43 | private static final long serialVersionUID = 1L; 44 | 45 | /** 46 | * The parameter used in the Cypher UNWIND statement. 47 | */ 48 | private String parameterName; 49 | 50 | /** 51 | * The keys to access the values in each map (e.g. {name:'Alice',age:42} has the keys [name, age]) 52 | */ 53 | private String[] elementKeys = new String[0]; 54 | 55 | /** 56 | * Data types of the values associated to the element keys. 57 | */ 58 | private Class[] elementTypes; 59 | 60 | /** 61 | * Payload is continuously build as long as tuples come in. If the batch is full, or there are no 62 | * more tuples, the payload replaces the parameter in the query and get send to Neo4j. 63 | */ 64 | private StringBuilder payload; 65 | 66 | /** 67 | * Used to build parameter maps. 68 | */ 69 | private JsonNodeFactory nodeFactory; 70 | 71 | /** 72 | * batchSize = -1: Batch is send once when {@link #close()} is called 73 | * batchSize > 0: Batch is send, when currentBatchSize == batchSize 74 | */ 75 | private int batchSize = -1; 76 | 77 | /** 78 | * The current number of elements in the batch. 79 | */ 80 | private int currentBatchSize = 0; 81 | 82 | public Neo4jOutputFormat(Builder builder) { 83 | super(builder); 84 | this.elementKeys = builder.elementKeys.toArray(this.elementKeys); 85 | this.batchSize = builder.batchSize; 86 | } 87 | 88 | @Override 89 | public void configure(Configuration configuration) { 90 | } 91 | 92 | @Override 93 | public void open(int taskNumber, int numTasks) throws IOException { 94 | parameterName = getParameterName(); 95 | nodeFactory = JsonNodeFactory.instance; 96 | initBatch(); 97 | } 98 | 99 | /** 100 | * Adds the tuple to the current batch. If the batch is full, it is send to the Neo4j server. 101 | * 102 | * @param tuple Flink tuple 103 | * @throws IOException 104 | */ 105 | @Override 106 | public void writeRecord(OUT tuple) throws IOException { 107 | addToBatch(tuple); 108 | if (isBatchFull()) { 109 | finalizeBatch(); 110 | sendBatch(); 111 | initBatch(); 112 | } 113 | } 114 | 115 | @Override 116 | public void close() throws IOException { 117 | if (!isBatchEmpty()) { 118 | finalizeBatch(); 119 | sendBatch(); 120 | } 121 | } 122 | 123 | /** 124 | * Returns the parameter name contained in the query, 125 | * e.g. "UNWIND {params} MATCH ..." returns "params" 126 | * 127 | * @return parameter name 128 | */ 129 | private String getParameterName() { 130 | Pattern pattern = Pattern.compile("^[uU][nN][wW][iI][nN][dD] \\{(.+)\\} .*"); 131 | Matcher matcher = pattern.matcher(getCypherQuery()); 132 | if (matcher.matches()) { 133 | return matcher.group(1); 134 | } 135 | throw new IllegalArgumentException("Query does not contain a parameter statement."); 136 | } 137 | 138 | private void initBatch() { 139 | payload = new StringBuilder(); 140 | currentBatchSize = 0; 141 | payload.append(String.format("\"%s\" : [", parameterName)); 142 | } 143 | 144 | private void addToBatch(OUT tuple) throws IOException { 145 | if (elementTypes == null) { 146 | initValueTypes(tuple); 147 | } 148 | 149 | if (!isBatchEmpty()) { 150 | payload.append(","); 151 | } 152 | 153 | ObjectNode node = nodeFactory.objectNode(); 154 | 155 | for (int i = 0; i < elementKeys.length; i++) { 156 | if (elementTypes[i].equals(Boolean.class)) { 157 | node.put(elementKeys[i], (Boolean) tuple.getField(i)); 158 | } else if (elementTypes[i].equals(Integer.class)) { 159 | node.put(elementKeys[i], (Integer) tuple.getField(i)); 160 | } else if (elementTypes[i].equals(Long.class)) { 161 | node.put(elementKeys[i], (Long) tuple.getField(i)); 162 | } else if (elementTypes[i].equals(Float.class)) { 163 | node.put(elementKeys[i], (Float) tuple.getField(i)); 164 | } else if (elementTypes[i].equals(Double.class)) { 165 | node.put(elementKeys[i], (Double) tuple.getField(i)); 166 | } else if (elementTypes[i].equals(String.class)) { 167 | node.put(elementKeys[i], (String) tuple.getField(i)); 168 | } else { 169 | throw new IOException("Unsupported field type for value: " + tuple.getField(i)); 170 | } 171 | } 172 | 173 | payload.append(node.toString()); 174 | 175 | currentBatchSize++; 176 | } 177 | 178 | private void finalizeBatch() { 179 | payload.append("]"); 180 | } 181 | 182 | private void initValueTypes(OUT tuple) throws IOException { 183 | elementTypes = new Class[tuple.getArity()]; 184 | for (int i = 0; i < tuple.getArity(); i++) { 185 | Object field = tuple.getField(i); 186 | if (field instanceof Boolean) { 187 | elementTypes[i] = Boolean.class; 188 | } else if (field instanceof Integer) { 189 | elementTypes[i] = Integer.class; 190 | } else if (field instanceof Long) { 191 | elementTypes[i] = Long.class; 192 | } else if (field instanceof Float) { 193 | elementTypes[i] = Float.class; 194 | } else if (field instanceof Double) { 195 | elementTypes[i] = Double.class; 196 | } else if (field instanceof String) { 197 | elementTypes[i] = String.class; 198 | } else { 199 | throw new IOException("Unsupported field type for value: " + field); 200 | } 201 | } 202 | } 203 | 204 | /** 205 | * Opens a connection to the Neo4j endpoint and sends the transactional batch query. 206 | * 207 | * @throws IOException 208 | */ 209 | private void sendBatch() throws IOException { 210 | Client client = createClient(); 211 | 212 | String requestPayload = String.format(PAYLOAD_TEMPLATE, getCypherQuery(), payload.toString()); 213 | 214 | ClientResponse response = client 215 | .resource(restURI + TRANSACTION_URI) 216 | .accept(MediaType.APPLICATION_JSON) 217 | .header("Content-Type", "application/json;charset=UTF-8") 218 | .entity(requestPayload) 219 | .post(ClientResponse.class); 220 | 221 | if (response.getStatus() != Response.Status.OK.getStatusCode()) { 222 | throw new IOException(String.format("Server returned status [%d]", response.getStatus())); 223 | } 224 | response.close(); 225 | } 226 | 227 | 228 | private boolean isBatchEmpty() { 229 | return currentBatchSize == 0; 230 | } 231 | 232 | private boolean isBatchFull() { 233 | return currentBatchSize == batchSize; 234 | } 235 | 236 | public static Builder buildNeo4jOutputFormat() { 237 | return new Builder(); 238 | } 239 | 240 | /** 241 | * Used to build instances of {@link Neo4jOutputFormat}. 242 | */ 243 | public static class Builder extends Neo4jFormatBase.Builder { 244 | 245 | private List elementKeys = Lists.newArrayList(); 246 | 247 | private int batchSize; 248 | 249 | /** 250 | * Used to tell the output format which parameter keys are used. 251 | * 252 | * Consider the following cypher query: 253 | * 254 | * UNWIND {inserts} AS i CREATE (a:User {name:i.name, born:i.born}) 255 | * 256 | * The parameter keys in that example are "name" and "born". 257 | * 258 | * @param key parameter key used to set a literal 259 | * @return builder 260 | */ 261 | public Builder addParameterKey(String key) { 262 | return addParameterKey(elementKeys.size(), key); 263 | } 264 | 265 | /** 266 | * Used to tell the output format which parameter keys are used. 267 | * 268 | * Consider the following cypher query: 269 | * 270 | * UNWIND {inserts} AS i CREATE (a:User {name:i.name, born:i.born}) 271 | * 272 | * The parameter keys in that example are "name" and "born". 273 | * 274 | * @param position index of the key in the query 275 | * @param key parameter key used to set a literal 276 | * @return builder 277 | */ 278 | public Builder addParameterKey(int position, String key) { 279 | checkArgument(!Strings.isNullOrEmpty(key), "Key must not be null or empty."); 280 | elementKeys.add(position, key); 281 | return getThis(); 282 | } 283 | 284 | /** 285 | * Sets the batch size per Flink task. Output are written in parallel batches where each 286 | * Flink task is responsible for a part of the dataset. Using this parameter, one can set 287 | * how many elements are contained in a single batch sent by a Flink task. 288 | * 289 | * @param batchSize batch size per task 290 | * @return builder 291 | */ 292 | public Builder setTaskBatchSize(int batchSize) { 293 | checkArgument(batchSize >= 0, "Batch size must be greater or equal than zero."); 294 | this.batchSize = batchSize; 295 | return getThis(); 296 | } 297 | 298 | @Override 299 | public Builder getThis() { 300 | return this; 301 | } 302 | 303 | /** 304 | * Creates the output format. 305 | * 306 | * @return output format 307 | */ 308 | public Neo4jOutputFormat finish() { 309 | validate(); 310 | if (elementKeys.size() == 0) { 311 | throw new IllegalArgumentException("No parameter keys were supplied."); 312 | } 313 | return new Neo4jOutputFormat(this); 314 | } 315 | } 316 | } 317 | -------------------------------------------------------------------------------- /src/test/java/org/apache/flink/api/java/io/neo4j/Neo4jFormatTest.java: -------------------------------------------------------------------------------- 1 | package org.apache.flink.api.java.io.neo4j; 2 | 3 | import org.junit.Rule; 4 | import org.neo4j.harness.junit.Neo4jRule; 5 | 6 | public abstract class Neo4jFormatTest { 7 | 8 | /** 9 | * Test database 10 | */ 11 | @Rule 12 | public Neo4jRule neo4j = new Neo4jRule() 13 | .withConfig("dbms.auth.enabled","false") 14 | .withFixture("CREATE" + 15 | "(alice:User { name : 'Alice', born : 1984, height : 1.72, trust : true })," + 16 | "(bob:User { name : 'Bob', born : 1983, height : 1.81, trust : true })," + 17 | "(eve:User { name : 'Eve', born : 1984, height : 1.62, trust : false })," + 18 | "(alice)-[:KNOWS {since : 2001}]->(bob)," + 19 | "(bob)-[:KNOWS {since : 2002}]->(alice)"); 20 | } 21 | -------------------------------------------------------------------------------- /src/test/java/org/apache/flink/api/java/io/neo4j/Neo4jInputTest.java: -------------------------------------------------------------------------------- 1 | package org.apache.flink.api.java.io.neo4j; 2 | 3 | import org.apache.flink.api.common.typeinfo.BasicTypeInfo; 4 | import org.apache.flink.api.java.DataSet; 5 | import org.apache.flink.api.java.ExecutionEnvironment; 6 | import org.apache.flink.api.java.io.LocalCollectionOutputFormat; 7 | import org.apache.flink.api.java.tuple.Tuple4; 8 | import org.apache.flink.api.java.tuple.Tuple5; 9 | import org.apache.flink.api.java.typeutils.TupleTypeInfo; 10 | import org.apache.flink.hadoop.shaded.com.google.common.collect.Lists; 11 | import org.junit.Test; 12 | 13 | import java.util.List; 14 | 15 | import static org.junit.Assert.assertEquals; 16 | 17 | public class Neo4jInputTest extends Neo4jFormatTest { 18 | 19 | @SuppressWarnings("unchecked") 20 | @Test 21 | public void readTest() throws Exception { 22 | ExecutionEnvironment environment = ExecutionEnvironment.getExecutionEnvironment(); 23 | 24 | String restURI = neo4j.httpURI().resolve("/db/data/").toString(); 25 | 26 | String vertexQuery = "MATCH (n:User) RETURN id(n), n.name, n.born, n.height, n.trust"; 27 | 28 | Neo4jInputFormat> vertexInput = 29 | Neo4jInputFormat.buildNeo4jInputFormat() 30 | .setRestURI(restURI) 31 | .setCypherQuery(vertexQuery) 32 | .setConnectTimeout(10000) 33 | .setReadTimeout(10000) 34 | .finish(); 35 | 36 | DataSet> vertexRows = environment 37 | .createInput(vertexInput, 38 | new TupleTypeInfo>( 39 | BasicTypeInfo.INT_TYPE_INFO, // id 40 | BasicTypeInfo.STRING_TYPE_INFO, // name 41 | BasicTypeInfo.INT_TYPE_INFO, // born 42 | BasicTypeInfo.DOUBLE_TYPE_INFO, // height 43 | BasicTypeInfo.BOOLEAN_TYPE_INFO // trust 44 | )); 45 | 46 | String edgeQuery = "MATCH (a:User)-[e]->(b:User) RETURN id(e), id(a), id(b), e.since"; 47 | 48 | Neo4jInputFormat> edgeInput = 49 | Neo4jInputFormat.buildNeo4jInputFormat() 50 | .setRestURI(restURI) 51 | .setCypherQuery(edgeQuery) 52 | .setConnectTimeout(10000) 53 | .setReadTimeout(10000) 54 | .finish(); 55 | 56 | DataSet> edgeRows = environment 57 | .createInput(edgeInput, new TupleTypeInfo>( 58 | BasicTypeInfo.INT_TYPE_INFO, // edge id 59 | BasicTypeInfo.INT_TYPE_INFO, // source id 60 | BasicTypeInfo.INT_TYPE_INFO, // target id 61 | BasicTypeInfo.INT_TYPE_INFO // since 62 | )); 63 | 64 | List> vertexList = Lists.newArrayList(); 65 | List> edgeList = Lists.newArrayList(); 66 | 67 | vertexRows.output(new LocalCollectionOutputFormat<>(vertexList)); 68 | edgeRows.output(new LocalCollectionOutputFormat<>(edgeList)); 69 | 70 | environment.execute(); 71 | 72 | assertEquals("wrong number of vertices", 3, vertexList.size()); 73 | assertEquals("wrong number of edges", 2, edgeList.size()); 74 | 75 | Integer idAlice = 0, idBob = 0; 76 | 77 | for (Tuple5 vertex : vertexList) { 78 | switch (vertex.f1) { 79 | case "Alice": 80 | idAlice = vertex.f0; 81 | validateVertex(vertex, 1984, 1.72, Boolean.TRUE); 82 | break; 83 | case "Bob": 84 | idBob = vertex.f0; 85 | validateVertex(vertex, 1983, 1.81, Boolean.TRUE); 86 | break; 87 | case "Eve": 88 | validateVertex(vertex, 1984, 1.62, Boolean.FALSE); 89 | break; 90 | } 91 | } 92 | 93 | for (Tuple4 edge : edgeList) { 94 | if (edge.f1.equals(idAlice)) { 95 | validateEdge(edge, idBob, 2001); 96 | } else if (edge.f1.equals(idBob)) { 97 | validateEdge(edge, idAlice, 2002); 98 | } 99 | } 100 | } 101 | 102 | private void validateEdge(Tuple4 edge, Integer targetId, 103 | int since) { 104 | assertEquals("wrong target vertex id", targetId, edge.f2); 105 | assertEquals("wrong property value (since)", Integer.valueOf(since), edge.f3); 106 | } 107 | 108 | private void validateVertex(Tuple5 vertex, int born, 109 | double weight, boolean trust) { 110 | assertEquals("wrong property value (since)", Integer.valueOf(born), vertex.f2); 111 | assertEquals("wrong property value (born)", Double.valueOf(weight), vertex.f3); 112 | assertEquals("wrong property value (weight)", trust, vertex.f4); 113 | } 114 | } 115 | -------------------------------------------------------------------------------- /src/test/java/org/apache/flink/api/java/io/neo4j/Neo4jOutputTest.java: -------------------------------------------------------------------------------- 1 | package org.apache.flink.api.java.io.neo4j; 2 | 3 | import org.apache.flink.api.java.ExecutionEnvironment; 4 | import org.apache.flink.api.java.tuple.Tuple4; 5 | import org.junit.Test; 6 | import org.neo4j.graphdb.GraphDatabaseService; 7 | import org.neo4j.graphdb.Result; 8 | import org.neo4j.graphdb.Transaction; 9 | 10 | import java.util.Map; 11 | 12 | import static org.junit.Assert.assertEquals; 13 | import static org.junit.Assert.assertTrue; 14 | 15 | public class Neo4jOutputTest extends Neo4jFormatTest { 16 | 17 | @SuppressWarnings("unchecked") 18 | @Test 19 | public void createTest() throws Exception { 20 | ExecutionEnvironment environment = ExecutionEnvironment.getExecutionEnvironment(); 21 | 22 | String restURI = neo4j.httpURI().resolve("/db/data/").toString(); 23 | 24 | String createQuery = "" + 25 | "UNWIND {inserts} AS i " + 26 | "CREATE (a:User {name:i.name, born:i.born, height:i.height, trust:i.trust})"; 27 | 28 | Neo4jOutputFormat> outputFormat = Neo4jOutputFormat 29 | .buildNeo4jOutputFormat() 30 | .setRestURI(restURI) 31 | .setConnectTimeout(1_000) 32 | .setReadTimeout(1_000) 33 | .setCypherQuery(createQuery) 34 | .addParameterKey(0, "name") 35 | .addParameterKey(1, "born") 36 | .addParameterKey(2, "height") 37 | .addParameterKey(3, "trust") 38 | .finish(); 39 | 40 | environment.fromElements( 41 | new Tuple4<>("Frank", 1982, 1.84d, true), 42 | new Tuple4<>("Dave", 1976, 1.82d, true)) 43 | .output(outputFormat); 44 | 45 | environment.execute(); 46 | 47 | // test it 48 | 49 | GraphDatabaseService graphDB = neo4j.getGraphDatabaseService(); 50 | try (Transaction tx = graphDB.beginTx()) { 51 | Result result = graphDB.execute("" + 52 | "MATCH (n:User) " + 53 | "WHERE n.name = 'Frank' OR n.name = 'Dave' " + 54 | "RETURN n.name AS name, n.born AS born, n.height AS height, n.trust AS trust"); 55 | int rows = 0; 56 | while(result.hasNext()) { 57 | rows++; 58 | Map row = result.next(); 59 | if (row.get("name").equals("Frank")) { 60 | assertEquals("wrong born attribute value", 1982, row.get("born")); 61 | assertEquals("wrong height attribute value", 1.84d, row.get("height")); 62 | assertEquals("wrong trust attribute value", true, row.get("trust")); 63 | } else if (row.get("name").equals("Dave")){ 64 | assertEquals("wrong born attribute value", 1976, row.get("born")); 65 | assertEquals("wrong height attribute value", 1.82d, row.get("height")); 66 | assertEquals("wrong trust attribute value", true, row.get("trust")); 67 | } else { 68 | assertTrue("Unexpected result", false); 69 | } 70 | } 71 | assertEquals("Unexpected row count", 2, rows); 72 | tx.success(); 73 | } 74 | } 75 | 76 | @SuppressWarnings("unchecked") 77 | @Test 78 | public void updateTest() throws Exception { 79 | ExecutionEnvironment environment = ExecutionEnvironment.getExecutionEnvironment(); 80 | 81 | String restURI = neo4j.httpURI().resolve("/db/data/").toString(); 82 | 83 | String updateQuery = "" + 84 | "UNWIND {updates} AS u " + 85 | "MATCH (p) " + 86 | "WHERE p.name = u.name " + 87 | "SET p.weight = u.weight, p.height = u.height, p.trust = u.trust"; 88 | 89 | Neo4jOutputFormat> outputFormat = Neo4jOutputFormat 90 | .buildNeo4jOutputFormat() 91 | .setRestURI(restURI) 92 | .setConnectTimeout(10_000) 93 | .setReadTimeout(10_000) 94 | .setCypherQuery(updateQuery) 95 | .addParameterKey("name") 96 | .addParameterKey("weight") 97 | .addParameterKey("height") 98 | .addParameterKey("trust") 99 | .finish(); 100 | 101 | environment.fromElements( 102 | new Tuple4<>("Alice", 42, 1.74d, false), 103 | new Tuple4<>("Bob", 75, 1.82d, true)) 104 | .output(outputFormat); 105 | 106 | environment.execute(); 107 | 108 | // test it 109 | 110 | GraphDatabaseService graphDB = neo4j.getGraphDatabaseService(); 111 | try (Transaction tx = graphDB.beginTx()) { 112 | Result result = graphDB.execute("" + 113 | "MATCH (n:User) " + 114 | "WHERE n.name = 'Alice' OR n.name = 'Bob' " + 115 | "RETURN n.name AS name, n.weight AS weight, n.height AS height, n.trust AS trust"); 116 | 117 | int rows = 0; 118 | while(result.hasNext()) { 119 | rows++; 120 | Map row = result.next(); 121 | if (row.get("name").equals("Alice")) { 122 | assertEquals("wrong weight attribute value", 42, row.get("weight")); 123 | assertEquals("wrong height attribute value", 1.74d, row.get("height")); 124 | assertEquals("wrong trust attribute value", false, row.get("trust")); 125 | } else if (row.get("name").equals("Bob")) { 126 | assertEquals("wrong weight attribute value", 75, row.get("weight")); 127 | assertEquals("wrong height attribute value", 1.82d, row.get("height")); 128 | assertEquals("wrong trust attribute value", true, row.get("trust")); 129 | } else { 130 | assertTrue("Unexpected result", false); 131 | } 132 | } 133 | assertEquals("Unexpected row count", 2, rows); 134 | 135 | tx.success(); 136 | } 137 | } 138 | } 139 | -------------------------------------------------------------------------------- /src/test/resources/log4j-test.properties: -------------------------------------------------------------------------------- 1 | # Set root logger level to OFF to not flood build logs 2 | # set manually to INFO for debugging purposes 3 | log4j.rootLogger=OFF, testlogger 4 | 5 | # A1 is set to be a ConsoleAppender. 6 | log4j.appender.testlogger=org.apache.log4j.ConsoleAppender 7 | log4j.appender.testlogger.target=System.err 8 | log4j.appender.testlogger.layout=org.apache.log4j.PatternLayout 9 | log4j.appender.testlogger.layout.ConversionPattern=%-4r [%t] %-5p %c %x - %m%n --------------------------------------------------------------------------------