Skip to content

Commit 6f917f9

Browse files
committed
Remove plan time from profiling
1 parent 3ce645d commit 6f917f9

File tree

6 files changed

+7
-44
lines changed

6 files changed

+7
-44
lines changed

x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/operator/PlanTimeProfile.java

Lines changed: 4 additions & 19 deletions
Original file line numberDiff line numberDiff line change
@@ -22,42 +22,35 @@
2222
*
2323
*/
2424
public final class PlanTimeProfile implements Writeable, ToXContentObject {
25-
private long planNanos;
2625
private long logicalOptimizationNanos;
2726
private long physicalOptimizationNanos;
2827

2928
/**
3029
* @param logicalOptimizationNanos Time spent on local logical plan optimization (in nanoseconds)
3130
* @param physicalOptimizationNanos Time spent on local physical plan optimization (in nanoseconds)
3231
*/
33-
public PlanTimeProfile(long planNanos, long logicalOptimizationNanos, long physicalOptimizationNanos) {
34-
this.planNanos = planNanos;
32+
public PlanTimeProfile(long logicalOptimizationNanos, long physicalOptimizationNanos) {
3533
this.logicalOptimizationNanos = logicalOptimizationNanos;
3634
this.physicalOptimizationNanos = physicalOptimizationNanos;
3735
}
3836

3937
public PlanTimeProfile() {
40-
this.planNanos = 0L;
4138
this.logicalOptimizationNanos = 0L;
4239
this.physicalOptimizationNanos = 0L;
4340
}
4441

4542
public PlanTimeProfile(StreamInput in) throws IOException {
46-
this(in.readVLong(), in.readVLong(), in.readVLong());
43+
this(in.readVLong(), in.readVLong());
4744
}
4845

4946
@Override
5047
public void writeTo(StreamOutput out) throws IOException {
51-
out.writeVLong(planNanos);
5248
out.writeVLong(logicalOptimizationNanos);
5349
out.writeVLong(physicalOptimizationNanos);
5450
}
5551

5652
@Override
5753
public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException {
58-
if (planNanos > 0) {
59-
builder.field("plan_nanos", planNanos);
60-
}
6154
if (logicalOptimizationNanos > 0) {
6255
builder.field("logical_optimization_nanos", logicalOptimizationNanos);
6356
}
@@ -67,10 +60,6 @@ public XContentBuilder toXContent(XContentBuilder builder, Params params) throws
6760
return builder;
6861
}
6962

70-
public void addPlanTime(long planNanos) {
71-
this.planNanos = this.planNanos + planNanos;
72-
}
73-
7463
public void addLogicalOptimizationPlanTime(long logicalOptimizationPlanTime) {
7564
this.logicalOptimizationNanos = this.logicalOptimizationNanos + logicalOptimizationPlanTime;
7665
}
@@ -84,22 +73,18 @@ public boolean equals(Object obj) {
8473
if (obj == this) return true;
8574
if (obj == null || obj.getClass() != this.getClass()) return false;
8675
var that = (PlanTimeProfile) obj;
87-
return this.planNanos == that.planNanos
88-
&& this.logicalOptimizationNanos == that.logicalOptimizationNanos
76+
return this.logicalOptimizationNanos == that.logicalOptimizationNanos
8977
&& this.physicalOptimizationNanos == that.physicalOptimizationNanos;
9078
}
9179

9280
@Override
9381
public int hashCode() {
94-
return Objects.hash(planNanos, logicalOptimizationNanos, physicalOptimizationNanos);
82+
return Objects.hash(logicalOptimizationNanos, physicalOptimizationNanos);
9583
}
9684

9785
@Override
9886
public String toString() {
9987
return "PlanTimeProfile["
100-
+ "planNanos="
101-
+ planNanos
102-
+ ", "
10388
+ "logicalOptimizationNanos="
10489
+ logicalOptimizationNanos
10590
+ ", "

x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/planner/PlannerUtils.java

Lines changed: 1 addition & 9 deletions
Original file line numberDiff line numberDiff line change
@@ -224,20 +224,11 @@ public static PhysicalPlan localPlan(
224224
) {
225225
final LocalMapper localMapper = new LocalMapper();
226226
var isCoordPlan = new Holder<>(Boolean.TRUE);
227-
long planStartNanos = 0L;
228-
boolean profilingEnabled = planTimeProfile != null;
229-
if (profilingEnabled) {
230-
planStartNanos = System.nanoTime();
231-
}
232227
Set<PhysicalPlan> lookupJoinExecRightChildren = plan.collect(LookupJoinExec.class::isInstance)
233228
.stream()
234229
.map(x -> ((LookupJoinExec) x).right())
235230
.collect(Collectors.toSet());
236231

237-
if (profilingEnabled) {
238-
planTimeProfile.addPlanTime(System.nanoTime() - planStartNanos);
239-
}
240-
241232
PhysicalPlan localPhysicalPlan = plan.transformUp(FragmentExec.class, f -> {
242233
if (lookupJoinExecRightChildren.contains(f)) {
243234
// Do not optimize the right child of a lookup join exec
@@ -248,6 +239,7 @@ public static PhysicalPlan localPlan(
248239
isCoordPlan.set(Boolean.FALSE);
249240

250241
// Logical optimization
242+
boolean profilingEnabled = planTimeProfile != null;
251243
long logicalStartNanos = profilingEnabled ? System.nanoTime() : 0;
252244
LogicalPlan optimizedFragment = logicalOptimizer.localOptimize(f.fragment());
253245
if (profilingEnabled) {

x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plugin/ComputeService.java

Lines changed: 0 additions & 7 deletions
Original file line numberDiff line numberDiff line change
@@ -327,17 +327,10 @@ public void executePlan(
327327
Supplier<ExchangeSink> exchangeSinkSupplier,
328328
PlanTimeProfile planTimeProfile
329329
) {
330-
long startTime = 0L;
331-
if (configuration.profile()) {
332-
startTime = System.nanoTime();
333-
}
334330
Tuple<PhysicalPlan, PhysicalPlan> coordinatorAndDataNodePlan = PlannerUtils.breakPlanBetweenCoordinatorAndDataNode(
335331
physicalPlan,
336332
configuration
337333
);
338-
if (configuration.profile()) {
339-
planTimeProfile.addPlanTime(System.nanoTime() - startTime);
340-
}
341334
final List<Page> collectedPages = Collections.synchronizedList(new ArrayList<>());
342335
listener = listener.delegateResponse((l, e) -> {
343336
collectedPages.forEach(p -> Releasables.closeExpectNoException(p::releaseBlocks));

x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plugin/DataNodeComputeHandler.java

Lines changed: 0 additions & 7 deletions
Original file line numberDiff line numberDiff line change
@@ -568,13 +568,8 @@ public void messageReceived(DataNodeRequest request, TransportChannel channel, T
568568
Configuration configuration = request.configuration();
569569
// We can avoid synchronization (for the most part) since the array elements are never modified, and the array is only added to,
570570
// with its size being known before we start the computation.
571-
long planTime = 0L;
572571
PlanTimeProfile planTimeProfile = null;
573572
if (request.plan() instanceof ExchangeSinkExec plan) {
574-
long startPlanTime = 0L;
575-
if (configuration.profile()) {
576-
startPlanTime = System.nanoTime();
577-
}
578573
reductionPlan = ComputeService.reductionPlan(
579574
computeService.plannerSettings(),
580575
computeService.createFlags(),
@@ -586,8 +581,6 @@ public void messageReceived(DataNodeRequest request, TransportChannel channel, T
586581
);
587582
if (configuration.profile()) {
588583
planTimeProfile = new PlanTimeProfile();
589-
planTimeProfile.addPlanTime(System.nanoTime() - startPlanTime);
590-
591584
}
592585
} else {
593586
listener.onFailure(new IllegalStateException("expected exchange sink for a remote compute; got " + request.plan()));

x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/action/EsqlQueryResponseProfileTests.java

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -74,7 +74,7 @@ private List<PlanProfile> randomPlanProfiles() {
7474
}
7575

7676
private PlanTimeProfile randomPlanTimeProfile() {
77-
return randomBoolean() ? null : new PlanTimeProfile(randomNonNegativeLong(), randomNonNegativeLong(), randomNonNegativeLong());
77+
return randomBoolean() ? null : new PlanTimeProfile(randomNonNegativeLong(), randomNonNegativeLong());
7878
}
7979

8080
private OperatorStatus randomOperatorStatus() {

x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/plugin/ComputeListenerTests.java

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -95,7 +95,7 @@ private DriverCompletionInfo randomCompletionInfo() {
9595
}
9696

9797
private PlanTimeProfile randomPlanTimeProfile() {
98-
return randomBoolean() ? null : new PlanTimeProfile(randomNonNegativeLong(), randomNonNegativeLong(), randomNonNegativeLong());
98+
return randomBoolean() ? null : new PlanTimeProfile(randomNonNegativeLong(), randomNonNegativeLong());
9999
}
100100

101101
public void testEmpty() {

0 commit comments

Comments
 (0)