You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@pinot.apache.org by xi...@apache.org on 2020/05/12 10:53:27 UTC

[incubator-pinot] branch template_ingestion_spec updated (48033d1 -> 42a1b6f)

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

xiangfu pushed a change to branch template_ingestion_spec
in repository https://gitbox.apache.org/repos/asf/incubator-pinot.git.


    omit 48033d1  Adding template support for Pinot Ingestion Job Spec
     add 1beaab5  upgrade superset to 0.36.0 (#5239)
     add 5703560  Adding Pinot Presto Helm (#5358)
     add c0cdb72  Adding template support for Pinot Ingestion Job Spec (#5341)
     add f686218  Introduced StreamPartitionMsgOffset class (#5360)
     add d54afba  Remove some timeFieldSpec constructors (#5355)
     add 1b28e5f  Initial implementation for support Theta Sketches (WIP). (#5316)
     add 0ec6463  Revert "Adding template support for Pinot Ingestion Job Spec (#5341)" (#5366)
     add 29ac151  [TE] Mock event generator for demo purposes (#5354)
     new 42a1b6f  Re-work on adding template support for Pinot Ingestion Job Spec

This update added new revisions after undoing existing revisions.
That is to say, some revisions that were in the old version of the
branch are not in the new version.  This situation occurs
when a user --force pushes a change and generates a repository
containing something like this:

 * -- * -- B -- O -- O -- O   (48033d1)
            \
             N -- N -- N   refs/heads/template_ingestion_spec (42a1b6f)

You should already have received notification emails for all of the O
revisions, and so the following emails describe only the N revisions
from the common base, B.

Any revisions marked "omit" are not gone; other references still
refer to them.  Any revisions marked "discard" are gone forever.

The 1 revisions listed above as "new" are entirely new to this
repository and will be described in separate emails.  The revisions
listed as "add" were already present in the repository and have only
been added to this reference.


Summary of changes:
 docker/images/pinot-superset/.dockerignore         |    1 +
 docker/images/pinot-superset/.python-version       |   20 -
 docker/images/pinot-superset/Dockerfile            |  178 +--
 docker/images/pinot-superset/Makefile              |   64 -
 docker/images/pinot-superset/README.md             |   22 +-
 .../examples/pinot_example_dashboard.json          | 1257 ++++++++------------
 .../examples/pinot_example_datasource.yaml         |  193 +++
 .../pinot_example_datasource_quickstart.yaml       |  193 +++
 docker/images/pinot-superset/requirements-db.txt   |   18 +-
 kubernetes/helm/index.yaml                         |   42 +-
 kubernetes/helm/pinot-0.2.0.tgz                    |  Bin 102559 -> 23445 bytes
 kubernetes/helm/{ => pinot}/Chart.yaml             |    0
 kubernetes/helm/{ => pinot}/README.md              |  173 +--
 .../helm/{ => pinot}/charts/zookeeper-2.1.3.tgz    |  Bin
 .../helm/{ => pinot}/pinot-github-events-setup.yml |    0
 .../helm/{ => pinot}/pinot-realtime-quickstart.yml |    0
 kubernetes/helm/{ => pinot}/query-pinot-data.sh    |    0
 kubernetes/helm/{ => pinot}/requirements.lock      |    0
 kubernetes/helm/{ => pinot}/requirements.yaml      |    0
 kubernetes/helm/{ => pinot}/templates/_helpers.tpl |    0
 .../{ => pinot}/templates/broker/configmap.yaml    |    0
 .../templates/broker/service-external.yaml         |    0
 .../templates/broker/service-headless.yaml         |    0
 .../helm/{ => pinot}/templates/broker/service.yaml |    0
 .../{ => pinot}/templates/broker/statefulset.yml   |    0
 .../templates/controller/configmap.yaml            |    0
 .../templates/controller/service-external.yaml     |    0
 .../templates/controller/service-headless.yaml     |    0
 .../{ => pinot}/templates/controller/service.yaml  |    0
 .../templates/controller/statefulset.yaml          |    0
 .../{ => pinot}/templates/server/configmap.yaml    |    0
 .../templates/server/service-headless.yaml         |    0
 .../helm/{ => pinot}/templates/server/service.yaml |    0
 .../{ => pinot}/templates/server/statefulset.yml   |    0
 kubernetes/helm/{ => pinot}/values.yaml            |    0
 kubernetes/helm/presto-0.2.0.tgz                   |  Bin 0 -> 8299 bytes
 kubernetes/helm/presto-cli.sh                      |   28 -
 kubernetes/helm/{ => presto}/Chart.yaml            |   11 +-
 kubernetes/helm/presto/README.md                   |  232 ++++
 kubernetes/helm/{ => presto}/launch-presto-ui.sh   |    0
 kubernetes/helm/{ => presto}/pinot-presto-cli.sh   |    0
 kubernetes/helm/presto/presto-cli.sh               |    8 +
 .../helm/{ => presto}/presto-coordinator.yaml      |    0
 kubernetes/helm/{ => presto}/presto-worker.yaml    |    0
 kubernetes/helm/presto/templates/_helpers.tpl      |   91 ++
 .../presto/templates/coordinator/configmap.yaml    |   60 +
 .../templates/coordinator}/service-external.yaml   |   20 +-
 .../templates/coordinator}/service-headless.yaml   |   14 +-
 .../templates/coordinator}/service.yaml            |   14 +-
 .../presto/templates/coordinator/statefulset.yml   |  121 ++
 .../templates/worker}/configmap.yaml               |   30 +-
 .../templates/worker}/service-headless.yaml        |   14 +-
 .../templates/worker}/service.yaml                 |   14 +-
 .../helm/presto/templates/worker/statefulset.yml   |  121 ++
 kubernetes/helm/presto/values.yaml                 |  160 +++
 kubernetes/helm/superset.yaml                      |   12 +-
 .../common/function/AggregationFunctionType.java   |    2 +
 .../apache/pinot/parsers/utils/ParserUtils.java    |  214 ++++
 .../parsers/PinotQuery2BrokerRequestConverter.java |  133 +--
 .../pinot/pql/parsers/pql2/ast/FilterKind.java     |   12 +-
 .../apache/pinot/sql/parsers/CalciteSqlParser.java |   34 +-
 .../apache/pinot/common/data/FieldSpecTest.java    |   76 +-
 .../org/apache/pinot/common/data/SchemaTest.java   |   10 +-
 pinot-core/pom.xml                                 |    4 +
 .../apache/pinot/core/common/ObjectSerDeUtils.java |   29 +-
 .../org/apache/pinot/core/common/Predicate.java    |    5 +
 .../manager/realtime/DefaultSegmentCommitter.java  |    2 +-
 .../realtime/LLRealtimeSegmentDataManager.java     |   70 +-
 .../realtime/SegmentBuildTimeLeaseExtender.java    |    5 +-
 .../data/manager/realtime/SegmentCommitter.java    |    4 +-
 .../manager/realtime/SplitSegmentCommitter.java    |    2 +-
 .../aggregation/DefaultAggregationExecutor.java    |   52 +-
 .../core/query/aggregation/ThetaSketchParams.java  |   66 +
 .../function/AggregationFunctionFactory.java       |   10 +-
 .../function/AggregationFunctionVisitorBase.java   |    3 +
 ...istinctCountThetaSketchAggregationFunction.java |  637 ++++++++++
 .../groupby/DefaultGroupByExecutor.java            |   20 +-
 .../segment/index/metadata/ColumnMetadata.java     |    3 +-
 .../executor/StarTreeAggregationExecutor.java      |   32 +-
 .../startree/executor/StarTreeGroupByExecutor.java |   31 +-
 .../startree/v2/AggregationFunctionColumnPair.java |    7 +-
 .../realtime/LLRealtimeSegmentDataManagerTest.java |   27 +-
 .../MultiplePinotSegmentRecordReaderTest.java      |   18 +-
 .../data/readers/PinotSegmentRecordReaderTest.java |   16 +-
 .../ExpressionTransformerTest.java                 |    7 +-
 .../minion/MergeRollupSegmentConverterTest.java    |   12 +-
 .../pinot/core/minion/SegmentConverterTest.java    |    9 +-
 .../function/BaseTransformFunctionTest.java        |   18 +-
 .../function/DateTruncTransformFunctionTest.java   |    5 +-
 .../SegmentGenerationWithTimeColumnTest.java       |   17 +-
 .../apache/pinot/core/util/SchemaUtilsTest.java    |   60 +-
 .../queries/DistinctCountThetaSketchTest.java      |  353 ++++++
 .../converter/RealtimeSegmentConverterTest.java    |   11 +-
 .../segments/v1/creator/SegmentTestUtils.java      |    1 -
 .../DefaultCommitterRealtimeIntegrationTest.java   |    6 +-
 .../pinot/plugin/inputformat/avro/AvroUtils.java   |    3 +-
 pinot-spi/pom.xml                                  |    4 -
 .../org/apache/pinot/spi/data/TimeFieldSpec.java   |   89 --
 .../spi/ingestion/batch/IngestionJobLauncher.java  |    8 +-
 .../pinot/spi/stream/StreamPartitionMsgOffset.java |   68 ++
 ...TemplateUtils.java => GroovyTemplateUtils.java} |   29 +-
 .../ingestion/batch/IngestionJobLauncherTest.java  |    8 +-
 ...UtilsTest.java => GroovyTemplateUtilsTest.java} |   35 +-
 .../test/resources/ingestionJobSpecTemplate.yaml   |    4 +-
 .../pinot/tools/data/generator/DataGenerator.java  |    2 +-
 .../pinot/tools/streams/AirlineDataStream.java     |    2 +-
 pom.xml                                            |   11 +-
 thirdeye/thirdeye-pinot/config/detector.yml        |   42 +
 thirdeye/thirdeye-pinot/config/rca.yml             |   26 +-
 .../anomaly/MockEventsLoaderConfiguration.java     |  134 +++
 .../anomaly/ThirdEyeAnomalyApplication.java        |    6 +
 .../anomaly/ThirdEyeAnomalyConfiguration.java      |   18 +
 .../thirdeye/anomaly/events/MockEventsLoader.java  |  200 ++++
 .../thirdeye/datalayer/entity/EventIndex.java      |    8 +-
 .../datasource/mock/MockThirdEyeDataSource.java    |    4 +-
 .../rootcause/impl/ThirdEyeEventEntity.java        |    2 +-
 .../rootcause/impl/ThirdEyeEventsPipeline.java     |   21 +-
 117 files changed, 4037 insertions(+), 1821 deletions(-)
 delete mode 100644 docker/images/pinot-superset/.python-version
 delete mode 100644 docker/images/pinot-superset/Makefile
 copy kubernetes/helm/superset.yaml => docker/images/pinot-superset/examples/pinot_example_dashboard.json (62%)
 create mode 100644 docker/images/pinot-superset/examples/pinot_example_datasource.yaml
 create mode 100644 docker/images/pinot-superset/examples/pinot_example_datasource_quickstart.yaml
 copy kubernetes/helm/{ => pinot}/Chart.yaml (100%)
 copy kubernetes/helm/{ => pinot}/README.md (89%)
 rename kubernetes/helm/{ => pinot}/charts/zookeeper-2.1.3.tgz (100%)
 rename kubernetes/helm/{ => pinot}/pinot-github-events-setup.yml (100%)
 rename kubernetes/helm/{ => pinot}/pinot-realtime-quickstart.yml (100%)
 rename kubernetes/helm/{ => pinot}/query-pinot-data.sh (100%)
 rename kubernetes/helm/{ => pinot}/requirements.lock (100%)
 rename kubernetes/helm/{ => pinot}/requirements.yaml (100%)
 rename kubernetes/helm/{ => pinot}/templates/_helpers.tpl (100%)
 rename kubernetes/helm/{ => pinot}/templates/broker/configmap.yaml (100%)
 copy kubernetes/helm/{ => pinot}/templates/broker/service-external.yaml (100%)
 copy kubernetes/helm/{ => pinot}/templates/broker/service-headless.yaml (100%)
 copy kubernetes/helm/{ => pinot}/templates/broker/service.yaml (100%)
 rename kubernetes/helm/{ => pinot}/templates/broker/statefulset.yml (100%)
 copy kubernetes/helm/{ => pinot}/templates/controller/configmap.yaml (100%)
 rename kubernetes/helm/{ => pinot}/templates/controller/service-external.yaml (100%)
 rename kubernetes/helm/{ => pinot}/templates/controller/service-headless.yaml (100%)
 rename kubernetes/helm/{ => pinot}/templates/controller/service.yaml (100%)
 rename kubernetes/helm/{ => pinot}/templates/controller/statefulset.yaml (100%)
 rename kubernetes/helm/{ => pinot}/templates/server/configmap.yaml (100%)
 copy kubernetes/helm/{ => pinot}/templates/server/service-headless.yaml (100%)
 copy kubernetes/helm/{ => pinot}/templates/server/service.yaml (100%)
 rename kubernetes/helm/{ => pinot}/templates/server/statefulset.yml (100%)
 rename kubernetes/helm/{ => pinot}/values.yaml (100%)
 create mode 100644 kubernetes/helm/presto-0.2.0.tgz
 delete mode 100644 kubernetes/helm/presto-cli.sh
 rename kubernetes/helm/{ => presto}/Chart.yaml (69%)
 create mode 100644 kubernetes/helm/presto/README.md
 rename kubernetes/helm/{ => presto}/launch-presto-ui.sh (100%)
 rename kubernetes/helm/{ => presto}/pinot-presto-cli.sh (100%)
 create mode 100755 kubernetes/helm/presto/presto-cli.sh
 rename kubernetes/helm/{ => presto}/presto-coordinator.yaml (100%)
 rename kubernetes/helm/{ => presto}/presto-worker.yaml (100%)
 create mode 100644 kubernetes/helm/presto/templates/_helpers.tpl
 create mode 100644 kubernetes/helm/presto/templates/coordinator/configmap.yaml
 rename kubernetes/helm/{templates/broker => presto/templates/coordinator}/service-external.yaml (68%)
 rename kubernetes/helm/{templates/broker => presto/templates/coordinator}/service-headless.yaml (76%)
 rename kubernetes/helm/{templates/server => presto/templates/coordinator}/service.yaml (76%)
 create mode 100644 kubernetes/helm/presto/templates/coordinator/statefulset.yml
 rename kubernetes/helm/{templates/controller => presto/templates/worker}/configmap.yaml (53%)
 rename kubernetes/helm/{templates/server => presto/templates/worker}/service-headless.yaml (77%)
 rename kubernetes/helm/{templates/broker => presto/templates/worker}/service.yaml (77%)
 create mode 100644 kubernetes/helm/presto/templates/worker/statefulset.yml
 create mode 100644 kubernetes/helm/presto/values.yaml
 create mode 100644 pinot-common/src/main/java/org/apache/pinot/parsers/utils/ParserUtils.java
 create mode 100644 pinot-core/src/main/java/org/apache/pinot/core/query/aggregation/ThetaSketchParams.java
 create mode 100644 pinot-core/src/main/java/org/apache/pinot/core/query/aggregation/function/DistinctCountThetaSketchAggregationFunction.java
 create mode 100644 pinot-core/src/test/java/org/apache/pinot/queries/DistinctCountThetaSketchTest.java
 create mode 100644 pinot-spi/src/main/java/org/apache/pinot/spi/stream/StreamPartitionMsgOffset.java
 rename pinot-spi/src/main/java/org/apache/pinot/spi/utils/{JinjaTemplateUtils.java => GroovyTemplateUtils.java} (68%)
 rename pinot-spi/src/test/java/org/apache/pinot/spi/utils/{JinjaTemplateUtilsTest.java => GroovyTemplateUtilsTest.java} (62%)
 create mode 100644 thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/anomaly/MockEventsLoaderConfiguration.java
 create mode 100644 thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/anomaly/events/MockEventsLoader.java


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


[incubator-pinot] 01/01: Re-work on adding template support for Pinot Ingestion Job Spec

Posted by xi...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

xiangfu pushed a commit to branch template_ingestion_spec
in repository https://gitbox.apache.org/repos/asf/incubator-pinot.git

commit 42a1b6f575f7b1749cee27947ef5bdf631145e3d
Author: Xiang Fu <fx...@gmail.com>
AuthorDate: Tue May 12 02:52:25 2020 -0700

    Re-work on adding template support for Pinot Ingestion Job Spec
---
 .../spi/ingestion/batch/IngestionJobLauncher.java  | 29 ++++++--
 .../pinot/spi/utils/GroovyTemplateUtils.java       | 78 +++++++++++++++++++
 .../ingestion/batch/IngestionJobLauncherTest.java  | 42 +++++++++++
 .../pinot/spi/utils/GroovyTemplateUtilsTest.java   | 87 ++++++++++++++++++++++
 .../test/resources/ingestionJobSpecTemplate.yaml   | 45 +++++++++++
 .../command/LaunchDataIngestionJobCommand.java     | 16 +++-
 6 files changed, 287 insertions(+), 10 deletions(-)

diff --git a/pinot-spi/src/main/java/org/apache/pinot/spi/ingestion/batch/IngestionJobLauncher.java b/pinot-spi/src/main/java/org/apache/pinot/spi/ingestion/batch/IngestionJobLauncher.java
index 9bb740a..03cef2f 100644
--- a/pinot-spi/src/main/java/org/apache/pinot/spi/ingestion/batch/IngestionJobLauncher.java
+++ b/pinot-spi/src/main/java/org/apache/pinot/spi/ingestion/batch/IngestionJobLauncher.java
@@ -20,13 +20,18 @@ package org.apache.pinot.spi.ingestion.batch;
 
 import java.io.BufferedReader;
 import java.io.FileReader;
-import java.io.Reader;
+import java.io.IOException;
 import java.io.StringWriter;
+import java.util.ArrayList;
 import java.util.Arrays;
+import java.util.List;
+import java.util.Map;
+import org.apache.commons.io.IOUtils;
 import org.apache.pinot.spi.ingestion.batch.runner.IngestionJobRunner;
 import org.apache.pinot.spi.ingestion.batch.spec.ExecutionFrameworkSpec;
 import org.apache.pinot.spi.ingestion.batch.spec.SegmentGenerationJobSpec;
 import org.apache.pinot.spi.plugin.PluginManager;
+import org.apache.pinot.spi.utils.GroovyTemplateUtils;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.yaml.snakeyaml.Yaml;
@@ -36,7 +41,7 @@ public class IngestionJobLauncher {
 
   public static final Logger LOGGER = LoggerFactory.getLogger(IngestionJobLauncher.class);
 
-  private static final String USAGE = "usage: [jobSpec.yaml]";
+  private static final String USAGE = "usage: [jobSpec.yaml] [template_key=template_value]...";
 
   private static void usage() {
     System.err.println(USAGE);
@@ -44,16 +49,26 @@ public class IngestionJobLauncher {
 
   public static void main(String[] args)
       throws Exception {
-    if (args.length != 1) {
+    if (args.length < 1) {
       usage();
       System.exit(1);
     }
     String jobSpecFilePath = args[0];
-
-    try (Reader reader = new BufferedReader(new FileReader(jobSpecFilePath))) {
-      SegmentGenerationJobSpec spec = new Yaml().loadAs(reader, SegmentGenerationJobSpec.class);
-      runIngestionJob(spec);
+    List<String> valueList = new ArrayList<>();
+    for (int i = 1; i < args.length; i++) {
+      valueList.add(args[i]);
     }
+    SegmentGenerationJobSpec spec =
+        getSegmentGenerationJobSpec(jobSpecFilePath, GroovyTemplateUtils.getTemplateContext(valueList));
+    runIngestionJob(spec);
+  }
+
+  public static SegmentGenerationJobSpec getSegmentGenerationJobSpec(String jobSpecFilePath,
+      Map<String, Object> context)
+      throws IOException, ClassNotFoundException {
+    String yamlTemplate = IOUtils.toString(new BufferedReader(new FileReader(jobSpecFilePath)));
+    String yamlStr = GroovyTemplateUtils.renderTemplate(yamlTemplate, context);
+    return new Yaml().loadAs(yamlStr, SegmentGenerationJobSpec.class);
   }
 
   public static void runIngestionJob(SegmentGenerationJobSpec spec)
diff --git a/pinot-spi/src/main/java/org/apache/pinot/spi/utils/GroovyTemplateUtils.java b/pinot-spi/src/main/java/org/apache/pinot/spi/utils/GroovyTemplateUtils.java
new file mode 100644
index 0000000..5319b1d
--- /dev/null
+++ b/pinot-spi/src/main/java/org/apache/pinot/spi/utils/GroovyTemplateUtils.java
@@ -0,0 +1,78 @@
+/**
+ * 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.pinot.spi.utils;
+
+import groovy.text.SimpleTemplateEngine;
+import groovy.text.TemplateEngine;
+import java.io.IOException;
+import java.text.SimpleDateFormat;
+import java.time.Instant;
+import java.time.temporal.ChronoUnit;
+import java.util.Collections;
+import java.util.Date;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.TimeZone;
+
+
+public class GroovyTemplateUtils {
+  private static final TemplateEngine GROOVY_TEMPLATE_ENGINE = new SimpleTemplateEngine();
+  private static final SimpleDateFormat DATE_FORMAT = new SimpleDateFormat("yyyy-MM-dd");
+
+  public static String renderTemplate(String template, Map<String, Object> newContext)
+      throws IOException, ClassNotFoundException {
+    Map<String, Object> contextMap = getDefaultContextMap();
+    contextMap.putAll(newContext);
+    return GROOVY_TEMPLATE_ENGINE.createTemplate(template).make(contextMap).toString();
+  }
+
+  /**
+   Construct default template context:
+   today : today's date in format `yyyy-MM-dd`, example value: '2020-05-06'
+   yesterday : yesterday's date in format `yyyy-MM-dd`, example value: '2020-05-06'
+   */
+  public static Map<String, Object> getDefaultContextMap() {
+    Map<String, Object> defaultContextMap = new HashMap<>();
+    Instant now = Instant.now();
+    defaultContextMap.put("today", DATE_FORMAT.format(new Date(now.toEpochMilli())));
+    defaultContextMap.put("yesterday", DATE_FORMAT.format(new Date(now.minus(1, ChronoUnit.DAYS).toEpochMilli())));
+    return defaultContextMap;
+  }
+
+  public static Map<String, Object> getTemplateContext(List<String> values) {
+    Map<String, Object> context = new HashMap<>();
+    for (String value : values) {
+      String[] splits = value.split("=", 2);
+      if (splits.length > 1) {
+        context.put(splits[0], splits[1]);
+      }
+    }
+    return context;
+  }
+
+  public static String renderTemplate(String template)
+      throws IOException, ClassNotFoundException {
+    return renderTemplate(template, Collections.emptyMap());
+  }
+
+  static {
+    DATE_FORMAT.setTimeZone(TimeZone.getTimeZone("UTC"));
+  }
+}
diff --git a/pinot-spi/src/test/java/org/apache/pinot/spi/ingestion/batch/IngestionJobLauncherTest.java b/pinot-spi/src/test/java/org/apache/pinot/spi/ingestion/batch/IngestionJobLauncherTest.java
new file mode 100644
index 0000000..5d98c06
--- /dev/null
+++ b/pinot-spi/src/test/java/org/apache/pinot/spi/ingestion/batch/IngestionJobLauncherTest.java
@@ -0,0 +1,42 @@
+/**
+ * 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.pinot.spi.ingestion.batch;
+
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.Map;
+import org.apache.pinot.spi.ingestion.batch.spec.SegmentGenerationJobSpec;
+import org.apache.pinot.spi.utils.GroovyTemplateUtils;
+import org.testng.Assert;
+import org.testng.annotations.Test;
+
+
+public class IngestionJobLauncherTest {
+
+  @Test
+  public void testIngestionJobLauncherWithTemplate()
+      throws IOException, ClassNotFoundException {
+    Map<String, Object> context =
+        GroovyTemplateUtils.getTemplateContext(Arrays.asList("year=2020", "month=05", "day=06"));
+    SegmentGenerationJobSpec spec = IngestionJobLauncher.getSegmentGenerationJobSpec(
+        GroovyTemplateUtils.class.getClassLoader().getResource("ingestionJobSpecTemplate.yaml").getFile(), context);
+    Assert.assertEquals(spec.getInputDirURI(), "file:///path/to/input/2020/05/06");
+    Assert.assertEquals(spec.getOutputDirURI(), "file:///path/to/output/2020/05/06");
+  }
+}
diff --git a/pinot-spi/src/test/java/org/apache/pinot/spi/utils/GroovyTemplateUtilsTest.java b/pinot-spi/src/test/java/org/apache/pinot/spi/utils/GroovyTemplateUtilsTest.java
new file mode 100644
index 0000000..4719db6
--- /dev/null
+++ b/pinot-spi/src/test/java/org/apache/pinot/spi/utils/GroovyTemplateUtilsTest.java
@@ -0,0 +1,87 @@
+/**
+ * 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.pinot.spi.utils;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.text.SimpleDateFormat;
+import java.time.Instant;
+import java.time.temporal.ChronoUnit;
+import java.util.Arrays;
+import java.util.Date;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.TimeZone;
+import org.apache.commons.io.IOUtils;
+import org.apache.pinot.spi.ingestion.batch.spec.SegmentGenerationJobSpec;
+import org.testng.Assert;
+import org.testng.annotations.Test;
+import org.yaml.snakeyaml.Yaml;
+
+
+public class GroovyTemplateUtilsTest {
+
+  @Test
+  public void testDefaultRenderTemplate()
+      throws IOException, ClassNotFoundException {
+    Date today = new Date(Instant.now().toEpochMilli());
+    Date yesterday = new Date(Instant.now().minus(1, ChronoUnit.DAYS).toEpochMilli());
+    SimpleDateFormat dateFormat = new SimpleDateFormat("yyyy-MM-dd");
+    dateFormat.setTimeZone(TimeZone.getTimeZone("UTC"));
+    Assert.assertEquals(GroovyTemplateUtils.renderTemplate("${ today }"), dateFormat.format(today));
+    Assert.assertEquals(GroovyTemplateUtils.renderTemplate("${ yesterday }"), dateFormat.format(yesterday));
+  }
+
+  @Test
+  public void testRenderTemplateWithGivenContextMap()
+      throws IOException, ClassNotFoundException {
+    Map<String, Object> contextMap = new HashMap<>();
+    contextMap.put("first_date_2020", "2020-01-01");
+    contextMap.put("name", "xiang");
+    contextMap.put("ts", 1577836800);
+    contextMap.put("yyyy", "2020");
+    contextMap.put("YYYY", "1919");
+    contextMap.put("MM", "05");
+    contextMap.put("dd", "06");
+    Assert.assertEquals(GroovyTemplateUtils.renderTemplate("$first_date_2020", contextMap), "2020-01-01");
+    Assert.assertEquals(GroovyTemplateUtils.renderTemplate("${first_date_2020}", contextMap), "2020-01-01");
+    Assert.assertEquals(GroovyTemplateUtils.renderTemplate("${ name }", contextMap), "xiang");
+    Assert.assertEquals(GroovyTemplateUtils.renderTemplate("${ ts }", contextMap), "1577836800");
+    Assert.assertEquals(GroovyTemplateUtils.renderTemplate("/var/rawdata/${ yyyy }/${ MM }/${ dd }", contextMap),
+        "/var/rawdata/2020/05/06");
+    Assert.assertEquals(GroovyTemplateUtils.renderTemplate("/var/rawdata/${yyyy}/${MM}/${dd}", contextMap),
+        "/var/rawdata/2020/05/06");
+    Assert.assertEquals(GroovyTemplateUtils.renderTemplate("/var/rawdata/${YYYY}/${MM}/${dd}", contextMap),
+        "/var/rawdata/1919/05/06");
+  }
+
+  @Test
+  public void testIngestionJobTemplate()
+      throws IOException, ClassNotFoundException {
+    InputStream resourceAsStream =
+        GroovyTemplateUtils.class.getClassLoader().getResourceAsStream("ingestionJobSpecTemplate.yaml");
+    String yamlTemplate = IOUtils.toString(resourceAsStream);
+    Map<String, Object> context =
+        GroovyTemplateUtils.getTemplateContext(Arrays.asList("year=2020", "month=05", "day=06"));
+    String yamlStr = GroovyTemplateUtils.renderTemplate(yamlTemplate, context);
+    SegmentGenerationJobSpec spec = new Yaml().loadAs(yamlStr, SegmentGenerationJobSpec.class);
+    Assert.assertEquals(spec.getInputDirURI(), "file:///path/to/input/2020/05/06");
+    Assert.assertEquals(spec.getOutputDirURI(), "file:///path/to/output/2020/05/06");
+  }
+}
diff --git a/pinot-spi/src/test/resources/ingestionJobSpecTemplate.yaml b/pinot-spi/src/test/resources/ingestionJobSpecTemplate.yaml
new file mode 100644
index 0000000..c20e88a
--- /dev/null
+++ b/pinot-spi/src/test/resources/ingestionJobSpecTemplate.yaml
@@ -0,0 +1,45 @@
+#
+# 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.
+#
+
+executionFrameworkSpec:
+  name: 'standalone'
+  segmentGenerationJobRunnerClassName: 'org.apache.pinot.plugin.ingestion.batch.standalone.SegmentGenerationJobRunner'
+  segmentTarPushJobRunnerClassName: 'org.apache.pinot.plugin.ingestion.batch.standalone.SegmentTarPushJobRunner'
+  segmentUriPushJobRunnerClassName: 'org.apache.pinot.plugin.ingestion.batch.standalone.SegmentUriPushJobRunner'
+jobType: SegmentCreationAndTarPush
+inputDirURI: 'file:///path/to/input/${ year }/${ month }/${ day }'
+includeFileNamePattern: 'glob:**/*.parquet'
+excludeFileNamePattern: 'glob:**/*.avro'
+outputDirURI: 'file:///path/to/output/${year}/${month}/${day}'
+overwriteOutput: true
+pinotFSSpecs:
+  - scheme: file
+    className: org.apache.pinot.spi.filesystem.LocalPinotFS
+recordReaderSpec:
+  dataFormat: 'parquet'
+  className: 'org.apache.pinot.parquet.data.readers.ParquetRecordReader'
+tableSpec:
+  tableName: 'myTable'
+  schemaURI: 'http://localhost:9000/tables/myTable/schema'
+  tableConfigURI: 'http://localhost:9000/tables/myTable'
+pinotClusterSpecs:
+  - controllerURI: 'localhost:9000'
+pushJobSpec:
+  pushAttempts: 2
+  pushRetryIntervalMillis: 1000
\ No newline at end of file
diff --git a/pinot-tools/src/main/java/org/apache/pinot/tools/admin/command/LaunchDataIngestionJobCommand.java b/pinot-tools/src/main/java/org/apache/pinot/tools/admin/command/LaunchDataIngestionJobCommand.java
index 49c31b2..bd37ebb 100644
--- a/pinot-tools/src/main/java/org/apache/pinot/tools/admin/command/LaunchDataIngestionJobCommand.java
+++ b/pinot-tools/src/main/java/org/apache/pinot/tools/admin/command/LaunchDataIngestionJobCommand.java
@@ -18,9 +18,13 @@
  */
 package org.apache.pinot.tools.admin.command;
 
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
 import org.apache.pinot.spi.ingestion.batch.IngestionJobLauncher;
 import org.apache.pinot.tools.Command;
 import org.kohsuke.args4j.Option;
+import org.kohsuke.args4j.spi.StringArrayOptionHandler;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -35,6 +39,9 @@ public class LaunchDataIngestionJobCommand extends AbstractBaseAdminCommand impl
   @Option(name = "-jobSpecFile", required = true, metaVar = "<string>", usage = "Ingestion job spec file")
   private String _jobSpecFile;
 
+  @Option(name = "-values", required = false, metaVar = "<template context>", handler = StringArrayOptionHandler.class, usage = "Context values set to the job spec template")
+  private List<String> _values;
+
   @Option(name = "-help", required = false, help = true, aliases = {"-h", "--h", "--help"}, usage = "Print this message.")
   private boolean _help = false;
 
@@ -47,9 +54,12 @@ public class LaunchDataIngestionJobCommand extends AbstractBaseAdminCommand impl
   public boolean execute()
       throws Exception {
     try {
-      IngestionJobLauncher.main(new String[]{_jobSpecFile});
+      List<String> arguments = new ArrayList();
+      arguments.add(_jobSpecFile);
+      arguments.addAll(_values);
+      IngestionJobLauncher.main(arguments.toArray(new String[0]));
     } catch (Exception e) {
-      LOGGER.error("Got exception to kick off standalone data ingestion job -", e);
+      LOGGER.error("Got exception to kick off standalone data ingestion job - ", e);
       throw e;
     }
     return true;
@@ -62,7 +72,7 @@ public class LaunchDataIngestionJobCommand extends AbstractBaseAdminCommand impl
 
   @Override
   public String toString() {
-    return ("LaunchDataIngestionJob -jobSpecFile " + _jobSpecFile);
+    return ("LaunchDataIngestionJob -jobSpecFile " + _jobSpecFile + " -values " + Arrays.toString(_values.toArray()));
   }
 
   @Override


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org