This is an automated email from the ASF dual-hosted git repository.

kfaraz pushed a commit to branch 31.0.0
in repository https://gitbox.apache.org/repos/asf/druid.git


The following commit(s) were added to refs/heads/31.0.0 by this push:
     new 06c1a6a31ea [Backport] Dart backports for (#17312) , (#17313) , 
(#17319) (#17322)
06c1a6a31ea is described below

commit 06c1a6a31eaa2339a8507658ea1baf378da48e4a
Author: Karan Kumar <[email protected]>
AuthorDate: Thu Oct 10 16:45:34 2024 +0530

    [Backport] Dart backports for (#17312) , (#17313) , (#17319) (#17322)
    
    * MSQ: Use leaf worker count for stages that have any leaf inputs. (#17312)
    (cherry picked from commit b27712933e07f1ae2977461b63aa26ddde018395)
    
    * MSQ: Call "onQueryComplete" after the query is closed. (#17313)
    (cherry picked from commit 4092f3fe47abb80b2b5bf2fd1e21482b79dc1057)
    
    * Dart: Only use historicals as workers. (#17319)
    (cherry picked from commit 074944e02c7fb29b9f4160626ff8dc6fd225b0a9)
    
    ---------
    Co-authored-by: Gian Merlino <[email protected]>
---
 .../msq/dart/controller/DartControllerContext.java |   5 +-
 .../org/apache/druid/msq/exec/ControllerImpl.java  |  11 +-
 .../org/apache/druid/msq/input/InputSpecs.java     |  20 ++++
 .../apache/druid/msq/querykit/DataSourcePlan.java  |  18 ++-
 .../apache/druid/msq/querykit/QueryKitSpec.java    |  21 ++--
 .../msq/querykit/groupby/GroupByQueryKit.java      |   5 +-
 .../druid/msq/querykit/scan/ScanQueryKit.java      |   5 +-
 .../dart/controller/DartControllerContextTest.java | 124 +++++++++++++++++++++
 .../org/apache/druid/msq/input/InputSpecsTest.java |  84 ++++++++++++++
 9 files changed, 263 insertions(+), 30 deletions(-)

diff --git 
a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/dart/controller/DartControllerContext.java
 
b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/dart/controller/DartControllerContext.java
index 0248e66fd22..52f21518b31 100644
--- 
a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/dart/controller/DartControllerContext.java
+++ 
b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/dart/controller/DartControllerContext.java
@@ -48,6 +48,7 @@ import org.apache.druid.query.Query;
 import org.apache.druid.query.QueryContext;
 import org.apache.druid.server.DruidNode;
 import org.apache.druid.server.coordination.DruidServerMetadata;
+import org.apache.druid.server.coordination.ServerType;
 
 import java.util.ArrayList;
 import java.util.Collections;
@@ -122,7 +123,9 @@ public class DartControllerContext implements 
ControllerContext
     // since the serverView is referenced shortly after the worker list is 
created.
     final List<String> workerIds = new ArrayList<>(servers.size());
     for (final DruidServerMetadata server : servers) {
-      workerIds.add(WorkerId.fromDruidServerMetadata(server, 
queryId).toString());
+      if (server.getType() == ServerType.HISTORICAL) {
+        workerIds.add(WorkerId.fromDruidServerMetadata(server, 
queryId).toString());
+      }
     }
 
     // Shuffle workerIds, so we don't bias towards specific servers when 
running multiple queries concurrently. For any
diff --git 
a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ControllerImpl.java
 
b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ControllerImpl.java
index 7c0a18f3db9..0043aaff294 100644
--- 
a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ControllerImpl.java
+++ 
b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ControllerImpl.java
@@ -323,9 +323,12 @@ public class ControllerImpl implements Controller
   @Override
   public void run(final QueryListener queryListener) throws Exception
   {
+    final MSQTaskReportPayload reportPayload;
     try (final Closer closer = Closer.create()) {
-      runInternal(queryListener, closer);
+      reportPayload = runInternal(queryListener, closer);
     }
+    // Call onQueryComplete after Closer is fully closed, ensuring no 
controller-related processing is ongoing.
+    queryListener.onQueryComplete(reportPayload);
   }
 
   @Override
@@ -348,7 +351,7 @@ public class ControllerImpl implements Controller
     }
   }
 
-  private void runInternal(final QueryListener queryListener, final Closer 
closer)
+  private MSQTaskReportPayload runInternal(final QueryListener queryListener, 
final Closer closer)
   {
     QueryDefinition queryDef = null;
     ControllerQueryKernel queryKernel = null;
@@ -511,7 +514,7 @@ public class ControllerImpl implements Controller
       stagesReport = null;
     }
 
-    final MSQTaskReportPayload taskReportPayload = new MSQTaskReportPayload(
+    return new MSQTaskReportPayload(
         makeStatusReport(
             taskStateForReport,
             errorForReport,
@@ -526,8 +529,6 @@ public class ControllerImpl implements Controller
         countersSnapshot,
         null
     );
-
-    queryListener.onQueryComplete(taskReportPayload);
   }
 
   /**
diff --git 
a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/InputSpecs.java
 
b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/InputSpecs.java
index 250f320118a..dc7589b6c42 100644
--- 
a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/InputSpecs.java
+++ 
b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/InputSpecs.java
@@ -22,6 +22,7 @@ package org.apache.druid.msq.input;
 import it.unimi.dsi.fastutil.ints.IntRBTreeSet;
 import it.unimi.dsi.fastutil.ints.IntSet;
 import org.apache.druid.msq.input.stage.StageInputSpec;
+import org.apache.druid.msq.kernel.StageDefinition;
 
 import java.util.List;
 
@@ -50,4 +51,23 @@ public class InputSpecs
 
     return retVal;
   }
+
+  /**
+   * Returns whether any of the provided input specs are leafs. Leafs are 
anything that is not broadcast and not another
+   * stage. For example, regular tables and external files are leafs.
+   *
+   * @param inputSpecs      list of input specs, corresponds to {@link 
StageDefinition#getInputSpecs()}
+   * @param broadcastInputs positions in "inputSpecs" which are broadcast 
specs, corresponds to
+   *                        {@link StageDefinition#getBroadcastInputNumbers()}
+   */
+  public static boolean hasLeafInputs(final List<InputSpec> inputSpecs, final 
IntSet broadcastInputs)
+  {
+    for (int i = 0; i < inputSpecs.size(); i++) {
+      final InputSpec spec = inputSpecs.get(i);
+      if (!broadcastInputs.contains(i) && !(spec instanceof StageInputSpec)) {
+        return true;
+      }
+    }
+    return false;
+  }
 }
diff --git 
a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/DataSourcePlan.java
 
b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/DataSourcePlan.java
index 21848813e5d..6b8fbcb6f35 100644
--- 
a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/DataSourcePlan.java
+++ 
b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/DataSourcePlan.java
@@ -32,6 +32,7 @@ import org.apache.druid.java.util.common.Intervals;
 import org.apache.druid.java.util.common.UOE;
 import org.apache.druid.java.util.common.logger.Logger;
 import org.apache.druid.msq.input.InputSpec;
+import org.apache.druid.msq.input.InputSpecs;
 import org.apache.druid.msq.input.external.ExternalInputSpec;
 import org.apache.druid.msq.input.inline.InlineInputSpec;
 import org.apache.druid.msq.input.lookup.LookupInputSpec;
@@ -123,7 +124,7 @@ public class DataSourcePlan
   /**
    * Build a plan.
    *
-   * @param queryKitSpec reference for recursive planning
+   * @param queryKitSpec     reference for recursive planning
    * @param queryContext     query context
    * @param dataSource       datasource to plan
    * @param querySegmentSpec intervals for mandatory pruning. Must be {@link 
MultipleIntervalSegmentSpec}. The returned
@@ -134,7 +135,6 @@ public class DataSourcePlan
    * @param minStageNumber   starting stage number for subqueries
    * @param broadcast        whether the plan should broadcast data for this 
datasource
    */
-  @SuppressWarnings("rawtypes")
   public static DataSourcePlan forDataSource(
       final QueryKitSpec queryKitSpec,
       final QueryContext queryContext,
@@ -274,6 +274,20 @@ public class DataSourcePlan
     return broadcastInputs;
   }
 
+  /**
+   * Figure for {@link StageDefinition#getMaxWorkerCount()} that should be 
used when processing.
+   */
+  public int getMaxWorkerCount(final QueryKitSpec queryKitSpec)
+  {
+    if (isSingleWorker()) {
+      return 1;
+    } else if (InputSpecs.hasLeafInputs(inputSpecs, broadcastInputs)) {
+      return queryKitSpec.getMaxLeafWorkerCount();
+    } else {
+      return queryKitSpec.getMaxNonLeafWorkerCount();
+    }
+  }
+
   /**
    * Returns a {@link QueryDefinitionBuilder} that includes any {@link 
StageInputSpec} from {@link #getInputSpecs()}.
    * Absent if this plan does not involve reading from prior stages.
diff --git 
a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/QueryKitSpec.java
 
b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/QueryKitSpec.java
index 7cae4ed7d7b..2d0361a037f 100644
--- 
a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/QueryKitSpec.java
+++ 
b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/QueryKitSpec.java
@@ -19,13 +19,11 @@
 
 package org.apache.druid.msq.querykit;
 
-import org.apache.druid.msq.input.InputSpec;
 import org.apache.druid.msq.input.InputSpecs;
 import org.apache.druid.msq.kernel.QueryDefinition;
+import org.apache.druid.msq.kernel.StageDefinition;
 import org.apache.druid.query.Query;
 
-import java.util.List;
-
 /**
  * Collection of parameters for {@link QueryKit#makeQueryDefinition}.
  */
@@ -42,9 +40,9 @@ public class QueryKitSpec
    *                                  {@link 
org.apache.druid.query.QueryDataSource}. Typically a {@link MultiQueryKit}.
    * @param queryId                   queryId of the resulting {@link 
QueryDefinition}
    * @param maxLeafWorkerCount        maximum number of workers for leaf 
stages: becomes
-   *                                  {@link 
org.apache.druid.msq.kernel.StageDefinition#getMaxWorkerCount()}
+   *                                  {@link 
StageDefinition#getMaxWorkerCount()}
    * @param maxNonLeafWorkerCount     maximum number of workers for non-leaf 
stages: becomes
-   *                                  {@link 
org.apache.druid.msq.kernel.StageDefinition#getMaxWorkerCount()}
+   *                                  {@link 
StageDefinition#getMaxWorkerCount()}
    * @param targetPartitionsPerWorker preferred number of partitions per 
worker for subqueries
    */
   public QueryKitSpec(
@@ -79,20 +77,15 @@ public class QueryKitSpec
   }
 
   /**
-   * Maximum worker count for a stage with the given inputs. Will use {@link 
#maxNonLeafWorkerCount} if there are
-   * any stage inputs, {@link #maxLeafWorkerCount} otherwise.
+   * Maximum number of workers for leaf stages. See {@link 
InputSpecs#hasLeafInputs}.
    */
-  public int getMaxWorkerCount(final List<InputSpec> inputSpecs)
+  public int getMaxLeafWorkerCount()
   {
-    if (InputSpecs.getStageNumbers(inputSpecs).isEmpty()) {
-      return maxLeafWorkerCount;
-    } else {
-      return maxNonLeafWorkerCount;
-    }
+    return maxLeafWorkerCount;
   }
 
   /**
-   * Maximum number of workers for non-leaf stages (where there are some stage 
inputs).
+   * Maximum number of workers for non-leaf stages. See {@link 
InputSpecs#hasLeafInputs}.
    */
   public int getMaxNonLeafWorkerCount()
   {
diff --git 
a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/groupby/GroupByQueryKit.java
 
b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/groupby/GroupByQueryKit.java
index db56bd02f74..42f39fb78e5 100644
--- 
a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/groupby/GroupByQueryKit.java
+++ 
b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/groupby/GroupByQueryKit.java
@@ -163,10 +163,7 @@ public class GroupByQueryKit implements 
QueryKit<GroupByQuery>
                        .broadcastInputs(dataSourcePlan.getBroadcastInputs())
                        .signature(intermediateSignature)
                        
.shuffleSpec(shuffleSpecFactoryPreAggregation.build(intermediateClusterBy, 
true))
-                       .maxWorkerCount(
-                           dataSourcePlan.isSingleWorker()
-                           ? 1
-                           : 
queryKitSpec.getMaxWorkerCount(dataSourcePlan.getInputSpecs()))
+                       
.maxWorkerCount(dataSourcePlan.getMaxWorkerCount(queryKitSpec))
                        .processorFactory(new 
GroupByPreShuffleFrameProcessorFactory(queryToRun))
     );
 
diff --git 
a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/scan/ScanQueryKit.java
 
b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/scan/ScanQueryKit.java
index 8d23e289bb6..d76eeb96c37 100644
--- 
a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/scan/ScanQueryKit.java
+++ 
b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/scan/ScanQueryKit.java
@@ -173,10 +173,7 @@ public class ScanQueryKit implements QueryKit<ScanQuery>
                        .broadcastInputs(dataSourcePlan.getBroadcastInputs())
                        .shuffleSpec(scanShuffleSpec)
                        .signature(signatureToUse)
-                       .maxWorkerCount(
-                           dataSourcePlan.isSingleWorker()
-                           ? 1
-                           : 
queryKitSpec.getMaxWorkerCount(dataSourcePlan.getInputSpecs()))
+                       
.maxWorkerCount(dataSourcePlan.getMaxWorkerCount(queryKitSpec))
                        .processorFactory(new 
ScanQueryFrameProcessorFactory(queryToRun))
     );
 
diff --git 
a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/dart/controller/DartControllerContextTest.java
 
b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/dart/controller/DartControllerContextTest.java
new file mode 100644
index 00000000000..0bf61054f32
--- /dev/null
+++ 
b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/dart/controller/DartControllerContextTest.java
@@ -0,0 +1,124 @@
+/*
+ * 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.druid.msq.dart.controller;
+
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableMap;
+import org.apache.druid.client.BrokerServerView;
+import org.apache.druid.msq.dart.worker.WorkerId;
+import org.apache.druid.msq.exec.MemoryIntrospector;
+import org.apache.druid.msq.exec.MemoryIntrospectorImpl;
+import org.apache.druid.msq.indexing.MSQSpec;
+import org.apache.druid.msq.indexing.destination.TaskReportMSQDestination;
+import org.apache.druid.msq.kernel.controller.ControllerQueryKernelConfig;
+import org.apache.druid.msq.util.MultiStageQueryContext;
+import org.apache.druid.query.Query;
+import org.apache.druid.query.QueryContext;
+import org.apache.druid.server.DruidNode;
+import org.apache.druid.server.coordination.DruidServerMetadata;
+import org.apache.druid.server.coordination.ServerType;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.Assertions;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+import org.mockito.Mock;
+import org.mockito.Mockito;
+import org.mockito.MockitoAnnotations;
+
+import java.util.List;
+import java.util.stream.Collectors;
+
+public class DartControllerContextTest
+{
+  private static final List<DruidServerMetadata> SERVERS = ImmutableList.of(
+      new DruidServerMetadata("no", "localhost:1001", null, 1, 
ServerType.HISTORICAL, "__default", 2), // plaintext
+      new DruidServerMetadata("no", null, "localhost:1002", 1, 
ServerType.HISTORICAL, "__default", 1), // TLS
+      new DruidServerMetadata("no", "localhost:1003", null, 1, 
ServerType.REALTIME, "__default", 0)
+  );
+  private static final DruidNode SELF_NODE = new DruidNode("none", 
"localhost", false, 8080, -1, true, false);
+  private static final String QUERY_ID = "abc";
+
+  /**
+   * Context returned by {@link #query}. Overrides "maxConcurrentStages".
+   */
+  private QueryContext queryContext =
+      
QueryContext.of(ImmutableMap.of(MultiStageQueryContext.CTX_MAX_CONCURRENT_STAGES,
 3));
+  private MemoryIntrospector memoryIntrospector;
+  private AutoCloseable mockCloser;
+
+  /**
+   * Server view that returns {@link #SERVERS}.
+   */
+  @Mock
+  private BrokerServerView serverView;
+
+  /**
+   * Query spec that exists mainly to test {@link 
DartControllerContext#queryKernelConfig}.
+   */
+  @Mock
+  private MSQSpec querySpec;
+
+  /**
+   * Query returned by {@link #querySpec}.
+   */
+  @Mock
+  private Query query;
+
+  @BeforeEach
+  public void setUp()
+  {
+    mockCloser = MockitoAnnotations.openMocks(this);
+    memoryIntrospector = new MemoryIntrospectorImpl(100_000_000, 0.75, 1, 1, 
null);
+    Mockito.when(serverView.getDruidServerMetadatas()).thenReturn(SERVERS);
+    Mockito.when(querySpec.getQuery()).thenReturn(query);
+    
Mockito.when(querySpec.getDestination()).thenReturn(TaskReportMSQDestination.instance());
+    Mockito.when(query.context()).thenReturn(queryContext);
+  }
+
+  @AfterEach
+  public void tearDown() throws Exception
+  {
+    mockCloser.close();
+  }
+
+  @Test
+  public void test_queryKernelConfig()
+  {
+    final DartControllerContext controllerContext =
+        new DartControllerContext(null, null, SELF_NODE, null, 
memoryIntrospector, serverView, null);
+    final ControllerQueryKernelConfig queryKernelConfig = 
controllerContext.queryKernelConfig(QUERY_ID, querySpec);
+
+    Assertions.assertFalse(queryKernelConfig.isFaultTolerant());
+    Assertions.assertFalse(queryKernelConfig.isDurableStorage());
+    Assertions.assertEquals(3, queryKernelConfig.getMaxConcurrentStages());
+    Assertions.assertEquals(TaskReportMSQDestination.instance(), 
queryKernelConfig.getDestination());
+    Assertions.assertTrue(queryKernelConfig.isPipeline());
+
+    // Check workerIds after sorting, because they've been shuffled.
+    Assertions.assertEquals(
+        ImmutableList.of(
+            // Only the HISTORICAL servers
+            WorkerId.fromDruidServerMetadata(SERVERS.get(0), 
QUERY_ID).toString(),
+            WorkerId.fromDruidServerMetadata(SERVERS.get(1), 
QUERY_ID).toString()
+        ),
+        
queryKernelConfig.getWorkerIds().stream().sorted().collect(Collectors.toList())
+    );
+  }
+}
diff --git 
a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/input/InputSpecsTest.java
 
b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/input/InputSpecsTest.java
index 94fb76f8064..221e315c7d6 100644
--- 
a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/input/InputSpecsTest.java
+++ 
b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/input/InputSpecsTest.java
@@ -21,7 +21,10 @@ package org.apache.druid.msq.input;
 
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.ImmutableSet;
+import it.unimi.dsi.fastutil.ints.IntSet;
+import it.unimi.dsi.fastutil.ints.IntSets;
 import org.apache.druid.msq.input.stage.StageInputSpec;
+import org.apache.druid.msq.input.table.TableInputSpec;
 import org.junit.Assert;
 import org.junit.Test;
 
@@ -40,4 +43,85 @@ public class InputSpecsTest
         )
     );
   }
+
+  @Test
+  public void test_getHasLeafInputs_allStages()
+  {
+    Assert.assertFalse(
+        InputSpecs.hasLeafInputs(
+            ImmutableList.of(
+                new StageInputSpec(1),
+                new StageInputSpec(2)
+            ),
+            IntSets.emptySet()
+        )
+    );
+  }
+
+  @Test
+  public void test_getHasLeafInputs_broadcastTable()
+  {
+    Assert.assertFalse(
+        InputSpecs.hasLeafInputs(
+            ImmutableList.of(new TableInputSpec("tbl", null, null, null)),
+            IntSet.of(0)
+        )
+    );
+  }
+
+  @Test
+  public void test_getHasLeafInputs_oneTableOneStage()
+  {
+    Assert.assertTrue(
+        InputSpecs.hasLeafInputs(
+            ImmutableList.of(
+                new TableInputSpec("tbl", null, null, null),
+                new StageInputSpec(0)
+            ),
+            IntSets.emptySet()
+        )
+    );
+  }
+
+  @Test
+  public void test_getHasLeafInputs_oneTableOneBroadcastStage()
+  {
+    Assert.assertTrue(
+        InputSpecs.hasLeafInputs(
+            ImmutableList.of(
+                new TableInputSpec("tbl", null, null, null),
+                new StageInputSpec(0)
+            ),
+            IntSet.of(1)
+        )
+    );
+  }
+
+  @Test
+  public void test_getHasLeafInputs_oneBroadcastTableOneStage()
+  {
+    Assert.assertFalse(
+        InputSpecs.hasLeafInputs(
+            ImmutableList.of(
+                new TableInputSpec("tbl", null, null, null),
+                new StageInputSpec(0)
+            ),
+            IntSet.of(0)
+        )
+    );
+  }
+
+  @Test
+  public void test_getHasLeafInputs_oneTableOneBroadcastTable()
+  {
+    Assert.assertTrue(
+        InputSpecs.hasLeafInputs(
+            ImmutableList.of(
+                new TableInputSpec("tbl", null, null, null),
+                new TableInputSpec("tbl2", null, null, null)
+            ),
+            IntSet.of(1)
+        )
+    );
+  }
 }


---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]

Reply via email to