You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@pinot.apache.org by ap...@apache.org on 2020/03/03 19:19:01 UTC

[incubator-pinot] 01/01: Extends pinot-tools data generator with realistic time series templates considering seasonality, trends, noise, and major outliers. Specifically, this adds support for: * seasonal (diurnal) patterns to simulate page views and clicks metrics * spiky (long-tailed) patterns to simulate error metrics * sequential patterns to deterministically populate time stamp columns * string dictionaries to deterministically populate dimension columns * mixture models of the above

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

apucher pushed a commit to branch pinot-tools-time-series-generator
in repository https://gitbox.apache.org/repos/asf/incubator-pinot.git

commit 903b82fa9bdae91415d496f495ce754d43f420a3
Author: Alexander Pucher <al...@alexpucher.com>
AuthorDate: Tue Mar 3 11:12:32 2020 -0800

    Extends pinot-tools data generator with realistic time series templates considering seasonality,
    trends, noise, and major outliers. Specifically, this adds support for:
    * seasonal (diurnal) patterns to simulate page views and clicks metrics
    * spiky (long-tailed) patterns to simulate error metrics
    * sequential patterns to deterministically populate time stamp columns
    * string dictionaries to deterministically populate dimension columns
    * mixture models of the above
    
    Additionally provides two sample configurations "simpleWebsite" and "complexWebsite" which
    generate non-dimensional and dimensional examples of time series respectively. Also provides
    instructions for generating and loading mock data onto an existing Pinot cluster.
---
 .../java/org/apache/pinot/spi/utils/JsonUtils.java |   8 ++
 .../tools/admin/command/GenerateDataCommand.java   |  59 ++++++----
 .../pinot/tools/data/generator/DataGenerator.java  |  50 ++++++--
 .../tools/data/generator/DataGeneratorSpec.java    |  17 ++-
 .../tools/data/generator/GeneratorFactory.java     |  48 ++++----
 .../tools/data/generator/SchemaAnnotation.java     |  16 +++
 .../data/generator/TemplateMixtureGenerator.java   | 109 +++++++++++++++++
 .../data/generator/TemplateSeasonalGenerator.java  | 129 +++++++++++++++++++++
 .../data/generator/TemplateSequenceGenerator.java  |  83 +++++++++++++
 .../data/generator/TemplateSpikeGenerator.java     | 108 +++++++++++++++++
 .../data/generator/TemplateStringGenerator.java    |  80 +++++++++++++
 .../pinot/tools/data/generator/TemplateType.java   |  27 +++++
 pinot-tools/src/main/resources/generator/README.md |  85 ++++++++++++++
 .../resources/generator/complexWebsite_config.json |  17 +++
 .../generator/complexWebsite_generator.json        | 113 ++++++++++++++++++
 .../resources/generator/complexWebsite_schema.json |  38 ++++++
 .../resources/generator/simpleWebsite_config.json  |  17 +++
 .../generator/simpleWebsite_generator.json         |  44 +++++++
 .../resources/generator/simpleWebsite_schema.json  |  26 +++++
 19 files changed, 1014 insertions(+), 60 deletions(-)

diff --git a/pinot-spi/src/main/java/org/apache/pinot/spi/utils/JsonUtils.java b/pinot-spi/src/main/java/org/apache/pinot/spi/utils/JsonUtils.java
index 13c242c..9e66f4a 100644
--- a/pinot-spi/src/main/java/org/apache/pinot/spi/utils/JsonUtils.java
+++ b/pinot-spi/src/main/java/org/apache/pinot/spi/utils/JsonUtils.java
@@ -33,6 +33,8 @@ import java.io.File;
 import java.io.FileInputStream;
 import java.io.IOException;
 import java.io.InputStream;
+import java.util.Collection;
+import java.util.List;
 import javax.annotation.Nullable;
 import org.apache.pinot.spi.data.FieldSpec;
 import org.apache.pinot.spi.data.FieldSpec.DataType;
@@ -68,6 +70,12 @@ public class JsonUtils {
     return DEFAULT_READER.forType(valueType).readValue(jsonFile);
   }
 
+  public static <T> List<T> fileToList(File jsonFile, Class<T> valueType)
+      throws IOException {
+    return DEFAULT_READER.forType(
+            DEFAULT_MAPPER.getTypeFactory().constructCollectionType(List.class, valueType)).readValue(jsonFile);
+  }
+
   public static JsonNode fileToJsonNode(File jsonFile)
       throws IOException {
     try (InputStream inputStream = new FileInputStream(jsonFile)) {
diff --git a/pinot-tools/src/main/java/org/apache/pinot/tools/admin/command/GenerateDataCommand.java b/pinot-tools/src/main/java/org/apache/pinot/tools/admin/command/GenerateDataCommand.java
index 19ff1ce..3ce0427 100644
--- a/pinot-tools/src/main/java/org/apache/pinot/tools/admin/command/GenerateDataCommand.java
+++ b/pinot-tools/src/main/java/org/apache/pinot/tools/admin/command/GenerateDataCommand.java
@@ -18,12 +18,6 @@
  */
 package org.apache.pinot.tools.admin.command;
 
-import java.io.File;
-import java.io.IOException;
-import java.util.HashMap;
-import java.util.LinkedList;
-import java.util.List;
-import java.util.concurrent.TimeUnit;
 import org.apache.commons.lang.math.IntRange;
 import org.apache.pinot.spi.data.FieldSpec;
 import org.apache.pinot.spi.data.FieldSpec.DataType;
@@ -31,8 +25,8 @@ import org.apache.pinot.spi.data.FieldSpec.FieldType;
 import org.apache.pinot.spi.data.Schema;
 import org.apache.pinot.spi.data.Schema.SchemaBuilder;
 import org.apache.pinot.spi.data.TimeFieldSpec;
-import org.apache.pinot.spi.utils.JsonUtils;
 import org.apache.pinot.spi.data.readers.FileFormat;
+import org.apache.pinot.spi.utils.JsonUtils;
 import org.apache.pinot.tools.Command;
 import org.apache.pinot.tools.data.generator.DataGenerator;
 import org.apache.pinot.tools.data.generator.DataGeneratorSpec;
@@ -41,6 +35,14 @@ import org.kohsuke.args4j.Option;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import java.io.File;
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.TimeUnit;
+
 
 /**
  * Class to implement GenerateData command.
@@ -49,6 +51,9 @@ import org.slf4j.LoggerFactory;
 public class GenerateDataCommand extends AbstractBaseAdminCommand implements Command {
   private static final Logger LOGGER = LoggerFactory.getLogger(GenerateDataCommand.class);
 
+  private final static String FORMAT_AVRO = "avro";
+  private final static String FORMAT_CSV = "csv";
+
   @Option(name = "-numRecords", required = true, metaVar = "<int>", usage = "Number of records to generate.")
   private int _numRecords = 0;
 
@@ -70,6 +75,9 @@ public class GenerateDataCommand extends AbstractBaseAdminCommand implements Com
   @Option(name = "-help", required = false, help = true, aliases = {"-h", "--h", "--help"}, usage = "Print this message.")
   private boolean _help = false;
 
+  @Option(name = "-format", required = false, help = true, usage = "Output format ('avro' or 'csv').")
+  private String _format = FORMAT_AVRO;
+
   @Override
   public boolean getHelp() {
     return _help;
@@ -115,39 +123,50 @@ public class GenerateDataCommand extends AbstractBaseAdminCommand implements Com
 
     Schema schema = Schema.fromFile(new File(_schemaFile));
 
-    List<String> columns = new LinkedList<String>();
-    final HashMap<String, DataType> dataTypes = new HashMap<String, DataType>();
-    final HashMap<String, FieldType> fieldTypes = new HashMap<String, FieldType>();
-    final HashMap<String, TimeUnit> timeUnits = new HashMap<String, TimeUnit>();
+    List<String> columns = new LinkedList<>();
+    final HashMap<String, DataType> dataTypes = new HashMap<>();
+    final HashMap<String, FieldType> fieldTypes = new HashMap<>();
+    final HashMap<String, TimeUnit> timeUnits = new HashMap<>();
+
+    final HashMap<String, Integer> cardinality = new HashMap<>();
+    final HashMap<String, IntRange> range = new HashMap<>();
+    final HashMap<String, Map<String, Object>> template = new HashMap<>();
 
-    final HashMap<String, Integer> cardinality = new HashMap<String, Integer>();
-    final HashMap<String, IntRange> range = new HashMap<String, IntRange>();
+    buildCardinalityRangeMaps(_schemaAnnFile, cardinality, range, template);
 
-    buildCardinalityRangeMaps(_schemaAnnFile, cardinality, range);
     final DataGeneratorSpec spec =
-        buildDataGeneratorSpec(schema, columns, dataTypes, fieldTypes, timeUnits, cardinality, range);
+        buildDataGeneratorSpec(schema, columns, dataTypes, fieldTypes, timeUnits, cardinality, range, template);
 
     final DataGenerator gen = new DataGenerator();
     gen.init(spec);
-    gen.generate(_numRecords, _numFiles);
+
+    if (FORMAT_AVRO.equals(_format)) {
+      gen.generateAvro(_numRecords, _numFiles);
+    } else if (FORMAT_CSV.equals(_format)) {
+      gen.generateCsv(_numRecords, _numFiles);
+    } else {
+      throw new IllegalArgumentException(String.format("Invalid output format '%s'", _format));
+    }
 
     return true;
   }
 
   private void buildCardinalityRangeMaps(String file, HashMap<String, Integer> cardinality,
-      HashMap<String, IntRange> range)
+      HashMap<String, IntRange> range, Map<String, Map<String, Object>> template)
       throws IOException {
     if (file == null) {
       return; // Nothing to do here.
     }
 
-    List<SchemaAnnotation> saList = JsonUtils.fileToObject(new File(file), List.class);
+    List<SchemaAnnotation> saList = JsonUtils.fileToList(new File(file), SchemaAnnotation.class);
 
     for (SchemaAnnotation sa : saList) {
       String column = sa.getColumn();
 
       if (sa.isRange()) {
         range.put(column, new IntRange(sa.getRangeStart(), sa.getRangeEnd()));
+      } else if (sa.getTemplate() != null) {
+        template.put(column, sa.getTemplate());
       } else {
         cardinality.put(column, sa.getCardinality());
       }
@@ -156,7 +175,7 @@ public class GenerateDataCommand extends AbstractBaseAdminCommand implements Com
 
   private DataGeneratorSpec buildDataGeneratorSpec(Schema schema, List<String> columns,
       HashMap<String, DataType> dataTypes, HashMap<String, FieldType> fieldTypes, HashMap<String, TimeUnit> timeUnits,
-      HashMap<String, Integer> cardinality, HashMap<String, IntRange> range) {
+      HashMap<String, Integer> cardinality, HashMap<String, IntRange> range, HashMap<String, Map<String, Object>> template) {
     for (final FieldSpec fs : schema.getAllFieldSpecs()) {
       String col = fs.getName();
 
@@ -190,7 +209,7 @@ public class GenerateDataCommand extends AbstractBaseAdminCommand implements Com
       }
     }
 
-    return new DataGeneratorSpec(columns, cardinality, range, dataTypes, fieldTypes, timeUnits, FileFormat.AVRO,
+    return new DataGeneratorSpec(columns, cardinality, range, template, dataTypes, fieldTypes, timeUnits, FileFormat.AVRO,
         _outDir, _overwrite);
   }
 
diff --git a/pinot-tools/src/main/java/org/apache/pinot/tools/data/generator/DataGenerator.java b/pinot-tools/src/main/java/org/apache/pinot/tools/data/generator/DataGenerator.java
index 182ae68..25375d4 100644
--- a/pinot-tools/src/main/java/org/apache/pinot/tools/data/generator/DataGenerator.java
+++ b/pinot-tools/src/main/java/org/apache/pinot/tools/data/generator/DataGenerator.java
@@ -19,13 +19,16 @@
 package org.apache.pinot.tools.data.generator;
 
 import java.io.File;
+import java.io.FileWriter;
 import java.io.IOException;
 import java.util.Arrays;
 import java.util.HashMap;
 import java.util.Map;
 import java.util.concurrent.TimeUnit;
 import org.apache.commons.io.FileUtils;
+import org.apache.commons.lang.StringUtils;
 import org.apache.commons.lang.math.IntRange;
+import org.apache.pinot.common.utils.StringUtil;
 import org.apache.pinot.spi.data.DimensionFieldSpec;
 import org.apache.pinot.spi.data.FieldSpec;
 import org.apache.pinot.spi.data.FieldSpec.DataType;
@@ -72,13 +75,20 @@ public class DataGenerator {
     for (final String column : genSpec.getColumns()) {
       DataType dataType = genSpec.getDataTypesMap().get(column);
 
-      if (genSpec.getCardinalityMap().containsKey(column)) {
+      if (genSpec.getTemplateMap().containsKey(column)) {
+        generators.put(column,
+                GeneratorFactory.getGeneratorFor(
+                        TemplateType.valueOf(genSpec.getTemplateMap().get(column).get("type").toString()),
+                        genSpec.getTemplateMap().get(column)));
+
+      } else if (genSpec.getCardinalityMap().containsKey(column)) {
         generators.put(column, GeneratorFactory.getGeneratorFor(dataType, genSpec.getCardinalityMap().get(column)));
+
       } else if (genSpec.getRangeMap().containsKey(column)) {
         IntRange range = genSpec.getRangeMap().get(column);
-
         generators.put(column,
-            GeneratorFactory.getGeneratorFor(dataType, range.getMinimumInteger(), range.getMaximumInteger()));
+                GeneratorFactory.getGeneratorFor(dataType, range.getMinimumInteger(), range.getMaximumInteger()));
+
       } else {
         LOGGER.error("cardinality for this column does not exist : " + column);
         throw new RuntimeException("cardinality for this column does not exist");
@@ -88,7 +98,7 @@ public class DataGenerator {
     }
   }
 
-  public void generate(long totalDocs, int numFiles)
+  public void generateAvro(long totalDocs, int numFiles)
       throws IOException {
     final int numPerFiles = (int) (totalDocs / numFiles);
     for (int i = 0; i < numFiles; i++) {
@@ -100,6 +110,23 @@ public class DataGenerator {
     }
   }
 
+  public void generateCsv(long totalDocs, int numFiles)
+      throws IOException {
+    final int numPerFiles = (int) (totalDocs / numFiles);
+    for (int i = 0; i < numFiles; i++) {
+      try (FileWriter writer = new FileWriter(outDir + "/output.csv")) {
+        writer.append(StringUtils.join(genSpec.getColumns(), ",")).append('\n');
+        for (int j = 0; j < numPerFiles; j++) {
+          Object[] values = new Object[genSpec.getColumns().size()];
+          for (int k = 0; k < genSpec.getColumns().size(); k++) {
+            values[k] = generators.get(genSpec.getColumns().get(k)).next();
+          }
+          writer.append(StringUtils.join(values, ",")).append('\n');
+        }
+      }
+    }
+  }
+
   public Schema fetchSchema() {
     final Schema schema = new Schema();
     for (final String column : genSpec.getColumns()) {
@@ -141,12 +168,13 @@ public class DataGenerator {
   public static void main(String[] args)
       throws IOException {
     final String[] columns = {"column1", "column2", "column3", "column4", "column5"};
-    final Map<String, DataType> dataTypes = new HashMap<String, DataType>();
-    final Map<String, FieldType> fieldTypes = new HashMap<String, FieldType>();
-    final Map<String, TimeUnit> timeUnits = new HashMap<String, TimeUnit>();
+    final Map<String, DataType> dataTypes = new HashMap<>();
+    final Map<String, FieldType> fieldTypes = new HashMap<>();
+    final Map<String, TimeUnit> timeUnits = new HashMap<>();
 
-    final Map<String, Integer> cardinality = new HashMap<String, Integer>();
-    final Map<String, IntRange> range = new HashMap<String, IntRange>();
+    final Map<String, Integer> cardinality = new HashMap<>();
+    final Map<String, IntRange> range = new HashMap<>();
+    final Map<String, Map<String, Object>> template = new HashMap<>();
 
     for (final String col : columns) {
       dataTypes.put(col, DataType.INT);
@@ -154,11 +182,11 @@ public class DataGenerator {
       cardinality.put(col, 1000);
     }
     final DataGeneratorSpec spec =
-        new DataGeneratorSpec(Arrays.asList(columns), cardinality, range, dataTypes, fieldTypes, timeUnits,
+        new DataGeneratorSpec(Arrays.asList(columns), cardinality, range, template, dataTypes, fieldTypes, timeUnits,
             FileFormat.AVRO, "/tmp/out", true);
 
     final DataGenerator gen = new DataGenerator();
     gen.init(spec);
-    gen.generate(1000000L, 2);
+    gen.generateAvro(1000000L, 2);
   }
 }
diff --git a/pinot-tools/src/main/java/org/apache/pinot/tools/data/generator/DataGeneratorSpec.java b/pinot-tools/src/main/java/org/apache/pinot/tools/data/generator/DataGeneratorSpec.java
index c4c3fb9..8bd6096 100644
--- a/pinot-tools/src/main/java/org/apache/pinot/tools/data/generator/DataGeneratorSpec.java
+++ b/pinot-tools/src/main/java/org/apache/pinot/tools/data/generator/DataGeneratorSpec.java
@@ -37,6 +37,7 @@ public class DataGeneratorSpec {
   private final List<String> columns;
   private final Map<String, Integer> cardinalityMap;
   private final Map<String, IntRange> rangeMap;
+  private final Map<String, Map<String, Object>> templateMap;
 
   private final Map<String, DataType> dataTypesMap;
   private final Map<String, FieldType> fieldTypesMap;
@@ -47,17 +48,19 @@ public class DataGeneratorSpec {
   private final boolean overrideOutDir;
 
   public DataGeneratorSpec() {
-    this(new ArrayList<String>(), new HashMap<String, Integer>(), new HashMap<String, IntRange>(),
-        new HashMap<String, DataType>(), new HashMap<String, FieldType>(), new HashMap<String, TimeUnit>(),
+    this(new ArrayList<String>(), new HashMap<>(), new HashMap<>(), new HashMap<>(),
+        new HashMap<>(), new HashMap<>(), new HashMap<>(),
         FileFormat.AVRO, "/tmp/dataGen", true);
   }
 
   public DataGeneratorSpec(List<String> columns, Map<String, Integer> cardinalityMap, Map<String, IntRange> rangeMap,
-      Map<String, DataType> dataTypesMap, Map<String, FieldType> fieldTypesMap, Map<String, TimeUnit> timeUnitMap,
+      Map<String, Map<String, Object>> templateMap, Map<String, DataType> dataTypesMap, Map<String, FieldType> fieldTypesMap, Map<String, TimeUnit> timeUnitMap,
       FileFormat format, String outputDir, boolean override) {
     this.columns = columns;
     this.cardinalityMap = cardinalityMap;
     this.rangeMap = rangeMap;
+    this.templateMap = templateMap;
+
     outputFileFormat = format;
     this.outputDir = outputDir;
     overrideOutDir = override;
@@ -95,6 +98,10 @@ public class DataGeneratorSpec {
     return rangeMap;
   }
 
+  public Map<String, Map<String, Object>> getTemplateMap() {
+    return templateMap;
+  }
+
   public FileFormat getOutputFileFormat() {
     return outputFileFormat;
   }
@@ -109,8 +116,10 @@ public class DataGeneratorSpec {
     for (final String column : columns) {
       if (cardinalityMap.get(column) != null) {
         builder.append(column + " : " + cardinalityMap.get(column) + " : " + dataTypesMap.get(column));
-      } else {
+      } else if (rangeMap.get(column) != null) {
         builder.append(column + " : " + rangeMap.get(column) + " : " + dataTypesMap.get(column));
+      } else {
+        builder.append(column + " : " + templateMap.get(column));
       }
     }
     builder.append("output file format : " + outputFileFormat);
diff --git a/pinot-tools/src/main/java/org/apache/pinot/tools/data/generator/GeneratorFactory.java b/pinot-tools/src/main/java/org/apache/pinot/tools/data/generator/GeneratorFactory.java
index fbf22fb..0c1b950 100644
--- a/pinot-tools/src/main/java/org/apache/pinot/tools/data/generator/GeneratorFactory.java
+++ b/pinot-tools/src/main/java/org/apache/pinot/tools/data/generator/GeneratorFactory.java
@@ -20,22 +20,14 @@ package org.apache.pinot.tools.data.generator;
 
 import org.apache.pinot.spi.data.FieldSpec.DataType;
 
+import java.util.Map;
+
 
 /**
  * Sep 13, 2014
  */
 
 public class GeneratorFactory {
-
-  public static Class getGeneratorFor(DataType type) {
-
-    if (type == DataType.STRING) {
-      return StringGenerator.class;
-    }
-
-    return NumberGenerator.class;
-  }
-
   public static Generator getGeneratorFor(DataType type, int cardinality) {
     if (type == DataType.STRING) {
       return new StringGenerator(cardinality);
@@ -45,28 +37,34 @@ public class GeneratorFactory {
   }
 
   public static Generator getGeneratorFor(DataType dataType, int start, int end) {
-    Generator generator;
-
     switch (dataType) {
       case INT:
-        generator = new RangeIntGenerator(start, end);
-        break;
-
+        return new RangeIntGenerator(start, end);
       case LONG:
-        generator = new RangeLongGenerator(start, end);
-        break;
-
+        return new RangeLongGenerator(start, end);
       case FLOAT:
-        generator = new RangeFloatGenerator(start, end);
-        break;
-
+        return new RangeFloatGenerator(start, end);
       case DOUBLE:
-        generator = new RangeDoubleGenerator(start, end);
-        break;
+        return new RangeDoubleGenerator(start, end);
+      default:
+        throw new RuntimeException(String.format("Invalid datatype '%s'", dataType));
+    }
+  }
 
+  public static Generator getGeneratorFor(TemplateType templateType, Map<String, Object> templateConfig) {
+    switch (templateType) {
+      case SEASONAL:
+        return new TemplateSeasonalGenerator(templateConfig);
+      case SPIKE:
+        return new TemplateSpikeGenerator(templateConfig);
+      case SEQUENCE:
+        return new TemplateSequenceGenerator(templateConfig);
+      case STRING:
+        return new TemplateStringGenerator(templateConfig);
+      case MIXTURE:
+        return new TemplateMixtureGenerator(templateConfig);
       default:
-        throw new RuntimeException("Invalid datatype");
+        throw new RuntimeException(String.format("Invalid template '%s'", templateType));
     }
-    return generator;
   }
 }
diff --git a/pinot-tools/src/main/java/org/apache/pinot/tools/data/generator/SchemaAnnotation.java b/pinot-tools/src/main/java/org/apache/pinot/tools/data/generator/SchemaAnnotation.java
index 9b61891..1c032b7 100644
--- a/pinot-tools/src/main/java/org/apache/pinot/tools/data/generator/SchemaAnnotation.java
+++ b/pinot-tools/src/main/java/org/apache/pinot/tools/data/generator/SchemaAnnotation.java
@@ -18,6 +18,8 @@
  */
 package org.apache.pinot.tools.data.generator;
 
+import java.util.Map;
+
 public class SchemaAnnotation {
   String _column;
   boolean _range;
@@ -25,6 +27,7 @@ public class SchemaAnnotation {
   private int _cardinality;
   private int _rangeStart;
   private int _rangeEnd;
+  private Map<String, Object> _template;
 
   public SchemaAnnotation() {
   }
@@ -40,6 +43,11 @@ public class SchemaAnnotation {
     _rangeEnd = rangeEnd;
   }
 
+  public SchemaAnnotation(String column, Map<String, Object> template) {
+    _column = column;
+    _template = template;
+  }
+
   public String getColumn() {
     return _column;
   }
@@ -79,4 +87,12 @@ public class SchemaAnnotation {
   public void setRangeEnd(int rangeEnd) {
     _rangeEnd = rangeEnd;
   }
+
+  public Map<String, Object> getTemplate() {
+    return _template;
+  }
+
+  public void setTemplate(Map<String, Object> template) {
+    _template = template;
+  }
 }
diff --git a/pinot-tools/src/main/java/org/apache/pinot/tools/data/generator/TemplateMixtureGenerator.java b/pinot-tools/src/main/java/org/apache/pinot/tools/data/generator/TemplateMixtureGenerator.java
new file mode 100644
index 0000000..b87a052
--- /dev/null
+++ b/pinot-tools/src/main/java/org/apache/pinot/tools/data/generator/TemplateMixtureGenerator.java
@@ -0,0 +1,109 @@
+/**
+ * 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.tools.data.generator;
+
+import java.util.*;
+import java.util.stream.Collectors;
+
+/**
+ * TemplateMixtureGenerator enables combination of multiple Generators in alternating and additive patterns, including
+ * nested mixture models. This is typically used to generate similar (but not exact copies of) time series for different
+ * dimension values of series or to simulate anomalous behavior in a otherwise regular time series.
+ *
+ * Generator example:
+ * <pre>
+ *     generator bins = [
+ *       [ { type = "SEASONAL", mean = 10, ... } ],
+ *       [ { type = "SEASONAL", mean = 30, ... }, { type = "SPIKE", arrivalMean = 2, ... } ],
+ *       [ { type = "SEASONAL", mean = 50, ... } ],
+ *     ]
+ *
+ *     returns [ 10, 30, 50, 11, 29, 52, 10, 114, 51, 9, 64, 50, 10, 35, 49, ... ]
+ * </pre>
+ *
+ * Configuration examples:
+ * <ul>
+ *     <li>./pinot-tools/src/main/resources/generator/complexWebsite_generator.json</li>
+ * </ul>
+ */
+public class TemplateMixtureGenerator implements Generator {
+    private final List<List<Generator>> generatorBins;
+
+    private long step = -1;
+
+    public TemplateMixtureGenerator(Map<String, Object> templateConfig) {
+        this(toGeneratorBins(
+                (List<List<Map<String, Object>>>) templateConfig.get("generatorBins"),
+                (Map<String, Object>) templateConfig.get("defaults")));
+    }
+
+    public TemplateMixtureGenerator(List<List<Generator>> generatorBins) {
+        this.generatorBins = generatorBins;
+    }
+
+    private static List<List<Generator>> toGeneratorBins(List<List<Map<String, Object>>> templateConfigBins, Map<String, Object> defaults) {
+        final List<List<Map<String, Object>>> safeBins = templateConfigBins == null ? new ArrayList<>() : templateConfigBins;
+        final Map<String, Object> safeDefaults = defaults == null ? new HashMap<>() : defaults;
+
+        return safeBins.stream().map(conf -> toGenerators(conf, safeDefaults)).collect(Collectors.toList());
+    }
+
+    private static List<Generator> toGenerators(List<Map<String, Object>> templateConfigs, Map<String, Object> defaults) {
+        return templateConfigs.stream()
+                .map(conf -> {
+                    Map<String, Object> augmentedConf = new HashMap<>(defaults);
+                    augmentedConf.putAll(conf);
+                    return toGenerator(augmentedConf);
+                }).collect(Collectors.toList());
+    }
+
+    private static Generator toGenerator(Map<String, Object> templateConfig) {
+        TemplateType type = TemplateType.valueOf(templateConfig.get("type").toString());
+        return GeneratorFactory.getGeneratorFor(type, templateConfig);
+    }
+
+    @Override
+    public void init() {
+        // left blank
+    }
+
+    @Override
+    public Object next() {
+        step++;
+        int bin = (int) step % generatorBins.size();
+        long output = 0;
+        for (Generator gen : generatorBins.get(bin)) {
+            output += (Long) gen.next();
+        }
+        return output;
+    }
+
+    public static void main(String[] args) {
+        TemplateMixtureGenerator generator = new TemplateMixtureGenerator(
+                Arrays.asList(
+                        Arrays.asList(new TemplateSeasonalGenerator(100, 1, 0, 24, 50, new double[] { 1 })),
+                        Arrays.asList(new TemplateSequenceGenerator(-10, 1, 2))
+                )
+        );
+
+        for (int i = 0; i < 1000; i++) {
+            System.out.println(generator.next());
+        }
+    }
+}
diff --git a/pinot-tools/src/main/java/org/apache/pinot/tools/data/generator/TemplateSeasonalGenerator.java b/pinot-tools/src/main/java/org/apache/pinot/tools/data/generator/TemplateSeasonalGenerator.java
new file mode 100644
index 0000000..55b88dd
--- /dev/null
+++ b/pinot-tools/src/main/java/org/apache/pinot/tools/data/generator/TemplateSeasonalGenerator.java
@@ -0,0 +1,129 @@
+/**
+ * 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.tools.data.generator;
+
+import org.apache.commons.math3.distribution.NormalDistribution;
+
+import java.util.Arrays;
+import java.util.List;
+import java.util.Map;
+
+/**
+ * TemplateSeasonalGenerator generates sinus wave patterns with a linear trend, gaussian noise, and cyclically repeating
+ * scaling factors. These patterns are typical for di-urnal usage patterns such as clicks and impressions of a website.
+ *
+ * Generator example:
+ * <pre>
+ *     mean = 10
+ *     sigma = 1
+ *     wavelength = 4
+ *     amplitude = 10
+ *     scaling factors = [ 0.5, 1.0, 1.0, 1.0, 1.0, 0.5 ] // e.g. high weekdays, low week ends
+ *
+ *     returns [ 5, 10, 12, 7, 11, 18, 21, 6, 2, 7, 12, 20, 21, 13, ... ]
+ * </pre>
+ *
+ * Configuration examples:
+ * <ul>
+ *     <li>./pinot-tools/src/main/resources/generator/simpleWebsite_generator.json</li>
+ *     <li>./pinot-tools/src/main/resources/generator/complexWebsite_generator.json</li>
+ * </ul>
+ */
+public class TemplateSeasonalGenerator implements Generator {
+    private final double trend;
+    private final double wavelength;
+    private final double amplitude;
+    private final double[] scalingFactors;
+
+    private final NormalDistribution generator;
+
+    private long step = -1;
+
+    public TemplateSeasonalGenerator(Map<String, Object> templateConfig) {
+        this(toDouble(templateConfig.get("mean"), 0),
+                toDouble(templateConfig.get("sigma"), 0),
+                toDouble(templateConfig.get("trend"), 0),
+                toDouble(templateConfig.get("wavelength"), 0),
+                toDouble(templateConfig.get("amplitude"), 0),
+                toDoubleArray(templateConfig.get("scalingFactors"), 1));
+    }
+
+    public TemplateSeasonalGenerator(double mean, double sigma, double trend, double wavelength, double amplitude,
+                                     double[] scalingFactors) {
+        this.trend = trend;
+        this.wavelength = wavelength;
+        this.amplitude = amplitude;
+        this.scalingFactors = scalingFactors;
+
+        this.generator = new NormalDistribution(mean, sigma);
+    }
+
+    @Override
+    public void init() {
+        // left blank
+    }
+
+    @Override
+    public Object next() {
+        step++;
+        return (long) Math.max((generator.sample()
+                + (trend * step)
+                + (wavelength == 0d ? 0 : Math.sin(step / wavelength * 2 * Math.PI) * amplitude))
+                * makeScalingFactor(step), 0);
+    }
+
+    private double makeScalingFactor(long step) {
+        double offset = step / wavelength - 0.5 + scalingFactors.length;
+        int i = (int) Math.floor(offset) % scalingFactors.length;
+        int j = (int) Math.ceil(offset) % scalingFactors.length;
+
+        double shift = offset - Math.floor(offset);
+
+        return (1 - shift) * scalingFactors[i] + shift * scalingFactors[j];
+    }
+
+    private static double toDouble(Object obj, double defaultValue) {
+        if (obj == null) {
+            return defaultValue;
+        }
+        return Double.valueOf(obj.toString());
+    }
+
+    private static double[] toDoubleArray(Object obj, double defaultValue) {
+        if (obj == null) {
+            double[] values = new double[1];
+            Arrays.fill(values, defaultValue);
+            return values;
+        }
+
+        List<Double> userValues = (List<Double>) obj;
+        double[] values = new double[userValues.size()];
+        for (int i = 0; i < userValues.size(); i++) {
+            values[i] = userValues.get(i);
+        }
+        return values;
+    }
+
+    public static void main(String[] args) {
+        TemplateSeasonalGenerator gen = new TemplateSeasonalGenerator(80, 5, 0.02, 24, 40, new double[] { 0.5, 0.9, 1, 1, 1, 0.8, 0.6 });
+        for (int i = 0; i < 336; i++) {
+            System.out.println(gen.next());
+        }
+    }
+}
diff --git a/pinot-tools/src/main/java/org/apache/pinot/tools/data/generator/TemplateSequenceGenerator.java b/pinot-tools/src/main/java/org/apache/pinot/tools/data/generator/TemplateSequenceGenerator.java
new file mode 100644
index 0000000..d52e56f
--- /dev/null
+++ b/pinot-tools/src/main/java/org/apache/pinot/tools/data/generator/TemplateSequenceGenerator.java
@@ -0,0 +1,83 @@
+/**
+ * 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.tools.data.generator;
+
+import java.util.Map;
+
+/**
+ * TemplateSequenceGenerator produces a series of sequentially increasing (decreasing) numbers, optionally with a fixed
+ * number of repetitions per values. This pattern is typical for monotonically increasing series such as timestamps.
+ *
+ * Generator example:
+ * <pre>
+ *     start = -10
+ *     stepsize = 3
+ *     repetitions = 2
+ *
+ *     returns [ -10, -10, -7, -7, -4, -4, -1, -1, 2, 2, ... ]
+ * </pre>
+ *
+ * Configuration examples:
+ * <ul>
+ *     <li>./pinot-tools/src/main/resources/generator/simpleWebsite_generator.json</li>
+ *     <li>./pinot-tools/src/main/resources/generator/complexWebsite_generator.json</li>
+ * </ul>
+ */
+public class TemplateSequenceGenerator implements Generator {
+    private final long start;
+    private final long stepsize;
+    private final long repetitions;
+
+    private long step = -1;
+
+    public TemplateSequenceGenerator(Map<String, Object> templateConfig) {
+        this(toLong(templateConfig.get("start"), 0), toLong(templateConfig.get("stepsize"), 1), toLong(templateConfig.get("repetitions"), 1));
+    }
+
+    public TemplateSequenceGenerator(long start, long stepsize, long repetitions) {
+        this.start = start;
+        this.stepsize = stepsize;
+        this.repetitions = repetitions;
+    }
+
+    @Override
+    public void init() {
+        // left blank
+    }
+
+    @Override
+    public Object next() {
+        step++;
+        return start + (step / repetitions) * stepsize;
+    }
+
+    private static long toLong(Object obj, long defaultValue) {
+        if (obj == null) {
+            return defaultValue;
+        }
+        return Long.valueOf(obj.toString());
+    }
+
+    public static void main(String[] args) {
+        TemplateSequenceGenerator gen = new TemplateSequenceGenerator(-10, 3, 2);
+        for (int i = 0; i < 100; i++) {
+            System.out.println(gen.next());
+        }
+    }
+}
diff --git a/pinot-tools/src/main/java/org/apache/pinot/tools/data/generator/TemplateSpikeGenerator.java b/pinot-tools/src/main/java/org/apache/pinot/tools/data/generator/TemplateSpikeGenerator.java
new file mode 100644
index 0000000..a78241f
--- /dev/null
+++ b/pinot-tools/src/main/java/org/apache/pinot/tools/data/generator/TemplateSpikeGenerator.java
@@ -0,0 +1,108 @@
+/**
+ * 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.tools.data.generator;
+
+import org.apache.commons.math3.distribution.LogNormalDistribution;
+
+import java.util.Map;
+
+/**
+ * TemplateSpikeGenerator produces a series of log-normal spikes with log-normal arrival times, with optional smoothing.
+ * This pattern is typical for rare even spikes, such as error counts. The generated values are sampled non-deterministically.
+ *
+ * Generator example:
+ * <pre>
+ *     baseline = 0
+ *     arrivalMean = ?
+ *     magnitudeMean = ?
+ *
+ *     returns [ 0, 0, 0, 0, 0, 0, 47, 15, 2, 1, 0, 0, ... ]
+ * </pre>
+ *
+ * Configuration examples:
+ * <ul>
+ *     <li>./pinot-tools/src/main/resources/generator/simpleWebsite_generator.json</li>
+ *     <li>./pinot-tools/src/main/resources/generator/complexWebsite_generator.json</li>
+ * </ul>
+ */
+public class TemplateSpikeGenerator implements Generator {
+    private final double baseline;
+    private final double smoothing;
+
+    private final LogNormalDistribution arrivalGenerator;
+    private final LogNormalDistribution magnitudeGenerator;
+
+    private long step = -1;
+
+    private long nextArrival;
+    private double lastValue;
+
+    public TemplateSpikeGenerator(Map<String, Object> templateConfig) {
+        this(toDouble(templateConfig.get("baseline"), 0),
+                toDouble(templateConfig.get("arrivalMean"), 2),
+                toDouble(templateConfig.get("arrivalSigma"), 1),
+                toDouble(templateConfig.get("magnitudeMean"), 2),
+                toDouble(templateConfig.get("magnitudeSigma"), 1),
+                toDouble(templateConfig.get("smoothing"), 0));
+    }
+
+    public TemplateSpikeGenerator(double baseline, double arrivalMean, double arrivalSigma, double magnitudeMean, double magnitudeSigma, double smoothing) {
+        this.baseline = baseline;
+        this.smoothing = smoothing;
+
+        this.arrivalGenerator = new LogNormalDistribution(arrivalMean, arrivalSigma);
+        this.magnitudeGenerator = new LogNormalDistribution(magnitudeMean, magnitudeSigma);
+
+        this.nextArrival = (long) arrivalGenerator.sample();
+        this.lastValue = baseline;
+    }
+
+    @Override
+    public void init() {
+        // left blank
+    }
+
+    @Override
+    public Object next() {
+        step++;
+
+        if (step < nextArrival) {
+            lastValue = (1 - smoothing) * baseline + smoothing * lastValue;
+            return (long) lastValue;
+        }
+
+        nextArrival += (long) arrivalGenerator.sample();
+        lastValue = baseline + this.magnitudeGenerator.sample();
+        return (long) lastValue;
+    }
+
+    private static double toDouble(Object obj, double defaultValue) {
+        if (obj == null) {
+            return defaultValue;
+        }
+        return Double.valueOf(obj.toString());
+    }
+
+    public static void main(String[] args) {
+        TemplateSpikeGenerator gen = new TemplateSpikeGenerator(15, 2, 1, 3, 1, 0.25);
+        for (int i = 0; i < 100; i++) {
+            System.out.println(gen.next());
+        }
+    }
+}
diff --git a/pinot-tools/src/main/java/org/apache/pinot/tools/data/generator/TemplateStringGenerator.java b/pinot-tools/src/main/java/org/apache/pinot/tools/data/generator/TemplateStringGenerator.java
new file mode 100644
index 0000000..eb052a8
--- /dev/null
+++ b/pinot-tools/src/main/java/org/apache/pinot/tools/data/generator/TemplateStringGenerator.java
@@ -0,0 +1,80 @@
+/**
+ * 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.tools.data.generator;
+
+import java.util.List;
+import java.util.Map;
+
+/**
+ * TemplateStringGenerator produces series of strings by cycling through a predefined list of values, optionally with
+ * a number of repetitions per value.
+ *
+ * Generator example:
+ * <pre>
+ *     values = [ "hello", "world" ]
+ *     repetitions = 2
+ *
+ *     returns [ "hello", "hello", "world", "world", "hello", ... ]
+ * </pre>
+ *
+ * Configuration examples:
+ * <ul>
+ *     <li>./pinot-tools/src/main/resources/generator/simpleWebsite_generator.json</li>
+ *     <li>./pinot-tools/src/main/resources/generator/complexWebsite_generator.json</li>
+ * </ul>
+ */
+public class TemplateStringGenerator implements Generator {
+    private final String[] values;
+    private final long repetitions;
+
+    private long step;
+
+    public TemplateStringGenerator(Map<String, Object> templateConfig) {
+        this(((List<String>) templateConfig.get("values")).toArray(new String[0]), toLong(templateConfig.get("repetitions"), 1));
+    }
+
+    public TemplateStringGenerator(String[] values, long repetitions) {
+        this.values = values;
+        this.repetitions = repetitions;
+    }
+
+    @Override
+    public void init() {
+        // left blank
+    }
+
+    @Override
+    public Object next() {
+        return values[(int) (step++ / repetitions) % values.length];
+    }
+
+    private static long toLong(Object obj, long defaultValue) {
+        if (obj == null) {
+            return defaultValue;
+        }
+        return Long.valueOf(obj.toString());
+    }
+
+    public static void main(String[] args) {
+        Generator gen = new TemplateStringGenerator(new String[] { "hello", "world", "!" }, 3);
+        for (int i = 0; i < 100; i++) {
+            System.out.println(gen.next());
+        }
+    }
+}
diff --git a/pinot-tools/src/main/java/org/apache/pinot/tools/data/generator/TemplateType.java b/pinot-tools/src/main/java/org/apache/pinot/tools/data/generator/TemplateType.java
new file mode 100644
index 0000000..deb2302
--- /dev/null
+++ b/pinot-tools/src/main/java/org/apache/pinot/tools/data/generator/TemplateType.java
@@ -0,0 +1,27 @@
+/**
+ * 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.tools.data.generator;
+
+public enum TemplateType {
+    SEASONAL,
+    SPIKE,
+    SEQUENCE,
+    STRING,
+    MIXTURE
+}
diff --git a/pinot-tools/src/main/resources/generator/README.md b/pinot-tools/src/main/resources/generator/README.md
new file mode 100644
index 0000000..bdf2f02
--- /dev/null
+++ b/pinot-tools/src/main/resources/generator/README.md
@@ -0,0 +1,85 @@
+<!--
+
+    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.
+
+-->
+# Generating and Loading Template Data
+
+Mock data has many use-cases from testing over benchmarking to portable application demos. The generator configs
+in this directory produce neat synthetic time series data of an imaginary website. You can generate gigabytes of mock
+data with these templates if you so desire.
+
+**simpleWebsite** generates non-dimensional data with views, clicks, and error count metrics
+
+**complexWebsite** generates similar metrics with a 3-dimensional breakdown across countries, browsers, and platforms
+
+The command line examples below are meant to be executed from the **pinot repository root**.
+(This was tested with pinot-quickstart in batch mode. Requires DefaultTenant and broker) 
+
+## Generate data via template
+This first step generates the raw data from a given template file. By default, we generate the data as CSV, and you can
+have a look manually with your favorite spreadsheet tool.
+
+(may require **rm -rf ./myTestData** to clear out existing mock data)
+
+```
+./pinot-tools/target/pinot-tools-pkg/bin/pinot-admin.sh GenerateData \
+-numFiles 1 -numRecords 354780  -format csv \
+-schemaFile ./pinot-tools/src/main/resources/generator/complexWebsite_schema.json \
+-schemaAnnotationFile ./pinot-tools/src/main/resources/generator/complexWebsite_generator.json \
+-outDir ./myTestData
+```
+
+## Generate Pinot Segment
+Now we turn the verbose CSV data into an efficiently packed segment ready for upload into pinot.
+
+```
+./pinot-tools/target/pinot-tools-pkg/bin/pinot-admin.sh CreateSegment \
+-tableName complexWebsite -segmentName complexWebsite -format CSV -overwrite \
+-schemaFile ./pinot-tools/src/main/resources/generator/complexWebsite_schema.json \
+-dataDir ./myTestData \
+-outDir ./myTestSegment 
+```
+
+## Create Pinot Table
+Before we push the segment, let's ensure that we have a table namespace ready. You can skip this step if you created a
+table earlier already.
+
+```
+./pinot-tools/target/pinot-tools-pkg/bin/pinot-admin.sh AddTable -exec \
+-tableConfigFile ./pinot-tools/src/main/resources/generator/complexWebsite_config.json \
+-schemaFile ./pinot-tools/src/main/resources/generator/complexWebsite_schema.json
+```
+
+## Upload Pinot Segment
+Now, we upload the segment. After this step, data should be available and query-able from the pinot console an any
+connected applications.
+
+```
+./pinot-tools/target/pinot-tools-pkg/bin/pinot-admin.sh UploadSegment \
+-tableName complexWebsite \
+-segmentDir ./myTestSegment
+```
+
+## Check data availability
+We can finally check data availability, e.g. by using pinot's built-in query console.
+If you're running a local pinot-quickstart image via docker the URL should be:
+
+```
+http://localhost:9000/query#
+```
diff --git a/pinot-tools/src/main/resources/generator/complexWebsite_config.json b/pinot-tools/src/main/resources/generator/complexWebsite_config.json
new file mode 100644
index 0000000..cbe7e5a
--- /dev/null
+++ b/pinot-tools/src/main/resources/generator/complexWebsite_config.json
@@ -0,0 +1,17 @@
+{
+  "tableName": "complexWebsite",
+  "segmentsConfig" : {
+    "replication" : "1",
+    "schemaName" : "complexWebsite"
+  },
+  "tableIndexConfig" : {
+    "invertedIndexColumns" : [],
+    "loadMode"  : "HEAP"
+  },
+  "tenants" : {
+    "broker":"DefaultTenant",
+    "server":"DefaultTenant"
+  },
+  "tableType":"OFFLINE",
+  "metadata": {}
+}
\ No newline at end of file
diff --git a/pinot-tools/src/main/resources/generator/complexWebsite_generator.json b/pinot-tools/src/main/resources/generator/complexWebsite_generator.json
new file mode 100644
index 0000000..94bb52c
--- /dev/null
+++ b/pinot-tools/src/main/resources/generator/complexWebsite_generator.json
@@ -0,0 +1,113 @@
+[
+  {
+    "column": "hoursSinceEpoch",
+    "template": {
+      "type": "SEQUENCE", "start": 420768, "stepsize": 1, "repetitions": 18
+    }
+  },
+  {
+    "column": "country",
+    "template": {
+      "type": "STRING", "values": [ "us", "cn", "in" ], "repetitions": 6
+    }
+  },
+  {
+    "column": "platform",
+    "template": {
+      "type": "STRING", "values": [ "desktop", "mobile" ], "repetitions": 3
+    }
+  },
+  {
+    "column": "browser",
+    "template": {
+      "type": "STRING", "values": [ "chrome", "safari", "firefox" ]
+    }
+  },
+  {
+    "column": "views",
+    "template": {
+      "type": "MIXTURE",
+      "defaults": {
+        "type": "SEASONAL", "wavelength": 24, "scalingFactors": [ 0.4, 0.9, 1.0, 1.0, 1.0, 0.8, 0.4 ]
+      },
+      "generatorBins": [
+        [ { "mean": 50, "sigma": 4.0, "amplitude": 40 } ],
+        [ { "mean": 25, "sigma": 2.5, "amplitude": 15 } ],
+        [ { "mean": 10, "sigma": 1.0, "amplitude":  5 } ],
+        [ { "mean": 90, "sigma": 6.0, "amplitude": 80 }, { "type": "SPIKE", "arrivalMean": 5, "arrivalSigma": 1, "magnitudeMean": 5, "magnitudeSigma": 1, "smoothing": 0.7 } ],
+        [ { "mean": 45, "sigma": 3.0, "amplitude": 35 } ],
+        [ { "mean": 20, "sigma": 1.5, "amplitude": 15 } ],
+        [ { "mean": 10, "sigma": 0.5, "amplitude":  8 } ],
+        [ { "mean": 25, "sigma": 2.5, "amplitude": 15 } ],
+        [ { "mean": 10, "sigma": 1.0, "amplitude":  5 } ],
+        [ { "mean": 20, "sigma": 1.0, "amplitude": 18 }, { "type": "SPIKE", "arrivalMean": 5, "arrivalSigma": 1, "magnitudeMean": 0.5, "magnitudeSigma": 0.5, "smoothing": 0.5 } ],
+        [ { "mean": 45, "sigma": 3.0, "amplitude": 35 } ],
+        [ { "mean": 20, "sigma": 1.5, "amplitude": 15 } ],
+        [ { "mean": 50, "sigma": 4.0, "amplitude": 40 } ],
+        [ { "mean": 10, "sigma": 1.0, "amplitude":  8 } ],
+        [ { "mean": 10, "sigma": 1.0, "amplitude":  5 } ],
+        [ { "mean": 90, "sigma": 6.0, "amplitude": 80 }, { "type": "SPIKE", "arrivalMean": 5, "arrivalSigma": 1, "magnitudeMean": 4, "magnitudeSigma": 1, "smoothing": 0.5 } ],
+        [ { "mean": 45, "sigma": 3.0, "amplitude": 35 } ],
+        [ { "mean": 10, "sigma": 0.5, "amplitude":  9 } ]
+      ]
+    }
+  },
+  {
+    "column": "clicks",
+    "template": {
+      "type": "MIXTURE",
+      "defaults": {
+        "type": "SEASONAL", "wavelength": 24, "scalingFactors": [ 0.5, 0.9, 1.0, 1.0, 1.0, 0.9, 0.5 ]
+      },
+      "generatorBins": [
+        [ { "mean": 12, "sigma": 1.0, "amplitude": 10 } ],
+        [ { "mean":  6, "sigma": 0.5, "amplitude":  5 } ],
+        [ { "mean":  2, "sigma": 0.3, "amplitude":  2 } ],
+        [ { "mean": 20, "sigma": 2.0, "amplitude": 18 }, { "type": "SPIKE", "arrivalMean": 4, "arrivalSigma": 1, "magnitudeMean": 2, "magnitudeSigma": 1, "smoothing": 0.5 } ],
+        [ { "mean":  9, "sigma": 1.0, "amplitude":  7 } ],
+        [ { "mean":  4, "sigma": 0.5, "amplitude":  3 } ],
+        [ { "mean":  3, "sigma": 0.5, "amplitude":  2 } ],
+        [ { "mean":  6, "sigma": 0.5, "amplitude":  5 } ],
+        [ { "mean":  2, "sigma": 0.3, "amplitude":  2 } ],
+        [ { "mean":  5, "sigma": 0.5, "amplitude":  4 }, { "type": "SPIKE", "arrivalMean": 5, "arrivalSigma": 1, "magnitudeMean": 0.5, "magnitudeSigma": 0.5, "smoothing": 0.3 } ],
+        [ { "mean":  9, "sigma": 1.0, "amplitude":  7 } ],
+        [ { "mean":  4, "sigma": 0.5, "amplitude":  3 } ],
+        [ { "mean": 12, "sigma": 1.0, "amplitude": 10 } ],
+        [ { "mean":  6, "sigma": 0.5, "amplitude":  5 } ],
+        [ { "mean":  2, "sigma": 0.3, "amplitude":  2 } ],
+        [ { "mean": 20, "sigma": 2.0, "amplitude": 18 }, { "type": "SPIKE", "arrivalMean": 5, "arrivalSigma": 1, "magnitudeMean": 3, "magnitudeSigma": 1, "smoothing": 0.3 } ],
+        [ { "mean":  9, "sigma": 1.0, "amplitude":  7 } ],
+        [ { "mean":  4, "sigma": 0.5, "amplitude":  3 } ]
+      ]
+    }
+  },
+  {
+    "column": "errors",
+    "template": {
+      "type": "MIXTURE",
+      "defaults": {
+        "type": "SPIKE", "arrivalMean": 4, "arrivalSigma": 1, "magnitudeMean": 2, "magnitudeSigma": 1, "smoothing": 0.3
+      },
+      "generatorBins": [
+        [ { "arrivalMean": 4, "magnitudeMean": 2.0 } ],
+        [ { "arrivalMean": 3, "magnitudeMean": 1.0 } ],
+        [ { "arrivalMean": 5, "magnitudeMean": 0.2 } ],
+        [ { "arrivalMean": 4, "magnitudeMean": 2.5 } ],
+        [ { "arrivalMean": 3, "magnitudeMean": 0.8 } ],
+        [ { "arrivalMean": 5, "magnitudeMean": 0.1 } ],
+        [ { "arrivalMean": 1, "magnitudeMean": 0.5 } ],
+        [ { "arrivalMean": 3, "magnitudeMean": 1.0 } ],
+        [ { "arrivalMean": 5, "magnitudeMean": 0.2 } ],
+        [ { "arrivalMean": 1, "magnitudeMean": 0.5 } ],
+        [ { "arrivalMean": 3, "magnitudeMean": 0.8 } ],
+        [ { "arrivalMean": 5, "magnitudeMean": 0.1 } ],
+        [ { "arrivalMean": 4, "magnitudeMean": 2.0 } ],
+        [ { "arrivalMean": 3, "magnitudeMean": 1.0 } ],
+        [ { "arrivalMean": 5, "magnitudeMean": 0.2 } ],
+        [ { "arrivalMean": 4, "magnitudeMean": 2.5 } ],
+        [ { "arrivalMean": 3, "magnitudeMean": 0.8 } ],
+        [ { "arrivalMean": 5, "magnitudeMean": 0.1 } ]
+      ]
+    }
+  }
+]
\ No newline at end of file
diff --git a/pinot-tools/src/main/resources/generator/complexWebsite_schema.json b/pinot-tools/src/main/resources/generator/complexWebsite_schema.json
new file mode 100644
index 0000000..99a349b
--- /dev/null
+++ b/pinot-tools/src/main/resources/generator/complexWebsite_schema.json
@@ -0,0 +1,38 @@
+{
+  "metricFieldSpecs": [
+    {
+      "dataType": "LONG",
+      "name": "views"
+    },
+    {
+      "dataType": "LONG",
+      "name": "clicks"
+    },
+    {
+      "dataType": "LONG",
+      "name": "errors"
+    }
+  ],
+  "dimensionFieldSpecs": [
+    {
+      "dataType": "STRING",
+      "name": "country"
+    },
+    {
+      "dataType": "STRING",
+      "name": "browser"
+    },
+    {
+      "dataType": "STRING",
+      "name": "platform"
+    }
+  ],
+  "timeFieldSpec": {
+    "incomingGranularitySpec": {
+      "timeType": "HOURS",
+      "dataType": "LONG",
+      "name": "hoursSinceEpoch"
+    }
+  },
+  "schemaName": "complexWebsite"
+}
diff --git a/pinot-tools/src/main/resources/generator/simpleWebsite_config.json b/pinot-tools/src/main/resources/generator/simpleWebsite_config.json
new file mode 100644
index 0000000..66f9cbe
--- /dev/null
+++ b/pinot-tools/src/main/resources/generator/simpleWebsite_config.json
@@ -0,0 +1,17 @@
+{
+  "tableName": "simpleWebsite",
+  "segmentsConfig" : {
+    "replication" : "1",
+    "schemaName" : "simpleWebsite"
+  },
+  "tableIndexConfig" : {
+    "invertedIndexColumns" : [],
+    "loadMode"  : "HEAP"
+  },
+  "tenants" : {
+    "broker":"DefaultTenant",
+    "server":"DefaultTenant"
+  },
+  "tableType":"OFFLINE",
+  "metadata": {}
+}
\ No newline at end of file
diff --git a/pinot-tools/src/main/resources/generator/simpleWebsite_generator.json b/pinot-tools/src/main/resources/generator/simpleWebsite_generator.json
new file mode 100644
index 0000000..b02bb71
--- /dev/null
+++ b/pinot-tools/src/main/resources/generator/simpleWebsite_generator.json
@@ -0,0 +1,44 @@
+[
+  {
+    "column": "hoursSinceEpoch",
+    "template": {
+      "type": "SEQUENCE",
+      "start": 420768,
+      "stepsize": 1
+    }
+  },
+  {
+    "column": "views",
+    "template": {
+      "type": "SEASONAL",
+      "mean": 80,
+      "sigma": 1.5,
+      "trend": 0.005,
+      "wavelength": 24,
+      "amplitude": 40
+    }
+  },
+  {
+    "column": "clicks",
+    "template": {
+      "type": "SEASONAL",
+      "mean": 20,
+      "sigma": 1,
+      "trend": 0.001,
+      "wavelength": 24,
+      "amplitude": 5
+    }
+  },
+  {
+    "column": "errors",
+    "template": {
+      "type": "SPIKE",
+      "baseline": 0,
+      "arrivalMean": 2,
+      "arrivalSigma": 1,
+      "magnitudeMean": 3,
+      "magnitudeSigma": 1,
+      "smoothing": 0.1
+    }
+  }
+]
\ No newline at end of file
diff --git a/pinot-tools/src/main/resources/generator/simpleWebsite_schema.json b/pinot-tools/src/main/resources/generator/simpleWebsite_schema.json
new file mode 100644
index 0000000..c7c015e
--- /dev/null
+++ b/pinot-tools/src/main/resources/generator/simpleWebsite_schema.json
@@ -0,0 +1,26 @@
+{
+  "metricFieldSpecs": [
+    {
+      "dataType": "LONG",
+      "name": "views"
+    },
+    {
+      "dataType": "LONG",
+      "name": "clicks"
+    },
+    {
+      "dataType": "LONG",
+      "name": "errors"
+    }
+  ],
+  "dimensionFieldSpecs": [
+  ],
+  "timeFieldSpec": {
+    "incomingGranularitySpec": {
+      "timeType": "HOURS",
+      "dataType": "LONG",
+      "name": "hoursSinceEpoch"
+    }
+  },
+  "schemaName": "simpleWebsite"
+}


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