Skip to content

Commit d22802b

Browse files
authored
fix(interactive): The replay doesn't need to fill the kafka again. (#4296)
1 parent fa07f86 commit d22802b

File tree

27 files changed

+338
-143
lines changed

27 files changed

+338
-143
lines changed

interactive_engine/common/src/main/java/com/alibaba/graphscope/groot/common/config/CoordinatorConfig.java

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -17,7 +17,7 @@
1717

1818
public class CoordinatorConfig {
1919
public static final Config<Long> SNAPSHOT_INCREASE_INTERVAL_MS =
20-
Config.longConfig("snapshot.increase.interval.ms", 1000L);
20+
Config.longConfig("snapshot.increase.interval.ms", 2000L);
2121

2222
public static final Config<Long> OFFSETS_PERSIST_INTERVAL_MS =
2323
Config.longConfig("offsets.persist.interval.ms", 1000L);

interactive_engine/compiler/src/main/java/com/alibaba/graphscope/common/client/ExecutionClient.java

Lines changed: 3 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -21,6 +21,7 @@
2121
import com.alibaba.graphscope.common.client.type.ExecutionResponseListener;
2222
import com.alibaba.graphscope.common.config.Configs;
2323
import com.alibaba.graphscope.common.config.QueryTimeoutConfig;
24+
import com.alibaba.graphscope.gremlin.plugin.QueryLogger;
2425

2526
/**
2627
* client to submit request to remote engine service
@@ -37,7 +38,8 @@ public ExecutionClient(ChannelFetcher<C> channelFetcher) {
3738
public abstract void submit(
3839
ExecutionRequest request,
3940
ExecutionResponseListener listener,
40-
QueryTimeoutConfig timeoutConfig)
41+
QueryTimeoutConfig timeoutConfig,
42+
QueryLogger queryLogger)
4143
throws Exception;
4244

4345
public abstract void close() throws Exception;

interactive_engine/compiler/src/main/java/com/alibaba/graphscope/common/client/HttpExecutionClient.java

Lines changed: 3 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -25,6 +25,7 @@
2525
import com.alibaba.graphscope.gaia.proto.GraphAlgebraPhysical;
2626
import com.alibaba.graphscope.gaia.proto.IrResult;
2727
import com.alibaba.graphscope.gaia.proto.StoredProcedure;
28+
import com.alibaba.graphscope.gremlin.plugin.QueryLogger;
2829
import com.alibaba.graphscope.interactive.client.Session;
2930
import com.alibaba.graphscope.interactive.client.common.Result;
3031
import com.alibaba.graphscope.interactive.client.impl.DefaultSession;
@@ -56,7 +57,8 @@ public HttpExecutionClient(Configs graphConfig, ChannelFetcher<URI> channelFetch
5657
public void submit(
5758
ExecutionRequest request,
5859
ExecutionResponseListener listener,
59-
QueryTimeoutConfig timeoutConfig)
60+
QueryTimeoutConfig timeoutConfig,
61+
QueryLogger queryLogger)
6062
throws Exception {
6163
List<CompletableFuture> responseFutures = Lists.newArrayList();
6264
for (URI httpURI : channelFetcher.fetch()) {

interactive_engine/compiler/src/main/java/com/alibaba/graphscope/common/client/RpcExecutionClient.java

Lines changed: 5 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -23,6 +23,7 @@
2323
import com.alibaba.graphscope.common.config.PegasusConfig;
2424
import com.alibaba.graphscope.common.config.QueryTimeoutConfig;
2525
import com.alibaba.graphscope.gaia.proto.IrResult;
26+
import com.alibaba.graphscope.gremlin.plugin.QueryLogger;
2627
import com.alibaba.pegasus.RpcChannel;
2728
import com.alibaba.pegasus.RpcClient;
2829
import com.alibaba.pegasus.intf.ResultProcessor;
@@ -54,7 +55,8 @@ public RpcExecutionClient(Configs graphConfig, ChannelFetcher<RpcChannel> channe
5455
public void submit(
5556
ExecutionRequest request,
5657
ExecutionResponseListener listener,
57-
QueryTimeoutConfig timeoutConfig)
58+
QueryTimeoutConfig timeoutConfig,
59+
QueryLogger queryLogger)
5860
throws Exception {
5961
if (rpcClientRef.get() == null) {
6062
rpcClientRef.compareAndSet(null, new RpcClient(channelFetcher.fetch()));
@@ -97,12 +99,13 @@ public void process(PegasusClient.JobResponse jobResponse) {
9799
@Override
98100
public void finish() {
99101
listener.onCompleted();
100-
logger.info("[compile]: received results from engine");
102+
queryLogger.info("[compile]: received results from engine");
101103
}
102104

103105
@Override
104106
public void error(Status status) {
105107
listener.onError(status.asException());
108+
queryLogger.error("[compile]: fail to receive results from engine");
106109
}
107110
},
108111
timeoutConfig.getChannelTimeoutMS());

interactive_engine/compiler/src/main/java/com/alibaba/graphscope/common/ir/meta/fetcher/DynamicIrMetaFetcher.java

Lines changed: 23 additions & 31 deletions
Original file line numberDiff line numberDiff line change
@@ -20,7 +20,7 @@
2020

2121
import com.alibaba.graphscope.common.config.Configs;
2222
import com.alibaba.graphscope.common.config.GraphConfig;
23-
import com.alibaba.graphscope.common.ir.meta.GraphId;
23+
import com.alibaba.graphscope.common.config.PlannerConfig;
2424
import com.alibaba.graphscope.common.ir.meta.IrMeta;
2525
import com.alibaba.graphscope.common.ir.meta.IrMetaStats;
2626
import com.alibaba.graphscope.common.ir.meta.IrMetaTracker;
@@ -46,21 +46,27 @@ public class DynamicIrMetaFetcher extends IrMetaFetcher implements AutoCloseable
4646
private volatile IrMetaStats currentState;
4747
// To manage the state changes of statistics resulting from different update operations.
4848
private volatile StatsState statsState;
49-
private volatile Boolean statsEnabled = null;
49+
private final boolean fetchStats;
5050

5151
public DynamicIrMetaFetcher(Configs configs, IrMetaReader dataReader, IrMetaTracker tracker) {
5252
super(dataReader, tracker);
53-
this.scheduler = new ScheduledThreadPoolExecutor(2);
53+
this.scheduler = new ScheduledThreadPoolExecutor(1);
5454
this.scheduler.scheduleAtFixedRate(
5555
() -> syncMeta(),
56-
2000,
56+
0,
5757
GraphConfig.GRAPH_META_SCHEMA_FETCH_INTERVAL_MS.get(configs),
5858
TimeUnit.MILLISECONDS);
59-
this.scheduler.scheduleAtFixedRate(
60-
() -> syncStats(statsEnabled == null ? false : statsEnabled),
61-
2000,
62-
GraphConfig.GRAPH_META_STATISTICS_FETCH_INTERVAL_MS.get(configs),
63-
TimeUnit.MILLISECONDS);
59+
this.fetchStats =
60+
PlannerConfig.GRAPH_PLANNER_IS_ON.get(configs)
61+
&& PlannerConfig.GRAPH_PLANNER_OPT.get(configs).equals("CBO");
62+
if (this.fetchStats) {
63+
logger.info("start to schedule statistics fetch task");
64+
this.scheduler.scheduleAtFixedRate(
65+
() -> syncStats(),
66+
2000,
67+
GraphConfig.GRAPH_META_STATISTICS_FETCH_INTERVAL_MS.get(configs),
68+
TimeUnit.MILLISECONDS);
69+
}
6470
}
6571

6672
@Override
@@ -94,32 +100,18 @@ private synchronized void syncMeta() {
94100
meta.getSchema(),
95101
meta.getStoredProcedures(),
96102
curStats);
97-
boolean statsEnabled = getStatsEnabled(this.currentState.getGraphId());
98-
if (statsEnabled && this.statsState != StatsState.SYNCED
99-
|| (!statsEnabled && this.statsState != StatsState.MOCKED)) {
100-
logger.debug("start to sync stats");
101-
syncStats(statsEnabled);
103+
if (this.fetchStats && this.statsState != StatsState.SYNCED) {
104+
logger.info("start to schedule statistics fetch task");
105+
syncStats();
102106
}
103107
} catch (Throwable e) {
104-
logger.warn("failed to read meta data, error is {}", e);
105-
}
106-
}
107-
108-
private boolean getStatsEnabled(GraphId graphId) {
109-
try {
110-
return this.statsEnabled == null
111-
? this.reader.syncStatsEnabled(graphId)
112-
: this.statsEnabled;
113-
} catch (
114-
Throwable e) { // if errors happen when reading stats enabled, we assume it is false
115-
logger.warn("failed to read stats enabled, error is {}", e);
116-
return false;
108+
logger.warn("failed to read meta data", e);
117109
}
118110
}
119111

120-
private synchronized void syncStats(boolean statsEnabled) {
112+
private synchronized void syncStats() {
121113
try {
122-
if (this.currentState != null && statsEnabled) {
114+
if (this.currentState != null) {
123115
GraphStatistics stats = this.reader.readStats(this.currentState.getGraphId());
124116
logger.debug("statistics from remote: {}", stats);
125117
if (stats != null && stats.getVertexCount() != 0) {
@@ -137,7 +129,7 @@ private synchronized void syncStats(boolean statsEnabled) {
137129
}
138130
}
139131
} catch (Throwable e) {
140-
logger.warn("failed to read graph statistics, error is {}", e);
132+
logger.warn("failed to read graph statistics, error is", e);
141133
} finally {
142134
try {
143135
if (this.currentState != null
@@ -148,7 +140,7 @@ private synchronized void syncStats(boolean statsEnabled) {
148140
this.statsState = StatsState.MOCKED;
149141
}
150142
} catch (Throwable t) {
151-
logger.warn("failed to mock the glogue, error is {}", t);
143+
logger.warn("failed to mock the glogue, error is", t);
152144
}
153145
}
154146
}

interactive_engine/compiler/src/main/java/com/alibaba/graphscope/cypher/executor/GraphQueryExecutor.java

Lines changed: 5 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -185,7 +185,11 @@ public void execute(
185185
jobName,
186186
summary.getLogicalPlan(),
187187
summary.getPhysicalPlan());
188-
client.submit(request, listener, timeoutConfig);
188+
client.submit(
189+
request,
190+
listener,
191+
timeoutConfig,
192+
statusCallback.getQueryLogger());
189193
}
190194
};
191195
}

interactive_engine/compiler/src/main/java/com/alibaba/graphscope/gremlin/integration/processor/IrTestOpProcessor.java

Lines changed: 2 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -187,7 +187,8 @@ public ThrowingConsumer<Context> select(Context ctx) {
187187
summary.getLogicalPlan(),
188188
summary.getPhysicalPlan()),
189189
listener,
190-
timeoutConfig);
190+
timeoutConfig,
191+
statusCallback.getQueryLogger());
191192
}
192193
// request results from remote engine in a blocking way
193194
listener.request();

interactive_engine/compiler/src/main/java/com/alibaba/graphscope/gremlin/plugin/QueryStatusCallback.java

Lines changed: 0 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -114,8 +114,6 @@ private JsonObject buildSimpleLog(boolean isSucceed, long elapsedMillis) {
114114
private void fillLogDetail(JsonObject logJson, String errorMsg) {
115115
try {
116116
if (this.metricsCollector.getElapsedMillis() > this.printThreshold) {
117-
// todo(siyuan): the invocation of the function can cause Exception when serializing
118-
// a gremlin vertex to json format
119117
fillLogDetail(logJson, errorMsg, metricsCollector.getStartMillis());
120118
}
121119
} catch (Throwable t) {

interactive_engine/compiler/src/main/java/com/alibaba/graphscope/gremlin/plugin/processor/LifeCycleSupplier.java

Lines changed: 2 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -118,7 +118,8 @@ public GremlinExecutor.LifeCycle get() {
118118
summary.getLogicalPlan(),
119119
summary.getPhysicalPlan()),
120120
listener,
121-
timeoutConfig);
121+
timeoutConfig,
122+
statusCallback.getQueryLogger());
122123
}
123124
// request results from remote engine in a blocking way
124125
listener.request();

interactive_engine/compiler/src/main/java/com/alibaba/graphscope/gremlin/result/processor/AbstractResultProcessor.java

Lines changed: 2 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -73,9 +73,8 @@ protected AbstractResultProcessor(
7373
msg.optionalArgs(Tokens.ARGS_BATCH_SIZE)
7474
.orElse(settings.resultIterationBatchSize);
7575
this.resultCollectors = new ArrayList<>(this.resultCollectorsBatchSize);
76-
this.responseStreamIterator =
77-
new StreamIterator<>(
78-
FrontendConfig.PER_QUERY_STREAM_BUFFER_MAX_CAPACITY.get(configs));
76+
int capacity = FrontendConfig.PER_QUERY_STREAM_BUFFER_MAX_CAPACITY.get(configs);
77+
this.responseStreamIterator = new StreamIterator<>(capacity);
7978
}
8079

8180
@Override

0 commit comments

Comments
 (0)