Skip to content
Open
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension


Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -32,6 +32,10 @@ public AbstractIncVertexCentricComputeAlgo(long iterations, String name) {
super(iterations, name);
}

public String getPythonTransformClassName() {
return null;
}

public abstract FUNC getIncComputeFunction();

}
Original file line number Diff line number Diff line change
Expand Up @@ -20,8 +20,10 @@
package org.apache.geaflow.operator.impl.graph.compute.dynamic;

import java.io.IOException;
import java.util.HashMap;
import java.util.HashSet;
import java.util.List;
import java.util.Map;
import java.util.Set;
import org.apache.geaflow.api.function.iterator.RichIteratorFunction;
import org.apache.geaflow.api.graph.base.algo.AbstractIncVertexCentricComputeAlgo;
Expand Down Expand Up @@ -164,7 +166,8 @@ class IncGraphInferComputeContextImpl<OUT> extends IncGraphComputeContextImpl im
public IncGraphInferComputeContextImpl() {
if (clientLocal.get() == null) {
try {
inferContext = new InferContext<>(runtimeContext.getConfiguration());
inferContext = new InferContext<>(buildInferConfiguration(runtimeContext.getConfiguration(),
function.getPythonTransformClassName()));
} catch (Exception e) {
throw new GeaflowRuntimeException(e);
}
Expand All @@ -191,4 +194,16 @@ public void close() throws IOException {
}
}
}

static Configuration buildInferConfiguration(Configuration baseConfig, String pythonTransformClassName) {
if (pythonTransformClassName == null || pythonTransformClassName.trim().isEmpty()) {
return baseConfig;
}
Map<String, String> configMap = new HashMap<>(baseConfig.getConfigMap());
configMap.put(FrameworkConfigKeys.INFER_ENV_USER_TRANSFORM_CLASSNAME.getKey(),
pythonTransformClassName);
Configuration configuration = new Configuration(configMap);
configuration.setMasterId(baseConfig.getMasterId());
return configuration;
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -34,6 +34,7 @@
import org.apache.geaflow.collector.ICollector;
import org.apache.geaflow.common.config.Configuration;
import org.apache.geaflow.common.config.keys.ExecutionConfigKeys;
import org.apache.geaflow.common.config.keys.FrameworkConfigKeys;
import org.apache.geaflow.common.task.TaskArgs;
import org.apache.geaflow.common.type.primitive.IntegerType;
import org.apache.geaflow.common.utils.ReflectionUtil;
Expand Down Expand Up @@ -105,6 +106,21 @@ public VertexCentricCombineFunction getCombineFunction() {
Assert.assertEquals(3L, ((RuntimeContext) ReflectionUtil.getField(operator, "runtimeContext")).getWindowId());
}

@Test
public void testBuildInferConfigurationOverride() {
Configuration config = new Configuration();
config.put(FrameworkConfigKeys.INFER_ENV_USER_TRANSFORM_CLASSNAME, "GlobalTransform");
Configuration overridden = DynamicGraphVertexCentricComputeOp.buildInferConfiguration(config,
"AlgoTransform");
Assert.assertEquals(config.getString(FrameworkConfigKeys.INFER_ENV_USER_TRANSFORM_CLASSNAME),
"GlobalTransform");
Assert.assertEquals(overridden.getString(FrameworkConfigKeys.INFER_ENV_USER_TRANSFORM_CLASSNAME),
"AlgoTransform");

Configuration unchanged = DynamicGraphVertexCentricComputeOp.buildInferConfiguration(config, null);
Assert.assertSame(unchanged, config);
}

public class TestRuntimeContext extends AbstractRuntimeContext {

public TestRuntimeContext() {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -151,6 +151,13 @@ public interface AlgorithmRuntimeContext<K, M> {
*/
Configuration getConfig();

/**
* Invoke model inference when runtime infer support is enabled.
*/
default <OUT> OUT infer(Object... modelInputs) {
throw new UnsupportedOperationException("Inference is not enabled. Set INFER_ENV_ENABLE=true to enable inference.");
}

/**
* Sends a termination vote to the coordinator to signal algorithm completion.
* This method allows vertices to vote for algorithm termination when they
Expand All @@ -160,4 +167,4 @@ public interface AlgorithmRuntimeContext<K, M> {
* @param voteValue The vote value (typically 1 for termination vote)
*/
void voteToTerminate(String terminationReason, Object voteValue);
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -39,6 +39,7 @@
import org.apache.geaflow.dsl.udf.graph.ClusterCoefficient;
import org.apache.geaflow.dsl.udf.graph.CommonNeighbors;
import org.apache.geaflow.dsl.udf.graph.ConnectedComponents;
import org.apache.geaflow.dsl.udf.graph.GCN;
import org.apache.geaflow.dsl.udf.graph.IncKHopAlgorithm;
import org.apache.geaflow.dsl.udf.graph.IncMinimumSpanningTree;
import org.apache.geaflow.dsl.udf.graph.IncWeakConnectedComponents;
Expand Down Expand Up @@ -241,6 +242,7 @@ public class BuildInSqlFunctionTable extends ListSqlOperatorTable {
.add(GeaFlowFunction.of(LabelPropagation.class))
.add(GeaFlowFunction.of(ConnectedComponents.class))
.add(GeaFlowFunction.of(Louvain.class))
.add(GeaFlowFunction.of(GCN.class))
.build();

public BuildInSqlFunctionTable(GQLJavaTypeFactory typeFactory) {
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,166 @@
/*
* 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.udf.graph;

import java.lang.reflect.InvocationTargetException;
import java.lang.reflect.Method;
import java.util.ArrayList;
import java.util.Collections;
import java.util.Iterator;
import java.util.List;
import java.util.Optional;
import org.apache.geaflow.common.type.Types;
import org.apache.geaflow.dsl.common.algo.AlgorithmRuntimeContext;
import org.apache.geaflow.dsl.common.algo.AlgorithmUserFunction;
import org.apache.geaflow.dsl.common.algo.IncrementalAlgorithmUserFunction;
import org.apache.geaflow.dsl.common.data.Row;
import org.apache.geaflow.dsl.common.data.RowEdge;
import org.apache.geaflow.dsl.common.data.RowVertex;
import org.apache.geaflow.dsl.common.data.impl.ObjectRow;
import org.apache.geaflow.dsl.common.function.Description;
import org.apache.geaflow.dsl.common.types.ArrayType;
import org.apache.geaflow.dsl.common.types.GraphSchema;
import org.apache.geaflow.dsl.common.types.StructType;
import org.apache.geaflow.dsl.common.types.TableField;
import org.apache.geaflow.dsl.udf.graph.gcn.GCNConfig;
import org.apache.geaflow.dsl.udf.graph.gcn.GCNFeatureCollector;
import org.apache.geaflow.dsl.udf.graph.gcn.GCNInferPayload;
import org.apache.geaflow.dsl.udf.graph.gcn.GCNInferResult;
import org.apache.geaflow.dsl.udf.graph.gcn.GCNResultParser;
import org.apache.geaflow.dsl.udf.graph.gcn.GCNSubgraphBuilder;
import org.apache.geaflow.model.graph.edge.EdgeDirection;

@Description(name = "gcn", description = "built-in udga for GCN inference")
public class GCN implements AlgorithmUserFunction<Object, Object>, IncrementalAlgorithmUserFunction {

private AlgorithmRuntimeContext<Object, Object> context;
private GCNConfig config;
private GCNSubgraphBuilder subgraphBuilder;
private final GCNFeatureCollector featureCollector = new GCNFeatureCollector();
private final GCNResultParser resultParser = new GCNResultParser();

@Override
public void init(AlgorithmRuntimeContext<Object, Object> context, Object[] params) {
this.context = context;
this.config = parseConfig(params);
this.subgraphBuilder = new GCNSubgraphBuilder(config);
}

@Override
public void process(RowVertex vertex, Optional<Row> updatedValues, Iterator<Object> messages) {
if (context.getCurrentIterationId() > 1) {
return;
}
GCNInferPayload payload = subgraphBuilder.build(vertex.getId(), new DynamicGraphAdapter(vertex));
Object rawResult = context.infer(payload);
GCNInferResult result = resultParser.parse(vertex.getId(), rawResult);
context.take(ObjectRow.create(result.toRowValues()));
}

@Override
public void finish(RowVertex graphVertex, Optional<Row> updatedValues) {
}

@Override
public StructType getOutputType(GraphSchema graphSchema) {
return new StructType(
new TableField("id", graphSchema.getIdType(), false),
new TableField("embedding", new ArrayType(Types.DOUBLE), false),
new TableField("predicted_class", Types.LONG, false),
new TableField("confidence", Types.DOUBLE, false)
);
}

private GCNConfig parseConfig(Object[] params) {
if (params.length == 0) {
return new GCNConfig();
}
if (params.length != 2 && params.length != 3) {
throw new IllegalArgumentException("GCN accepts 0, 2 or 3 parameters");
}
int numHops = Integer.parseInt(String.valueOf(params[0]));
int numSamplesPerHop = Integer.parseInt(String.valueOf(params[1]));
String className = params.length == 3 ? String.valueOf(params[2])
: GCNConfig.DEFAULT_PYTHON_TRANSFORM_CLASS;
return new GCNConfig(numHops, numSamplesPerHop, className);
}

private class DynamicGraphAdapter implements GCNSubgraphBuilder.GraphAdapter {

private final GraphSchema graphSchema = context.getGraphSchema();
private final RowVertex rootVertex;

DynamicGraphAdapter(RowVertex rootVertex) {
this.rootVertex = rootVertex;
}

@Override
public List<Object> loadNeighbors(Object nodeId) {
RowVertex current = switchVertex(nodeId);
if (current == null) {
restoreVertex(rootVertex.getId());
return Collections.emptyList();
}
List<RowEdge> edges = context.loadEdges(EdgeDirection.BOTH);
List<Object> neighbors = new ArrayList<>(edges.size());
for (RowEdge edge : edges) {
Object neighborId = nodeId.equals(edge.getSrcId()) ? edge.getTargetId()
: edge.getSrcId();
neighbors.add(neighborId);
}
restoreVertex(rootVertex.getId());
return neighbors;
}

@Override
public double[] loadFeatures(Object nodeId) {
RowVertex vertex = nodeId.equals(rootVertex.getId()) ? rootVertex : switchVertex(nodeId);
try {
return vertex == null ? new double[config.getFeatureDimLimit()]
: featureCollector.collectFromRowVertex(vertex, graphSchema,
config.getFeatureDimLimit());
} finally {
restoreVertex(rootVertex.getId());
}
}

private RowVertex switchVertex(Object nodeId) {
try {
Method setVertexId = context.getClass().getMethod("setVertexId", Object.class);
setVertexId.invoke(context, nodeId);
Method loadVertex = context.getClass().getMethod("loadVertex");
return (RowVertex) loadVertex.invoke(context);
} catch (NoSuchMethodException e) {
throw new IllegalStateException("GCN requires dynamic runtime context with loadVertex support", e);
} catch (IllegalAccessException | InvocationTargetException e) {
throw new IllegalStateException("Failed to switch runtime context vertex", e);
}
}

private void restoreVertex(Object rootId) {
try {
Method setVertexId = context.getClass().getMethod("setVertexId", Object.class);
setVertexId.invoke(context, rootId);
} catch (NoSuchMethodException | IllegalAccessException | InvocationTargetException e) {
throw new IllegalStateException("Failed to restore runtime context vertex", e);
}
}
}
}
Loading
Loading