You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@beam.apache.org by "ASF GitHub Bot (JIRA)" <ji...@apache.org> on 2018/05/07 04:04:01 UTC

[jira] [Work logged] (BEAM-2953) Create more advanced Timeseries processing examples using state API

     [ https://issues.apache.org/jira/browse/BEAM-2953?focusedWorklogId=98864&page=com.atlassian.jira.plugin.system.issuetabpanels:worklog-tabpanel#worklog-98864 ]

ASF GitHub Bot logged work on BEAM-2953:
----------------------------------------

                Author: ASF GitHub Bot
            Created on: 07/May/18 04:03
            Start Date: 07/May/18 04:03
    Worklog Time Spent: 10m 
      Work Description: rezarokni closed pull request #4116: [BEAM-2953] Part 1 of Multipart advanced timeseries examples
URL: https://github.com/apache/beam/pull/4116
 
 
   

This is a PR merged from a forked repository.
As GitHub hides the original diff on merge, it is displayed below for
the sake of provenance:

As this is a foreign pull request (from a fork), the diff is supplied
below (as it won't show otherwise due to GitHub magic):

diff --git a/examples/java8/src/main/java/org/apache/beam/examples/advanced/timeseries/configuration/TSConfiguration.java b/examples/java8/src/main/java/org/apache/beam/examples/advanced/timeseries/configuration/TSConfiguration.java
new file mode 100644
index 00000000000..9119fbd423d
--- /dev/null
+++ b/examples/java8/src/main/java/org/apache/beam/examples/advanced/timeseries/configuration/TSConfiguration.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.beam.examples.advanced.timeseries.configuration;
+
+import com.google.auto.value.AutoValue;
+
+import java.io.Serializable;
+
+import org.apache.beam.sdk.annotations.Experimental;
+import org.joda.time.Duration;
+import org.joda.time.Instant;
+
+/**
+ * Configuration options for dealing with time series.
+ */
+@SuppressWarnings("serial")
+@Experimental
+@AutoValue
+public abstract class TSConfiguration implements Serializable {
+
+public static final String HEARTBEAT = "HB";
+
+/**
+ * Used to determine how empty values should be populated.
+ * Currently not implemented
+ */
+public enum FillOptions {
+NONE, LAST_SEEN_WINDOW
+}
+public abstract FillOptions fillOption();
+
+// The down sample period which must be set.
+public abstract Duration downSampleDuration();
+// Set if this is a streaming pipeline.
+public abstract boolean isStreaming();
+// The start time which defines when back fill starts.
+public abstract Instant startTime();
+// The end time of if this is Batch mode, do not set for stream mode.
+public abstract Instant endTime();
+
+abstract Builder toBuilder();
+
+/**
+* Set start time to NOW in the absence of any other data.
+* Set the back fill option to create Zero or Null by default
+*/
+public static Builder builder() {
+  return new AutoValue_TSConfiguration.Builder().setStartTime(Instant.now())
+      .setFillOption(TSConfiguration.FillOptions.NONE);
+}
+
+/**
+ * Builder.
+ */
+@AutoValue.Builder
+public abstract static class Builder {
+  public abstract Builder setFillOption(FillOptions fillOption);
+  public abstract Builder setDownSampleDuration(Duration downSampleDuration);
+  public abstract Builder setIsStreaming(boolean isStreaming);
+  public abstract Builder setStartTime(Instant startTime);
+  public abstract Builder setEndTime(Instant endTime);
+  public abstract TSConfiguration build();
+  }
+
+}
diff --git a/examples/java8/src/main/java/org/apache/beam/examples/advanced/timeseries/pipelines/TimeseriesFillPipeline.java b/examples/java8/src/main/java/org/apache/beam/examples/advanced/timeseries/pipelines/TimeseriesFillPipeline.java
new file mode 100644
index 00000000000..9d87aaade39
--- /dev/null
+++ b/examples/java8/src/main/java/org/apache/beam/examples/advanced/timeseries/pipelines/TimeseriesFillPipeline.java
@@ -0,0 +1,187 @@
+/*
+ * 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.beam.examples.advanced.timeseries.pipelines;
+
+import com.google.protobuf.Timestamp;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.beam.examples.advanced.timeseries.configuration.TSConfiguration;
+import org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos;
+import org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSDataPoint;
+import org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSDataPointSequence.Builder;
+import org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSKey;
+import org.apache.beam.examples.advanced.timeseries.transform.library.BackFillAllWindowsAndKeys;
+import org.apache.beam.examples.advanced.timeseries.transform.library.Utils;
+import org.apache.beam.sdk.Pipeline;
+import org.apache.beam.sdk.options.PipelineOptions;
+import org.apache.beam.sdk.options.PipelineOptionsFactory;
+import org.apache.beam.sdk.transforms.Create;
+import org.apache.beam.sdk.transforms.DoFn;
+import org.apache.beam.sdk.transforms.GroupByKey;
+import org.apache.beam.sdk.transforms.ParDo;
+import org.apache.beam.sdk.transforms.Sum;
+import org.apache.beam.sdk.transforms.View;
+import org.apache.beam.sdk.transforms.windowing.IntervalWindow;
+import org.apache.beam.sdk.values.KV;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.PCollectionView;
+import org.joda.time.Duration;
+import org.joda.time.Instant;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * This pipeline demonstrates how you can backfill.
+ */
+public class TimeseriesFillPipeline {
+
+  private static final Logger LOG = LoggerFactory.getLogger(TimeseriesFillPipeline.class);
+
+  @SuppressWarnings("serial")
+  public static void main(String[] args) {
+
+    // Setup start and end time for example Pipeline
+
+    Instant startTime = new Instant("2000-01-01T00:00:00");
+
+    Instant endTime = startTime.plus(Duration.standardSeconds(60));
+
+    Duration downSampleDuration = Duration.standardSeconds(20);
+
+    // Setup configuration
+    TSConfiguration configuration = TSConfiguration.builder().setStartTime(startTime)
+        .setEndTime(endTime).setDownSampleDuration(downSampleDuration)
+        .setFillOption(TSConfiguration.FillOptions.NONE).setIsStreaming(false).build();
+
+    // Create pipeline
+    PipelineOptions options = PipelineOptionsFactory.create();
+    Pipeline p = Pipeline.create(options);
+
+    // Define a view of all possible keys
+    PCollectionView<List<String>> allKeys =
+        p.apply(Create.of("TS1", "TS2", "TS3")).apply(View.asList());
+
+    // Create 3 mock time series
+    PCollection<TSDataPoint> dataPoints = p.apply(Create.of(createMockTimeSeries(configuration)));
+
+    // As this example is not using a unbounded source which will set the
+    // time stamp we will manually set it
+    PCollection<TSDataPoint> dataPointsWithTimestamp = dataPoints.apply(Utils.extractTimeStamp());
+
+    // Generate a tick for each window for each key even if there was no
+    // data
+    PCollection<TSDataPoint> dataPointsWithbackFill =
+        dataPointsWithTimestamp.apply(new BackFillAllWindowsAndKeys(configuration, allKeys));
+
+    dataPointsWithbackFill.apply(ParDo.of(new DoFn<TSDataPoint, KV<String, TSDataPoint>>() {
+
+      // Print the output of the back fill
+      // In order to print all data from all collections we need to
+      // re-key and re-window into a Global Window
+
+      @ProcessElement
+      public void process(ProcessContext c) {
+        c.output(KV.of(c.element().getKey().getKey(), c.element()));
+      }
+    })).apply(GroupByKey.create())
+        .apply(ParDo.of(new DoFn<KV<String, Iterable<TSDataPoint>>, KV<String, Double>>() {
+
+          @ProcessElement
+          public void process(ProcessContext c, IntervalWindow w) {
+
+            StringBuffer sb = new StringBuffer();
+
+            sb.append(String.format("Key is %s Time Window is %s \n", c.element().getKey(),
+                w.toString()));
+
+            for (TSDataPoint ts : c.element().getValue()) {
+
+              Builder list = TimeSeriesProtos.TSDataPointSequence.newBuilder();
+
+              list.addAccums(ts);
+
+              c.output(KV.of(c.element().getKey(), ts.getDoubleVal()));
+
+              sb.append(String.format("Time is %s Value is %s  is heart beat? %s \n",
+                  ts.getDoubleVal(), new Instant(ts.getTimestamp().getSeconds() * 1000),
+                  ts.containsMetadata(TSConfiguration.HEARTBEAT)));
+            }
+
+            LOG.info(sb.toString());
+          }
+        })).apply(Sum.doublesPerKey())
+        .apply(ParDo.of(new DoFn<KV<String, Double>, KV<String, Double>>() {
+
+          // Print the per wind
+          // In order to print all data from all collections we need to
+          // re-key and re-window into a Global Window
+
+          @ProcessElement
+          public void process(ProcessContext c, IntervalWindow w) {
+
+            LOG.info(String.format(" Sum of values for key %s in window %s was %s",
+                c.element().getKey(), w.toString(), c.element().getValue()));
+
+          }
+        }));
+
+    p.run();
+  }
+
+  /**
+   * Create three mock time series with missing ticks.
+   * @param configuration
+   * @return List
+   */
+  public static List<TSDataPoint> createMockTimeSeries(TSConfiguration configuration) {
+
+    // Create Time series
+    List<TSDataPoint> ts1 = new ArrayList<>();
+
+    double numElements =
+        Math.floor(configuration.endTime().getMillis() - configuration.startTime().getMillis())
+            / 5000;
+
+    long startTime = configuration.startTime().getMillis();
+
+    // Create a Data point every 5 seconds for TS1
+    // Remove a data point at position 4,5,6,7
+    for (long i = 0; i < numElements; i++) {
+      if (i < 4 || i > 7) {
+        ts1.add(TSDataPoint.newBuilder().setKey(TSKey.newBuilder().setKey("TS1")).setDoubleVal(i)
+            .setTimestamp(Timestamp.newBuilder().setSeconds((startTime / 1000) + (5 * i))).build());
+      }
+    }
+
+    // Create a Data point every 5 seconds for TS2
+    // Remove data point at min 0
+    for (long i = 0; i < numElements; i++) {
+      if (i != 0) {
+        ts1.add(TSDataPoint.newBuilder().setKey(TSKey.newBuilder().setKey("TS2"))
+            .setDoubleVal(i + 100)
+            .setTimestamp(Timestamp.newBuilder().setSeconds((startTime / 1000) + (5 * i))).build());
+      }
+    }
+
+    // Add only one value at point 0
+    ts1.add(TSDataPoint.newBuilder().setKey(TSKey.newBuilder().setKey("TS3")).setDoubleVal(1000)
+        .setTimestamp(Timestamp.newBuilder().setSeconds((startTime / 1000))).build());
+
+    return ts1;
+  }
+
+}
diff --git a/examples/java8/src/main/java/org/apache/beam/examples/advanced/timeseries/protos/TimeSeries.proto b/examples/java8/src/main/java/org/apache/beam/examples/advanced/timeseries/protos/TimeSeries.proto
new file mode 100644
index 00000000000..eba8b9ae352
--- /dev/null
+++ b/examples/java8/src/main/java/org/apache/beam/examples/advanced/timeseries/protos/TimeSeries.proto
@@ -0,0 +1,90 @@
+/*
+ * 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.
+ */
+syntax = "proto3";
+
+import "google/protobuf/timestamp.proto";
+option java_outer_classname = "TimeSeriesProtos";
+option java_multiple_files = false;
+option java_package = "org.apache.beam.examples.advanced.timeseries.protos";
+package protos;
+
+message TSKey {
+	string key = 1;
+}
+
+// Generic TS data point
+message TSDataPoint {
+	TSKey key = 1;
+	oneof value {
+	// DT_FLOAT.
+		float float_val = 3;
+
+		// DT_DOUBLE.
+		double double_val = 4;
+
+		// DT_INT32, DT_INT16, DT_INT8, DT_UINT8.
+		int32 int_val = 5;
+	}
+	google.protobuf.Timestamp timestamp = 7;
+	map < string, string > metadata = 8;
+}
+
+message TSDataSchema {
+	TSKey key = 1;
+	map < string, TSKey > features = 2;
+}
+
+// Time Series sequence holder
+message TSDataPointSequence {
+	TSKey key = 1;
+	PipelineVersion pipelineVersion = 2;
+	repeated TSDataPoint accums = 3;
+	int64 lowerWindowBoundary = 8;
+	int64 upperWindowBoundary = 9;
+}
+
+message PipelineVersion {
+	string majorVersion = 1;
+	string minorVersion = 2;
+	string pipelineName = 3;
+	string transformName = 4;
+}
+
+enum DownSampleType {
+	SUM = 0;
+	FIRST = 1;
+	LAST = 2;
+	AVERAGE = 3;
+}
+
+// Accumulator used in combiners
+message TSAccum {
+	TSKey key = 1;
+	PipelineVersion pipelineVersion = 2;
+	TSDataPoint data = 3;
+	int64 durationMills = 4;
+	google.protobuf.Timestamp lowerWindowBoundary = 5;
+	google.protobuf.Timestamp upperWindowBoundary = 6;
+}
+
+// Time Series sequence holder
+message TSAccumSequence {
+	TSKey key = 1;
+	PipelineVersion pipelineVersion = 2;
+	repeated TSAccum accums = 3;
+	google.protobuf.Timestamp lowerWindowBoundary = 4;
+	google.protobuf.Timestamp upperWindowBoundary = 5;
+}
+
diff --git a/examples/java8/src/main/java/org/apache/beam/examples/advanced/timeseries/protos/TimeSeriesProtos.java b/examples/java8/src/main/java/org/apache/beam/examples/advanced/timeseries/protos/TimeSeriesProtos.java
new file mode 100644
index 00000000000..c2aebd69dda
--- /dev/null
+++ b/examples/java8/src/main/java/org/apache/beam/examples/advanced/timeseries/protos/TimeSeriesProtos.java
@@ -0,0 +1,8230 @@
+/*
+ * 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.
+ */
+// Generated by the protocol buffer compiler.  DO NOT EDIT!
+// source: TimeSeries.proto
+
+package org.apache.beam.examples.advanced.timeseries.protos;
+
+public final class TimeSeriesProtos {
+  private TimeSeriesProtos() {}
+  public static void registerAllExtensions(
+      com.google.protobuf.ExtensionRegistryLite registry) {
+  }
+
+  public static void registerAllExtensions(
+      com.google.protobuf.ExtensionRegistry registry) {
+    registerAllExtensions(
+        (com.google.protobuf.ExtensionRegistryLite) registry);
+  }
+  /**
+   * Protobuf enum {@code protos.DownSampleType}
+   */
+  public enum DownSampleType
+      implements com.google.protobuf.ProtocolMessageEnum {
+    /**
+     * <code>SUM = 0;</code>
+     */
+    SUM(0),
+    /**
+     * <code>FIRST = 1;</code>
+     */
+    FIRST(1),
+    /**
+     * <code>LAST = 2;</code>
+     */
+    LAST(2),
+    /**
+     * <code>AVERAGE = 3;</code>
+     */
+    AVERAGE(3),
+    UNRECOGNIZED(-1),
+    ;
+
+    /**
+     * <code>SUM = 0;</code>
+     */
+    public static final int SUM_VALUE = 0;
+    /**
+     * <code>FIRST = 1;</code>
+     */
+    public static final int FIRST_VALUE = 1;
+    /**
+     * <code>LAST = 2;</code>
+     */
+    public static final int LAST_VALUE = 2;
+    /**
+     * <code>AVERAGE = 3;</code>
+     */
+    public static final int AVERAGE_VALUE = 3;
+
+
+    public final int getNumber() {
+      if (this == UNRECOGNIZED) {
+        throw new java.lang.IllegalArgumentException(
+            "Can't get the number of an unknown enum value.");
+      }
+      return value;
+    }
+
+    /**
+     * @deprecated Use {@link #forNumber(int)} instead.
+     */
+    @java.lang.Deprecated
+    public static DownSampleType valueOf(int value) {
+      return forNumber(value);
+    }
+
+    public static DownSampleType forNumber(int value) {
+      switch (value) {
+        case 0: return SUM;
+        case 1: return FIRST;
+        case 2: return LAST;
+        case 3: return AVERAGE;
+        default: return null;
+      }
+    }
+
+    public static com.google.protobuf.Internal.EnumLiteMap<DownSampleType>
+        internalGetValueMap() {
+      return internalValueMap;
+    }
+    private static final com.google.protobuf.Internal.EnumLiteMap<
+        DownSampleType> internalValueMap =
+          new com.google.protobuf.Internal.EnumLiteMap<DownSampleType>() {
+            public DownSampleType findValueByNumber(int number) {
+              return DownSampleType.forNumber(number);
+            }
+          };
+
+    public final com.google.protobuf.Descriptors.EnumValueDescriptor
+        getValueDescriptor() {
+      return getDescriptor().getValues().get(ordinal());
+    }
+    public final com.google.protobuf.Descriptors.EnumDescriptor
+        getDescriptorForType() {
+      return getDescriptor();
+    }
+    public static final com.google.protobuf.Descriptors.EnumDescriptor
+        getDescriptor() {
+      return org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.getDescriptor().getEnumTypes().get(0);
+    }
+
+    private static final DownSampleType[] VALUES = values();
+
+    public static DownSampleType valueOf(
+        com.google.protobuf.Descriptors.EnumValueDescriptor desc) {
+      if (desc.getType() != getDescriptor()) {
+        throw new java.lang.IllegalArgumentException(
+          "EnumValueDescriptor is not for this type.");
+      }
+      if (desc.getIndex() == -1) {
+        return UNRECOGNIZED;
+      }
+      return VALUES[desc.getIndex()];
+    }
+
+    private final int value;
+
+    private DownSampleType(int value) {
+      this.value = value;
+    }
+
+    // @@protoc_insertion_point(enum_scope:protos.DownSampleType)
+  }
+
+  public interface TSKeyOrBuilder extends
+      // @@protoc_insertion_point(interface_extends:protos.TSKey)
+      com.google.protobuf.MessageOrBuilder {
+
+    /**
+     * <code>string key = 1;</code>
+     */
+    java.lang.String getKey();
+    /**
+     * <code>string key = 1;</code>
+     */
+    com.google.protobuf.ByteString
+        getKeyBytes();
+  }
+  /**
+   * Protobuf type {@code protos.TSKey}
+   */
+  public  static final class TSKey extends
+      com.google.protobuf.GeneratedMessageV3 implements
+      // @@protoc_insertion_point(message_implements:protos.TSKey)
+      TSKeyOrBuilder {
+    // Use TSKey.newBuilder() to construct.
+    private TSKey(com.google.protobuf.GeneratedMessageV3.Builder<?> builder) {
+      super(builder);
+    }
+    private TSKey() {
+      key_ = "";
+    }
+
+    @java.lang.Override
+    public final com.google.protobuf.UnknownFieldSet
+    getUnknownFields() {
+      return com.google.protobuf.UnknownFieldSet.getDefaultInstance();
+    }
+    private TSKey(
+        com.google.protobuf.CodedInputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      this();
+      int mutable_bitField0_ = 0;
+      try {
+        boolean done = false;
+        while (!done) {
+          int tag = input.readTag();
+          switch (tag) {
+            case 0:
+              done = true;
+              break;
+            default: {
+              if (!input.skipField(tag)) {
+                done = true;
+              }
+              break;
+            }
+            case 10: {
+              java.lang.String s = input.readStringRequireUtf8();
+
+              key_ = s;
+              break;
+            }
+          }
+        }
+      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        throw e.setUnfinishedMessage(this);
+      } catch (java.io.IOException e) {
+        throw new com.google.protobuf.InvalidProtocolBufferException(
+            e).setUnfinishedMessage(this);
+      } finally {
+        makeExtensionsImmutable();
+      }
+    }
+    public static final com.google.protobuf.Descriptors.Descriptor
+        getDescriptor() {
+      return org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.internal_static_protos_TSKey_descriptor;
+    }
+
+    protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+        internalGetFieldAccessorTable() {
+      return org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.internal_static_protos_TSKey_fieldAccessorTable
+          .ensureFieldAccessorsInitialized(
+              org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSKey.class, org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSKey.Builder.class);
+    }
+
+    public static final int KEY_FIELD_NUMBER = 1;
+    private volatile java.lang.Object key_;
+    /**
+     * <code>string key = 1;</code>
+     */
+    public java.lang.String getKey() {
+      java.lang.Object ref = key_;
+      if (ref instanceof java.lang.String) {
+        return (java.lang.String) ref;
+      } else {
+        com.google.protobuf.ByteString bs = 
+            (com.google.protobuf.ByteString) ref;
+        java.lang.String s = bs.toStringUtf8();
+        key_ = s;
+        return s;
+      }
+    }
+    /**
+     * <code>string key = 1;</code>
+     */
+    public com.google.protobuf.ByteString
+        getKeyBytes() {
+      java.lang.Object ref = key_;
+      if (ref instanceof java.lang.String) {
+        com.google.protobuf.ByteString b = 
+            com.google.protobuf.ByteString.copyFromUtf8(
+                (java.lang.String) ref);
+        key_ = b;
+        return b;
+      } else {
+        return (com.google.protobuf.ByteString) ref;
+      }
+    }
+
+    private byte memoizedIsInitialized = -1;
+    public final boolean isInitialized() {
+      byte isInitialized = memoizedIsInitialized;
+      if (isInitialized == 1) return true;
+      if (isInitialized == 0) return false;
+
+      memoizedIsInitialized = 1;
+      return true;
+    }
+
+    public void writeTo(com.google.protobuf.CodedOutputStream output)
+                        throws java.io.IOException {
+      if (!getKeyBytes().isEmpty()) {
+        com.google.protobuf.GeneratedMessageV3.writeString(output, 1, key_);
+      }
+    }
+
+    public int getSerializedSize() {
+      int size = memoizedSize;
+      if (size != -1) return size;
+
+      size = 0;
+      if (!getKeyBytes().isEmpty()) {
+        size += com.google.protobuf.GeneratedMessageV3.computeStringSize(1, key_);
+      }
+      memoizedSize = size;
+      return size;
+    }
+
+    private static final long serialVersionUID = 0L;
+    @java.lang.Override
+    public boolean equals(final java.lang.Object obj) {
+      if (obj == this) {
+       return true;
+      }
+      if (!(obj instanceof org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSKey)) {
+        return super.equals(obj);
+      }
+      org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSKey other = (org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSKey) obj;
+
+      boolean result = true;
+      result = result && getKey()
+          .equals(other.getKey());
+      return result;
+    }
+
+    @java.lang.Override
+    public int hashCode() {
+      if (memoizedHashCode != 0) {
+        return memoizedHashCode;
+      }
+      int hash = 41;
+      hash = (19 * hash) + getDescriptor().hashCode();
+      hash = (37 * hash) + KEY_FIELD_NUMBER;
+      hash = (53 * hash) + getKey().hashCode();
+      hash = (29 * hash) + unknownFields.hashCode();
+      memoizedHashCode = hash;
+      return hash;
+    }
+
+    public static org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSKey parseFrom(
+        com.google.protobuf.ByteString data)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data);
+    }
+    public static org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSKey parseFrom(
+        com.google.protobuf.ByteString data,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data, extensionRegistry);
+    }
+    public static org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSKey parseFrom(byte[] data)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data);
+    }
+    public static org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSKey parseFrom(
+        byte[] data,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data, extensionRegistry);
+    }
+    public static org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSKey parseFrom(java.io.InputStream input)
+        throws java.io.IOException {
+      return com.google.protobuf.GeneratedMessageV3
+          .parseWithIOException(PARSER, input);
+    }
+    public static org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSKey parseFrom(
+        java.io.InputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws java.io.IOException {
+      return com.google.protobuf.GeneratedMessageV3
+          .parseWithIOException(PARSER, input, extensionRegistry);
+    }
+    public static org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSKey parseDelimitedFrom(java.io.InputStream input)
+        throws java.io.IOException {
+      return com.google.protobuf.GeneratedMessageV3
+          .parseDelimitedWithIOException(PARSER, input);
+    }
+    public static org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSKey parseDelimitedFrom(
+        java.io.InputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws java.io.IOException {
+      return com.google.protobuf.GeneratedMessageV3
+          .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
+    }
+    public static org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSKey parseFrom(
+        com.google.protobuf.CodedInputStream input)
+        throws java.io.IOException {
+      return com.google.protobuf.GeneratedMessageV3
+          .parseWithIOException(PARSER, input);
+    }
+    public static org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSKey parseFrom(
+        com.google.protobuf.CodedInputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws java.io.IOException {
+      return com.google.protobuf.GeneratedMessageV3
+          .parseWithIOException(PARSER, input, extensionRegistry);
+    }
+
+    public Builder newBuilderForType() { return newBuilder(); }
+    public static Builder newBuilder() {
+      return DEFAULT_INSTANCE.toBuilder();
+    }
+    public static Builder newBuilder(org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSKey prototype) {
+      return DEFAULT_INSTANCE.toBuilder().mergeFrom(prototype);
+    }
+    public Builder toBuilder() {
+      return this == DEFAULT_INSTANCE
+          ? new Builder() : new Builder().mergeFrom(this);
+    }
+
+    @java.lang.Override
+    protected Builder newBuilderForType(
+        com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+      Builder builder = new Builder(parent);
+      return builder;
+    }
+    /**
+     * Protobuf type {@code protos.TSKey}
+     */
+    public static final class Builder extends
+        com.google.protobuf.GeneratedMessageV3.Builder<Builder> implements
+        // @@protoc_insertion_point(builder_implements:protos.TSKey)
+        org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSKeyOrBuilder {
+      public static final com.google.protobuf.Descriptors.Descriptor
+          getDescriptor() {
+        return org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.internal_static_protos_TSKey_descriptor;
+      }
+
+      protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+          internalGetFieldAccessorTable() {
+        return org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.internal_static_protos_TSKey_fieldAccessorTable
+            .ensureFieldAccessorsInitialized(
+                org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSKey.class, org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSKey.Builder.class);
+      }
+
+      // Construct using org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSKey.newBuilder()
+      private Builder() {
+        maybeForceBuilderInitialization();
+      }
+
+      private Builder(
+          com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+        super(parent);
+        maybeForceBuilderInitialization();
+      }
+      private void maybeForceBuilderInitialization() {
+        if (com.google.protobuf.GeneratedMessageV3
+                .alwaysUseFieldBuilders) {
+        }
+      }
+      public Builder clear() {
+        super.clear();
+        key_ = "";
+
+        return this;
+      }
+
+      public com.google.protobuf.Descriptors.Descriptor
+          getDescriptorForType() {
+        return org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.internal_static_protos_TSKey_descriptor;
+      }
+
+      public org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSKey getDefaultInstanceForType() {
+        return org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSKey.getDefaultInstance();
+      }
+
+      public org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSKey build() {
+        org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSKey result = buildPartial();
+        if (!result.isInitialized()) {
+          throw newUninitializedMessageException(result);
+        }
+        return result;
+      }
+
+      public org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSKey buildPartial() {
+        org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSKey result = new org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSKey(this);
+        result.key_ = key_;
+        onBuilt();
+        return result;
+      }
+
+      public Builder clone() {
+        return (Builder) super.clone();
+      }
+      public Builder setField(
+          com.google.protobuf.Descriptors.FieldDescriptor field,
+          Object value) {
+        return (Builder) super.setField(field, value);
+      }
+      public Builder clearField(
+          com.google.protobuf.Descriptors.FieldDescriptor field) {
+        return (Builder) super.clearField(field);
+      }
+      public Builder clearOneof(
+          com.google.protobuf.Descriptors.OneofDescriptor oneof) {
+        return (Builder) super.clearOneof(oneof);
+      }
+      public Builder setRepeatedField(
+          com.google.protobuf.Descriptors.FieldDescriptor field,
+          int index, Object value) {
+        return (Builder) super.setRepeatedField(field, index, value);
+      }
+      public Builder addRepeatedField(
+          com.google.protobuf.Descriptors.FieldDescriptor field,
+          Object value) {
+        return (Builder) super.addRepeatedField(field, value);
+      }
+      public Builder mergeFrom(com.google.protobuf.Message other) {
+        if (other instanceof org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSKey) {
+          return mergeFrom((org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSKey)other);
+        } else {
+          super.mergeFrom(other);
+          return this;
+        }
+      }
+
+      public Builder mergeFrom(org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSKey other) {
+        if (other == org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSKey.getDefaultInstance()) return this;
+        if (!other.getKey().isEmpty()) {
+          key_ = other.key_;
+          onChanged();
+        }
+        onChanged();
+        return this;
+      }
+
+      public final boolean isInitialized() {
+        return true;
+      }
+
+      public Builder mergeFrom(
+          com.google.protobuf.CodedInputStream input,
+          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws java.io.IOException {
+        org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSKey parsedMessage = null;
+        try {
+          parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
+        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+          parsedMessage = (org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSKey) e.getUnfinishedMessage();
+          throw e.unwrapIOException();
+        } finally {
+          if (parsedMessage != null) {
+            mergeFrom(parsedMessage);
+          }
+        }
+        return this;
+      }
+
+      private java.lang.Object key_ = "";
+      /**
+       * <code>string key = 1;</code>
+       */
+      public java.lang.String getKey() {
+        java.lang.Object ref = key_;
+        if (!(ref instanceof java.lang.String)) {
+          com.google.protobuf.ByteString bs =
+              (com.google.protobuf.ByteString) ref;
+          java.lang.String s = bs.toStringUtf8();
+          key_ = s;
+          return s;
+        } else {
+          return (java.lang.String) ref;
+        }
+      }
+      /**
+       * <code>string key = 1;</code>
+       */
+      public com.google.protobuf.ByteString
+          getKeyBytes() {
+        java.lang.Object ref = key_;
+        if (ref instanceof String) {
+          com.google.protobuf.ByteString b = 
+              com.google.protobuf.ByteString.copyFromUtf8(
+                  (java.lang.String) ref);
+          key_ = b;
+          return b;
+        } else {
+          return (com.google.protobuf.ByteString) ref;
+        }
+      }
+      /**
+       * <code>string key = 1;</code>
+       */
+      public Builder setKey(
+          java.lang.String value) {
+        if (value == null) {
+    throw new NullPointerException();
+  }
+  
+        key_ = value;
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>string key = 1;</code>
+       */
+      public Builder clearKey() {
+        
+        key_ = getDefaultInstance().getKey();
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>string key = 1;</code>
+       */
+      public Builder setKeyBytes(
+          com.google.protobuf.ByteString value) {
+        if (value == null) {
+    throw new NullPointerException();
+  }
+  checkByteStringIsUtf8(value);
+        
+        key_ = value;
+        onChanged();
+        return this;
+      }
+      public final Builder setUnknownFields(
+          final com.google.protobuf.UnknownFieldSet unknownFields) {
+        return this;
+      }
+
+      public final Builder mergeUnknownFields(
+          final com.google.protobuf.UnknownFieldSet unknownFields) {
+        return this;
+      }
+
+
+      // @@protoc_insertion_point(builder_scope:protos.TSKey)
+    }
+
+    // @@protoc_insertion_point(class_scope:protos.TSKey)
+    private static final org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSKey DEFAULT_INSTANCE;
+    static {
+      DEFAULT_INSTANCE = new org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSKey();
+    }
+
+    public static org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSKey getDefaultInstance() {
+      return DEFAULT_INSTANCE;
+    }
+
+    private static final com.google.protobuf.Parser<TSKey>
+        PARSER = new com.google.protobuf.AbstractParser<TSKey>() {
+      public TSKey parsePartialFrom(
+          com.google.protobuf.CodedInputStream input,
+          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws com.google.protobuf.InvalidProtocolBufferException {
+          return new TSKey(input, extensionRegistry);
+      }
+    };
+
+    public static com.google.protobuf.Parser<TSKey> parser() {
+      return PARSER;
+    }
+
+    @java.lang.Override
+    public com.google.protobuf.Parser<TSKey> getParserForType() {
+      return PARSER;
+    }
+
+    public org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSKey getDefaultInstanceForType() {
+      return DEFAULT_INSTANCE;
+    }
+
+  }
+
+  public interface TSDataPointOrBuilder extends
+      // @@protoc_insertion_point(interface_extends:protos.TSDataPoint)
+      com.google.protobuf.MessageOrBuilder {
+
+    /**
+     * <code>.protos.TSKey key = 1;</code>
+     */
+    boolean hasKey();
+    /**
+     * <code>.protos.TSKey key = 1;</code>
+     */
+    org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSKey getKey();
+    /**
+     * <code>.protos.TSKey key = 1;</code>
+     */
+    org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSKeyOrBuilder getKeyOrBuilder();
+
+    /**
+     * <pre>
+     * DT_FLOAT.
+     * </pre>
+     *
+     * <code>float float_val = 3;</code>
+     */
+    float getFloatVal();
+
+    /**
+     * <pre>
+     * DT_DOUBLE.
+     * </pre>
+     *
+     * <code>double double_val = 4;</code>
+     */
+    double getDoubleVal();
+
+    /**
+     * <pre>
+     * DT_INT32, DT_INT16, DT_INT8, DT_UINT8.
+     * </pre>
+     *
+     * <code>int32 int_val = 5;</code>
+     */
+    int getIntVal();
+
+    /**
+     * <code>.google.protobuf.Timestamp timestamp = 7;</code>
+     */
+    boolean hasTimestamp();
+    /**
+     * <code>.google.protobuf.Timestamp timestamp = 7;</code>
+     */
+    com.google.protobuf.Timestamp getTimestamp();
+    /**
+     * <code>.google.protobuf.Timestamp timestamp = 7;</code>
+     */
+    com.google.protobuf.TimestampOrBuilder getTimestampOrBuilder();
+
+    /**
+     * <code>map&lt;string, string&gt; metadata = 8;</code>
+     */
+    int getMetadataCount();
+    /**
+     * <code>map&lt;string, string&gt; metadata = 8;</code>
+     */
+    boolean containsMetadata(
+        java.lang.String key);
+    /**
+     * Use {@link #getMetadataMap()} instead.
+     */
+    @java.lang.Deprecated
+    java.util.Map<java.lang.String, java.lang.String>
+    getMetadata();
+    /**
+     * <code>map&lt;string, string&gt; metadata = 8;</code>
+     */
+    java.util.Map<java.lang.String, java.lang.String>
+    getMetadataMap();
+    /**
+     * <code>map&lt;string, string&gt; metadata = 8;</code>
+     */
+
+    java.lang.String getMetadataOrDefault(
+        java.lang.String key,
+        java.lang.String defaultValue);
+    /**
+     * <code>map&lt;string, string&gt; metadata = 8;</code>
+     */
+
+    java.lang.String getMetadataOrThrow(
+        java.lang.String key);
+
+    public org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSDataPoint.ValueCase getValueCase();
+  }
+  /**
+   * <pre>
+   * Generic TS data point
+   * </pre>
+   *
+   * Protobuf type {@code protos.TSDataPoint}
+   */
+  public  static final class TSDataPoint extends
+      com.google.protobuf.GeneratedMessageV3 implements
+      // @@protoc_insertion_point(message_implements:protos.TSDataPoint)
+      TSDataPointOrBuilder {
+    // Use TSDataPoint.newBuilder() to construct.
+    private TSDataPoint(com.google.protobuf.GeneratedMessageV3.Builder<?> builder) {
+      super(builder);
+    }
+    private TSDataPoint() {
+    }
+
+    @java.lang.Override
+    public final com.google.protobuf.UnknownFieldSet
+    getUnknownFields() {
+      return com.google.protobuf.UnknownFieldSet.getDefaultInstance();
+    }
+    private TSDataPoint(
+        com.google.protobuf.CodedInputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      this();
+      int mutable_bitField0_ = 0;
+      try {
+        boolean done = false;
+        while (!done) {
+          int tag = input.readTag();
+          switch (tag) {
+            case 0:
+              done = true;
+              break;
+            default: {
+              if (!input.skipField(tag)) {
+                done = true;
+              }
+              break;
+            }
+            case 10: {
+              org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSKey.Builder subBuilder = null;
+              if (key_ != null) {
+                subBuilder = key_.toBuilder();
+              }
+              key_ = input.readMessage(org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSKey.parser(), extensionRegistry);
+              if (subBuilder != null) {
+                subBuilder.mergeFrom(key_);
+                key_ = subBuilder.buildPartial();
+              }
+
+              break;
+            }
+            case 29: {
+              valueCase_ = 3;
+              value_ = input.readFloat();
+              break;
+            }
+            case 33: {
+              valueCase_ = 4;
+              value_ = input.readDouble();
+              break;
+            }
+            case 40: {
+              valueCase_ = 5;
+              value_ = input.readInt32();
+              break;
+            }
+            case 58: {
+              com.google.protobuf.Timestamp.Builder subBuilder = null;
+              if (timestamp_ != null) {
+                subBuilder = timestamp_.toBuilder();
+              }
+              timestamp_ = input.readMessage(com.google.protobuf.Timestamp.parser(), extensionRegistry);
+              if (subBuilder != null) {
+                subBuilder.mergeFrom(timestamp_);
+                timestamp_ = subBuilder.buildPartial();
+              }
+
+              break;
+            }
+            case 66: {
+              if (!((mutable_bitField0_ & 0x00000020) == 0x00000020)) {
+                metadata_ = com.google.protobuf.MapField.newMapField(
+                    MetadataDefaultEntryHolder.defaultEntry);
+                mutable_bitField0_ |= 0x00000020;
+              }
+              com.google.protobuf.MapEntry<java.lang.String, java.lang.String>
+              metadata__ = input.readMessage(
+                  MetadataDefaultEntryHolder.defaultEntry.getParserForType(), extensionRegistry);
+              metadata_.getMutableMap().put(
+                  metadata__.getKey(), metadata__.getValue());
+              break;
+            }
+          }
+        }
+      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        throw e.setUnfinishedMessage(this);
+      } catch (java.io.IOException e) {
+        throw new com.google.protobuf.InvalidProtocolBufferException(
+            e).setUnfinishedMessage(this);
+      } finally {
+        makeExtensionsImmutable();
+      }
+    }
+    public static final com.google.protobuf.Descriptors.Descriptor
+        getDescriptor() {
+      return org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.internal_static_protos_TSDataPoint_descriptor;
+    }
+
+    @SuppressWarnings({"rawtypes"})
+    protected com.google.protobuf.MapField internalGetMapField(
+        int number) {
+      switch (number) {
+        case 8:
+          return internalGetMetadata();
+        default:
+          throw new RuntimeException(
+              "Invalid map field number: " + number);
+      }
+    }
+    protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+        internalGetFieldAccessorTable() {
+      return org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.internal_static_protos_TSDataPoint_fieldAccessorTable
+          .ensureFieldAccessorsInitialized(
+              org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSDataPoint.class, org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSDataPoint.Builder.class);
+    }
+
+    private int bitField0_;
+    private int valueCase_ = 0;
+    private java.lang.Object value_;
+    public enum ValueCase
+        implements com.google.protobuf.Internal.EnumLite {
+      FLOAT_VAL(3),
+      DOUBLE_VAL(4),
+      INT_VAL(5),
+      VALUE_NOT_SET(0);
+      private final int value;
+      private ValueCase(int value) {
+        this.value = value;
+      }
+      /**
+       * @deprecated Use {@link #forNumber(int)} instead.
+       */
+      @java.lang.Deprecated
+      public static ValueCase valueOf(int value) {
+        return forNumber(value);
+      }
+
+      public static ValueCase forNumber(int value) {
+        switch (value) {
+          case 3: return FLOAT_VAL;
+          case 4: return DOUBLE_VAL;
+          case 5: return INT_VAL;
+          case 0: return VALUE_NOT_SET;
+          default: return null;
+        }
+      }
+      public int getNumber() {
+        return this.value;
+      }
+    };
+
+    public ValueCase
+    getValueCase() {
+      return ValueCase.forNumber(
+          valueCase_);
+    }
+
+    public static final int KEY_FIELD_NUMBER = 1;
+    private org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSKey key_;
+    /**
+     * <code>.protos.TSKey key = 1;</code>
+     */
+    public boolean hasKey() {
+      return key_ != null;
+    }
+    /**
+     * <code>.protos.TSKey key = 1;</code>
+     */
+    public org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSKey getKey() {
+      return key_ == null ? org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSKey.getDefaultInstance() : key_;
+    }
+    /**
+     * <code>.protos.TSKey key = 1;</code>
+     */
+    public org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSKeyOrBuilder getKeyOrBuilder() {
+      return getKey();
+    }
+
+    public static final int FLOAT_VAL_FIELD_NUMBER = 3;
+    /**
+     * <pre>
+     * DT_FLOAT.
+     * </pre>
+     *
+     * <code>float float_val = 3;</code>
+     */
+    public float getFloatVal() {
+      if (valueCase_ == 3) {
+        return (java.lang.Float) value_;
+      }
+      return 0F;
+    }
+
+    public static final int DOUBLE_VAL_FIELD_NUMBER = 4;
+    /**
+     * <pre>
+     * DT_DOUBLE.
+     * </pre>
+     *
+     * <code>double double_val = 4;</code>
+     */
+    public double getDoubleVal() {
+      if (valueCase_ == 4) {
+        return (java.lang.Double) value_;
+      }
+      return 0D;
+    }
+
+    public static final int INT_VAL_FIELD_NUMBER = 5;
+    /**
+     * <pre>
+     * DT_INT32, DT_INT16, DT_INT8, DT_UINT8.
+     * </pre>
+     *
+     * <code>int32 int_val = 5;</code>
+     */
+    public int getIntVal() {
+      if (valueCase_ == 5) {
+        return (java.lang.Integer) value_;
+      }
+      return 0;
+    }
+
+    public static final int TIMESTAMP_FIELD_NUMBER = 7;
+    private com.google.protobuf.Timestamp timestamp_;
+    /**
+     * <code>.google.protobuf.Timestamp timestamp = 7;</code>
+     */
+    public boolean hasTimestamp() {
+      return timestamp_ != null;
+    }
+    /**
+     * <code>.google.protobuf.Timestamp timestamp = 7;</code>
+     */
+    public com.google.protobuf.Timestamp getTimestamp() {
+      return timestamp_ == null ? com.google.protobuf.Timestamp.getDefaultInstance() : timestamp_;
+    }
+    /**
+     * <code>.google.protobuf.Timestamp timestamp = 7;</code>
+     */
+    public com.google.protobuf.TimestampOrBuilder getTimestampOrBuilder() {
+      return getTimestamp();
+    }
+
+    public static final int METADATA_FIELD_NUMBER = 8;
+    private static final class MetadataDefaultEntryHolder {
+      static final com.google.protobuf.MapEntry<
+          java.lang.String, java.lang.String> defaultEntry =
+              com.google.protobuf.MapEntry
+              .<java.lang.String, java.lang.String>newDefaultInstance(
+                  org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.internal_static_protos_TSDataPoint_MetadataEntry_descriptor, 
+                  com.google.protobuf.WireFormat.FieldType.STRING,
+                  "",
+                  com.google.protobuf.WireFormat.FieldType.STRING,
+                  "");
+    }
+    private com.google.protobuf.MapField<
+        java.lang.String, java.lang.String> metadata_;
+    private com.google.protobuf.MapField<java.lang.String, java.lang.String>
+    internalGetMetadata() {
+      if (metadata_ == null) {
+        return com.google.protobuf.MapField.emptyMapField(
+            MetadataDefaultEntryHolder.defaultEntry);
+      }
+      return metadata_;
+    }
+
+    public int getMetadataCount() {
+      return internalGetMetadata().getMap().size();
+    }
+    /**
+     * <code>map&lt;string, string&gt; metadata = 8;</code>
+     */
+
+    public boolean containsMetadata(
+        java.lang.String key) {
+      if (key == null) { throw new java.lang.NullPointerException(); }
+      return internalGetMetadata().getMap().containsKey(key);
+    }
+    /**
+     * Use {@link #getMetadataMap()} instead.
+     */
+    @java.lang.Deprecated
+    public java.util.Map<java.lang.String, java.lang.String> getMetadata() {
+      return getMetadataMap();
+    }
+    /**
+     * <code>map&lt;string, string&gt; metadata = 8;</code>
+     */
+
+    public java.util.Map<java.lang.String, java.lang.String> getMetadataMap() {
+      return internalGetMetadata().getMap();
+    }
+    /**
+     * <code>map&lt;string, string&gt; metadata = 8;</code>
+     */
+
+    public java.lang.String getMetadataOrDefault(
+        java.lang.String key,
+        java.lang.String defaultValue) {
+      if (key == null) { throw new java.lang.NullPointerException(); }
+      java.util.Map<java.lang.String, java.lang.String> map =
+          internalGetMetadata().getMap();
+      return map.containsKey(key) ? map.get(key) : defaultValue;
+    }
+    /**
+     * <code>map&lt;string, string&gt; metadata = 8;</code>
+     */
+
+    public java.lang.String getMetadataOrThrow(
+        java.lang.String key) {
+      if (key == null) { throw new java.lang.NullPointerException(); }
+      java.util.Map<java.lang.String, java.lang.String> map =
+          internalGetMetadata().getMap();
+      if (!map.containsKey(key)) {
+        throw new java.lang.IllegalArgumentException();
+      }
+      return map.get(key);
+    }
+
+    private byte memoizedIsInitialized = -1;
+    public final boolean isInitialized() {
+      byte isInitialized = memoizedIsInitialized;
+      if (isInitialized == 1) return true;
+      if (isInitialized == 0) return false;
+
+      memoizedIsInitialized = 1;
+      return true;
+    }
+
+    public void writeTo(com.google.protobuf.CodedOutputStream output)
+                        throws java.io.IOException {
+      if (key_ != null) {
+        output.writeMessage(1, getKey());
+      }
+      if (valueCase_ == 3) {
+        output.writeFloat(
+            3, (float)((java.lang.Float) value_));
+      }
+      if (valueCase_ == 4) {
+        output.writeDouble(
+            4, (double)((java.lang.Double) value_));
+      }
+      if (valueCase_ == 5) {
+        output.writeInt32(
+            5, (int)((java.lang.Integer) value_));
+      }
+      if (timestamp_ != null) {
+        output.writeMessage(7, getTimestamp());
+      }
+      com.google.protobuf.GeneratedMessageV3
+        .serializeStringMapTo(
+          output,
+          internalGetMetadata(),
+          MetadataDefaultEntryHolder.defaultEntry,
+          8);
+    }
+
+    public int getSerializedSize() {
+      int size = memoizedSize;
+      if (size != -1) return size;
+
+      size = 0;
+      if (key_ != null) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeMessageSize(1, getKey());
+      }
+      if (valueCase_ == 3) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeFloatSize(
+              3, (float)((java.lang.Float) value_));
+      }
+      if (valueCase_ == 4) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeDoubleSize(
+              4, (double)((java.lang.Double) value_));
+      }
+      if (valueCase_ == 5) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeInt32Size(
+              5, (int)((java.lang.Integer) value_));
+      }
+      if (timestamp_ != null) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeMessageSize(7, getTimestamp());
+      }
+      for (java.util.Map.Entry<java.lang.String, java.lang.String> entry
+           : internalGetMetadata().getMap().entrySet()) {
+        com.google.protobuf.MapEntry<java.lang.String, java.lang.String>
+        metadata__ = MetadataDefaultEntryHolder.defaultEntry.newBuilderForType()
+            .setKey(entry.getKey())
+            .setValue(entry.getValue())
+            .build();
+        size += com.google.protobuf.CodedOutputStream
+            .computeMessageSize(8, metadata__);
+      }
+      memoizedSize = size;
+      return size;
+    }
+
+    private static final long serialVersionUID = 0L;
+    @java.lang.Override
+    public boolean equals(final java.lang.Object obj) {
+      if (obj == this) {
+       return true;
+      }
+      if (!(obj instanceof org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSDataPoint)) {
+        return super.equals(obj);
+      }
+      org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSDataPoint other = (org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSDataPoint) obj;
+
+      boolean result = true;
+      result = result && (hasKey() == other.hasKey());
+      if (hasKey()) {
+        result = result && getKey()
+            .equals(other.getKey());
+      }
+      result = result && (hasTimestamp() == other.hasTimestamp());
+      if (hasTimestamp()) {
+        result = result && getTimestamp()
+            .equals(other.getTimestamp());
+      }
+      result = result && internalGetMetadata().equals(
+          other.internalGetMetadata());
+      result = result && getValueCase().equals(
+          other.getValueCase());
+      if (!result) return false;
+      switch (valueCase_) {
+        case 3:
+          result = result && (
+              java.lang.Float.floatToIntBits(getFloatVal())
+              == java.lang.Float.floatToIntBits(
+                  other.getFloatVal()));
+          break;
+        case 4:
+          result = result && (
+              java.lang.Double.doubleToLongBits(getDoubleVal())
+              == java.lang.Double.doubleToLongBits(
+                  other.getDoubleVal()));
+          break;
+        case 5:
+          result = result && (getIntVal()
+              == other.getIntVal());
+          break;
+        case 0:
+        default:
+      }
+      return result;
+    }
+
+    @java.lang.Override
+    public int hashCode() {
+      if (memoizedHashCode != 0) {
+        return memoizedHashCode;
+      }
+      int hash = 41;
+      hash = (19 * hash) + getDescriptor().hashCode();
+      if (hasKey()) {
+        hash = (37 * hash) + KEY_FIELD_NUMBER;
+        hash = (53 * hash) + getKey().hashCode();
+      }
+      if (hasTimestamp()) {
+        hash = (37 * hash) + TIMESTAMP_FIELD_NUMBER;
+        hash = (53 * hash) + getTimestamp().hashCode();
+      }
+      if (!internalGetMetadata().getMap().isEmpty()) {
+        hash = (37 * hash) + METADATA_FIELD_NUMBER;
+        hash = (53 * hash) + internalGetMetadata().hashCode();
+      }
+      switch (valueCase_) {
+        case 3:
+          hash = (37 * hash) + FLOAT_VAL_FIELD_NUMBER;
+          hash = (53 * hash) + java.lang.Float.floatToIntBits(
+              getFloatVal());
+          break;
+        case 4:
+          hash = (37 * hash) + DOUBLE_VAL_FIELD_NUMBER;
+          hash = (53 * hash) + com.google.protobuf.Internal.hashLong(
+              java.lang.Double.doubleToLongBits(getDoubleVal()));
+          break;
+        case 5:
+          hash = (37 * hash) + INT_VAL_FIELD_NUMBER;
+          hash = (53 * hash) + getIntVal();
+          break;
+        case 0:
+        default:
+      }
+      hash = (29 * hash) + unknownFields.hashCode();
+      memoizedHashCode = hash;
+      return hash;
+    }
+
+    public static org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSDataPoint parseFrom(
+        com.google.protobuf.ByteString data)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data);
+    }
+    public static org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSDataPoint parseFrom(
+        com.google.protobuf.ByteString data,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data, extensionRegistry);
+    }
+    public static org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSDataPoint parseFrom(byte[] data)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data);
+    }
+    public static org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSDataPoint parseFrom(
+        byte[] data,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data, extensionRegistry);
+    }
+    public static org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSDataPoint parseFrom(java.io.InputStream input)
+        throws java.io.IOException {
+      return com.google.protobuf.GeneratedMessageV3
+          .parseWithIOException(PARSER, input);
+    }
+    public static org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSDataPoint parseFrom(
+        java.io.InputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws java.io.IOException {
+      return com.google.protobuf.GeneratedMessageV3
+          .parseWithIOException(PARSER, input, extensionRegistry);
+    }
+    public static org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSDataPoint parseDelimitedFrom(java.io.InputStream input)
+        throws java.io.IOException {
+      return com.google.protobuf.GeneratedMessageV3
+          .parseDelimitedWithIOException(PARSER, input);
+    }
+    public static org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSDataPoint parseDelimitedFrom(
+        java.io.InputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws java.io.IOException {
+      return com.google.protobuf.GeneratedMessageV3
+          .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
+    }
+    public static org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSDataPoint parseFrom(
+        com.google.protobuf.CodedInputStream input)
+        throws java.io.IOException {
+      return com.google.protobuf.GeneratedMessageV3
+          .parseWithIOException(PARSER, input);
+    }
+    public static org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSDataPoint parseFrom(
+        com.google.protobuf.CodedInputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws java.io.IOException {
+      return com.google.protobuf.GeneratedMessageV3
+          .parseWithIOException(PARSER, input, extensionRegistry);
+    }
+
+    public Builder newBuilderForType() { return newBuilder(); }
+    public static Builder newBuilder() {
+      return DEFAULT_INSTANCE.toBuilder();
+    }
+    public static Builder newBuilder(org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSDataPoint prototype) {
+      return DEFAULT_INSTANCE.toBuilder().mergeFrom(prototype);
+    }
+    public Builder toBuilder() {
+      return this == DEFAULT_INSTANCE
+          ? new Builder() : new Builder().mergeFrom(this);
+    }
+
+    @java.lang.Override
+    protected Builder newBuilderForType(
+        com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+      Builder builder = new Builder(parent);
+      return builder;
+    }
+    /**
+     * <pre>
+     * Generic TS data point
+     * </pre>
+     *
+     * Protobuf type {@code protos.TSDataPoint}
+     */
+    public static final class Builder extends
+        com.google.protobuf.GeneratedMessageV3.Builder<Builder> implements
+        // @@protoc_insertion_point(builder_implements:protos.TSDataPoint)
+        org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSDataPointOrBuilder {
+      public static final com.google.protobuf.Descriptors.Descriptor
+          getDescriptor() {
+        return org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.internal_static_protos_TSDataPoint_descriptor;
+      }
+
+      @SuppressWarnings({"rawtypes"})
+      protected com.google.protobuf.MapField internalGetMapField(
+          int number) {
+        switch (number) {
+          case 8:
+            return internalGetMetadata();
+          default:
+            throw new RuntimeException(
+                "Invalid map field number: " + number);
+        }
+      }
+      @SuppressWarnings({"rawtypes"})
+      protected com.google.protobuf.MapField internalGetMutableMapField(
+          int number) {
+        switch (number) {
+          case 8:
+            return internalGetMutableMetadata();
+          default:
+            throw new RuntimeException(
+                "Invalid map field number: " + number);
+        }
+      }
+      protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+          internalGetFieldAccessorTable() {
+        return org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.internal_static_protos_TSDataPoint_fieldAccessorTable
+            .ensureFieldAccessorsInitialized(
+                org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSDataPoint.class, org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSDataPoint.Builder.class);
+      }
+
+      // Construct using org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSDataPoint.newBuilder()
+      private Builder() {
+        maybeForceBuilderInitialization();
+      }
+
+      private Builder(
+          com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+        super(parent);
+        maybeForceBuilderInitialization();
+      }
+      private void maybeForceBuilderInitialization() {
+        if (com.google.protobuf.GeneratedMessageV3
+                .alwaysUseFieldBuilders) {
+        }
+      }
+      public Builder clear() {
+        super.clear();
+        if (keyBuilder_ == null) {
+          key_ = null;
+        } else {
+          key_ = null;
+          keyBuilder_ = null;
+        }
+        if (timestampBuilder_ == null) {
+          timestamp_ = null;
+        } else {
+          timestamp_ = null;
+          timestampBuilder_ = null;
+        }
+        internalGetMutableMetadata().clear();
+        valueCase_ = 0;
+        value_ = null;
+        return this;
+      }
+
+      public com.google.protobuf.Descriptors.Descriptor
+          getDescriptorForType() {
+        return org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.internal_static_protos_TSDataPoint_descriptor;
+      }
+
+      public org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSDataPoint getDefaultInstanceForType() {
+        return org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSDataPoint.getDefaultInstance();
+      }
+
+      public org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSDataPoint build() {
+        org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSDataPoint result = buildPartial();
+        if (!result.isInitialized()) {
+          throw newUninitializedMessageException(result);
+        }
+        return result;
+      }
+
+      public org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSDataPoint buildPartial() {
+        org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSDataPoint result = new org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSDataPoint(this);
+        int from_bitField0_ = bitField0_;
+        int to_bitField0_ = 0;
+        if (keyBuilder_ == null) {
+          result.key_ = key_;
+        } else {
+          result.key_ = keyBuilder_.build();
+        }
+        if (valueCase_ == 3) {
+          result.value_ = value_;
+        }
+        if (valueCase_ == 4) {
+          result.value_ = value_;
+        }
+        if (valueCase_ == 5) {
+          result.value_ = value_;
+        }
+        if (timestampBuilder_ == null) {
+          result.timestamp_ = timestamp_;
+        } else {
+          result.timestamp_ = timestampBuilder_.build();
+        }
+        result.metadata_ = internalGetMetadata();
+        result.metadata_.makeImmutable();
+        result.bitField0_ = to_bitField0_;
+        result.valueCase_ = valueCase_;
+        onBuilt();
+        return result;
+      }
+
+      public Builder clone() {
+        return (Builder) super.clone();
+      }
+      public Builder setField(
+          com.google.protobuf.Descriptors.FieldDescriptor field,
+          Object value) {
+        return (Builder) super.setField(field, value);
+      }
+      public Builder clearField(
+          com.google.protobuf.Descriptors.FieldDescriptor field) {
+        return (Builder) super.clearField(field);
+      }
+      public Builder clearOneof(
+          com.google.protobuf.Descriptors.OneofDescriptor oneof) {
+        return (Builder) super.clearOneof(oneof);
+      }
+      public Builder setRepeatedField(
+          com.google.protobuf.Descriptors.FieldDescriptor field,
+          int index, Object value) {
+        return (Builder) super.setRepeatedField(field, index, value);
+      }
+      public Builder addRepeatedField(
+          com.google.protobuf.Descriptors.FieldDescriptor field,
+          Object value) {
+        return (Builder) super.addRepeatedField(field, value);
+      }
+      public Builder mergeFrom(com.google.protobuf.Message other) {
+        if (other instanceof org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSDataPoint) {
+          return mergeFrom((org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSDataPoint)other);
+        } else {
+          super.mergeFrom(other);
+          return this;
+        }
+      }
+
+      public Builder mergeFrom(org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSDataPoint other) {
+        if (other == org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSDataPoint.getDefaultInstance()) return this;
+        if (other.hasKey()) {
+          mergeKey(other.getKey());
+        }
+        if (other.hasTimestamp()) {
+          mergeTimestamp(other.getTimestamp());
+        }
+        internalGetMutableMetadata().mergeFrom(
+            other.internalGetMetadata());
+        switch (other.getValueCase()) {
+          case FLOAT_VAL: {
+            setFloatVal(other.getFloatVal());
+            break;
+          }
+          case DOUBLE_VAL: {
+            setDoubleVal(other.getDoubleVal());
+            break;
+          }
+          case INT_VAL: {
+            setIntVal(other.getIntVal());
+            break;
+          }
+          case VALUE_NOT_SET: {
+            break;
+          }
+        }
+        onChanged();
+        return this;
+      }
+
+      public final boolean isInitialized() {
+        return true;
+      }
+
+      public Builder mergeFrom(
+          com.google.protobuf.CodedInputStream input,
+          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws java.io.IOException {
+        org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSDataPoint parsedMessage = null;
+        try {
+          parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
+        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+          parsedMessage = (org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSDataPoint) e.getUnfinishedMessage();
+          throw e.unwrapIOException();
+        } finally {
+          if (parsedMessage != null) {
+            mergeFrom(parsedMessage);
+          }
+        }
+        return this;
+      }
+      private int valueCase_ = 0;
+      private java.lang.Object value_;
+      public ValueCase
+          getValueCase() {
+        return ValueCase.forNumber(
+            valueCase_);
+      }
+
+      public Builder clearValue() {
+        valueCase_ = 0;
+        value_ = null;
+        onChanged();
+        return this;
+      }
+
+      private int bitField0_;
+
+      private org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSKey key_ = null;
+      private com.google.protobuf.SingleFieldBuilderV3<
+          org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSKey, org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSKey.Builder, org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSKeyOrBuilder> keyBuilder_;
+      /**
+       * <code>.protos.TSKey key = 1;</code>
+       */
+      public boolean hasKey() {
+        return keyBuilder_ != null || key_ != null;
+      }
+      /**
+       * <code>.protos.TSKey key = 1;</code>
+       */
+      public org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSKey getKey() {
+        if (keyBuilder_ == null) {
+          return key_ == null ? org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSKey.getDefaultInstance() : key_;
+        } else {
+          return keyBuilder_.getMessage();
+        }
+      }
+      /**
+       * <code>.protos.TSKey key = 1;</code>
+       */
+      public Builder setKey(org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSKey value) {
+        if (keyBuilder_ == null) {
+          if (value == null) {
+            throw new NullPointerException();
+          }
+          key_ = value;
+          onChanged();
+        } else {
+          keyBuilder_.setMessage(value);
+        }
+
+        return this;
+      }
+      /**
+       * <code>.protos.TSKey key = 1;</code>
+       */
+      public Builder setKey(
+          org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSKey.Builder builderForValue) {
+        if (keyBuilder_ == null) {
+          key_ = builderForValue.build();
+          onChanged();
+        } else {
+          keyBuilder_.setMessage(builderForValue.build());
+        }
+
+        return this;
+      }
+      /**
+       * <code>.protos.TSKey key = 1;</code>
+       */
+      public Builder mergeKey(org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSKey value) {
+        if (keyBuilder_ == null) {
+          if (key_ != null) {
+            key_ =
+              org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSKey.newBuilder(key_).mergeFrom(value).buildPartial();
+          } else {
+            key_ = value;
+          }
+          onChanged();
+        } else {
+          keyBuilder_.mergeFrom(value);
+        }
+
+        return this;
+      }
+      /**
+       * <code>.protos.TSKey key = 1;</code>
+       */
+      public Builder clearKey() {
+        if (keyBuilder_ == null) {
+          key_ = null;
+          onChanged();
+        } else {
+          key_ = null;
+          keyBuilder_ = null;
+        }
+
+        return this;
+      }
+      /**
+       * <code>.protos.TSKey key = 1;</code>
+       */
+      public org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSKey.Builder getKeyBuilder() {
+        
+        onChanged();
+        return getKeyFieldBuilder().getBuilder();
+      }
+      /**
+       * <code>.protos.TSKey key = 1;</code>
+       */
+      public org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSKeyOrBuilder getKeyOrBuilder() {
+        if (keyBuilder_ != null) {
+          return keyBuilder_.getMessageOrBuilder();
+        } else {
+          return key_ == null ?
+              org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSKey.getDefaultInstance() : key_;
+        }
+      }
+      /**
+       * <code>.protos.TSKey key = 1;</code>
+       */
+      private com.google.protobuf.SingleFieldBuilderV3<
+          org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSKey, org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSKey.Builder, org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSKeyOrBuilder> 
+          getKeyFieldBuilder() {
+        if (keyBuilder_ == null) {
+          keyBuilder_ = new com.google.protobuf.SingleFieldBuilderV3<
+              org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSKey, org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSKey.Builder, org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSKeyOrBuilder>(
+                  getKey(),
+                  getParentForChildren(),
+                  isClean());
+          key_ = null;
+        }
+        return keyBuilder_;
+      }
+
+      /**
+       * <pre>
+       * DT_FLOAT.
+       * </pre>
+       *
+       * <code>float float_val = 3;</code>
+       */
+      public float getFloatVal() {
+        if (valueCase_ == 3) {
+          return (java.lang.Float) value_;
+        }
+        return 0F;
+      }
+      /**
+       * <pre>
+       * DT_FLOAT.
+       * </pre>
+       *
+       * <code>float float_val = 3;</code>
+       */
+      public Builder setFloatVal(float value) {
+        valueCase_ = 3;
+        value_ = value;
+        onChanged();
+        return this;
+      }
+      /**
+       * <pre>
+       * DT_FLOAT.
+       * </pre>
+       *
+       * <code>float float_val = 3;</code>
+       */
+      public Builder clearFloatVal() {
+        if (valueCase_ == 3) {
+          valueCase_ = 0;
+          value_ = null;
+          onChanged();
+        }
+        return this;
+      }
+
+      /**
+       * <pre>
+       * DT_DOUBLE.
+       * </pre>
+       *
+       * <code>double double_val = 4;</code>
+       */
+      public double getDoubleVal() {
+        if (valueCase_ == 4) {
+          return (java.lang.Double) value_;
+        }
+        return 0D;
+      }
+      /**
+       * <pre>
+       * DT_DOUBLE.
+       * </pre>
+       *
+       * <code>double double_val = 4;</code>
+       */
+      public Builder setDoubleVal(double value) {
+        valueCase_ = 4;
+        value_ = value;
+        onChanged();
+        return this;
+      }
+      /**
+       * <pre>
+       * DT_DOUBLE.
+       * </pre>
+       *
+       * <code>double double_val = 4;</code>
+       */
+      public Builder clearDoubleVal() {
+        if (valueCase_ == 4) {
+          valueCase_ = 0;
+          value_ = null;
+          onChanged();
+        }
+        return this;
+      }
+
+      /**
+       * <pre>
+       * DT_INT32, DT_INT16, DT_INT8, DT_UINT8.
+       * </pre>
+       *
+       * <code>int32 int_val = 5;</code>
+       */
+      public int getIntVal() {
+        if (valueCase_ == 5) {
+          return (java.lang.Integer) value_;
+        }
+        return 0;
+      }
+      /**
+       * <pre>
+       * DT_INT32, DT_INT16, DT_INT8, DT_UINT8.
+       * </pre>
+       *
+       * <code>int32 int_val = 5;</code>
+       */
+      public Builder setIntVal(int value) {
+        valueCase_ = 5;
+        value_ = value;
+        onChanged();
+        return this;
+      }
+      /**
+       * <pre>
+       * DT_INT32, DT_INT16, DT_INT8, DT_UINT8.
+       * </pre>
+       *
+       * <code>int32 int_val = 5;</code>
+       */
+      public Builder clearIntVal() {
+        if (valueCase_ == 5) {
+          valueCase_ = 0;
+          value_ = null;
+          onChanged();
+        }
+        return this;
+      }
+
+      private com.google.protobuf.Timestamp timestamp_ = null;
+      private com.google.protobuf.SingleFieldBuilderV3<
+          com.google.protobuf.Timestamp, com.google.protobuf.Timestamp.Builder, com.google.protobuf.TimestampOrBuilder> timestampBuilder_;
+      /**
+       * <code>.google.protobuf.Timestamp timestamp = 7;</code>
+       */
+      public boolean hasTimestamp() {
+        return timestampBuilder_ != null || timestamp_ != null;
+      }
+      /**
+       * <code>.google.protobuf.Timestamp timestamp = 7;</code>
+       */
+      public com.google.protobuf.Timestamp getTimestamp() {
+        if (timestampBuilder_ == null) {
+          return timestamp_ == null ? com.google.protobuf.Timestamp.getDefaultInstance() : timestamp_;
+        } else {
+          return timestampBuilder_.getMessage();
+        }
+      }
+      /**
+       * <code>.google.protobuf.Timestamp timestamp = 7;</code>
+       */
+      public Builder setTimestamp(com.google.protobuf.Timestamp value) {
+        if (timestampBuilder_ == null) {
+          if (value == null) {
+            throw new NullPointerException();
+          }
+          timestamp_ = value;
+          onChanged();
+        } else {
+          timestampBuilder_.setMessage(value);
+        }
+
+        return this;
+      }
+      /**
+       * <code>.google.protobuf.Timestamp timestamp = 7;</code>
+       */
+      public Builder setTimestamp(
+          com.google.protobuf.Timestamp.Builder builderForValue) {
+        if (timestampBuilder_ == null) {
+          timestamp_ = builderForValue.build();
+          onChanged();
+        } else {
+          timestampBuilder_.setMessage(builderForValue.build());
+        }
+
+        return this;
+      }
+      /**
+       * <code>.google.protobuf.Timestamp timestamp = 7;</code>
+       */
+      public Builder mergeTimestamp(com.google.protobuf.Timestamp value) {
+        if (timestampBuilder_ == null) {
+          if (timestamp_ != null) {
+            timestamp_ =
+              com.google.protobuf.Timestamp.newBuilder(timestamp_).mergeFrom(value).buildPartial();
+          } else {
+            timestamp_ = value;
+          }
+          onChanged();
+        } else {
+          timestampBuilder_.mergeFrom(value);
+        }
+
+        return this;
+      }
+      /**
+       * <code>.google.protobuf.Timestamp timestamp = 7;</code>
+       */
+      public Builder clearTimestamp() {
+        if (timestampBuilder_ == null) {
+          timestamp_ = null;
+          onChanged();
+        } else {
+          timestamp_ = null;
+          timestampBuilder_ = null;
+        }
+
+        return this;
+      }
+      /**
+       * <code>.google.protobuf.Timestamp timestamp = 7;</code>
+       */
+      public com.google.protobuf.Timestamp.Builder getTimestampBuilder() {
+        
+        onChanged();
+        return getTimestampFieldBuilder().getBuilder();
+      }
+      /**
+       * <code>.google.protobuf.Timestamp timestamp = 7;</code>
+       */
+      public com.google.protobuf.TimestampOrBuilder getTimestampOrBuilder() {
+        if (timestampBuilder_ != null) {
+          return timestampBuilder_.getMessageOrBuilder();
+        } else {
+          return timestamp_ == null ?
+              com.google.protobuf.Timestamp.getDefaultInstance() : timestamp_;
+        }
+      }
+      /**
+       * <code>.google.protobuf.Timestamp timestamp = 7;</code>
+       */
+      private com.google.protobuf.SingleFieldBuilderV3<
+          com.google.protobuf.Timestamp, com.google.protobuf.Timestamp.Builder, com.google.protobuf.TimestampOrBuilder> 
+          getTimestampFieldBuilder() {
+        if (timestampBuilder_ == null) {
+          timestampBuilder_ = new com.google.protobuf.SingleFieldBuilderV3<
+              com.google.protobuf.Timestamp, com.google.protobuf.Timestamp.Builder, com.google.protobuf.TimestampOrBuilder>(
+                  getTimestamp(),
+                  getParentForChildren(),
+                  isClean());
+          timestamp_ = null;
+        }
+        return timestampBuilder_;
+      }
+
+      private com.google.protobuf.MapField<
+          java.lang.String, java.lang.String> metadata_;
+      private com.google.protobuf.MapField<java.lang.String, java.lang.String>
+      internalGetMetadata() {
+        if (metadata_ == null) {
+          return com.google.protobuf.MapField.emptyMapField(
+              MetadataDefaultEntryHolder.defaultEntry);
+        }
+        return metadata_;
+      }
+      private com.google.protobuf.MapField<java.lang.String, java.lang.String>
+      internalGetMutableMetadata() {
+        onChanged();;
+        if (metadata_ == null) {
+          metadata_ = com.google.protobuf.MapField.newMapField(
+              MetadataDefaultEntryHolder.defaultEntry);
+        }
+        if (!metadata_.isMutable()) {
+          metadata_ = metadata_.copy();
+        }
+        return metadata_;
+      }
+
+      public int getMetadataCount() {
+        return internalGetMetadata().getMap().size();
+      }
+      /**
+       * <code>map&lt;string, string&gt; metadata = 8;</code>
+       */
+
+      public boolean containsMetadata(
+          java.lang.String key) {
+        if (key == null) { throw new java.lang.NullPointerException(); }
+        return internalGetMetadata().getMap().containsKey(key);
+      }
+      /**
+       * Use {@link #getMetadataMap()} instead.
+       */
+      @java.lang.Deprecated
+      public java.util.Map<java.lang.String, java.lang.String> getMetadata() {
+        return getMetadataMap();
+      }
+      /**
+       * <code>map&lt;string, string&gt; metadata = 8;</code>
+       */
+
+      public java.util.Map<java.lang.String, java.lang.String> getMetadataMap() {
+        return internalGetMetadata().getMap();
+      }
+      /**
+       * <code>map&lt;string, string&gt; metadata = 8;</code>
+       */
+
+      public java.lang.String getMetadataOrDefault(
+          java.lang.String key,
+          java.lang.String defaultValue) {
+        if (key == null) { throw new java.lang.NullPointerException(); }
+        java.util.Map<java.lang.String, java.lang.String> map =
+            internalGetMetadata().getMap();
+        return map.containsKey(key) ? map.get(key) : defaultValue;
+      }
+      /**
+       * <code>map&lt;string, string&gt; metadata = 8;</code>
+       */
+
+      public java.lang.String getMetadataOrThrow(
+          java.lang.String key) {
+        if (key == null) { throw new java.lang.NullPointerException(); }
+        java.util.Map<java.lang.String, java.lang.String> map =
+            internalGetMetadata().getMap();
+        if (!map.containsKey(key)) {
+          throw new java.lang.IllegalArgumentException();
+        }
+        return map.get(key);
+      }
+
+      public Builder clearMetadata() {
+        internalGetMutableMetadata().getMutableMap()
+            .clear();
+        return this;
+      }
+      /**
+       * <code>map&lt;string, string&gt; metadata = 8;</code>
+       */
+
+      public Builder removeMetadata(
+          java.lang.String key) {
+        if (key == null) { throw new java.lang.NullPointerException(); }
+        internalGetMutableMetadata().getMutableMap()
+            .remove(key);
+        return this;
+      }
+      /**
+       * Use alternate mutation accessors instead.
+       */
+      @java.lang.Deprecated
+      public java.util.Map<java.lang.String, java.lang.String>
+      getMutableMetadata() {
+        return internalGetMutableMetadata().getMutableMap();
+      }
+      /**
+       * <code>map&lt;string, string&gt; metadata = 8;</code>
+       */
+      public Builder putMetadata(
+          java.lang.String key,
+          java.lang.String value) {
+        if (key == null) { throw new java.lang.NullPointerException(); }
+        if (value == null) { throw new java.lang.NullPointerException(); }
+        internalGetMutableMetadata().getMutableMap()
+            .put(key, value);
+        return this;
+      }
+      /**
+       * <code>map&lt;string, string&gt; metadata = 8;</code>
+       */
+
+      public Builder putAllMetadata(
+          java.util.Map<java.lang.String, java.lang.String> values) {
+        internalGetMutableMetadata().getMutableMap()
+            .putAll(values);
+        return this;
+      }
+      public final Builder setUnknownFields(
+          final com.google.protobuf.UnknownFieldSet unknownFields) {
+        return this;
+      }
+
+      public final Builder mergeUnknownFields(
+          final com.google.protobuf.UnknownFieldSet unknownFields) {
+        return this;
+      }
+
+
+      // @@protoc_insertion_point(builder_scope:protos.TSDataPoint)
+    }
+
+    // @@protoc_insertion_point(class_scope:protos.TSDataPoint)
+    private static final org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSDataPoint DEFAULT_INSTANCE;
+    static {
+      DEFAULT_INSTANCE = new org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSDataPoint();
+    }
+
+    public static org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSDataPoint getDefaultInstance() {
+      return DEFAULT_INSTANCE;
+    }
+
+    private static final com.google.protobuf.Parser<TSDataPoint>
+        PARSER = new com.google.protobuf.AbstractParser<TSDataPoint>() {
+      public TSDataPoint parsePartialFrom(
+          com.google.protobuf.CodedInputStream input,
+          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws com.google.protobuf.InvalidProtocolBufferException {
+          return new TSDataPoint(input, extensionRegistry);
+      }
+    };
+
+    public static com.google.protobuf.Parser<TSDataPoint> parser() {
+      return PARSER;
+    }
+
+    @java.lang.Override
+    public com.google.protobuf.Parser<TSDataPoint> getParserForType() {
+      return PARSER;
+    }
+
+    public org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSDataPoint getDefaultInstanceForType() {
+      return DEFAULT_INSTANCE;
+    }
+
+  }
+
+  public interface TSDataSchemaOrBuilder extends
+      // @@protoc_insertion_point(interface_extends:protos.TSDataSchema)
+      com.google.protobuf.MessageOrBuilder {
+
+    /**
+     * <code>.protos.TSKey key = 1;</code>
+     */
+    boolean hasKey();
+    /**
+     * <code>.protos.TSKey key = 1;</code>
+     */
+    org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSKey getKey();
+    /**
+     * <code>.protos.TSKey key = 1;</code>
+     */
+    org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSKeyOrBuilder getKeyOrBuilder();
+
+    /**
+     * <code>map&lt;string, .protos.TSKey&gt; features = 2;</code>
+     */
+    int getFeaturesCount();
+    /**
+     * <code>map&lt;string, .protos.TSKey&gt; features = 2;</code>
+     */
+    boolean containsFeatures(
+        java.lang.String key);
+    /**
+     * Use {@link #getFeaturesMap()} instead.
+     */
+    @java.lang.Deprecated
+    java.util.Map<java.lang.String, org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSKey>
+    getFeatures();
+    /**
+     * <code>map&lt;string, .protos.TSKey&gt; features = 2;</code>
+     */
+    java.util.Map<java.lang.String, org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSKey>
+    getFeaturesMap();
+    /**
+     * <code>map&lt;string, .protos.TSKey&gt; features = 2;</code>
+     */
+
+    org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSKey getFeaturesOrDefault(
+        java.lang.String key,
+        org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSKey defaultValue);
+    /**
+     * <code>map&lt;string, .protos.TSKey&gt; features = 2;</code>
+     */
+
+    org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSKey getFeaturesOrThrow(
+        java.lang.String key);
+  }
+  /**
+   * Protobuf type {@code protos.TSDataSchema}
+   */
+  public  static final class TSDataSchema extends
+      com.google.protobuf.GeneratedMessageV3 implements
+      // @@protoc_insertion_point(message_implements:protos.TSDataSchema)
+      TSDataSchemaOrBuilder {
+    // Use TSDataSchema.newBuilder() to construct.
+    private TSDataSchema(com.google.protobuf.GeneratedMessageV3.Builder<?> builder) {
+      super(builder);
+    }
+    private TSDataSchema() {
+    }
+
+    @java.lang.Override
+    public final com.google.protobuf.UnknownFieldSet
+    getUnknownFields() {
+      return com.google.protobuf.UnknownFieldSet.getDefaultInstance();
+    }
+    private TSDataSchema(
+        com.google.protobuf.CodedInputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      this();
+      int mutable_bitField0_ = 0;
+      try {
+        boolean done = false;
+        while (!done) {
+          int tag = input.readTag();
+          switch (tag) {
+            case 0:
+              done = true;
+              break;
+            default: {
+              if (!input.skipField(tag)) {
+                done = true;
+              }
+              break;
+            }
+            case 10: {
+              org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSKey.Builder subBuilder = null;
+              if (key_ != null) {
+                subBuilder = key_.toBuilder();
+              }
+              key_ = input.readMessage(org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSKey.parser(), extensionRegistry);
+              if (subBuilder != null) {
+                subBuilder.mergeFrom(key_);
+                key_ = subBuilder.buildPartial();
+              }
+
+              break;
+            }
+            case 18: {
+              if (!((mutable_bitField0_ & 0x00000002) == 0x00000002)) {
+                features_ = com.google.protobuf.MapField.newMapField(
+                    FeaturesDefaultEntryHolder.defaultEntry);
+                mutable_bitField0_ |= 0x00000002;
+              }
+              com.google.protobuf.MapEntry<java.lang.String, org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSKey>
+              features__ = input.readMessage(
+                  FeaturesDefaultEntryHolder.defaultEntry.getParserForType(), extensionRegistry);
+              features_.getMutableMap().put(
+                  features__.getKey(), features__.getValue());
+              break;
+            }
+          }
+        }
+      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        throw e.setUnfinishedMessage(this);
+      } catch (java.io.IOException e) {
+        throw new com.google.protobuf.InvalidProtocolBufferException(
+            e).setUnfinishedMessage(this);
+      } finally {
+        makeExtensionsImmutable();
+      }
+    }
+    public static final com.google.protobuf.Descriptors.Descriptor
+        getDescriptor() {
+      return org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.internal_static_protos_TSDataSchema_descriptor;
+    }
+
+    @SuppressWarnings({"rawtypes"})
+    protected com.google.protobuf.MapField internalGetMapField(
+        int number) {
+      switch (number) {
+        case 2:
+          return internalGetFeatures();
+        default:
+          throw new RuntimeException(
+              "Invalid map field number: " + number);
+      }
+    }
+    protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+        internalGetFieldAccessorTable() {
+      return org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.internal_static_protos_TSDataSchema_fieldAccessorTable
+          .ensureFieldAccessorsInitialized(
+              org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSDataSchema.class, org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSDataSchema.Builder.class);
+    }
+
+    private int bitField0_;
+    public static final int KEY_FIELD_NUMBER = 1;
+    private org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSKey key_;
+    /**
+     * <code>.protos.TSKey key = 1;</code>
+     */
+    public boolean hasKey() {
+      return key_ != null;
+    }
+    /**
+     * <code>.protos.TSKey key = 1;</code>
+     */
+    public org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSKey getKey() {
+      return key_ == null ? org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSKey.getDefaultInstance() : key_;
+    }
+    /**
+     * <code>.protos.TSKey key = 1;</code>
+     */
+    public org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSKeyOrBuilder getKeyOrBuilder() {
+      return getKey();
+    }
+
+    public static final int FEATURES_FIELD_NUMBER = 2;
+    private static final class FeaturesDefaultEntryHolder {
+      static final com.google.protobuf.MapEntry<
+          java.lang.String, org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSKey> defaultEntry =
+              com.google.protobuf.MapEntry
+              .<java.lang.String, org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSKey>newDefaultInstance(
+                  org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.internal_static_protos_TSDataSchema_FeaturesEntry_descriptor, 
+                  com.google.protobuf.WireFormat.FieldType.STRING,
+                  "",
+                  com.google.protobuf.WireFormat.FieldType.MESSAGE,
+                  org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSKey.getDefaultInstance());
+    }
+    private com.google.protobuf.MapField<
+        java.lang.String, org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSKey> features_;
+    private com.google.protobuf.MapField<java.lang.String, org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSKey>
+    internalGetFeatures() {
+      if (features_ == null) {
+        return com.google.protobuf.MapField.emptyMapField(
+            FeaturesDefaultEntryHolder.defaultEntry);
+      }
+      return features_;
+    }
+
+    public int getFeaturesCount() {
+      return internalGetFeatures().getMap().size();
+    }
+    /**
+     * <code>map&lt;string, .protos.TSKey&gt; features = 2;</code>
+     */
+
+    public boolean containsFeatures(
+        java.lang.String key) {
+      if (key == null) { throw new java.lang.NullPointerException(); }
+      return internalGetFeatures().getMap().containsKey(key);
+    }
+    /**
+     * Use {@link #getFeaturesMap()} instead.
+     */
+    @java.lang.Deprecated
+    public java.util.Map<java.lang.String, org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSKey> getFeatures() {
+      return getFeaturesMap();
+    }
+    /**
+     * <code>map&lt;string, .protos.TSKey&gt; features = 2;</code>
+     */
+
+    public java.util.Map<java.lang.String, org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSKey> getFeaturesMap() {
+      return internalGetFeatures().getMap();
+    }
+    /**
+     * <code>map&lt;string, .protos.TSKey&gt; features = 2;</code>
+     */
+
+    public org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSKey getFeaturesOrDefault(
+        java.lang.String key,
+        org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSKey defaultValue) {
+      if (key == null) { throw new java.lang.NullPointerException(); }
+      java.util.Map<java.lang.String, org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSKey> map =
+          internalGetFeatures().getMap();
+      return map.containsKey(key) ? map.get(key) : defaultValue;
+    }
+    /**
+     * <code>map&lt;string, .protos.TSKey&gt; features = 2;</code>
+     */
+
+    public org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSKey getFeaturesOrThrow(
+        java.lang.String key) {
+      if (key == null) { throw new java.lang.NullPointerException(); }
+      java.util.Map<java.lang.String, org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSKey> map =
+          internalGetFeatures().getMap();
+      if (!map.containsKey(key)) {
+        throw new java.lang.IllegalArgumentException();
+      }
+      return map.get(key);
+    }
+
+    private byte memoizedIsInitialized = -1;
+    public final boolean isInitialized() {
+      byte isInitialized = memoizedIsInitialized;
+      if (isInitialized == 1) return true;
+      if (isInitialized == 0) return false;
+
+      memoizedIsInitialized = 1;
+      return true;
+    }
+
+    public void writeTo(com.google.protobuf.CodedOutputStream output)
+                        throws java.io.IOException {
+      if (key_ != null) {
+        output.writeMessage(1, getKey());
+      }
+      com.google.protobuf.GeneratedMessageV3
+        .serializeStringMapTo(
+          output,
+          internalGetFeatures(),
+          FeaturesDefaultEntryHolder.defaultEntry,
+          2);
+    }
+
+    public int getSerializedSize() {
+      int size = memoizedSize;
+      if (size != -1) return size;
+
+      size = 0;
+      if (key_ != null) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeMessageSize(1, getKey());
+      }
+      for (java.util.Map.Entry<java.lang.String, org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSKey> entry
+           : internalGetFeatures().getMap().entrySet()) {
+        com.google.protobuf.MapEntry<java.lang.String, org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSKey>
+        features__ = FeaturesDefaultEntryHolder.defaultEntry.newBuilderForType()
+            .setKey(entry.getKey())
+            .setValue(entry.getValue())
+            .build();
+        size += com.google.protobuf.CodedOutputStream
+            .computeMessageSize(2, features__);
+      }
+      memoizedSize = size;
+      return size;
+    }
+
+    private static final long serialVersionUID = 0L;
+    @java.lang.Override
+    public boolean equals(final java.lang.Object obj) {
+      if (obj == this) {
+       return true;
+      }
+      if (!(obj instanceof org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSDataSchema)) {
+        return super.equals(obj);
+      }
+      org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSDataSchema other = (org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSDataSchema) obj;
+
+      boolean result = true;
+      result = result && (hasKey() == other.hasKey());
+      if (hasKey()) {
+        result = result && getKey()
+            .equals(other.getKey());
+      }
+      result = result && internalGetFeatures().equals(
+          other.internalGetFeatures());
+      return result;
+    }
+
+    @java.lang.Override
+    public int hashCode() {
+      if (memoizedHashCode != 0) {
+        return memoizedHashCode;
+      }
+      int hash = 41;
+      hash = (19 * hash) + getDescriptor().hashCode();
+      if (hasKey()) {
+        hash = (37 * hash) + KEY_FIELD_NUMBER;
+        hash = (53 * hash) + getKey().hashCode();
+      }
+      if (!internalGetFeatures().getMap().isEmpty()) {
+        hash = (37 * hash) + FEATURES_FIELD_NUMBER;
+        hash = (53 * hash) + internalGetFeatures().hashCode();
+      }
+      hash = (29 * hash) + unknownFields.hashCode();
+      memoizedHashCode = hash;
+      return hash;
+    }
+
+    public static org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSDataSchema parseFrom(
+        com.google.protobuf.ByteString data)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data);
+    }
+    public static org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSDataSchema parseFrom(
+        com.google.protobuf.ByteString data,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data, extensionRegistry);
+    }
+    public static org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSDataSchema parseFrom(byte[] data)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data);
+    }
+    public static org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSDataSchema parseFrom(
+        byte[] data,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data, extensionRegistry);
+    }
+    public static org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSDataSchema parseFrom(java.io.InputStream input)
+        throws java.io.IOException {
+      return com.google.protobuf.GeneratedMessageV3
+          .parseWithIOException(PARSER, input);
+    }
+    public static org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSDataSchema parseFrom(
+        java.io.InputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws java.io.IOException {
+      return com.google.protobuf.GeneratedMessageV3
+          .parseWithIOException(PARSER, input, extensionRegistry);
+    }
+    public static org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSDataSchema parseDelimitedFrom(java.io.InputStream input)
+        throws java.io.IOException {
+      return com.google.protobuf.GeneratedMessageV3
+          .parseDelimitedWithIOException(PARSER, input);
+    }
+    public static org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSDataSchema parseDelimitedFrom(
+        java.io.InputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws java.io.IOException {
+      return com.google.protobuf.GeneratedMessageV3
+          .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
+    }
+    public static org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSDataSchema parseFrom(
+        com.google.protobuf.CodedInputStream input)
+        throws java.io.IOException {
+      return com.google.protobuf.GeneratedMessageV3
+          .parseWithIOException(PARSER, input);
+    }
+    public static org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSDataSchema parseFrom(
+        com.google.protobuf.CodedInputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws java.io.IOException {
+      return com.google.protobuf.GeneratedMessageV3
+          .parseWithIOException(PARSER, input, extensionRegistry);
+    }
+
+    public Builder newBuilderForType() { return newBuilder(); }
+    public static Builder newBuilder() {
+      return DEFAULT_INSTANCE.toBuilder();
+    }
+    public static Builder newBuilder(org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSDataSchema prototype) {
+      return DEFAULT_INSTANCE.toBuilder().mergeFrom(prototype);
+    }
+    public Builder toBuilder() {
+      return this == DEFAULT_INSTANCE
+          ? new Builder() : new Builder().mergeFrom(this);
+    }
+
+    @java.lang.Override
+    protected Builder newBuilderForType(
+        com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+      Builder builder = new Builder(parent);
+      return builder;
+    }
+    /**
+     * Protobuf type {@code protos.TSDataSchema}
+     */
+    public static final class Builder extends
+        com.google.protobuf.GeneratedMessageV3.Builder<Builder> implements
+        // @@protoc_insertion_point(builder_implements:protos.TSDataSchema)
+        org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSDataSchemaOrBuilder {
+      public static final com.google.protobuf.Descriptors.Descriptor
+          getDescriptor() {
+        return org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.internal_static_protos_TSDataSchema_descriptor;
+      }
+
+      @SuppressWarnings({"rawtypes"})
+      protected com.google.protobuf.MapField internalGetMapField(
+          int number) {
+        switch (number) {
+          case 2:
+            return internalGetFeatures();
+          default:
+            throw new RuntimeException(
+                "Invalid map field number: " + number);
+        }
+      }
+      @SuppressWarnings({"rawtypes"})
+      protected com.google.protobuf.MapField internalGetMutableMapField(
+          int number) {
+        switch (number) {
+          case 2:
+            return internalGetMutableFeatures();
+          default:
+            throw new RuntimeException(
+                "Invalid map field number: " + number);
+        }
+      }
+      protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+          internalGetFieldAccessorTable() {
+        return org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.internal_static_protos_TSDataSchema_fieldAccessorTable
+            .ensureFieldAccessorsInitialized(
+                org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSDataSchema.class, org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSDataSchema.Builder.class);
+      }
+
+      // Construct using org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSDataSchema.newBuilder()
+      private Builder() {
+        maybeForceBuilderInitialization();
+      }
+
+      private Builder(
+          com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+        super(parent);
+        maybeForceBuilderInitialization();
+      }
+      private void maybeForceBuilderInitialization() {
+        if (com.google.protobuf.GeneratedMessageV3
+                .alwaysUseFieldBuilders) {
+        }
+      }
+      public Builder clear() {
+        super.clear();
+        if (keyBuilder_ == null) {
+          key_ = null;
+        } else {
+          key_ = null;
+          keyBuilder_ = null;
+        }
+        internalGetMutableFeatures().clear();
+        return this;
+      }
+
+      public com.google.protobuf.Descriptors.Descriptor
+          getDescriptorForType() {
+        return org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.internal_static_protos_TSDataSchema_descriptor;
+      }
+
+      public org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSDataSchema getDefaultInstanceForType() {
+        return org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSDataSchema.getDefaultInstance();
+      }
+
+      public org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSDataSchema build() {
+        org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSDataSchema result = buildPartial();
+        if (!result.isInitialized()) {
+          throw newUninitializedMessageException(result);
+        }
+        return result;
+      }
+
+      public org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSDataSchema buildPartial() {
+        org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSDataSchema result = new org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSDataSchema(this);
+        int from_bitField0_ = bitField0_;
+        int to_bitField0_ = 0;
+        if (keyBuilder_ == null) {
+          result.key_ = key_;
+        } else {
+          result.key_ = keyBuilder_.build();
+        }
+        result.features_ = internalGetFeatures();
+        result.features_.makeImmutable();
+        result.bitField0_ = to_bitField0_;
+        onBuilt();
+        return result;
+      }
+
+      public Builder clone() {
+        return (Builder) super.clone();
+      }
+      public Builder setField(
+          com.google.protobuf.Descriptors.FieldDescriptor field,
+          Object value) {
+        return (Builder) super.setField(field, value);
+      }
+      public Builder clearField(
+          com.google.protobuf.Descriptors.FieldDescriptor field) {
+        return (Builder) super.clearField(field);
+      }
+      public Builder clearOneof(
+          com.google.protobuf.Descriptors.OneofDescriptor oneof) {
+        return (Builder) super.clearOneof(oneof);
+      }
+      public Builder setRepeatedField(
+          com.google.protobuf.Descriptors.FieldDescriptor field,
+          int index, Object value) {
+        return (Builder) super.setRepeatedField(field, index, value);
+      }
+      public Builder addRepeatedField(
+          com.google.protobuf.Descriptors.FieldDescriptor field,
+          Object value) {
+        return (Builder) super.addRepeatedField(field, value);
+      }
+      public Builder mergeFrom(com.google.protobuf.Message other) {
+        if (other instanceof org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSDataSchema) {
+          return mergeFrom((org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSDataSchema)other);
+        } else {
+          super.mergeFrom(other);
+          return this;
+        }
+      }
+
+      public Builder mergeFrom(org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSDataSchema other) {
+        if (other == org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSDataSchema.getDefaultInstance()) return this;
+        if (other.hasKey()) {
+          mergeKey(other.getKey());
+        }
+        internalGetMutableFeatures().mergeFrom(
+            other.internalGetFeatures());
+        onChanged();
+        return this;
+      }
+
+      public final boolean isInitialized() {
+        return true;
+      }
+
+      public Builder mergeFrom(
+          com.google.protobuf.CodedInputStream input,
+          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws java.io.IOException {
+        org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSDataSchema parsedMessage = null;
+        try {
+          parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
+        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+          parsedMessage = (org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSDataSchema) e.getUnfinishedMessage();
+          throw e.unwrapIOException();
+        } finally {
+          if (parsedMessage != null) {
+            mergeFrom(parsedMessage);
+          }
+        }
+        return this;
+      }
+      private int bitField0_;
+
+      private org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSKey key_ = null;
+      private com.google.protobuf.SingleFieldBuilderV3<
+          org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSKey, org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSKey.Builder, org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSKeyOrBuilder> keyBuilder_;
+      /**
+       * <code>.protos.TSKey key = 1;</code>
+       */
+      public boolean hasKey() {
+        return keyBuilder_ != null || key_ != null;
+      }
+      /**
+       * <code>.protos.TSKey key = 1;</code>
+       */
+      public org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSKey getKey() {
+        if (keyBuilder_ == null) {
+          return key_ == null ? org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSKey.getDefaultInstance() : key_;
+        } else {
+          return keyBuilder_.getMessage();
+        }
+      }
+      /**
+       * <code>.protos.TSKey key = 1;</code>
+       */
+      public Builder setKey(org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSKey value) {
+        if (keyBuilder_ == null) {
+          if (value == null) {
+            throw new NullPointerException();
+          }
+          key_ = value;
+          onChanged();
+        } else {
+          keyBuilder_.setMessage(value);
+        }
+
+        return this;
+      }
+      /**
+       * <code>.protos.TSKey key = 1;</code>
+       */
+      public Builder setKey(
+          org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSKey.Builder builderForValue) {
+        if (keyBuilder_ == null) {
+          key_ = builderForValue.build();
+          onChanged();
+        } else {
+          keyBuilder_.setMessage(builderForValue.build());
+        }
+
+        return this;
+      }
+      /**
+       * <code>.protos.TSKey key = 1;</code>
+       */
+      public Builder mergeKey(org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSKey value) {
+        if (keyBuilder_ == null) {
+          if (key_ != null) {
+            key_ =
+              org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSKey.newBuilder(key_).mergeFrom(value).buildPartial();
+          } else {
+            key_ = value;
+          }
+          onChanged();
+        } else {
+          keyBuilder_.mergeFrom(value);
+        }
+
+        return this;
+      }
+      /**
+       * <code>.protos.TSKey key = 1;</code>
+       */
+      public Builder clearKey() {
+        if (keyBuilder_ == null) {
+          key_ = null;
+          onChanged();
+        } else {
+          key_ = null;
+          keyBuilder_ = null;
+        }
+
+        return this;
+      }
+      /**
+       * <code>.protos.TSKey key = 1;</code>
+       */
+      public org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSKey.Builder getKeyBuilder() {
+        
+        onChanged();
+        return getKeyFieldBuilder().getBuilder();
+      }
+      /**
+       * <code>.protos.TSKey key = 1;</code>
+       */
+      public org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSKeyOrBuilder getKeyOrBuilder() {
+        if (keyBuilder_ != null) {
+          return keyBuilder_.getMessageOrBuilder();
+        } else {
+          return key_ == null ?
+              org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSKey.getDefaultInstance() : key_;
+        }
+      }
+      /**
+       * <code>.protos.TSKey key = 1;</code>
+       */
+      private com.google.protobuf.SingleFieldBuilderV3<
+          org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSKey, org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSKey.Builder, org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSKeyOrBuilder> 
+          getKeyFieldBuilder() {
+        if (keyBuilder_ == null) {
+          keyBuilder_ = new com.google.protobuf.SingleFieldBuilderV3<
+              org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSKey, org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSKey.Builder, org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSKeyOrBuilder>(
+                  getKey(),
+                  getParentForChildren(),
+                  isClean());
+          key_ = null;
+        }
+        return keyBuilder_;
+      }
+
+      private com.google.protobuf.MapField<
+          java.lang.String, org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSKey> features_;
+      private com.google.protobuf.MapField<java.lang.String, org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSKey>
+      internalGetFeatures() {
+        if (features_ == null) {
+          return com.google.protobuf.MapField.emptyMapField(
+              FeaturesDefaultEntryHolder.defaultEntry);
+        }
+        return features_;
+      }
+      private com.google.protobuf.MapField<java.lang.String, org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSKey>
+      internalGetMutableFeatures() {
+        onChanged();;
+        if (features_ == null) {
+          features_ = com.google.protobuf.MapField.newMapField(
+              FeaturesDefaultEntryHolder.defaultEntry);
+        }
+        if (!features_.isMutable()) {
+          features_ = features_.copy();
+        }
+        return features_;
+      }
+
+      public int getFeaturesCount() {
+        return internalGetFeatures().getMap().size();
+      }
+      /**
+       * <code>map&lt;string, .protos.TSKey&gt; features = 2;</code>
+       */
+
+      public boolean containsFeatures(
+          java.lang.String key) {
+        if (key == null) { throw new java.lang.NullPointerException(); }
+        return internalGetFeatures().getMap().containsKey(key);
+      }
+      /**
+       * Use {@link #getFeaturesMap()} instead.
+       */
+      @java.lang.Deprecated
+      public java.util.Map<java.lang.String, org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSKey> getFeatures() {
+        return getFeaturesMap();
+      }
+      /**
+       * <code>map&lt;string, .protos.TSKey&gt; features = 2;</code>
+       */
+
+      public java.util.Map<java.lang.String, org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSKey> getFeaturesMap() {
+        return internalGetFeatures().getMap();
+      }
+      /**
+       * <code>map&lt;string, .protos.TSKey&gt; features = 2;</code>
+       */
+
+      public org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSKey getFeaturesOrDefault(
+          java.lang.String key,
+          org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSKey defaultValue) {
+        if (key == null) { throw new java.lang.NullPointerException(); }
+        java.util.Map<java.lang.String, org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSKey> map =
+            internalGetFeatures().getMap();
+        return map.containsKey(key) ? map.get(key) : defaultValue;
+      }
+      /**
+       * <code>map&lt;string, .protos.TSKey&gt; features = 2;</code>
+       */
+
+      public org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSKey getFeaturesOrThrow(
+          java.lang.String key) {
+        if (key == null) { throw new java.lang.NullPointerException(); }
+        java.util.Map<java.lang.String, org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSKey> map =
+            internalGetFeatures().getMap();
+        if (!map.containsKey(key)) {
+          throw new java.lang.IllegalArgumentException();
+        }
+        return map.get(key);
+      }
+
+      public Builder clearFeatures() {
+        internalGetMutableFeatures().getMutableMap()
+            .clear();
+        return this;
+      }
+      /**
+       * <code>map&lt;string, .protos.TSKey&gt; features = 2;</code>
+       */
+
+      public Builder removeFeatures(
+          java.lang.String key) {
+        if (key == null) { throw new java.lang.NullPointerException(); }
+        internalGetMutableFeatures().getMutableMap()
+            .remove(key);
+        return this;
+      }
+      /**
+       * Use alternate mutation accessors instead.
+       */
+      @java.lang.Deprecated
+      public java.util.Map<java.lang.String, org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSKey>
+      getMutableFeatures() {
+        return internalGetMutableFeatures().getMutableMap();
+      }
+      /**
+       * <code>map&lt;string, .protos.TSKey&gt; features = 2;</code>
+       */
+      public Builder putFeatures(
+          java.lang.String key,
+          org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSKey value) {
+        if (key == null) { throw new java.lang.NullPointerException(); }
+        if (value == null) { throw new java.lang.NullPointerException(); }
+        internalGetMutableFeatures().getMutableMap()
+            .put(key, value);
+        return this;
+      }
+      /**
+       * <code>map&lt;string, .protos.TSKey&gt; features = 2;</code>
+       */
+
+      public Builder putAllFeatures(
+          java.util.Map<java.lang.String, org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSKey> values) {
+        internalGetMutableFeatures().getMutableMap()
+            .putAll(values);
+        return this;
+      }
+      public final Builder setUnknownFields(
+          final com.google.protobuf.UnknownFieldSet unknownFields) {
+        return this;
+      }
+
+      public final Builder mergeUnknownFields(
+          final com.google.protobuf.UnknownFieldSet unknownFields) {
+        return this;
+      }
+
+
+      // @@protoc_insertion_point(builder_scope:protos.TSDataSchema)
+    }
+
+    // @@protoc_insertion_point(class_scope:protos.TSDataSchema)
+    private static final org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSDataSchema DEFAULT_INSTANCE;
+    static {
+      DEFAULT_INSTANCE = new org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSDataSchema();
+    }
+
+    public static org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSDataSchema getDefaultInstance() {
+      return DEFAULT_INSTANCE;
+    }
+
+    private static final com.google.protobuf.Parser<TSDataSchema>
+        PARSER = new com.google.protobuf.AbstractParser<TSDataSchema>() {
+      public TSDataSchema parsePartialFrom(
+          com.google.protobuf.CodedInputStream input,
+          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws com.google.protobuf.InvalidProtocolBufferException {
+          return new TSDataSchema(input, extensionRegistry);
+      }
+    };
+
+    public static com.google.protobuf.Parser<TSDataSchema> parser() {
+      return PARSER;
+    }
+
+    @java.lang.Override
+    public com.google.protobuf.Parser<TSDataSchema> getParserForType() {
+      return PARSER;
+    }
+
+    public org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSDataSchema getDefaultInstanceForType() {
+      return DEFAULT_INSTANCE;
+    }
+
+  }
+
+  public interface TSDataPointSequenceOrBuilder extends
+      // @@protoc_insertion_point(interface_extends:protos.TSDataPointSequence)
+      com.google.protobuf.MessageOrBuilder {
+
+    /**
+     * <code>.protos.TSKey key = 1;</code>
+     */
+    boolean hasKey();
+    /**
+     * <code>.protos.TSKey key = 1;</code>
+     */
+    org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSKey getKey();
+    /**
+     * <code>.protos.TSKey key = 1;</code>
+     */
+    org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSKeyOrBuilder getKeyOrBuilder();
+
+    /**
+     * <code>.protos.PipelineVersion pipelineVersion = 2;</code>
+     */
+    boolean hasPipelineVersion();
+    /**
+     * <code>.protos.PipelineVersion pipelineVersion = 2;</code>
+     */
+    org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.PipelineVersion getPipelineVersion();
+    /**
+     * <code>.protos.PipelineVersion pipelineVersion = 2;</code>
+     */
+    org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.PipelineVersionOrBuilder getPipelineVersionOrBuilder();
+
+    /**
+     * <code>repeated .protos.TSDataPoint accums = 3;</code>
+     */
+    java.util.List<org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSDataPoint> 
+        getAccumsList();
+    /**
+     * <code>repeated .protos.TSDataPoint accums = 3;</code>
+     */
+    org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSDataPoint getAccums(int index);
+    /**
+     * <code>repeated .protos.TSDataPoint accums = 3;</code>
+     */
+    int getAccumsCount();
+    /**
+     * <code>repeated .protos.TSDataPoint accums = 3;</code>
+     */
+    java.util.List<? extends org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSDataPointOrBuilder> 
+        getAccumsOrBuilderList();
+    /**
+     * <code>repeated .protos.TSDataPoint accums = 3;</code>
+     */
+    org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSDataPointOrBuilder getAccumsOrBuilder(
+        int index);
+
+    /**
+     * <code>int64 lowerWindowBoundary = 8;</code>
+     */
+    long getLowerWindowBoundary();
+
+    /**
+     * <code>int64 upperWindowBoundary = 9;</code>
+     */
+    long getUpperWindowBoundary();
+  }
+  /**
+   * <pre>
+   * Time Series sequence holder
+   * </pre>
+   *
+   * Protobuf type {@code protos.TSDataPointSequence}
+   */
+  public  static final class TSDataPointSequence extends
+      com.google.protobuf.GeneratedMessageV3 implements
+      // @@protoc_insertion_point(message_implements:protos.TSDataPointSequence)
+      TSDataPointSequenceOrBuilder {
+    // Use TSDataPointSequence.newBuilder() to construct.
+    private TSDataPointSequence(com.google.protobuf.GeneratedMessageV3.Builder<?> builder) {
+      super(builder);
+    }
+    private TSDataPointSequence() {
+      accums_ = java.util.Collections.emptyList();
+      lowerWindowBoundary_ = 0L;
+      upperWindowBoundary_ = 0L;
+    }
+
+    @java.lang.Override
+    public final com.google.protobuf.UnknownFieldSet
+    getUnknownFields() {
+      return com.google.protobuf.UnknownFieldSet.getDefaultInstance();
+    }
+    private TSDataPointSequence(
+        com.google.protobuf.CodedInputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      this();
+      int mutable_bitField0_ = 0;
+      try {
+        boolean done = false;
+        while (!done) {
+          int tag = input.readTag();
+          switch (tag) {
+            case 0:
+              done = true;
+              break;
+            default: {
+              if (!input.skipField(tag)) {
+                done = true;
+              }
+              break;
+            }
+            case 10: {
+              org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSKey.Builder subBuilder = null;
+              if (key_ != null) {
+                subBuilder = key_.toBuilder();
+              }
+              key_ = input.readMessage(org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSKey.parser(), extensionRegistry);
+              if (subBuilder != null) {
+                subBuilder.mergeFrom(key_);
+                key_ = subBuilder.buildPartial();
+              }
+
+              break;
+            }
+            case 18: {
+              org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.PipelineVersion.Builder subBuilder = null;
+              if (pipelineVersion_ != null) {
+                subBuilder = pipelineVersion_.toBuilder();
+              }
+              pipelineVersion_ = input.readMessage(org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.PipelineVersion.parser(), extensionRegistry);
+              if (subBuilder != null) {
+                subBuilder.mergeFrom(pipelineVersion_);
+                pipelineVersion_ = subBuilder.buildPartial();
+              }
+
+              break;
+            }
+            case 26: {
+              if (!((mutable_bitField0_ & 0x00000004) == 0x00000004)) {
+                accums_ = new java.util.ArrayList<org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSDataPoint>();
+                mutable_bitField0_ |= 0x00000004;
+              }
+              accums_.add(
+                  input.readMessage(org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSDataPoint.parser(), extensionRegistry));
+              break;
+            }
+            case 64: {
+
+              lowerWindowBoundary_ = input.readInt64();
+              break;
+            }
+            case 72: {
+
+              upperWindowBoundary_ = input.readInt64();
+              break;
+            }
+          }
+        }
+      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        throw e.setUnfinishedMessage(this);
+      } catch (java.io.IOException e) {
+        throw new com.google.protobuf.InvalidProtocolBufferException(
+            e).setUnfinishedMessage(this);
+      } finally {
+        if (((mutable_bitField0_ & 0x00000004) == 0x00000004)) {
+          accums_ = java.util.Collections.unmodifiableList(accums_);
+        }
+        makeExtensionsImmutable();
+      }
+    }
+    public static final com.google.protobuf.Descriptors.Descriptor
+        getDescriptor() {
+      return org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.internal_static_protos_TSDataPointSequence_descriptor;
+    }
+
+    protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+        internalGetFieldAccessorTable() {
+      return org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.internal_static_protos_TSDataPointSequence_fieldAccessorTable
+          .ensureFieldAccessorsInitialized(
+              org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSDataPointSequence.class, org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSDataPointSequence.Builder.class);
+    }
+
+    private int bitField0_;
+    public static final int KEY_FIELD_NUMBER = 1;
+    private org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSKey key_;
+    /**
+     * <code>.protos.TSKey key = 1;</code>
+     */
+    public boolean hasKey() {
+      return key_ != null;
+    }
+    /**
+     * <code>.protos.TSKey key = 1;</code>
+     */
+    public org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSKey getKey() {
+      return key_ == null ? org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSKey.getDefaultInstance() : key_;
+    }
+    /**
+     * <code>.protos.TSKey key = 1;</code>
+     */
+    public org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSKeyOrBuilder getKeyOrBuilder() {
+      return getKey();
+    }
+
+    public static final int PIPELINEVERSION_FIELD_NUMBER = 2;
+    private org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.PipelineVersion pipelineVersion_;
+    /**
+     * <code>.protos.PipelineVersion pipelineVersion = 2;</code>
+     */
+    public boolean hasPipelineVersion() {
+      return pipelineVersion_ != null;
+    }
+    /**
+     * <code>.protos.PipelineVersion pipelineVersion = 2;</code>
+     */
+    public org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.PipelineVersion getPipelineVersion() {
+      return pipelineVersion_ == null ? org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.PipelineVersion.getDefaultInstance() : pipelineVersion_;
+    }
+    /**
+     * <code>.protos.PipelineVersion pipelineVersion = 2;</code>
+     */
+    public org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.PipelineVersionOrBuilder getPipelineVersionOrBuilder() {
+      return getPipelineVersion();
+    }
+
+    public static final int ACCUMS_FIELD_NUMBER = 3;
+    private java.util.List<org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSDataPoint> accums_;
+    /**
+     * <code>repeated .protos.TSDataPoint accums = 3;</code>
+     */
+    public java.util.List<org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSDataPoint> getAccumsList() {
+      return accums_;
+    }
+    /**
+     * <code>repeated .protos.TSDataPoint accums = 3;</code>
+     */
+    public java.util.List<? extends org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSDataPointOrBuilder> 
+        getAccumsOrBuilderList() {
+      return accums_;
+    }
+    /**
+     * <code>repeated .protos.TSDataPoint accums = 3;</code>
+     */
+    public int getAccumsCount() {
+      return accums_.size();
+    }
+    /**
+     * <code>repeated .protos.TSDataPoint accums = 3;</code>
+     */
+    public org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSDataPoint getAccums(int index) {
+      return accums_.get(index);
+    }
+    /**
+     * <code>repeated .protos.TSDataPoint accums = 3;</code>
+     */
+    public org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSDataPointOrBuilder getAccumsOrBuilder(
+        int index) {
+      return accums_.get(index);
+    }
+
+    public static final int LOWERWINDOWBOUNDARY_FIELD_NUMBER = 8;
+    private long lowerWindowBoundary_;
+    /**
+     * <code>int64 lowerWindowBoundary = 8;</code>
+     */
+    public long getLowerWindowBoundary() {
+      return lowerWindowBoundary_;
+    }
+
+    public static final int UPPERWINDOWBOUNDARY_FIELD_NUMBER = 9;
+    private long upperWindowBoundary_;
+    /**
+     * <code>int64 upperWindowBoundary = 9;</code>
+     */
+    public long getUpperWindowBoundary() {
+      return upperWindowBoundary_;
+    }
+
+    private byte memoizedIsInitialized = -1;
+    public final boolean isInitialized() {
+      byte isInitialized = memoizedIsInitialized;
+      if (isInitialized == 1) return true;
+      if (isInitialized == 0) return false;
+
+      memoizedIsInitialized = 1;
+      return true;
+    }
+
+    public void writeTo(com.google.protobuf.CodedOutputStream output)
+                        throws java.io.IOException {
+      if (key_ != null) {
+        output.writeMessage(1, getKey());
+      }
+      if (pipelineVersion_ != null) {
+        output.writeMessage(2, getPipelineVersion());
+      }
+      for (int i = 0; i < accums_.size(); i++) {
+        output.writeMessage(3, accums_.get(i));
+      }
+      if (lowerWindowBoundary_ != 0L) {
+        output.writeInt64(8, lowerWindowBoundary_);
+      }
+      if (upperWindowBoundary_ != 0L) {
+        output.writeInt64(9, upperWindowBoundary_);
+      }
+    }
+
+    public int getSerializedSize() {
+      int size = memoizedSize;
+      if (size != -1) return size;
+
+      size = 0;
+      if (key_ != null) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeMessageSize(1, getKey());
+      }
+      if (pipelineVersion_ != null) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeMessageSize(2, getPipelineVersion());
+      }
+      for (int i = 0; i < accums_.size(); i++) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeMessageSize(3, accums_.get(i));
+      }
+      if (lowerWindowBoundary_ != 0L) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeInt64Size(8, lowerWindowBoundary_);
+      }
+      if (upperWindowBoundary_ != 0L) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeInt64Size(9, upperWindowBoundary_);
+      }
+      memoizedSize = size;
+      return size;
+    }
+
+    private static final long serialVersionUID = 0L;
+    @java.lang.Override
+    public boolean equals(final java.lang.Object obj) {
+      if (obj == this) {
+       return true;
+      }
+      if (!(obj instanceof org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSDataPointSequence)) {
+        return super.equals(obj);
+      }
+      org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSDataPointSequence other = (org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSDataPointSequence) obj;
+
+      boolean result = true;
+      result = result && (hasKey() == other.hasKey());
+      if (hasKey()) {
+        result = result && getKey()
+            .equals(other.getKey());
+      }
+      result = result && (hasPipelineVersion() == other.hasPipelineVersion());
+      if (hasPipelineVersion()) {
+        result = result && getPipelineVersion()
+            .equals(other.getPipelineVersion());
+      }
+      result = result && getAccumsList()
+          .equals(other.getAccumsList());
+      result = result && (getLowerWindowBoundary()
+          == other.getLowerWindowBoundary());
+      result = result && (getUpperWindowBoundary()
+          == other.getUpperWindowBoundary());
+      return result;
+    }
+
+    @java.lang.Override
+    public int hashCode() {
+      if (memoizedHashCode != 0) {
+        return memoizedHashCode;
+      }
+      int hash = 41;
+      hash = (19 * hash) + getDescriptor().hashCode();
+      if (hasKey()) {
+        hash = (37 * hash) + KEY_FIELD_NUMBER;
+        hash = (53 * hash) + getKey().hashCode();
+      }
+      if (hasPipelineVersion()) {
+        hash = (37 * hash) + PIPELINEVERSION_FIELD_NUMBER;
+        hash = (53 * hash) + getPipelineVersion().hashCode();
+      }
+      if (getAccumsCount() > 0) {
+        hash = (37 * hash) + ACCUMS_FIELD_NUMBER;
+        hash = (53 * hash) + getAccumsList().hashCode();
+      }
+      hash = (37 * hash) + LOWERWINDOWBOUNDARY_FIELD_NUMBER;
+      hash = (53 * hash) + com.google.protobuf.Internal.hashLong(
+          getLowerWindowBoundary());
+      hash = (37 * hash) + UPPERWINDOWBOUNDARY_FIELD_NUMBER;
+      hash = (53 * hash) + com.google.protobuf.Internal.hashLong(
+          getUpperWindowBoundary());
+      hash = (29 * hash) + unknownFields.hashCode();
+      memoizedHashCode = hash;
+      return hash;
+    }
+
+    public static org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSDataPointSequence parseFrom(
+        com.google.protobuf.ByteString data)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data);
+    }
+    public static org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSDataPointSequence parseFrom(
+        com.google.protobuf.ByteString data,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data, extensionRegistry);
+    }
+    public static org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSDataPointSequence parseFrom(byte[] data)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data);
+    }
+    public static org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSDataPointSequence parseFrom(
+        byte[] data,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data, extensionRegistry);
+    }
+    public static org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSDataPointSequence parseFrom(java.io.InputStream input)
+        throws java.io.IOException {
+      return com.google.protobuf.GeneratedMessageV3
+          .parseWithIOException(PARSER, input);
+    }
+    public static org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSDataPointSequence parseFrom(
+        java.io.InputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws java.io.IOException {
+      return com.google.protobuf.GeneratedMessageV3
+          .parseWithIOException(PARSER, input, extensionRegistry);
+    }
+    public static org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSDataPointSequence parseDelimitedFrom(java.io.InputStream input)
+        throws java.io.IOException {
+      return com.google.protobuf.GeneratedMessageV3
+          .parseDelimitedWithIOException(PARSER, input);
+    }
+    public static org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSDataPointSequence parseDelimitedFrom(
+        java.io.InputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws java.io.IOException {
+      return com.google.protobuf.GeneratedMessageV3
+          .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
+    }
+    public static org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSDataPointSequence parseFrom(
+        com.google.protobuf.CodedInputStream input)
+        throws java.io.IOException {
+      return com.google.protobuf.GeneratedMessageV3
+          .parseWithIOException(PARSER, input);
+    }
+    public static org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSDataPointSequence parseFrom(
+        com.google.protobuf.CodedInputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws java.io.IOException {
+      return com.google.protobuf.GeneratedMessageV3
+          .parseWithIOException(PARSER, input, extensionRegistry);
+    }
+
+    public Builder newBuilderForType() { return newBuilder(); }
+    public static Builder newBuilder() {
+      return DEFAULT_INSTANCE.toBuilder();
+    }
+    public static Builder newBuilder(org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSDataPointSequence prototype) {
+      return DEFAULT_INSTANCE.toBuilder().mergeFrom(prototype);
+    }
+    public Builder toBuilder() {
+      return this == DEFAULT_INSTANCE
+          ? new Builder() : new Builder().mergeFrom(this);
+    }
+
+    @java.lang.Override
+    protected Builder newBuilderForType(
+        com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+      Builder builder = new Builder(parent);
+      return builder;
+    }
+    /**
+     * <pre>
+     * Time Series sequence holder
+     * </pre>
+     *
+     * Protobuf type {@code protos.TSDataPointSequence}
+     */
+    public static final class Builder extends
+        com.google.protobuf.GeneratedMessageV3.Builder<Builder> implements
+        // @@protoc_insertion_point(builder_implements:protos.TSDataPointSequence)
+        org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSDataPointSequenceOrBuilder {
+      public static final com.google.protobuf.Descriptors.Descriptor
+          getDescriptor() {
+        return org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.internal_static_protos_TSDataPointSequence_descriptor;
+      }
+
+      protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+          internalGetFieldAccessorTable() {
+        return org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.internal_static_protos_TSDataPointSequence_fieldAccessorTable
+            .ensureFieldAccessorsInitialized(
+                org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSDataPointSequence.class, org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSDataPointSequence.Builder.class);
+      }
+
+      // Construct using org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSDataPointSequence.newBuilder()
+      private Builder() {
+        maybeForceBuilderInitialization();
+      }
+
+      private Builder(
+          com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+        super(parent);
+        maybeForceBuilderInitialization();
+      }
+      private void maybeForceBuilderInitialization() {
+        if (com.google.protobuf.GeneratedMessageV3
+                .alwaysUseFieldBuilders) {
+          getAccumsFieldBuilder();
+        }
+      }
+      public Builder clear() {
+        super.clear();
+        if (keyBuilder_ == null) {
+          key_ = null;
+        } else {
+          key_ = null;
+          keyBuilder_ = null;
+        }
+        if (pipelineVersionBuilder_ == null) {
+          pipelineVersion_ = null;
+        } else {
+          pipelineVersion_ = null;
+          pipelineVersionBuilder_ = null;
+        }
+        if (accumsBuilder_ == null) {
+          accums_ = java.util.Collections.emptyList();
+          bitField0_ = (bitField0_ & ~0x00000004);
+        } else {
+          accumsBuilder_.clear();
+        }
+        lowerWindowBoundary_ = 0L;
+
+        upperWindowBoundary_ = 0L;
+
+        return this;
+      }
+
+      public com.google.protobuf.Descriptors.Descriptor
+          getDescriptorForType() {
+        return org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.internal_static_protos_TSDataPointSequence_descriptor;
+      }
+
+      public org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSDataPointSequence getDefaultInstanceForType() {
+        return org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSDataPointSequence.getDefaultInstance();
+      }
+
+      public org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSDataPointSequence build() {
+        org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSDataPointSequence result = buildPartial();
+        if (!result.isInitialized()) {
+          throw newUninitializedMessageException(result);
+        }
+        return result;
+      }
+
+      public org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSDataPointSequence buildPartial() {
+        org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSDataPointSequence result = new org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSDataPointSequence(this);
+        int from_bitField0_ = bitField0_;
+        int to_bitField0_ = 0;
+        if (keyBuilder_ == null) {
+          result.key_ = key_;
+        } else {
+          result.key_ = keyBuilder_.build();
+        }
+        if (pipelineVersionBuilder_ == null) {
+          result.pipelineVersion_ = pipelineVersion_;
+        } else {
+          result.pipelineVersion_ = pipelineVersionBuilder_.build();
+        }
+        if (accumsBuilder_ == null) {
+          if (((bitField0_ & 0x00000004) == 0x00000004)) {
+            accums_ = java.util.Collections.unmodifiableList(accums_);
+            bitField0_ = (bitField0_ & ~0x00000004);
+          }
+          result.accums_ = accums_;
+        } else {
+          result.accums_ = accumsBuilder_.build();
+        }
+        result.lowerWindowBoundary_ = lowerWindowBoundary_;
+        result.upperWindowBoundary_ = upperWindowBoundary_;
+        result.bitField0_ = to_bitField0_;
+        onBuilt();
+        return result;
+      }
+
+      public Builder clone() {
+        return (Builder) super.clone();
+      }
+      public Builder setField(
+          com.google.protobuf.Descriptors.FieldDescriptor field,
+          Object value) {
+        return (Builder) super.setField(field, value);
+      }
+      public Builder clearField(
+          com.google.protobuf.Descriptors.FieldDescriptor field) {
+        return (Builder) super.clearField(field);
+      }
+      public Builder clearOneof(
+          com.google.protobuf.Descriptors.OneofDescriptor oneof) {
+        return (Builder) super.clearOneof(oneof);
+      }
+      public Builder setRepeatedField(
+          com.google.protobuf.Descriptors.FieldDescriptor field,
+          int index, Object value) {
+        return (Builder) super.setRepeatedField(field, index, value);
+      }
+      public Builder addRepeatedField(
+          com.google.protobuf.Descriptors.FieldDescriptor field,
+          Object value) {
+        return (Builder) super.addRepeatedField(field, value);
+      }
+      public Builder mergeFrom(com.google.protobuf.Message other) {
+        if (other instanceof org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSDataPointSequence) {
+          return mergeFrom((org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSDataPointSequence)other);
+        } else {
+          super.mergeFrom(other);
+          return this;
+        }
+      }
+
+      public Builder mergeFrom(org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSDataPointSequence other) {
+        if (other == org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSDataPointSequence.getDefaultInstance()) return this;
+        if (other.hasKey()) {
+          mergeKey(other.getKey());
+        }
+        if (other.hasPipelineVersion()) {
+          mergePipelineVersion(other.getPipelineVersion());
+        }
+        if (accumsBuilder_ == null) {
+          if (!other.accums_.isEmpty()) {
+            if (accums_.isEmpty()) {
+              accums_ = other.accums_;
+              bitField0_ = (bitField0_ & ~0x00000004);
+            } else {
+              ensureAccumsIsMutable();
+              accums_.addAll(other.accums_);
+            }
+            onChanged();
+          }
+        } else {
+          if (!other.accums_.isEmpty()) {
+            if (accumsBuilder_.isEmpty()) {
+              accumsBuilder_.dispose();
+              accumsBuilder_ = null;
+              accums_ = other.accums_;
+              bitField0_ = (bitField0_ & ~0x00000004);
+              accumsBuilder_ = 
+                com.google.protobuf.GeneratedMessageV3.alwaysUseFieldBuilders ?
+                   getAccumsFieldBuilder() : null;
+            } else {
+              accumsBuilder_.addAllMessages(other.accums_);
+            }
+          }
+        }
+        if (other.getLowerWindowBoundary() != 0L) {
+          setLowerWindowBoundary(other.getLowerWindowBoundary());
+        }
+        if (other.getUpperWindowBoundary() != 0L) {
+          setUpperWindowBoundary(other.getUpperWindowBoundary());
+        }
+        onChanged();
+        return this;
+      }
+
+      public final boolean isInitialized() {
+        return true;
+      }
+
+      public Builder mergeFrom(
+          com.google.protobuf.CodedInputStream input,
+          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws java.io.IOException {
+        org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSDataPointSequence parsedMessage = null;
+        try {
+          parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
+        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+          parsedMessage = (org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSDataPointSequence) e.getUnfinishedMessage();
+          throw e.unwrapIOException();
+        } finally {
+          if (parsedMessage != null) {
+            mergeFrom(parsedMessage);
+          }
+        }
+        return this;
+      }
+      private int bitField0_;
+
+      private org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSKey key_ = null;
+      private com.google.protobuf.SingleFieldBuilderV3<
+          org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSKey, org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSKey.Builder, org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSKeyOrBuilder> keyBuilder_;
+      /**
+       * <code>.protos.TSKey key = 1;</code>
+       */
+      public boolean hasKey() {
+        return keyBuilder_ != null || key_ != null;
+      }
+      /**
+       * <code>.protos.TSKey key = 1;</code>
+       */
+      public org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSKey getKey() {
+        if (keyBuilder_ == null) {
+          return key_ == null ? org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSKey.getDefaultInstance() : key_;
+        } else {
+          return keyBuilder_.getMessage();
+        }
+      }
+      /**
+       * <code>.protos.TSKey key = 1;</code>
+       */
+      public Builder setKey(org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSKey value) {
+        if (keyBuilder_ == null) {
+          if (value == null) {
+            throw new NullPointerException();
+          }
+          key_ = value;
+          onChanged();
+        } else {
+          keyBuilder_.setMessage(value);
+        }
+
+        return this;
+      }
+      /**
+       * <code>.protos.TSKey key = 1;</code>
+       */
+      public Builder setKey(
+          org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSKey.Builder builderForValue) {
+        if (keyBuilder_ == null) {
+          key_ = builderForValue.build();
+          onChanged();
+        } else {
+          keyBuilder_.setMessage(builderForValue.build());
+        }
+
+        return this;
+      }
+      /**
+       * <code>.protos.TSKey key = 1;</code>
+       */
+      public Builder mergeKey(org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSKey value) {
+        if (keyBuilder_ == null) {
+          if (key_ != null) {
+            key_ =
+              org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSKey.newBuilder(key_).mergeFrom(value).buildPartial();
+          } else {
+            key_ = value;
+          }
+          onChanged();
+        } else {
+          keyBuilder_.mergeFrom(value);
+        }
+
+        return this;
+      }
+      /**
+       * <code>.protos.TSKey key = 1;</code>
+       */
+      public Builder clearKey() {
+        if (keyBuilder_ == null) {
+          key_ = null;
+          onChanged();
+        } else {
+          key_ = null;
+          keyBuilder_ = null;
+        }
+
+        return this;
+      }
+      /**
+       * <code>.protos.TSKey key = 1;</code>
+       */
+      public org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSKey.Builder getKeyBuilder() {
+        
+        onChanged();
+        return getKeyFieldBuilder().getBuilder();
+      }
+      /**
+       * <code>.protos.TSKey key = 1;</code>
+       */
+      public org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSKeyOrBuilder getKeyOrBuilder() {
+        if (keyBuilder_ != null) {
+          return keyBuilder_.getMessageOrBuilder();
+        } else {
+          return key_ == null ?
+              org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSKey.getDefaultInstance() : key_;
+        }
+      }
+      /**
+       * <code>.protos.TSKey key = 1;</code>
+       */
+      private com.google.protobuf.SingleFieldBuilderV3<
+          org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSKey, org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSKey.Builder, org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSKeyOrBuilder> 
+          getKeyFieldBuilder() {
+        if (keyBuilder_ == null) {
+          keyBuilder_ = new com.google.protobuf.SingleFieldBuilderV3<
+              org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSKey, org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSKey.Builder, org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSKeyOrBuilder>(
+                  getKey(),
+                  getParentForChildren(),
+                  isClean());
+          key_ = null;
+        }
+        return keyBuilder_;
+      }
+
+      private org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.PipelineVersion pipelineVersion_ = null;
+      private com.google.protobuf.SingleFieldBuilderV3<
+          org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.PipelineVersion, org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.PipelineVersion.Builder, org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.PipelineVersionOrBuilder> pipelineVersionBuilder_;
+      /**
+       * <code>.protos.PipelineVersion pipelineVersion = 2;</code>
+       */
+      public boolean hasPipelineVersion() {
+        return pipelineVersionBuilder_ != null || pipelineVersion_ != null;
+      }
+      /**
+       * <code>.protos.PipelineVersion pipelineVersion = 2;</code>
+       */
+      public org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.PipelineVersion getPipelineVersion() {
+        if (pipelineVersionBuilder_ == null) {
+          return pipelineVersion_ == null ? org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.PipelineVersion.getDefaultInstance() : pipelineVersion_;
+        } else {
+          return pipelineVersionBuilder_.getMessage();
+        }
+      }
+      /**
+       * <code>.protos.PipelineVersion pipelineVersion = 2;</code>
+       */
+      public Builder setPipelineVersion(org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.PipelineVersion value) {
+        if (pipelineVersionBuilder_ == null) {
+          if (value == null) {
+            throw new NullPointerException();
+          }
+          pipelineVersion_ = value;
+          onChanged();
+        } else {
+          pipelineVersionBuilder_.setMessage(value);
+        }
+
+        return this;
+      }
+      /**
+       * <code>.protos.PipelineVersion pipelineVersion = 2;</code>
+       */
+      public Builder setPipelineVersion(
+          org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.PipelineVersion.Builder builderForValue) {
+        if (pipelineVersionBuilder_ == null) {
+          pipelineVersion_ = builderForValue.build();
+          onChanged();
+        } else {
+          pipelineVersionBuilder_.setMessage(builderForValue.build());
+        }
+
+        return this;
+      }
+      /**
+       * <code>.protos.PipelineVersion pipelineVersion = 2;</code>
+       */
+      public Builder mergePipelineVersion(org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.PipelineVersion value) {
+        if (pipelineVersionBuilder_ == null) {
+          if (pipelineVersion_ != null) {
+            pipelineVersion_ =
+              org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.PipelineVersion.newBuilder(pipelineVersion_).mergeFrom(value).buildPartial();
+          } else {
+            pipelineVersion_ = value;
+          }
+          onChanged();
+        } else {
+          pipelineVersionBuilder_.mergeFrom(value);
+        }
+
+        return this;
+      }
+      /**
+       * <code>.protos.PipelineVersion pipelineVersion = 2;</code>
+       */
+      public Builder clearPipelineVersion() {
+        if (pipelineVersionBuilder_ == null) {
+          pipelineVersion_ = null;
+          onChanged();
+        } else {
+          pipelineVersion_ = null;
+          pipelineVersionBuilder_ = null;
+        }
+
+        return this;
+      }
+      /**
+       * <code>.protos.PipelineVersion pipelineVersion = 2;</code>
+       */
+      public org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.PipelineVersion.Builder getPipelineVersionBuilder() {
+        
+        onChanged();
+        return getPipelineVersionFieldBuilder().getBuilder();
+      }
+      /**
+       * <code>.protos.PipelineVersion pipelineVersion = 2;</code>
+       */
+      public org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.PipelineVersionOrBuilder getPipelineVersionOrBuilder() {
+        if (pipelineVersionBuilder_ != null) {
+          return pipelineVersionBuilder_.getMessageOrBuilder();
+        } else {
+          return pipelineVersion_ == null ?
+              org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.PipelineVersion.getDefaultInstance() : pipelineVersion_;
+        }
+      }
+      /**
+       * <code>.protos.PipelineVersion pipelineVersion = 2;</code>
+       */
+      private com.google.protobuf.SingleFieldBuilderV3<
+          org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.PipelineVersion, org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.PipelineVersion.Builder, org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.PipelineVersionOrBuilder> 
+          getPipelineVersionFieldBuilder() {
+        if (pipelineVersionBuilder_ == null) {
+          pipelineVersionBuilder_ = new com.google.protobuf.SingleFieldBuilderV3<
+              org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.PipelineVersion, org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.PipelineVersion.Builder, org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.PipelineVersionOrBuilder>(
+                  getPipelineVersion(),
+                  getParentForChildren(),
+                  isClean());
+          pipelineVersion_ = null;
+        }
+        return pipelineVersionBuilder_;
+      }
+
+      private java.util.List<org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSDataPoint> accums_ =
+        java.util.Collections.emptyList();
+      private void ensureAccumsIsMutable() {
+        if (!((bitField0_ & 0x00000004) == 0x00000004)) {
+          accums_ = new java.util.ArrayList<org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSDataPoint>(accums_);
+          bitField0_ |= 0x00000004;
+         }
+      }
+
+      private com.google.protobuf.RepeatedFieldBuilderV3<
+          org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSDataPoint, org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSDataPoint.Builder, org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSDataPointOrBuilder> accumsBuilder_;
+
+      /**
+       * <code>repeated .protos.TSDataPoint accums = 3;</code>
+       */
+      public java.util.List<org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSDataPoint> getAccumsList() {
+        if (accumsBuilder_ == null) {
+          return java.util.Collections.unmodifiableList(accums_);
+        } else {
+          return accumsBuilder_.getMessageList();
+        }
+      }
+      /**
+       * <code>repeated .protos.TSDataPoint accums = 3;</code>
+       */
+      public int getAccumsCount() {
+        if (accumsBuilder_ == null) {
+          return accums_.size();
+        } else {
+          return accumsBuilder_.getCount();
+        }
+      }
+      /**
+       * <code>repeated .protos.TSDataPoint accums = 3;</code>
+       */
+      public org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSDataPoint getAccums(int index) {
+        if (accumsBuilder_ == null) {
+          return accums_.get(index);
+        } else {
+          return accumsBuilder_.getMessage(index);
+        }
+      }
+      /**
+       * <code>repeated .protos.TSDataPoint accums = 3;</code>
+       */
+      public Builder setAccums(
+          int index, org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSDataPoint value) {
+        if (accumsBuilder_ == null) {
+          if (value == null) {
+            throw new NullPointerException();
+          }
+          ensureAccumsIsMutable();
+          accums_.set(index, value);
+          onChanged();
+        } else {
+          accumsBuilder_.setMessage(index, value);
+        }
+        return this;
+      }
+      /**
+       * <code>repeated .protos.TSDataPoint accums = 3;</code>
+       */
+      public Builder setAccums(
+          int index, org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSDataPoint.Builder builderForValue) {
+        if (accumsBuilder_ == null) {
+          ensureAccumsIsMutable();
+          accums_.set(index, builderForValue.build());
+          onChanged();
+        } else {
+          accumsBuilder_.setMessage(index, builderForValue.build());
+        }
+        return this;
+      }
+      /**
+       * <code>repeated .protos.TSDataPoint accums = 3;</code>
+       */
+      public Builder addAccums(org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSDataPoint value) {
+        if (accumsBuilder_ == null) {
+          if (value == null) {
+            throw new NullPointerException();
+          }
+          ensureAccumsIsMutable();
+          accums_.add(value);
+          onChanged();
+        } else {
+          accumsBuilder_.addMessage(value);
+        }
+        return this;
+      }
+      /**
+       * <code>repeated .protos.TSDataPoint accums = 3;</code>
+       */
+      public Builder addAccums(
+          int index, org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSDataPoint value) {
+        if (accumsBuilder_ == null) {
+          if (value == null) {
+            throw new NullPointerException();
+          }
+          ensureAccumsIsMutable();
+          accums_.add(index, value);
+          onChanged();
+        } else {
+          accumsBuilder_.addMessage(index, value);
+        }
+        return this;
+      }
+      /**
+       * <code>repeated .protos.TSDataPoint accums = 3;</code>
+       */
+      public Builder addAccums(
+          org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSDataPoint.Builder builderForValue) {
+        if (accumsBuilder_ == null) {
+          ensureAccumsIsMutable();
+          accums_.add(builderForValue.build());
+          onChanged();
+        } else {
+          accumsBuilder_.addMessage(builderForValue.build());
+        }
+        return this;
+      }
+      /**
+       * <code>repeated .protos.TSDataPoint accums = 3;</code>
+       */
+      public Builder addAccums(
+          int index, org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSDataPoint.Builder builderForValue) {
+        if (accumsBuilder_ == null) {
+          ensureAccumsIsMutable();
+          accums_.add(index, builderForValue.build());
+          onChanged();
+        } else {
+          accumsBuilder_.addMessage(index, builderForValue.build());
+        }
+        return this;
+      }
+      /**
+       * <code>repeated .protos.TSDataPoint accums = 3;</code>
+       */
+      public Builder addAllAccums(
+          java.lang.Iterable<? extends org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSDataPoint> values) {
+        if (accumsBuilder_ == null) {
+          ensureAccumsIsMutable();
+          com.google.protobuf.AbstractMessageLite.Builder.addAll(
+              values, accums_);
+          onChanged();
+        } else {
+          accumsBuilder_.addAllMessages(values);
+        }
+        return this;
+      }
+      /**
+       * <code>repeated .protos.TSDataPoint accums = 3;</code>
+       */
+      public Builder clearAccums() {
+        if (accumsBuilder_ == null) {
+          accums_ = java.util.Collections.emptyList();
+          bitField0_ = (bitField0_ & ~0x00000004);
+          onChanged();
+        } else {
+          accumsBuilder_.clear();
+        }
+        return this;
+      }
+      /**
+       * <code>repeated .protos.TSDataPoint accums = 3;</code>
+       */
+      public Builder removeAccums(int index) {
+        if (accumsBuilder_ == null) {
+          ensureAccumsIsMutable();
+          accums_.remove(index);
+          onChanged();
+        } else {
+          accumsBuilder_.remove(index);
+        }
+        return this;
+      }
+      /**
+       * <code>repeated .protos.TSDataPoint accums = 3;</code>
+       */
+      public org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSDataPoint.Builder getAccumsBuilder(
+          int index) {
+        return getAccumsFieldBuilder().getBuilder(index);
+      }
+      /**
+       * <code>repeated .protos.TSDataPoint accums = 3;</code>
+       */
+      public org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSDataPointOrBuilder getAccumsOrBuilder(
+          int index) {
+        if (accumsBuilder_ == null) {
+          return accums_.get(index);  } else {
+          return accumsBuilder_.getMessageOrBuilder(index);
+        }
+      }
+      /**
+       * <code>repeated .protos.TSDataPoint accums = 3;</code>
+       */
+      public java.util.List<? extends org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSDataPointOrBuilder> 
+           getAccumsOrBuilderList() {
+        if (accumsBuilder_ != null) {
+          return accumsBuilder_.getMessageOrBuilderList();
+        } else {
+          return java.util.Collections.unmodifiableList(accums_);
+        }
+      }
+      /**
+       * <code>repeated .protos.TSDataPoint accums = 3;</code>
+       */
+      public org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSDataPoint.Builder addAccumsBuilder() {
+        return getAccumsFieldBuilder().addBuilder(
+            org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSDataPoint.getDefaultInstance());
+      }
+      /**
+       * <code>repeated .protos.TSDataPoint accums = 3;</code>
+       */
+      public org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSDataPoint.Builder addAccumsBuilder(
+          int index) {
+        return getAccumsFieldBuilder().addBuilder(
+            index, org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSDataPoint.getDefaultInstance());
+      }
+      /**
+       * <code>repeated .protos.TSDataPoint accums = 3;</code>
+       */
+      public java.util.List<org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSDataPoint.Builder> 
+           getAccumsBuilderList() {
+        return getAccumsFieldBuilder().getBuilderList();
+      }
+      private com.google.protobuf.RepeatedFieldBuilderV3<
+          org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSDataPoint, org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSDataPoint.Builder, org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSDataPointOrBuilder> 
+          getAccumsFieldBuilder() {
+        if (accumsBuilder_ == null) {
+          accumsBuilder_ = new com.google.protobuf.RepeatedFieldBuilderV3<
+              org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSDataPoint, org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSDataPoint.Builder, org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSDataPointOrBuilder>(
+                  accums_,
+                  ((bitField0_ & 0x00000004) == 0x00000004),
+                  getParentForChildren(),
+                  isClean());
+          accums_ = null;
+        }
+        return accumsBuilder_;
+      }
+
+      private long lowerWindowBoundary_ ;
+      /**
+       * <code>int64 lowerWindowBoundary = 8;</code>
+       */
+      public long getLowerWindowBoundary() {
+        return lowerWindowBoundary_;
+      }
+      /**
+       * <code>int64 lowerWindowBoundary = 8;</code>
+       */
+      public Builder setLowerWindowBoundary(long value) {
+        
+        lowerWindowBoundary_ = value;
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>int64 lowerWindowBoundary = 8;</code>
+       */
+      public Builder clearLowerWindowBoundary() {
+        
+        lowerWindowBoundary_ = 0L;
+        onChanged();
+        return this;
+      }
+
+      private long upperWindowBoundary_ ;
+      /**
+       * <code>int64 upperWindowBoundary = 9;</code>
+       */
+      public long getUpperWindowBoundary() {
+        return upperWindowBoundary_;
+      }
+      /**
+       * <code>int64 upperWindowBoundary = 9;</code>
+       */
+      public Builder setUpperWindowBoundary(long value) {
+        
+        upperWindowBoundary_ = value;
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>int64 upperWindowBoundary = 9;</code>
+       */
+      public Builder clearUpperWindowBoundary() {
+        
+        upperWindowBoundary_ = 0L;
+        onChanged();
+        return this;
+      }
+      public final Builder setUnknownFields(
+          final com.google.protobuf.UnknownFieldSet unknownFields) {
+        return this;
+      }
+
+      public final Builder mergeUnknownFields(
+          final com.google.protobuf.UnknownFieldSet unknownFields) {
+        return this;
+      }
+
+
+      // @@protoc_insertion_point(builder_scope:protos.TSDataPointSequence)
+    }
+
+    // @@protoc_insertion_point(class_scope:protos.TSDataPointSequence)
+    private static final org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSDataPointSequence DEFAULT_INSTANCE;
+    static {
+      DEFAULT_INSTANCE = new org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSDataPointSequence();
+    }
+
+    public static org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSDataPointSequence getDefaultInstance() {
+      return DEFAULT_INSTANCE;
+    }
+
+    private static final com.google.protobuf.Parser<TSDataPointSequence>
+        PARSER = new com.google.protobuf.AbstractParser<TSDataPointSequence>() {
+      public TSDataPointSequence parsePartialFrom(
+          com.google.protobuf.CodedInputStream input,
+          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws com.google.protobuf.InvalidProtocolBufferException {
+          return new TSDataPointSequence(input, extensionRegistry);
+      }
+    };
+
+    public static com.google.protobuf.Parser<TSDataPointSequence> parser() {
+      return PARSER;
+    }
+
+    @java.lang.Override
+    public com.google.protobuf.Parser<TSDataPointSequence> getParserForType() {
+      return PARSER;
+    }
+
+    public org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSDataPointSequence getDefaultInstanceForType() {
+      return DEFAULT_INSTANCE;
+    }
+
+  }
+
+  public interface PipelineVersionOrBuilder extends
+      // @@protoc_insertion_point(interface_extends:protos.PipelineVersion)
+      com.google.protobuf.MessageOrBuilder {
+
+    /**
+     * <code>string majorVersion = 1;</code>
+     */
+    java.lang.String getMajorVersion();
+    /**
+     * <code>string majorVersion = 1;</code>
+     */
+    com.google.protobuf.ByteString
+        getMajorVersionBytes();
+
+    /**
+     * <code>string minorVersion = 2;</code>
+     */
+    java.lang.String getMinorVersion();
+    /**
+     * <code>string minorVersion = 2;</code>
+     */
+    com.google.protobuf.ByteString
+        getMinorVersionBytes();
+
+    /**
+     * <code>string pipelineName = 3;</code>
+     */
+    java.lang.String getPipelineName();
+    /**
+     * <code>string pipelineName = 3;</code>
+     */
+    com.google.protobuf.ByteString
+        getPipelineNameBytes();
+
+    /**
+     * <code>string transformName = 4;</code>
+     */
+    java.lang.String getTransformName();
+    /**
+     * <code>string transformName = 4;</code>
+     */
+    com.google.protobuf.ByteString
+        getTransformNameBytes();
+  }
+  /**
+   * Protobuf type {@code protos.PipelineVersion}
+   */
+  public  static final class PipelineVersion extends
+      com.google.protobuf.GeneratedMessageV3 implements
+      // @@protoc_insertion_point(message_implements:protos.PipelineVersion)
+      PipelineVersionOrBuilder {
+    // Use PipelineVersion.newBuilder() to construct.
+    private PipelineVersion(com.google.protobuf.GeneratedMessageV3.Builder<?> builder) {
+      super(builder);
+    }
+    private PipelineVersion() {
+      majorVersion_ = "";
+      minorVersion_ = "";
+      pipelineName_ = "";
+      transformName_ = "";
+    }
+
+    @java.lang.Override
+    public final com.google.protobuf.UnknownFieldSet
+    getUnknownFields() {
+      return com.google.protobuf.UnknownFieldSet.getDefaultInstance();
+    }
+    private PipelineVersion(
+        com.google.protobuf.CodedInputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      this();
+      int mutable_bitField0_ = 0;
+      try {
+        boolean done = false;
+        while (!done) {
+          int tag = input.readTag();
+          switch (tag) {
+            case 0:
+              done = true;
+              break;
+            default: {
+              if (!input.skipField(tag)) {
+                done = true;
+              }
+              break;
+            }
+            case 10: {
+              java.lang.String s = input.readStringRequireUtf8();
+
+              majorVersion_ = s;
+              break;
+            }
+            case 18: {
+              java.lang.String s = input.readStringRequireUtf8();
+
+              minorVersion_ = s;
+              break;
+            }
+            case 26: {
+              java.lang.String s = input.readStringRequireUtf8();
+
+              pipelineName_ = s;
+              break;
+            }
+            case 34: {
+              java.lang.String s = input.readStringRequireUtf8();
+
+              transformName_ = s;
+              break;
+            }
+          }
+        }
+      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        throw e.setUnfinishedMessage(this);
+      } catch (java.io.IOException e) {
+        throw new com.google.protobuf.InvalidProtocolBufferException(
+            e).setUnfinishedMessage(this);
+      } finally {
+        makeExtensionsImmutable();
+      }
+    }
+    public static final com.google.protobuf.Descriptors.Descriptor
+        getDescriptor() {
+      return org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.internal_static_protos_PipelineVersion_descriptor;
+    }
+
+    protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+        internalGetFieldAccessorTable() {
+      return org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.internal_static_protos_PipelineVersion_fieldAccessorTable
+          .ensureFieldAccessorsInitialized(
+              org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.PipelineVersion.class, org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.PipelineVersion.Builder.class);
+    }
+
+    public static final int MAJORVERSION_FIELD_NUMBER = 1;
+    private volatile java.lang.Object majorVersion_;
+    /**
+     * <code>string majorVersion = 1;</code>
+     */
+    public java.lang.String getMajorVersion() {
+      java.lang.Object ref = majorVersion_;
+      if (ref instanceof java.lang.String) {
+        return (java.lang.String) ref;
+      } else {
+        com.google.protobuf.ByteString bs = 
+            (com.google.protobuf.ByteString) ref;
+        java.lang.String s = bs.toStringUtf8();
+        majorVersion_ = s;
+        return s;
+      }
+    }
+    /**
+     * <code>string majorVersion = 1;</code>
+     */
+    public com.google.protobuf.ByteString
+        getMajorVersionBytes() {
+      java.lang.Object ref = majorVersion_;
+      if (ref instanceof java.lang.String) {
+        com.google.protobuf.ByteString b = 
+            com.google.protobuf.ByteString.copyFromUtf8(
+                (java.lang.String) ref);
+        majorVersion_ = b;
+        return b;
+      } else {
+        return (com.google.protobuf.ByteString) ref;
+      }
+    }
+
+    public static final int MINORVERSION_FIELD_NUMBER = 2;
+    private volatile java.lang.Object minorVersion_;
+    /**
+     * <code>string minorVersion = 2;</code>
+     */
+    public java.lang.String getMinorVersion() {
+      java.lang.Object ref = minorVersion_;
+      if (ref instanceof java.lang.String) {
+        return (java.lang.String) ref;
+      } else {
+        com.google.protobuf.ByteString bs = 
+            (com.google.protobuf.ByteString) ref;
+        java.lang.String s = bs.toStringUtf8();
+        minorVersion_ = s;
+        return s;
+      }
+    }
+    /**
+     * <code>string minorVersion = 2;</code>
+     */
+    public com.google.protobuf.ByteString
+        getMinorVersionBytes() {
+      java.lang.Object ref = minorVersion_;
+      if (ref instanceof java.lang.String) {
+        com.google.protobuf.ByteString b = 
+            com.google.protobuf.ByteString.copyFromUtf8(
+                (java.lang.String) ref);
+        minorVersion_ = b;
+        return b;
+      } else {
+        return (com.google.protobuf.ByteString) ref;
+      }
+    }
+
+    public static final int PIPELINENAME_FIELD_NUMBER = 3;
+    private volatile java.lang.Object pipelineName_;
+    /**
+     * <code>string pipelineName = 3;</code>
+     */
+    public java.lang.String getPipelineName() {
+      java.lang.Object ref = pipelineName_;
+      if (ref instanceof java.lang.String) {
+        return (java.lang.String) ref;
+      } else {
+        com.google.protobuf.ByteString bs = 
+            (com.google.protobuf.ByteString) ref;
+        java.lang.String s = bs.toStringUtf8();
+        pipelineName_ = s;
+        return s;
+      }
+    }
+    /**
+     * <code>string pipelineName = 3;</code>
+     */
+    public com.google.protobuf.ByteString
+        getPipelineNameBytes() {
+      java.lang.Object ref = pipelineName_;
+      if (ref instanceof java.lang.String) {
+        com.google.protobuf.ByteString b = 
+            com.google.protobuf.ByteString.copyFromUtf8(
+                (java.lang.String) ref);
+        pipelineName_ = b;
+        return b;
+      } else {
+        return (com.google.protobuf.ByteString) ref;
+      }
+    }
+
+    public static final int TRANSFORMNAME_FIELD_NUMBER = 4;
+    private volatile java.lang.Object transformName_;
+    /**
+     * <code>string transformName = 4;</code>
+     */
+    public java.lang.String getTransformName() {
+      java.lang.Object ref = transformName_;
+      if (ref instanceof java.lang.String) {
+        return (java.lang.String) ref;
+      } else {
+        com.google.protobuf.ByteString bs = 
+            (com.google.protobuf.ByteString) ref;
+        java.lang.String s = bs.toStringUtf8();
+        transformName_ = s;
+        return s;
+      }
+    }
+    /**
+     * <code>string transformName = 4;</code>
+     */
+    public com.google.protobuf.ByteString
+        getTransformNameBytes() {
+      java.lang.Object ref = transformName_;
+      if (ref instanceof java.lang.String) {
+        com.google.protobuf.ByteString b = 
+            com.google.protobuf.ByteString.copyFromUtf8(
+                (java.lang.String) ref);
+        transformName_ = b;
+        return b;
+      } else {
+        return (com.google.protobuf.ByteString) ref;
+      }
+    }
+
+    private byte memoizedIsInitialized = -1;
+    public final boolean isInitialized() {
+      byte isInitialized = memoizedIsInitialized;
+      if (isInitialized == 1) return true;
+      if (isInitialized == 0) return false;
+
+      memoizedIsInitialized = 1;
+      return true;
+    }
+
+    public void writeTo(com.google.protobuf.CodedOutputStream output)
+                        throws java.io.IOException {
+      if (!getMajorVersionBytes().isEmpty()) {
+        com.google.protobuf.GeneratedMessageV3.writeString(output, 1, majorVersion_);
+      }
+      if (!getMinorVersionBytes().isEmpty()) {
+        com.google.protobuf.GeneratedMessageV3.writeString(output, 2, minorVersion_);
+      }
+      if (!getPipelineNameBytes().isEmpty()) {
+        com.google.protobuf.GeneratedMessageV3.writeString(output, 3, pipelineName_);
+      }
+      if (!getTransformNameBytes().isEmpty()) {
+        com.google.protobuf.GeneratedMessageV3.writeString(output, 4, transformName_);
+      }
+    }
+
+    public int getSerializedSize() {
+      int size = memoizedSize;
+      if (size != -1) return size;
+
+      size = 0;
+      if (!getMajorVersionBytes().isEmpty()) {
+        size += com.google.protobuf.GeneratedMessageV3.computeStringSize(1, majorVersion_);
+      }
+      if (!getMinorVersionBytes().isEmpty()) {
+        size += com.google.protobuf.GeneratedMessageV3.computeStringSize(2, minorVersion_);
+      }
+      if (!getPipelineNameBytes().isEmpty()) {
+        size += com.google.protobuf.GeneratedMessageV3.computeStringSize(3, pipelineName_);
+      }
+      if (!getTransformNameBytes().isEmpty()) {
+        size += com.google.protobuf.GeneratedMessageV3.computeStringSize(4, transformName_);
+      }
+      memoizedSize = size;
+      return size;
+    }
+
+    private static final long serialVersionUID = 0L;
+    @java.lang.Override
+    public boolean equals(final java.lang.Object obj) {
+      if (obj == this) {
+       return true;
+      }
+      if (!(obj instanceof org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.PipelineVersion)) {
+        return super.equals(obj);
+      }
+      org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.PipelineVersion other = (org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.PipelineVersion) obj;
+
+      boolean result = true;
+      result = result && getMajorVersion()
+          .equals(other.getMajorVersion());
+      result = result && getMinorVersion()
+          .equals(other.getMinorVersion());
+      result = result && getPipelineName()
+          .equals(other.getPipelineName());
+      result = result && getTransformName()
+          .equals(other.getTransformName());
+      return result;
+    }
+
+    @java.lang.Override
+    public int hashCode() {
+      if (memoizedHashCode != 0) {
+        return memoizedHashCode;
+      }
+      int hash = 41;
+      hash = (19 * hash) + getDescriptor().hashCode();
+      hash = (37 * hash) + MAJORVERSION_FIELD_NUMBER;
+      hash = (53 * hash) + getMajorVersion().hashCode();
+      hash = (37 * hash) + MINORVERSION_FIELD_NUMBER;
+      hash = (53 * hash) + getMinorVersion().hashCode();
+      hash = (37 * hash) + PIPELINENAME_FIELD_NUMBER;
+      hash = (53 * hash) + getPipelineName().hashCode();
+      hash = (37 * hash) + TRANSFORMNAME_FIELD_NUMBER;
+      hash = (53 * hash) + getTransformName().hashCode();
+      hash = (29 * hash) + unknownFields.hashCode();
+      memoizedHashCode = hash;
+      return hash;
+    }
+
+    public static org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.PipelineVersion parseFrom(
+        com.google.protobuf.ByteString data)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data);
+    }
+    public static org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.PipelineVersion parseFrom(
+        com.google.protobuf.ByteString data,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data, extensionRegistry);
+    }
+    public static org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.PipelineVersion parseFrom(byte[] data)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data);
+    }
+    public static org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.PipelineVersion parseFrom(
+        byte[] data,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data, extensionRegistry);
+    }
+    public static org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.PipelineVersion parseFrom(java.io.InputStream input)
+        throws java.io.IOException {
+      return com.google.protobuf.GeneratedMessageV3
+          .parseWithIOException(PARSER, input);
+    }
+    public static org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.PipelineVersion parseFrom(
+        java.io.InputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws java.io.IOException {
+      return com.google.protobuf.GeneratedMessageV3
+          .parseWithIOException(PARSER, input, extensionRegistry);
+    }
+    public static org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.PipelineVersion parseDelimitedFrom(java.io.InputStream input)
+        throws java.io.IOException {
+      return com.google.protobuf.GeneratedMessageV3
+          .parseDelimitedWithIOException(PARSER, input);
+    }
+    public static org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.PipelineVersion parseDelimitedFrom(
+        java.io.InputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws java.io.IOException {
+      return com.google.protobuf.GeneratedMessageV3
+          .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
+    }
+    public static org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.PipelineVersion parseFrom(
+        com.google.protobuf.CodedInputStream input)
+        throws java.io.IOException {
+      return com.google.protobuf.GeneratedMessageV3
+          .parseWithIOException(PARSER, input);
+    }
+    public static org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.PipelineVersion parseFrom(
+        com.google.protobuf.CodedInputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws java.io.IOException {
+      return com.google.protobuf.GeneratedMessageV3
+          .parseWithIOException(PARSER, input, extensionRegistry);
+    }
+
+    public Builder newBuilderForType() { return newBuilder(); }
+    public static Builder newBuilder() {
+      return DEFAULT_INSTANCE.toBuilder();
+    }
+    public static Builder newBuilder(org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.PipelineVersion prototype) {
+      return DEFAULT_INSTANCE.toBuilder().mergeFrom(prototype);
+    }
+    public Builder toBuilder() {
+      return this == DEFAULT_INSTANCE
+          ? new Builder() : new Builder().mergeFrom(this);
+    }
+
+    @java.lang.Override
+    protected Builder newBuilderForType(
+        com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+      Builder builder = new Builder(parent);
+      return builder;
+    }
+    /**
+     * Protobuf type {@code protos.PipelineVersion}
+     */
+    public static final class Builder extends
+        com.google.protobuf.GeneratedMessageV3.Builder<Builder> implements
+        // @@protoc_insertion_point(builder_implements:protos.PipelineVersion)
+        org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.PipelineVersionOrBuilder {
+      public static final com.google.protobuf.Descriptors.Descriptor
+          getDescriptor() {
+        return org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.internal_static_protos_PipelineVersion_descriptor;
+      }
+
+      protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+          internalGetFieldAccessorTable() {
+        return org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.internal_static_protos_PipelineVersion_fieldAccessorTable
+            .ensureFieldAccessorsInitialized(
+                org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.PipelineVersion.class, org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.PipelineVersion.Builder.class);
+      }
+
+      // Construct using org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.PipelineVersion.newBuilder()
+      private Builder() {
+        maybeForceBuilderInitialization();
+      }
+
+      private Builder(
+          com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+        super(parent);
+        maybeForceBuilderInitialization();
+      }
+      private void maybeForceBuilderInitialization() {
+        if (com.google.protobuf.GeneratedMessageV3
+                .alwaysUseFieldBuilders) {
+        }
+      }
+      public Builder clear() {
+        super.clear();
+        majorVersion_ = "";
+
+        minorVersion_ = "";
+
+        pipelineName_ = "";
+
+        transformName_ = "";
+
+        return this;
+      }
+
+      public com.google.protobuf.Descriptors.Descriptor
+          getDescriptorForType() {
+        return org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.internal_static_protos_PipelineVersion_descriptor;
+      }
+
+      public org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.PipelineVersion getDefaultInstanceForType() {
+        return org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.PipelineVersion.getDefaultInstance();
+      }
+
+      public org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.PipelineVersion build() {
+        org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.PipelineVersion result = buildPartial();
+        if (!result.isInitialized()) {
+          throw newUninitializedMessageException(result);
+        }
+        return result;
+      }
+
+      public org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.PipelineVersion buildPartial() {
+        org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.PipelineVersion result = new org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.PipelineVersion(this);
+        result.majorVersion_ = majorVersion_;
+        result.minorVersion_ = minorVersion_;
+        result.pipelineName_ = pipelineName_;
+        result.transformName_ = transformName_;
+        onBuilt();
+        return result;
+      }
+
+      public Builder clone() {
+        return (Builder) super.clone();
+      }
+      public Builder setField(
+          com.google.protobuf.Descriptors.FieldDescriptor field,
+          Object value) {
+        return (Builder) super.setField(field, value);
+      }
+      public Builder clearField(
+          com.google.protobuf.Descriptors.FieldDescriptor field) {
+        return (Builder) super.clearField(field);
+      }
+      public Builder clearOneof(
+          com.google.protobuf.Descriptors.OneofDescriptor oneof) {
+        return (Builder) super.clearOneof(oneof);
+      }
+      public Builder setRepeatedField(
+          com.google.protobuf.Descriptors.FieldDescriptor field,
+          int index, Object value) {
+        return (Builder) super.setRepeatedField(field, index, value);
+      }
+      public Builder addRepeatedField(
+          com.google.protobuf.Descriptors.FieldDescriptor field,
+          Object value) {
+        return (Builder) super.addRepeatedField(field, value);
+      }
+      public Builder mergeFrom(com.google.protobuf.Message other) {
+        if (other instanceof org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.PipelineVersion) {
+          return mergeFrom((org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.PipelineVersion)other);
+        } else {
+          super.mergeFrom(other);
+          return this;
+        }
+      }
+
+      public Builder mergeFrom(org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.PipelineVersion other) {
+        if (other == org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.PipelineVersion.getDefaultInstance()) return this;
+        if (!other.getMajorVersion().isEmpty()) {
+          majorVersion_ = other.majorVersion_;
+          onChanged();
+        }
+        if (!other.getMinorVersion().isEmpty()) {
+          minorVersion_ = other.minorVersion_;
+          onChanged();
+        }
+        if (!other.getPipelineName().isEmpty()) {
+          pipelineName_ = other.pipelineName_;
+          onChanged();
+        }
+        if (!other.getTransformName().isEmpty()) {
+          transformName_ = other.transformName_;
+          onChanged();
+        }
+        onChanged();
+        return this;
+      }
+
+      public final boolean isInitialized() {
+        return true;
+      }
+
+      public Builder mergeFrom(
+          com.google.protobuf.CodedInputStream input,
+          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws java.io.IOException {
+        org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.PipelineVersion parsedMessage = null;
+        try {
+          parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
+        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+          parsedMessage = (org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.PipelineVersion) e.getUnfinishedMessage();
+          throw e.unwrapIOException();
+        } finally {
+          if (parsedMessage != null) {
+            mergeFrom(parsedMessage);
+          }
+        }
+        return this;
+      }
+
+      private java.lang.Object majorVersion_ = "";
+      /**
+       * <code>string majorVersion = 1;</code>
+       */
+      public java.lang.String getMajorVersion() {
+        java.lang.Object ref = majorVersion_;
+        if (!(ref instanceof java.lang.String)) {
+          com.google.protobuf.ByteString bs =
+              (com.google.protobuf.ByteString) ref;
+          java.lang.String s = bs.toStringUtf8();
+          majorVersion_ = s;
+          return s;
+        } else {
+          return (java.lang.String) ref;
+        }
+      }
+      /**
+       * <code>string majorVersion = 1;</code>
+       */
+      public com.google.protobuf.ByteString
+          getMajorVersionBytes() {
+        java.lang.Object ref = majorVersion_;
+        if (ref instanceof String) {
+          com.google.protobuf.ByteString b = 
+              com.google.protobuf.ByteString.copyFromUtf8(
+                  (java.lang.String) ref);
+          majorVersion_ = b;
+          return b;
+        } else {
+          return (com.google.protobuf.ByteString) ref;
+        }
+      }
+      /**
+       * <code>string majorVersion = 1;</code>
+       */
+      public Builder setMajorVersion(
+          java.lang.String value) {
+        if (value == null) {
+    throw new NullPointerException();
+  }
+  
+        majorVersion_ = value;
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>string majorVersion = 1;</code>
+       */
+      public Builder clearMajorVersion() {
+        
+        majorVersion_ = getDefaultInstance().getMajorVersion();
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>string majorVersion = 1;</code>
+       */
+      public Builder setMajorVersionBytes(
+          com.google.protobuf.ByteString value) {
+        if (value == null) {
+    throw new NullPointerException();
+  }
+  checkByteStringIsUtf8(value);
+        
+        majorVersion_ = value;
+        onChanged();
+        return this;
+      }
+
+      private java.lang.Object minorVersion_ = "";
+      /**
+       * <code>string minorVersion = 2;</code>
+       */
+      public java.lang.String getMinorVersion() {
+        java.lang.Object ref = minorVersion_;
+        if (!(ref instanceof java.lang.String)) {
+          com.google.protobuf.ByteString bs =
+              (com.google.protobuf.ByteString) ref;
+          java.lang.String s = bs.toStringUtf8();
+          minorVersion_ = s;
+          return s;
+        } else {
+          return (java.lang.String) ref;
+        }
+      }
+      /**
+       * <code>string minorVersion = 2;</code>
+       */
+      public com.google.protobuf.ByteString
+          getMinorVersionBytes() {
+        java.lang.Object ref = minorVersion_;
+        if (ref instanceof String) {
+          com.google.protobuf.ByteString b = 
+              com.google.protobuf.ByteString.copyFromUtf8(
+                  (java.lang.String) ref);
+          minorVersion_ = b;
+          return b;
+        } else {
+          return (com.google.protobuf.ByteString) ref;
+        }
+      }
+      /**
+       * <code>string minorVersion = 2;</code>
+       */
+      public Builder setMinorVersion(
+          java.lang.String value) {
+        if (value == null) {
+    throw new NullPointerException();
+  }
+  
+        minorVersion_ = value;
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>string minorVersion = 2;</code>
+       */
+      public Builder clearMinorVersion() {
+        
+        minorVersion_ = getDefaultInstance().getMinorVersion();
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>string minorVersion = 2;</code>
+       */
+      public Builder setMinorVersionBytes(
+          com.google.protobuf.ByteString value) {
+        if (value == null) {
+    throw new NullPointerException();
+  }
+  checkByteStringIsUtf8(value);
+        
+        minorVersion_ = value;
+        onChanged();
+        return this;
+      }
+
+      private java.lang.Object pipelineName_ = "";
+      /**
+       * <code>string pipelineName = 3;</code>
+       */
+      public java.lang.String getPipelineName() {
+        java.lang.Object ref = pipelineName_;
+        if (!(ref instanceof java.lang.String)) {
+          com.google.protobuf.ByteString bs =
+              (com.google.protobuf.ByteString) ref;
+          java.lang.String s = bs.toStringUtf8();
+          pipelineName_ = s;
+          return s;
+        } else {
+          return (java.lang.String) ref;
+        }
+      }
+      /**
+       * <code>string pipelineName = 3;</code>
+       */
+      public com.google.protobuf.ByteString
+          getPipelineNameBytes() {
+        java.lang.Object ref = pipelineName_;
+        if (ref instanceof String) {
+          com.google.protobuf.ByteString b = 
+              com.google.protobuf.ByteString.copyFromUtf8(
+                  (java.lang.String) ref);
+          pipelineName_ = b;
+          return b;
+        } else {
+          return (com.google.protobuf.ByteString) ref;
+        }
+      }
+      /**
+       * <code>string pipelineName = 3;</code>
+       */
+      public Builder setPipelineName(
+          java.lang.String value) {
+        if (value == null) {
+    throw new NullPointerException();
+  }
+  
+        pipelineName_ = value;
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>string pipelineName = 3;</code>
+       */
+      public Builder clearPipelineName() {
+        
+        pipelineName_ = getDefaultInstance().getPipelineName();
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>string pipelineName = 3;</code>
+       */
+      public Builder setPipelineNameBytes(
+          com.google.protobuf.ByteString value) {
+        if (value == null) {
+    throw new NullPointerException();
+  }
+  checkByteStringIsUtf8(value);
+        
+        pipelineName_ = value;
+        onChanged();
+        return this;
+      }
+
+      private java.lang.Object transformName_ = "";
+      /**
+       * <code>string transformName = 4;</code>
+       */
+      public java.lang.String getTransformName() {
+        java.lang.Object ref = transformName_;
+        if (!(ref instanceof java.lang.String)) {
+          com.google.protobuf.ByteString bs =
+              (com.google.protobuf.ByteString) ref;
+          java.lang.String s = bs.toStringUtf8();
+          transformName_ = s;
+          return s;
+        } else {
+          return (java.lang.String) ref;
+        }
+      }
+      /**
+       * <code>string transformName = 4;</code>
+       */
+      public com.google.protobuf.ByteString
+          getTransformNameBytes() {
+        java.lang.Object ref = transformName_;
+        if (ref instanceof String) {
+          com.google.protobuf.ByteString b = 
+              com.google.protobuf.ByteString.copyFromUtf8(
+                  (java.lang.String) ref);
+          transformName_ = b;
+          return b;
+        } else {
+          return (com.google.protobuf.ByteString) ref;
+        }
+      }
+      /**
+       * <code>string transformName = 4;</code>
+       */
+      public Builder setTransformName(
+          java.lang.String value) {
+        if (value == null) {
+    throw new NullPointerException();
+  }
+  
+        transformName_ = value;
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>string transformName = 4;</code>
+       */
+      public Builder clearTransformName() {
+        
+        transformName_ = getDefaultInstance().getTransformName();
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>string transformName = 4;</code>
+       */
+      public Builder setTransformNameBytes(
+          com.google.protobuf.ByteString value) {
+        if (value == null) {
+    throw new NullPointerException();
+  }
+  checkByteStringIsUtf8(value);
+        
+        transformName_ = value;
+        onChanged();
+        return this;
+      }
+      public final Builder setUnknownFields(
+          final com.google.protobuf.UnknownFieldSet unknownFields) {
+        return this;
+      }
+
+      public final Builder mergeUnknownFields(
+          final com.google.protobuf.UnknownFieldSet unknownFields) {
+        return this;
+      }
+
+
+      // @@protoc_insertion_point(builder_scope:protos.PipelineVersion)
+    }
+
+    // @@protoc_insertion_point(class_scope:protos.PipelineVersion)
+    private static final org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.PipelineVersion DEFAULT_INSTANCE;
+    static {
+      DEFAULT_INSTANCE = new org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.PipelineVersion();
+    }
+
+    public static org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.PipelineVersion getDefaultInstance() {
+      return DEFAULT_INSTANCE;
+    }
+
+    private static final com.google.protobuf.Parser<PipelineVersion>
+        PARSER = new com.google.protobuf.AbstractParser<PipelineVersion>() {
+      public PipelineVersion parsePartialFrom(
+          com.google.protobuf.CodedInputStream input,
+          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws com.google.protobuf.InvalidProtocolBufferException {
+          return new PipelineVersion(input, extensionRegistry);
+      }
+    };
+
+    public static com.google.protobuf.Parser<PipelineVersion> parser() {
+      return PARSER;
+    }
+
+    @java.lang.Override
+    public com.google.protobuf.Parser<PipelineVersion> getParserForType() {
+      return PARSER;
+    }
+
+    public org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.PipelineVersion getDefaultInstanceForType() {
+      return DEFAULT_INSTANCE;
+    }
+
+  }
+
+  public interface TSAccumOrBuilder extends
+      // @@protoc_insertion_point(interface_extends:protos.TSAccum)
+      com.google.protobuf.MessageOrBuilder {
+
+    /**
+     * <code>.protos.TSKey key = 1;</code>
+     */
+    boolean hasKey();
+    /**
+     * <code>.protos.TSKey key = 1;</code>
+     */
+    org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSKey getKey();
+    /**
+     * <code>.protos.TSKey key = 1;</code>
+     */
+    org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSKeyOrBuilder getKeyOrBuilder();
+
+    /**
+     * <code>.protos.PipelineVersion pipelineVersion = 2;</code>
+     */
+    boolean hasPipelineVersion();
+    /**
+     * <code>.protos.PipelineVersion pipelineVersion = 2;</code>
+     */
+    org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.PipelineVersion getPipelineVersion();
+    /**
+     * <code>.protos.PipelineVersion pipelineVersion = 2;</code>
+     */
+    org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.PipelineVersionOrBuilder getPipelineVersionOrBuilder();
+
+    /**
+     * <code>.protos.TSDataPoint data = 3;</code>
+     */
+    boolean hasData();
+    /**
+     * <code>.protos.TSDataPoint data = 3;</code>
+     */
+    org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSDataPoint getData();
+    /**
+     * <code>.protos.TSDataPoint data = 3;</code>
+     */
+    org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSDataPointOrBuilder getDataOrBuilder();
+
+    /**
+     * <code>int64 durationMills = 4;</code>
+     */
+    long getDurationMills();
+
+    /**
+     * <code>.google.protobuf.Timestamp lowerWindowBoundary = 5;</code>
+     */
+    boolean hasLowerWindowBoundary();
+    /**
+     * <code>.google.protobuf.Timestamp lowerWindowBoundary = 5;</code>
+     */
+    com.google.protobuf.Timestamp getLowerWindowBoundary();
+    /**
+     * <code>.google.protobuf.Timestamp lowerWindowBoundary = 5;</code>
+     */
+    com.google.protobuf.TimestampOrBuilder getLowerWindowBoundaryOrBuilder();
+
+    /**
+     * <code>.google.protobuf.Timestamp upperWindowBoundary = 6;</code>
+     */
+    boolean hasUpperWindowBoundary();
+    /**
+     * <code>.google.protobuf.Timestamp upperWindowBoundary = 6;</code>
+     */
+    com.google.protobuf.Timestamp getUpperWindowBoundary();
+    /**
+     * <code>.google.protobuf.Timestamp upperWindowBoundary = 6;</code>
+     */
+    com.google.protobuf.TimestampOrBuilder getUpperWindowBoundaryOrBuilder();
+  }
+  /**
+   * <pre>
+   * Accumulator used in combiners
+   * </pre>
+   *
+   * Protobuf type {@code protos.TSAccum}
+   */
+  public  static final class TSAccum extends
+      com.google.protobuf.GeneratedMessageV3 implements
+      // @@protoc_insertion_point(message_implements:protos.TSAccum)
+      TSAccumOrBuilder {
+    // Use TSAccum.newBuilder() to construct.
+    private TSAccum(com.google.protobuf.GeneratedMessageV3.Builder<?> builder) {
+      super(builder);
+    }
+    private TSAccum() {
+      durationMills_ = 0L;
+    }
+
+    @java.lang.Override
+    public final com.google.protobuf.UnknownFieldSet
+    getUnknownFields() {
+      return com.google.protobuf.UnknownFieldSet.getDefaultInstance();
+    }
+    private TSAccum(
+        com.google.protobuf.CodedInputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      this();
+      int mutable_bitField0_ = 0;
+      try {
+        boolean done = false;
+        while (!done) {
+          int tag = input.readTag();
+          switch (tag) {
+            case 0:
+              done = true;
+              break;
+            default: {
+              if (!input.skipField(tag)) {
+                done = true;
+              }
+              break;
+            }
+            case 10: {
+              org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSKey.Builder subBuilder = null;
+              if (key_ != null) {
+                subBuilder = key_.toBuilder();
+              }
+              key_ = input.readMessage(org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSKey.parser(), extensionRegistry);
+              if (subBuilder != null) {
+                subBuilder.mergeFrom(key_);
+                key_ = subBuilder.buildPartial();
+              }
+
+              break;
+            }
+            case 18: {
+              org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.PipelineVersion.Builder subBuilder = null;
+              if (pipelineVersion_ != null) {
+                subBuilder = pipelineVersion_.toBuilder();
+              }
+              pipelineVersion_ = input.readMessage(org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.PipelineVersion.parser(), extensionRegistry);
+              if (subBuilder != null) {
+                subBuilder.mergeFrom(pipelineVersion_);
+                pipelineVersion_ = subBuilder.buildPartial();
+              }
+
+              break;
+            }
+            case 26: {
+              org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSDataPoint.Builder subBuilder = null;
+              if (data_ != null) {
+                subBuilder = data_.toBuilder();
+              }
+              data_ = input.readMessage(org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSDataPoint.parser(), extensionRegistry);
+              if (subBuilder != null) {
+                subBuilder.mergeFrom(data_);
+                data_ = subBuilder.buildPartial();
+              }
+
+              break;
+            }
+            case 32: {
+
+              durationMills_ = input.readInt64();
+              break;
+            }
+            case 42: {
+              com.google.protobuf.Timestamp.Builder subBuilder = null;
+              if (lowerWindowBoundary_ != null) {
+                subBuilder = lowerWindowBoundary_.toBuilder();
+              }
+              lowerWindowBoundary_ = input.readMessage(com.google.protobuf.Timestamp.parser(), extensionRegistry);
+              if (subBuilder != null) {
+                subBuilder.mergeFrom(lowerWindowBoundary_);
+                lowerWindowBoundary_ = subBuilder.buildPartial();
+              }
+
+              break;
+            }
+            case 50: {
+              com.google.protobuf.Timestamp.Builder subBuilder = null;
+              if (upperWindowBoundary_ != null) {
+                subBuilder = upperWindowBoundary_.toBuilder();
+              }
+              upperWindowBoundary_ = input.readMessage(com.google.protobuf.Timestamp.parser(), extensionRegistry);
+              if (subBuilder != null) {
+                subBuilder.mergeFrom(upperWindowBoundary_);
+                upperWindowBoundary_ = subBuilder.buildPartial();
+              }
+
+              break;
+            }
+          }
+        }
+      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        throw e.setUnfinishedMessage(this);
+      } catch (java.io.IOException e) {
+        throw new com.google.protobuf.InvalidProtocolBufferException(
+            e).setUnfinishedMessage(this);
+      } finally {
+        makeExtensionsImmutable();
+      }
+    }
+    public static final com.google.protobuf.Descriptors.Descriptor
+        getDescriptor() {
+      return org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.internal_static_protos_TSAccum_descriptor;
+    }
+
+    protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+        internalGetFieldAccessorTable() {
+      return org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.internal_static_protos_TSAccum_fieldAccessorTable
+          .ensureFieldAccessorsInitialized(
+              org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSAccum.class, org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSAccum.Builder.class);
+    }
+
+    public static final int KEY_FIELD_NUMBER = 1;
+    private org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSKey key_;
+    /**
+     * <code>.protos.TSKey key = 1;</code>
+     */
+    public boolean hasKey() {
+      return key_ != null;
+    }
+    /**
+     * <code>.protos.TSKey key = 1;</code>
+     */
+    public org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSKey getKey() {
+      return key_ == null ? org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSKey.getDefaultInstance() : key_;
+    }
+    /**
+     * <code>.protos.TSKey key = 1;</code>
+     */
+    public org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSKeyOrBuilder getKeyOrBuilder() {
+      return getKey();
+    }
+
+    public static final int PIPELINEVERSION_FIELD_NUMBER = 2;
+    private org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.PipelineVersion pipelineVersion_;
+    /**
+     * <code>.protos.PipelineVersion pipelineVersion = 2;</code>
+     */
+    public boolean hasPipelineVersion() {
+      return pipelineVersion_ != null;
+    }
+    /**
+     * <code>.protos.PipelineVersion pipelineVersion = 2;</code>
+     */
+    public org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.PipelineVersion getPipelineVersion() {
+      return pipelineVersion_ == null ? org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.PipelineVersion.getDefaultInstance() : pipelineVersion_;
+    }
+    /**
+     * <code>.protos.PipelineVersion pipelineVersion = 2;</code>
+     */
+    public org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.PipelineVersionOrBuilder getPipelineVersionOrBuilder() {
+      return getPipelineVersion();
+    }
+
+    public static final int DATA_FIELD_NUMBER = 3;
+    private org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSDataPoint data_;
+    /**
+     * <code>.protos.TSDataPoint data = 3;</code>
+     */
+    public boolean hasData() {
+      return data_ != null;
+    }
+    /**
+     * <code>.protos.TSDataPoint data = 3;</code>
+     */
+    public org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSDataPoint getData() {
+      return data_ == null ? org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSDataPoint.getDefaultInstance() : data_;
+    }
+    /**
+     * <code>.protos.TSDataPoint data = 3;</code>
+     */
+    public org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSDataPointOrBuilder getDataOrBuilder() {
+      return getData();
+    }
+
+    public static final int DURATIONMILLS_FIELD_NUMBER = 4;
+    private long durationMills_;
+    /**
+     * <code>int64 durationMills = 4;</code>
+     */
+    public long getDurationMills() {
+      return durationMills_;
+    }
+
+    public static final int LOWERWINDOWBOUNDARY_FIELD_NUMBER = 5;
+    private com.google.protobuf.Timestamp lowerWindowBoundary_;
+    /**
+     * <code>.google.protobuf.Timestamp lowerWindowBoundary = 5;</code>
+     */
+    public boolean hasLowerWindowBoundary() {
+      return lowerWindowBoundary_ != null;
+    }
+    /**
+     * <code>.google.protobuf.Timestamp lowerWindowBoundary = 5;</code>
+     */
+    public com.google.protobuf.Timestamp getLowerWindowBoundary() {
+      return lowerWindowBoundary_ == null ? com.google.protobuf.Timestamp.getDefaultInstance() : lowerWindowBoundary_;
+    }
+    /**
+     * <code>.google.protobuf.Timestamp lowerWindowBoundary = 5;</code>
+     */
+    public com.google.protobuf.TimestampOrBuilder getLowerWindowBoundaryOrBuilder() {
+      return getLowerWindowBoundary();
+    }
+
+    public static final int UPPERWINDOWBOUNDARY_FIELD_NUMBER = 6;
+    private com.google.protobuf.Timestamp upperWindowBoundary_;
+    /**
+     * <code>.google.protobuf.Timestamp upperWindowBoundary = 6;</code>
+     */
+    public boolean hasUpperWindowBoundary() {
+      return upperWindowBoundary_ != null;
+    }
+    /**
+     * <code>.google.protobuf.Timestamp upperWindowBoundary = 6;</code>
+     */
+    public com.google.protobuf.Timestamp getUpperWindowBoundary() {
+      return upperWindowBoundary_ == null ? com.google.protobuf.Timestamp.getDefaultInstance() : upperWindowBoundary_;
+    }
+    /**
+     * <code>.google.protobuf.Timestamp upperWindowBoundary = 6;</code>
+     */
+    public com.google.protobuf.TimestampOrBuilder getUpperWindowBoundaryOrBuilder() {
+      return getUpperWindowBoundary();
+    }
+
+    private byte memoizedIsInitialized = -1;
+    public final boolean isInitialized() {
+      byte isInitialized = memoizedIsInitialized;
+      if (isInitialized == 1) return true;
+      if (isInitialized == 0) return false;
+
+      memoizedIsInitialized = 1;
+      return true;
+    }
+
+    public void writeTo(com.google.protobuf.CodedOutputStream output)
+                        throws java.io.IOException {
+      if (key_ != null) {
+        output.writeMessage(1, getKey());
+      }
+      if (pipelineVersion_ != null) {
+        output.writeMessage(2, getPipelineVersion());
+      }
+      if (data_ != null) {
+        output.writeMessage(3, getData());
+      }
+      if (durationMills_ != 0L) {
+        output.writeInt64(4, durationMills_);
+      }
+      if (lowerWindowBoundary_ != null) {
+        output.writeMessage(5, getLowerWindowBoundary());
+      }
+      if (upperWindowBoundary_ != null) {
+        output.writeMessage(6, getUpperWindowBoundary());
+      }
+    }
+
+    public int getSerializedSize() {
+      int size = memoizedSize;
+      if (size != -1) return size;
+
+      size = 0;
+      if (key_ != null) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeMessageSize(1, getKey());
+      }
+      if (pipelineVersion_ != null) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeMessageSize(2, getPipelineVersion());
+      }
+      if (data_ != null) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeMessageSize(3, getData());
+      }
+      if (durationMills_ != 0L) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeInt64Size(4, durationMills_);
+      }
+      if (lowerWindowBoundary_ != null) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeMessageSize(5, getLowerWindowBoundary());
+      }
+      if (upperWindowBoundary_ != null) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeMessageSize(6, getUpperWindowBoundary());
+      }
+      memoizedSize = size;
+      return size;
+    }
+
+    private static final long serialVersionUID = 0L;
+    @java.lang.Override
+    public boolean equals(final java.lang.Object obj) {
+      if (obj == this) {
+       return true;
+      }
+      if (!(obj instanceof org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSAccum)) {
+        return super.equals(obj);
+      }
+      org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSAccum other = (org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSAccum) obj;
+
+      boolean result = true;
+      result = result && (hasKey() == other.hasKey());
+      if (hasKey()) {
+        result = result && getKey()
+            .equals(other.getKey());
+      }
+      result = result && (hasPipelineVersion() == other.hasPipelineVersion());
+      if (hasPipelineVersion()) {
+        result = result && getPipelineVersion()
+            .equals(other.getPipelineVersion());
+      }
+      result = result && (hasData() == other.hasData());
+      if (hasData()) {
+        result = result && getData()
+            .equals(other.getData());
+      }
+      result = result && (getDurationMills()
+          == other.getDurationMills());
+      result = result && (hasLowerWindowBoundary() == other.hasLowerWindowBoundary());
+      if (hasLowerWindowBoundary()) {
+        result = result && getLowerWindowBoundary()
+            .equals(other.getLowerWindowBoundary());
+      }
+      result = result && (hasUpperWindowBoundary() == other.hasUpperWindowBoundary());
+      if (hasUpperWindowBoundary()) {
+        result = result && getUpperWindowBoundary()
+            .equals(other.getUpperWindowBoundary());
+      }
+      return result;
+    }
+
+    @java.lang.Override
+    public int hashCode() {
+      if (memoizedHashCode != 0) {
+        return memoizedHashCode;
+      }
+      int hash = 41;
+      hash = (19 * hash) + getDescriptor().hashCode();
+      if (hasKey()) {
+        hash = (37 * hash) + KEY_FIELD_NUMBER;
+        hash = (53 * hash) + getKey().hashCode();
+      }
+      if (hasPipelineVersion()) {
+        hash = (37 * hash) + PIPELINEVERSION_FIELD_NUMBER;
+        hash = (53 * hash) + getPipelineVersion().hashCode();
+      }
+      if (hasData()) {
+        hash = (37 * hash) + DATA_FIELD_NUMBER;
+        hash = (53 * hash) + getData().hashCode();
+      }
+      hash = (37 * hash) + DURATIONMILLS_FIELD_NUMBER;
+      hash = (53 * hash) + com.google.protobuf.Internal.hashLong(
+          getDurationMills());
+      if (hasLowerWindowBoundary()) {
+        hash = (37 * hash) + LOWERWINDOWBOUNDARY_FIELD_NUMBER;
+        hash = (53 * hash) + getLowerWindowBoundary().hashCode();
+      }
+      if (hasUpperWindowBoundary()) {
+        hash = (37 * hash) + UPPERWINDOWBOUNDARY_FIELD_NUMBER;
+        hash = (53 * hash) + getUpperWindowBoundary().hashCode();
+      }
+      hash = (29 * hash) + unknownFields.hashCode();
+      memoizedHashCode = hash;
+      return hash;
+    }
+
+    public static org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSAccum parseFrom(
+        com.google.protobuf.ByteString data)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data);
+    }
+    public static org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSAccum parseFrom(
+        com.google.protobuf.ByteString data,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data, extensionRegistry);
+    }
+    public static org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSAccum parseFrom(byte[] data)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data);
+    }
+    public static org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSAccum parseFrom(
+        byte[] data,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data, extensionRegistry);
+    }
+    public static org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSAccum parseFrom(java.io.InputStream input)
+        throws java.io.IOException {
+      return com.google.protobuf.GeneratedMessageV3
+          .parseWithIOException(PARSER, input);
+    }
+    public static org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSAccum parseFrom(
+        java.io.InputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws java.io.IOException {
+      return com.google.protobuf.GeneratedMessageV3
+          .parseWithIOException(PARSER, input, extensionRegistry);
+    }
+    public static org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSAccum parseDelimitedFrom(java.io.InputStream input)
+        throws java.io.IOException {
+      return com.google.protobuf.GeneratedMessageV3
+          .parseDelimitedWithIOException(PARSER, input);
+    }
+    public static org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSAccum parseDelimitedFrom(
+        java.io.InputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws java.io.IOException {
+      return com.google.protobuf.GeneratedMessageV3
+          .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
+    }
+    public static org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSAccum parseFrom(
+        com.google.protobuf.CodedInputStream input)
+        throws java.io.IOException {
+      return com.google.protobuf.GeneratedMessageV3
+          .parseWithIOException(PARSER, input);
+    }
+    public static org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSAccum parseFrom(
+        com.google.protobuf.CodedInputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws java.io.IOException {
+      return com.google.protobuf.GeneratedMessageV3
+          .parseWithIOException(PARSER, input, extensionRegistry);
+    }
+
+    public Builder newBuilderForType() { return newBuilder(); }
+    public static Builder newBuilder() {
+      return DEFAULT_INSTANCE.toBuilder();
+    }
+    public static Builder newBuilder(org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSAccum prototype) {
+      return DEFAULT_INSTANCE.toBuilder().mergeFrom(prototype);
+    }
+    public Builder toBuilder() {
+      return this == DEFAULT_INSTANCE
+          ? new Builder() : new Builder().mergeFrom(this);
+    }
+
+    @java.lang.Override
+    protected Builder newBuilderForType(
+        com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+      Builder builder = new Builder(parent);
+      return builder;
+    }
+    /**
+     * <pre>
+     * Accumulator used in combiners
+     * </pre>
+     *
+     * Protobuf type {@code protos.TSAccum}
+     */
+    public static final class Builder extends
+        com.google.protobuf.GeneratedMessageV3.Builder<Builder> implements
+        // @@protoc_insertion_point(builder_implements:protos.TSAccum)
+        org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSAccumOrBuilder {
+      public static final com.google.protobuf.Descriptors.Descriptor
+          getDescriptor() {
+        return org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.internal_static_protos_TSAccum_descriptor;
+      }
+
+      protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+          internalGetFieldAccessorTable() {
+        return org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.internal_static_protos_TSAccum_fieldAccessorTable
+            .ensureFieldAccessorsInitialized(
+                org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSAccum.class, org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSAccum.Builder.class);
+      }
+
+      // Construct using org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSAccum.newBuilder()
+      private Builder() {
+        maybeForceBuilderInitialization();
+      }
+
+      private Builder(
+          com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+        super(parent);
+        maybeForceBuilderInitialization();
+      }
+      private void maybeForceBuilderInitialization() {
+        if (com.google.protobuf.GeneratedMessageV3
+                .alwaysUseFieldBuilders) {
+        }
+      }
+      public Builder clear() {
+        super.clear();
+        if (keyBuilder_ == null) {
+          key_ = null;
+        } else {
+          key_ = null;
+          keyBuilder_ = null;
+        }
+        if (pipelineVersionBuilder_ == null) {
+          pipelineVersion_ = null;
+        } else {
+          pipelineVersion_ = null;
+          pipelineVersionBuilder_ = null;
+        }
+        if (dataBuilder_ == null) {
+          data_ = null;
+        } else {
+          data_ = null;
+          dataBuilder_ = null;
+        }
+        durationMills_ = 0L;
+
+        if (lowerWindowBoundaryBuilder_ == null) {
+          lowerWindowBoundary_ = null;
+        } else {
+          lowerWindowBoundary_ = null;
+          lowerWindowBoundaryBuilder_ = null;
+        }
+        if (upperWindowBoundaryBuilder_ == null) {
+          upperWindowBoundary_ = null;
+        } else {
+          upperWindowBoundary_ = null;
+          upperWindowBoundaryBuilder_ = null;
+        }
+        return this;
+      }
+
+      public com.google.protobuf.Descriptors.Descriptor
+          getDescriptorForType() {
+        return org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.internal_static_protos_TSAccum_descriptor;
+      }
+
+      public org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSAccum getDefaultInstanceForType() {
+        return org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSAccum.getDefaultInstance();
+      }
+
+      public org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSAccum build() {
+        org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSAccum result = buildPartial();
+        if (!result.isInitialized()) {
+          throw newUninitializedMessageException(result);
+        }
+        return result;
+      }
+
+      public org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSAccum buildPartial() {
+        org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSAccum result = new org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSAccum(this);
+        if (keyBuilder_ == null) {
+          result.key_ = key_;
+        } else {
+          result.key_ = keyBuilder_.build();
+        }
+        if (pipelineVersionBuilder_ == null) {
+          result.pipelineVersion_ = pipelineVersion_;
+        } else {
+          result.pipelineVersion_ = pipelineVersionBuilder_.build();
+        }
+        if (dataBuilder_ == null) {
+          result.data_ = data_;
+        } else {
+          result.data_ = dataBuilder_.build();
+        }
+        result.durationMills_ = durationMills_;
+        if (lowerWindowBoundaryBuilder_ == null) {
+          result.lowerWindowBoundary_ = lowerWindowBoundary_;
+        } else {
+          result.lowerWindowBoundary_ = lowerWindowBoundaryBuilder_.build();
+        }
+        if (upperWindowBoundaryBuilder_ == null) {
+          result.upperWindowBoundary_ = upperWindowBoundary_;
+        } else {
+          result.upperWindowBoundary_ = upperWindowBoundaryBuilder_.build();
+        }
+        onBuilt();
+        return result;
+      }
+
+      public Builder clone() {
+        return (Builder) super.clone();
+      }
+      public Builder setField(
+          com.google.protobuf.Descriptors.FieldDescriptor field,
+          Object value) {
+        return (Builder) super.setField(field, value);
+      }
+      public Builder clearField(
+          com.google.protobuf.Descriptors.FieldDescriptor field) {
+        return (Builder) super.clearField(field);
+      }
+      public Builder clearOneof(
+          com.google.protobuf.Descriptors.OneofDescriptor oneof) {
+        return (Builder) super.clearOneof(oneof);
+      }
+      public Builder setRepeatedField(
+          com.google.protobuf.Descriptors.FieldDescriptor field,
+          int index, Object value) {
+        return (Builder) super.setRepeatedField(field, index, value);
+      }
+      public Builder addRepeatedField(
+          com.google.protobuf.Descriptors.FieldDescriptor field,
+          Object value) {
+        return (Builder) super.addRepeatedField(field, value);
+      }
+      public Builder mergeFrom(com.google.protobuf.Message other) {
+        if (other instanceof org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSAccum) {
+          return mergeFrom((org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSAccum)other);
+        } else {
+          super.mergeFrom(other);
+          return this;
+        }
+      }
+
+      public Builder mergeFrom(org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSAccum other) {
+        if (other == org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSAccum.getDefaultInstance()) return this;
+        if (other.hasKey()) {
+          mergeKey(other.getKey());
+        }
+        if (other.hasPipelineVersion()) {
+          mergePipelineVersion(other.getPipelineVersion());
+        }
+        if (other.hasData()) {
+          mergeData(other.getData());
+        }
+        if (other.getDurationMills() != 0L) {
+          setDurationMills(other.getDurationMills());
+        }
+        if (other.hasLowerWindowBoundary()) {
+          mergeLowerWindowBoundary(other.getLowerWindowBoundary());
+        }
+        if (other.hasUpperWindowBoundary()) {
+          mergeUpperWindowBoundary(other.getUpperWindowBoundary());
+        }
+        onChanged();
+        return this;
+      }
+
+      public final boolean isInitialized() {
+        return true;
+      }
+
+      public Builder mergeFrom(
+          com.google.protobuf.CodedInputStream input,
+          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws java.io.IOException {
+        org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSAccum parsedMessage = null;
+        try {
+          parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
+        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+          parsedMessage = (org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSAccum) e.getUnfinishedMessage();
+          throw e.unwrapIOException();
+        } finally {
+          if (parsedMessage != null) {
+            mergeFrom(parsedMessage);
+          }
+        }
+        return this;
+      }
+
+      private org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSKey key_ = null;
+      private com.google.protobuf.SingleFieldBuilderV3<
+          org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSKey, org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSKey.Builder, org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSKeyOrBuilder> keyBuilder_;
+      /**
+       * <code>.protos.TSKey key = 1;</code>
+       */
+      public boolean hasKey() {
+        return keyBuilder_ != null || key_ != null;
+      }
+      /**
+       * <code>.protos.TSKey key = 1;</code>
+       */
+      public org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSKey getKey() {
+        if (keyBuilder_ == null) {
+          return key_ == null ? org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSKey.getDefaultInstance() : key_;
+        } else {
+          return keyBuilder_.getMessage();
+        }
+      }
+      /**
+       * <code>.protos.TSKey key = 1;</code>
+       */
+      public Builder setKey(org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSKey value) {
+        if (keyBuilder_ == null) {
+          if (value == null) {
+            throw new NullPointerException();
+          }
+          key_ = value;
+          onChanged();
+        } else {
+          keyBuilder_.setMessage(value);
+        }
+
+        return this;
+      }
+      /**
+       * <code>.protos.TSKey key = 1;</code>
+       */
+      public Builder setKey(
+          org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSKey.Builder builderForValue) {
+        if (keyBuilder_ == null) {
+          key_ = builderForValue.build();
+          onChanged();
+        } else {
+          keyBuilder_.setMessage(builderForValue.build());
+        }
+
+        return this;
+      }
+      /**
+       * <code>.protos.TSKey key = 1;</code>
+       */
+      public Builder mergeKey(org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSKey value) {
+        if (keyBuilder_ == null) {
+          if (key_ != null) {
+            key_ =
+              org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSKey.newBuilder(key_).mergeFrom(value).buildPartial();
+          } else {
+            key_ = value;
+          }
+          onChanged();
+        } else {
+          keyBuilder_.mergeFrom(value);
+        }
+
+        return this;
+      }
+      /**
+       * <code>.protos.TSKey key = 1;</code>
+       */
+      public Builder clearKey() {
+        if (keyBuilder_ == null) {
+          key_ = null;
+          onChanged();
+        } else {
+          key_ = null;
+          keyBuilder_ = null;
+        }
+
+        return this;
+      }
+      /**
+       * <code>.protos.TSKey key = 1;</code>
+       */
+      public org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSKey.Builder getKeyBuilder() {
+        
+        onChanged();
+        return getKeyFieldBuilder().getBuilder();
+      }
+      /**
+       * <code>.protos.TSKey key = 1;</code>
+       */
+      public org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSKeyOrBuilder getKeyOrBuilder() {
+        if (keyBuilder_ != null) {
+          return keyBuilder_.getMessageOrBuilder();
+        } else {
+          return key_ == null ?
+              org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSKey.getDefaultInstance() : key_;
+        }
+      }
+      /**
+       * <code>.protos.TSKey key = 1;</code>
+       */
+      private com.google.protobuf.SingleFieldBuilderV3<
+          org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSKey, org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSKey.Builder, org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSKeyOrBuilder> 
+          getKeyFieldBuilder() {
+        if (keyBuilder_ == null) {
+          keyBuilder_ = new com.google.protobuf.SingleFieldBuilderV3<
+              org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSKey, org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSKey.Builder, org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSKeyOrBuilder>(
+                  getKey(),
+                  getParentForChildren(),
+                  isClean());
+          key_ = null;
+        }
+        return keyBuilder_;
+      }
+
+      private org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.PipelineVersion pipelineVersion_ = null;
+      private com.google.protobuf.SingleFieldBuilderV3<
+          org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.PipelineVersion, org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.PipelineVersion.Builder, org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.PipelineVersionOrBuilder> pipelineVersionBuilder_;
+      /**
+       * <code>.protos.PipelineVersion pipelineVersion = 2;</code>
+       */
+      public boolean hasPipelineVersion() {
+        return pipelineVersionBuilder_ != null || pipelineVersion_ != null;
+      }
+      /**
+       * <code>.protos.PipelineVersion pipelineVersion = 2;</code>
+       */
+      public org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.PipelineVersion getPipelineVersion() {
+        if (pipelineVersionBuilder_ == null) {
+          return pipelineVersion_ == null ? org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.PipelineVersion.getDefaultInstance() : pipelineVersion_;
+        } else {
+          return pipelineVersionBuilder_.getMessage();
+        }
+      }
+      /**
+       * <code>.protos.PipelineVersion pipelineVersion = 2;</code>
+       */
+      public Builder setPipelineVersion(org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.PipelineVersion value) {
+        if (pipelineVersionBuilder_ == null) {
+          if (value == null) {
+            throw new NullPointerException();
+          }
+          pipelineVersion_ = value;
+          onChanged();
+        } else {
+          pipelineVersionBuilder_.setMessage(value);
+        }
+
+        return this;
+      }
+      /**
+       * <code>.protos.PipelineVersion pipelineVersion = 2;</code>
+       */
+      public Builder setPipelineVersion(
+          org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.PipelineVersion.Builder builderForValue) {
+        if (pipelineVersionBuilder_ == null) {
+          pipelineVersion_ = builderForValue.build();
+          onChanged();
+        } else {
+          pipelineVersionBuilder_.setMessage(builderForValue.build());
+        }
+
+        return this;
+      }
+      /**
+       * <code>.protos.PipelineVersion pipelineVersion = 2;</code>
+       */
+      public Builder mergePipelineVersion(org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.PipelineVersion value) {
+        if (pipelineVersionBuilder_ == null) {
+          if (pipelineVersion_ != null) {
+            pipelineVersion_ =
+              org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.PipelineVersion.newBuilder(pipelineVersion_).mergeFrom(value).buildPartial();
+          } else {
+            pipelineVersion_ = value;
+          }
+          onChanged();
+        } else {
+          pipelineVersionBuilder_.mergeFrom(value);
+        }
+
+        return this;
+      }
+      /**
+       * <code>.protos.PipelineVersion pipelineVersion = 2;</code>
+       */
+      public Builder clearPipelineVersion() {
+        if (pipelineVersionBuilder_ == null) {
+          pipelineVersion_ = null;
+          onChanged();
+        } else {
+          pipelineVersion_ = null;
+          pipelineVersionBuilder_ = null;
+        }
+
+        return this;
+      }
+      /**
+       * <code>.protos.PipelineVersion pipelineVersion = 2;</code>
+       */
+      public org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.PipelineVersion.Builder getPipelineVersionBuilder() {
+        
+        onChanged();
+        return getPipelineVersionFieldBuilder().getBuilder();
+      }
+      /**
+       * <code>.protos.PipelineVersion pipelineVersion = 2;</code>
+       */
+      public org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.PipelineVersionOrBuilder getPipelineVersionOrBuilder() {
+        if (pipelineVersionBuilder_ != null) {
+          return pipelineVersionBuilder_.getMessageOrBuilder();
+        } else {
+          return pipelineVersion_ == null ?
+              org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.PipelineVersion.getDefaultInstance() : pipelineVersion_;
+        }
+      }
+      /**
+       * <code>.protos.PipelineVersion pipelineVersion = 2;</code>
+       */
+      private com.google.protobuf.SingleFieldBuilderV3<
+          org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.PipelineVersion, org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.PipelineVersion.Builder, org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.PipelineVersionOrBuilder> 
+          getPipelineVersionFieldBuilder() {
+        if (pipelineVersionBuilder_ == null) {
+          pipelineVersionBuilder_ = new com.google.protobuf.SingleFieldBuilderV3<
+              org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.PipelineVersion, org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.PipelineVersion.Builder, org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.PipelineVersionOrBuilder>(
+                  getPipelineVersion(),
+                  getParentForChildren(),
+                  isClean());
+          pipelineVersion_ = null;
+        }
+        return pipelineVersionBuilder_;
+      }
+
+      private org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSDataPoint data_ = null;
+      private com.google.protobuf.SingleFieldBuilderV3<
+          org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSDataPoint, org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSDataPoint.Builder, org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSDataPointOrBuilder> dataBuilder_;
+      /**
+       * <code>.protos.TSDataPoint data = 3;</code>
+       */
+      public boolean hasData() {
+        return dataBuilder_ != null || data_ != null;
+      }
+      /**
+       * <code>.protos.TSDataPoint data = 3;</code>
+       */
+      public org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSDataPoint getData() {
+        if (dataBuilder_ == null) {
+          return data_ == null ? org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSDataPoint.getDefaultInstance() : data_;
+        } else {
+          return dataBuilder_.getMessage();
+        }
+      }
+      /**
+       * <code>.protos.TSDataPoint data = 3;</code>
+       */
+      public Builder setData(org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSDataPoint value) {
+        if (dataBuilder_ == null) {
+          if (value == null) {
+            throw new NullPointerException();
+          }
+          data_ = value;
+          onChanged();
+        } else {
+          dataBuilder_.setMessage(value);
+        }
+
+        return this;
+      }
+      /**
+       * <code>.protos.TSDataPoint data = 3;</code>
+       */
+      public Builder setData(
+          org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSDataPoint.Builder builderForValue) {
+        if (dataBuilder_ == null) {
+          data_ = builderForValue.build();
+          onChanged();
+        } else {
+          dataBuilder_.setMessage(builderForValue.build());
+        }
+
+        return this;
+      }
+      /**
+       * <code>.protos.TSDataPoint data = 3;</code>
+       */
+      public Builder mergeData(org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSDataPoint value) {
+        if (dataBuilder_ == null) {
+          if (data_ != null) {
+            data_ =
+              org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSDataPoint.newBuilder(data_).mergeFrom(value).buildPartial();
+          } else {
+            data_ = value;
+          }
+          onChanged();
+        } else {
+          dataBuilder_.mergeFrom(value);
+        }
+
+        return this;
+      }
+      /**
+       * <code>.protos.TSDataPoint data = 3;</code>
+       */
+      public Builder clearData() {
+        if (dataBuilder_ == null) {
+          data_ = null;
+          onChanged();
+        } else {
+          data_ = null;
+          dataBuilder_ = null;
+        }
+
+        return this;
+      }
+      /**
+       * <code>.protos.TSDataPoint data = 3;</code>
+       */
+      public org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSDataPoint.Builder getDataBuilder() {
+        
+        onChanged();
+        return getDataFieldBuilder().getBuilder();
+      }
+      /**
+       * <code>.protos.TSDataPoint data = 3;</code>
+       */
+      public org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSDataPointOrBuilder getDataOrBuilder() {
+        if (dataBuilder_ != null) {
+          return dataBuilder_.getMessageOrBuilder();
+        } else {
+          return data_ == null ?
+              org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSDataPoint.getDefaultInstance() : data_;
+        }
+      }
+      /**
+       * <code>.protos.TSDataPoint data = 3;</code>
+       */
+      private com.google.protobuf.SingleFieldBuilderV3<
+          org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSDataPoint, org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSDataPoint.Builder, org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSDataPointOrBuilder> 
+          getDataFieldBuilder() {
+        if (dataBuilder_ == null) {
+          dataBuilder_ = new com.google.protobuf.SingleFieldBuilderV3<
+              org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSDataPoint, org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSDataPoint.Builder, org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSDataPointOrBuilder>(
+                  getData(),
+                  getParentForChildren(),
+                  isClean());
+          data_ = null;
+        }
+        return dataBuilder_;
+      }
+
+      private long durationMills_ ;
+      /**
+       * <code>int64 durationMills = 4;</code>
+       */
+      public long getDurationMills() {
+        return durationMills_;
+      }
+      /**
+       * <code>int64 durationMills = 4;</code>
+       */
+      public Builder setDurationMills(long value) {
+        
+        durationMills_ = value;
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>int64 durationMills = 4;</code>
+       */
+      public Builder clearDurationMills() {
+        
+        durationMills_ = 0L;
+        onChanged();
+        return this;
+      }
+
+      private com.google.protobuf.Timestamp lowerWindowBoundary_ = null;
+      private com.google.protobuf.SingleFieldBuilderV3<
+          com.google.protobuf.Timestamp, com.google.protobuf.Timestamp.Builder, com.google.protobuf.TimestampOrBuilder> lowerWindowBoundaryBuilder_;
+      /**
+       * <code>.google.protobuf.Timestamp lowerWindowBoundary = 5;</code>
+       */
+      public boolean hasLowerWindowBoundary() {
+        return lowerWindowBoundaryBuilder_ != null || lowerWindowBoundary_ != null;
+      }
+      /**
+       * <code>.google.protobuf.Timestamp lowerWindowBoundary = 5;</code>
+       */
+      public com.google.protobuf.Timestamp getLowerWindowBoundary() {
+        if (lowerWindowBoundaryBuilder_ == null) {
+          return lowerWindowBoundary_ == null ? com.google.protobuf.Timestamp.getDefaultInstance() : lowerWindowBoundary_;
+        } else {
+          return lowerWindowBoundaryBuilder_.getMessage();
+        }
+      }
+      /**
+       * <code>.google.protobuf.Timestamp lowerWindowBoundary = 5;</code>
+       */
+      public Builder setLowerWindowBoundary(com.google.protobuf.Timestamp value) {
+        if (lowerWindowBoundaryBuilder_ == null) {
+          if (value == null) {
+            throw new NullPointerException();
+          }
+          lowerWindowBoundary_ = value;
+          onChanged();
+        } else {
+          lowerWindowBoundaryBuilder_.setMessage(value);
+        }
+
+        return this;
+      }
+      /**
+       * <code>.google.protobuf.Timestamp lowerWindowBoundary = 5;</code>
+       */
+      public Builder setLowerWindowBoundary(
+          com.google.protobuf.Timestamp.Builder builderForValue) {
+        if (lowerWindowBoundaryBuilder_ == null) {
+          lowerWindowBoundary_ = builderForValue.build();
+          onChanged();
+        } else {
+          lowerWindowBoundaryBuilder_.setMessage(builderForValue.build());
+        }
+
+        return this;
+      }
+      /**
+       * <code>.google.protobuf.Timestamp lowerWindowBoundary = 5;</code>
+       */
+      public Builder mergeLowerWindowBoundary(com.google.protobuf.Timestamp value) {
+        if (lowerWindowBoundaryBuilder_ == null) {
+          if (lowerWindowBoundary_ != null) {
+            lowerWindowBoundary_ =
+              com.google.protobuf.Timestamp.newBuilder(lowerWindowBoundary_).mergeFrom(value).buildPartial();
+          } else {
+            lowerWindowBoundary_ = value;
+          }
+          onChanged();
+        } else {
+          lowerWindowBoundaryBuilder_.mergeFrom(value);
+        }
+
+        return this;
+      }
+      /**
+       * <code>.google.protobuf.Timestamp lowerWindowBoundary = 5;</code>
+       */
+      public Builder clearLowerWindowBoundary() {
+        if (lowerWindowBoundaryBuilder_ == null) {
+          lowerWindowBoundary_ = null;
+          onChanged();
+        } else {
+          lowerWindowBoundary_ = null;
+          lowerWindowBoundaryBuilder_ = null;
+        }
+
+        return this;
+      }
+      /**
+       * <code>.google.protobuf.Timestamp lowerWindowBoundary = 5;</code>
+       */
+      public com.google.protobuf.Timestamp.Builder getLowerWindowBoundaryBuilder() {
+        
+        onChanged();
+        return getLowerWindowBoundaryFieldBuilder().getBuilder();
+      }
+      /**
+       * <code>.google.protobuf.Timestamp lowerWindowBoundary = 5;</code>
+       */
+      public com.google.protobuf.TimestampOrBuilder getLowerWindowBoundaryOrBuilder() {
+        if (lowerWindowBoundaryBuilder_ != null) {
+          return lowerWindowBoundaryBuilder_.getMessageOrBuilder();
+        } else {
+          return lowerWindowBoundary_ == null ?
+              com.google.protobuf.Timestamp.getDefaultInstance() : lowerWindowBoundary_;
+        }
+      }
+      /**
+       * <code>.google.protobuf.Timestamp lowerWindowBoundary = 5;</code>
+       */
+      private com.google.protobuf.SingleFieldBuilderV3<
+          com.google.protobuf.Timestamp, com.google.protobuf.Timestamp.Builder, com.google.protobuf.TimestampOrBuilder> 
+          getLowerWindowBoundaryFieldBuilder() {
+        if (lowerWindowBoundaryBuilder_ == null) {
+          lowerWindowBoundaryBuilder_ = new com.google.protobuf.SingleFieldBuilderV3<
+              com.google.protobuf.Timestamp, com.google.protobuf.Timestamp.Builder, com.google.protobuf.TimestampOrBuilder>(
+                  getLowerWindowBoundary(),
+                  getParentForChildren(),
+                  isClean());
+          lowerWindowBoundary_ = null;
+        }
+        return lowerWindowBoundaryBuilder_;
+      }
+
+      private com.google.protobuf.Timestamp upperWindowBoundary_ = null;
+      private com.google.protobuf.SingleFieldBuilderV3<
+          com.google.protobuf.Timestamp, com.google.protobuf.Timestamp.Builder, com.google.protobuf.TimestampOrBuilder> upperWindowBoundaryBuilder_;
+      /**
+       * <code>.google.protobuf.Timestamp upperWindowBoundary = 6;</code>
+       */
+      public boolean hasUpperWindowBoundary() {
+        return upperWindowBoundaryBuilder_ != null || upperWindowBoundary_ != null;
+      }
+      /**
+       * <code>.google.protobuf.Timestamp upperWindowBoundary = 6;</code>
+       */
+      public com.google.protobuf.Timestamp getUpperWindowBoundary() {
+        if (upperWindowBoundaryBuilder_ == null) {
+          return upperWindowBoundary_ == null ? com.google.protobuf.Timestamp.getDefaultInstance() : upperWindowBoundary_;
+        } else {
+          return upperWindowBoundaryBuilder_.getMessage();
+        }
+      }
+      /**
+       * <code>.google.protobuf.Timestamp upperWindowBoundary = 6;</code>
+       */
+      public Builder setUpperWindowBoundary(com.google.protobuf.Timestamp value) {
+        if (upperWindowBoundaryBuilder_ == null) {
+          if (value == null) {
+            throw new NullPointerException();
+          }
+          upperWindowBoundary_ = value;
+          onChanged();
+        } else {
+          upperWindowBoundaryBuilder_.setMessage(value);
+        }
+
+        return this;
+      }
+      /**
+       * <code>.google.protobuf.Timestamp upperWindowBoundary = 6;</code>
+       */
+      public Builder setUpperWindowBoundary(
+          com.google.protobuf.Timestamp.Builder builderForValue) {
+        if (upperWindowBoundaryBuilder_ == null) {
+          upperWindowBoundary_ = builderForValue.build();
+          onChanged();
+        } else {
+          upperWindowBoundaryBuilder_.setMessage(builderForValue.build());
+        }
+
+        return this;
+      }
+      /**
+       * <code>.google.protobuf.Timestamp upperWindowBoundary = 6;</code>
+       */
+      public Builder mergeUpperWindowBoundary(com.google.protobuf.Timestamp value) {
+        if (upperWindowBoundaryBuilder_ == null) {
+          if (upperWindowBoundary_ != null) {
+            upperWindowBoundary_ =
+              com.google.protobuf.Timestamp.newBuilder(upperWindowBoundary_).mergeFrom(value).buildPartial();
+          } else {
+            upperWindowBoundary_ = value;
+          }
+          onChanged();
+        } else {
+          upperWindowBoundaryBuilder_.mergeFrom(value);
+        }
+
+        return this;
+      }
+      /**
+       * <code>.google.protobuf.Timestamp upperWindowBoundary = 6;</code>
+       */
+      public Builder clearUpperWindowBoundary() {
+        if (upperWindowBoundaryBuilder_ == null) {
+          upperWindowBoundary_ = null;
+          onChanged();
+        } else {
+          upperWindowBoundary_ = null;
+          upperWindowBoundaryBuilder_ = null;
+        }
+
+        return this;
+      }
+      /**
+       * <code>.google.protobuf.Timestamp upperWindowBoundary = 6;</code>
+       */
+      public com.google.protobuf.Timestamp.Builder getUpperWindowBoundaryBuilder() {
+        
+        onChanged();
+        return getUpperWindowBoundaryFieldBuilder().getBuilder();
+      }
+      /**
+       * <code>.google.protobuf.Timestamp upperWindowBoundary = 6;</code>
+       */
+      public com.google.protobuf.TimestampOrBuilder getUpperWindowBoundaryOrBuilder() {
+        if (upperWindowBoundaryBuilder_ != null) {
+          return upperWindowBoundaryBuilder_.getMessageOrBuilder();
+        } else {
+          return upperWindowBoundary_ == null ?
+              com.google.protobuf.Timestamp.getDefaultInstance() : upperWindowBoundary_;
+        }
+      }
+      /**
+       * <code>.google.protobuf.Timestamp upperWindowBoundary = 6;</code>
+       */
+      private com.google.protobuf.SingleFieldBuilderV3<
+          com.google.protobuf.Timestamp, com.google.protobuf.Timestamp.Builder, com.google.protobuf.TimestampOrBuilder> 
+          getUpperWindowBoundaryFieldBuilder() {
+        if (upperWindowBoundaryBuilder_ == null) {
+          upperWindowBoundaryBuilder_ = new com.google.protobuf.SingleFieldBuilderV3<
+              com.google.protobuf.Timestamp, com.google.protobuf.Timestamp.Builder, com.google.protobuf.TimestampOrBuilder>(
+                  getUpperWindowBoundary(),
+                  getParentForChildren(),
+                  isClean());
+          upperWindowBoundary_ = null;
+        }
+        return upperWindowBoundaryBuilder_;
+      }
+      public final Builder setUnknownFields(
+          final com.google.protobuf.UnknownFieldSet unknownFields) {
+        return this;
+      }
+
+      public final Builder mergeUnknownFields(
+          final com.google.protobuf.UnknownFieldSet unknownFields) {
+        return this;
+      }
+
+
+      // @@protoc_insertion_point(builder_scope:protos.TSAccum)
+    }
+
+    // @@protoc_insertion_point(class_scope:protos.TSAccum)
+    private static final org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSAccum DEFAULT_INSTANCE;
+    static {
+      DEFAULT_INSTANCE = new org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSAccum();
+    }
+
+    public static org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSAccum getDefaultInstance() {
+      return DEFAULT_INSTANCE;
+    }
+
+    private static final com.google.protobuf.Parser<TSAccum>
+        PARSER = new com.google.protobuf.AbstractParser<TSAccum>() {
+      public TSAccum parsePartialFrom(
+          com.google.protobuf.CodedInputStream input,
+          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws com.google.protobuf.InvalidProtocolBufferException {
+          return new TSAccum(input, extensionRegistry);
+      }
+    };
+
+    public static com.google.protobuf.Parser<TSAccum> parser() {
+      return PARSER;
+    }
+
+    @java.lang.Override
+    public com.google.protobuf.Parser<TSAccum> getParserForType() {
+      return PARSER;
+    }
+
+    public org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSAccum getDefaultInstanceForType() {
+      return DEFAULT_INSTANCE;
+    }
+
+  }
+
+  public interface TSAccumSequenceOrBuilder extends
+      // @@protoc_insertion_point(interface_extends:protos.TSAccumSequence)
+      com.google.protobuf.MessageOrBuilder {
+
+    /**
+     * <code>.protos.TSKey key = 1;</code>
+     */
+    boolean hasKey();
+    /**
+     * <code>.protos.TSKey key = 1;</code>
+     */
+    org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSKey getKey();
+    /**
+     * <code>.protos.TSKey key = 1;</code>
+     */
+    org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSKeyOrBuilder getKeyOrBuilder();
+
+    /**
+     * <code>.protos.PipelineVersion pipelineVersion = 2;</code>
+     */
+    boolean hasPipelineVersion();
+    /**
+     * <code>.protos.PipelineVersion pipelineVersion = 2;</code>
+     */
+    org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.PipelineVersion getPipelineVersion();
+    /**
+     * <code>.protos.PipelineVersion pipelineVersion = 2;</code>
+     */
+    org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.PipelineVersionOrBuilder getPipelineVersionOrBuilder();
+
+    /**
+     * <code>repeated .protos.TSAccum accums = 3;</code>
+     */
+    java.util.List<org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSAccum> 
+        getAccumsList();
+    /**
+     * <code>repeated .protos.TSAccum accums = 3;</code>
+     */
+    org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSAccum getAccums(int index);
+    /**
+     * <code>repeated .protos.TSAccum accums = 3;</code>
+     */
+    int getAccumsCount();
+    /**
+     * <code>repeated .protos.TSAccum accums = 3;</code>
+     */
+    java.util.List<? extends org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSAccumOrBuilder> 
+        getAccumsOrBuilderList();
+    /**
+     * <code>repeated .protos.TSAccum accums = 3;</code>
+     */
+    org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSAccumOrBuilder getAccumsOrBuilder(
+        int index);
+
+    /**
+     * <code>.google.protobuf.Timestamp lowerWindowBoundary = 4;</code>
+     */
+    boolean hasLowerWindowBoundary();
+    /**
+     * <code>.google.protobuf.Timestamp lowerWindowBoundary = 4;</code>
+     */
+    com.google.protobuf.Timestamp getLowerWindowBoundary();
+    /**
+     * <code>.google.protobuf.Timestamp lowerWindowBoundary = 4;</code>
+     */
+    com.google.protobuf.TimestampOrBuilder getLowerWindowBoundaryOrBuilder();
+
+    /**
+     * <code>.google.protobuf.Timestamp upperWindowBoundary = 5;</code>
+     */
+    boolean hasUpperWindowBoundary();
+    /**
+     * <code>.google.protobuf.Timestamp upperWindowBoundary = 5;</code>
+     */
+    com.google.protobuf.Timestamp getUpperWindowBoundary();
+    /**
+     * <code>.google.protobuf.Timestamp upperWindowBoundary = 5;</code>
+     */
+    com.google.protobuf.TimestampOrBuilder getUpperWindowBoundaryOrBuilder();
+  }
+  /**
+   * <pre>
+   * Time Series sequence holder
+   * </pre>
+   *
+   * Protobuf type {@code protos.TSAccumSequence}
+   */
+  public  static final class TSAccumSequence extends
+      com.google.protobuf.GeneratedMessageV3 implements
+      // @@protoc_insertion_point(message_implements:protos.TSAccumSequence)
+      TSAccumSequenceOrBuilder {
+    // Use TSAccumSequence.newBuilder() to construct.
+    private TSAccumSequence(com.google.protobuf.GeneratedMessageV3.Builder<?> builder) {
+      super(builder);
+    }
+    private TSAccumSequence() {
+      accums_ = java.util.Collections.emptyList();
+    }
+
+    @java.lang.Override
+    public final com.google.protobuf.UnknownFieldSet
+    getUnknownFields() {
+      return com.google.protobuf.UnknownFieldSet.getDefaultInstance();
+    }
+    private TSAccumSequence(
+        com.google.protobuf.CodedInputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      this();
+      int mutable_bitField0_ = 0;
+      try {
+        boolean done = false;
+        while (!done) {
+          int tag = input.readTag();
+          switch (tag) {
+            case 0:
+              done = true;
+              break;
+            default: {
+              if (!input.skipField(tag)) {
+                done = true;
+              }
+              break;
+            }
+            case 10: {
+              org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSKey.Builder subBuilder = null;
+              if (key_ != null) {
+                subBuilder = key_.toBuilder();
+              }
+              key_ = input.readMessage(org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSKey.parser(), extensionRegistry);
+              if (subBuilder != null) {
+                subBuilder.mergeFrom(key_);
+                key_ = subBuilder.buildPartial();
+              }
+
+              break;
+            }
+            case 18: {
+              org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.PipelineVersion.Builder subBuilder = null;
+              if (pipelineVersion_ != null) {
+                subBuilder = pipelineVersion_.toBuilder();
+              }
+              pipelineVersion_ = input.readMessage(org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.PipelineVersion.parser(), extensionRegistry);
+              if (subBuilder != null) {
+                subBuilder.mergeFrom(pipelineVersion_);
+                pipelineVersion_ = subBuilder.buildPartial();
+              }
+
+              break;
+            }
+            case 26: {
+              if (!((mutable_bitField0_ & 0x00000004) == 0x00000004)) {
+                accums_ = new java.util.ArrayList<org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSAccum>();
+                mutable_bitField0_ |= 0x00000004;
+              }
+              accums_.add(
+                  input.readMessage(org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSAccum.parser(), extensionRegistry));
+              break;
+            }
+            case 34: {
+              com.google.protobuf.Timestamp.Builder subBuilder = null;
+              if (lowerWindowBoundary_ != null) {
+                subBuilder = lowerWindowBoundary_.toBuilder();
+              }
+              lowerWindowBoundary_ = input.readMessage(com.google.protobuf.Timestamp.parser(), extensionRegistry);
+              if (subBuilder != null) {
+                subBuilder.mergeFrom(lowerWindowBoundary_);
+                lowerWindowBoundary_ = subBuilder.buildPartial();
+              }
+
+              break;
+            }
+            case 42: {
+              com.google.protobuf.Timestamp.Builder subBuilder = null;
+              if (upperWindowBoundary_ != null) {
+                subBuilder = upperWindowBoundary_.toBuilder();
+              }
+              upperWindowBoundary_ = input.readMessage(com.google.protobuf.Timestamp.parser(), extensionRegistry);
+              if (subBuilder != null) {
+                subBuilder.mergeFrom(upperWindowBoundary_);
+                upperWindowBoundary_ = subBuilder.buildPartial();
+              }
+
+              break;
+            }
+          }
+        }
+      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        throw e.setUnfinishedMessage(this);
+      } catch (java.io.IOException e) {
+        throw new com.google.protobuf.InvalidProtocolBufferException(
+            e).setUnfinishedMessage(this);
+      } finally {
+        if (((mutable_bitField0_ & 0x00000004) == 0x00000004)) {
+          accums_ = java.util.Collections.unmodifiableList(accums_);
+        }
+        makeExtensionsImmutable();
+      }
+    }
+    public static final com.google.protobuf.Descriptors.Descriptor
+        getDescriptor() {
+      return org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.internal_static_protos_TSAccumSequence_descriptor;
+    }
+
+    protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+        internalGetFieldAccessorTable() {
+      return org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.internal_static_protos_TSAccumSequence_fieldAccessorTable
+          .ensureFieldAccessorsInitialized(
+              org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSAccumSequence.class, org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSAccumSequence.Builder.class);
+    }
+
+    private int bitField0_;
+    public static final int KEY_FIELD_NUMBER = 1;
+    private org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSKey key_;
+    /**
+     * <code>.protos.TSKey key = 1;</code>
+     */
+    public boolean hasKey() {
+      return key_ != null;
+    }
+    /**
+     * <code>.protos.TSKey key = 1;</code>
+     */
+    public org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSKey getKey() {
+      return key_ == null ? org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSKey.getDefaultInstance() : key_;
+    }
+    /**
+     * <code>.protos.TSKey key = 1;</code>
+     */
+    public org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSKeyOrBuilder getKeyOrBuilder() {
+      return getKey();
+    }
+
+    public static final int PIPELINEVERSION_FIELD_NUMBER = 2;
+    private org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.PipelineVersion pipelineVersion_;
+    /**
+     * <code>.protos.PipelineVersion pipelineVersion = 2;</code>
+     */
+    public boolean hasPipelineVersion() {
+      return pipelineVersion_ != null;
+    }
+    /**
+     * <code>.protos.PipelineVersion pipelineVersion = 2;</code>
+     */
+    public org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.PipelineVersion getPipelineVersion() {
+      return pipelineVersion_ == null ? org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.PipelineVersion.getDefaultInstance() : pipelineVersion_;
+    }
+    /**
+     * <code>.protos.PipelineVersion pipelineVersion = 2;</code>
+     */
+    public org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.PipelineVersionOrBuilder getPipelineVersionOrBuilder() {
+      return getPipelineVersion();
+    }
+
+    public static final int ACCUMS_FIELD_NUMBER = 3;
+    private java.util.List<org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSAccum> accums_;
+    /**
+     * <code>repeated .protos.TSAccum accums = 3;</code>
+     */
+    public java.util.List<org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSAccum> getAccumsList() {
+      return accums_;
+    }
+    /**
+     * <code>repeated .protos.TSAccum accums = 3;</code>
+     */
+    public java.util.List<? extends org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSAccumOrBuilder> 
+        getAccumsOrBuilderList() {
+      return accums_;
+    }
+    /**
+     * <code>repeated .protos.TSAccum accums = 3;</code>
+     */
+    public int getAccumsCount() {
+      return accums_.size();
+    }
+    /**
+     * <code>repeated .protos.TSAccum accums = 3;</code>
+     */
+    public org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSAccum getAccums(int index) {
+      return accums_.get(index);
+    }
+    /**
+     * <code>repeated .protos.TSAccum accums = 3;</code>
+     */
+    public org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSAccumOrBuilder getAccumsOrBuilder(
+        int index) {
+      return accums_.get(index);
+    }
+
+    public static final int LOWERWINDOWBOUNDARY_FIELD_NUMBER = 4;
+    private com.google.protobuf.Timestamp lowerWindowBoundary_;
+    /**
+     * <code>.google.protobuf.Timestamp lowerWindowBoundary = 4;</code>
+     */
+    public boolean hasLowerWindowBoundary() {
+      return lowerWindowBoundary_ != null;
+    }
+    /**
+     * <code>.google.protobuf.Timestamp lowerWindowBoundary = 4;</code>
+     */
+    public com.google.protobuf.Timestamp getLowerWindowBoundary() {
+      return lowerWindowBoundary_ == null ? com.google.protobuf.Timestamp.getDefaultInstance() : lowerWindowBoundary_;
+    }
+    /**
+     * <code>.google.protobuf.Timestamp lowerWindowBoundary = 4;</code>
+     */
+    public com.google.protobuf.TimestampOrBuilder getLowerWindowBoundaryOrBuilder() {
+      return getLowerWindowBoundary();
+    }
+
+    public static final int UPPERWINDOWBOUNDARY_FIELD_NUMBER = 5;
+    private com.google.protobuf.Timestamp upperWindowBoundary_;
+    /**
+     * <code>.google.protobuf.Timestamp upperWindowBoundary = 5;</code>
+     */
+    public boolean hasUpperWindowBoundary() {
+      return upperWindowBoundary_ != null;
+    }
+    /**
+     * <code>.google.protobuf.Timestamp upperWindowBoundary = 5;</code>
+     */
+    public com.google.protobuf.Timestamp getUpperWindowBoundary() {
+      return upperWindowBoundary_ == null ? com.google.protobuf.Timestamp.getDefaultInstance() : upperWindowBoundary_;
+    }
+    /**
+     * <code>.google.protobuf.Timestamp upperWindowBoundary = 5;</code>
+     */
+    public com.google.protobuf.TimestampOrBuilder getUpperWindowBoundaryOrBuilder() {
+      return getUpperWindowBoundary();
+    }
+
+    private byte memoizedIsInitialized = -1;
+    public final boolean isInitialized() {
+      byte isInitialized = memoizedIsInitialized;
+      if (isInitialized == 1) return true;
+      if (isInitialized == 0) return false;
+
+      memoizedIsInitialized = 1;
+      return true;
+    }
+
+    public void writeTo(com.google.protobuf.CodedOutputStream output)
+                        throws java.io.IOException {
+      if (key_ != null) {
+        output.writeMessage(1, getKey());
+      }
+      if (pipelineVersion_ != null) {
+        output.writeMessage(2, getPipelineVersion());
+      }
+      for (int i = 0; i < accums_.size(); i++) {
+        output.writeMessage(3, accums_.get(i));
+      }
+      if (lowerWindowBoundary_ != null) {
+        output.writeMessage(4, getLowerWindowBoundary());
+      }
+      if (upperWindowBoundary_ != null) {
+        output.writeMessage(5, getUpperWindowBoundary());
+      }
+    }
+
+    public int getSerializedSize() {
+      int size = memoizedSize;
+      if (size != -1) return size;
+
+      size = 0;
+      if (key_ != null) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeMessageSize(1, getKey());
+      }
+      if (pipelineVersion_ != null) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeMessageSize(2, getPipelineVersion());
+      }
+      for (int i = 0; i < accums_.size(); i++) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeMessageSize(3, accums_.get(i));
+      }
+      if (lowerWindowBoundary_ != null) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeMessageSize(4, getLowerWindowBoundary());
+      }
+      if (upperWindowBoundary_ != null) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeMessageSize(5, getUpperWindowBoundary());
+      }
+      memoizedSize = size;
+      return size;
+    }
+
+    private static final long serialVersionUID = 0L;
+    @java.lang.Override
+    public boolean equals(final java.lang.Object obj) {
+      if (obj == this) {
+       return true;
+      }
+      if (!(obj instanceof org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSAccumSequence)) {
+        return super.equals(obj);
+      }
+      org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSAccumSequence other = (org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSAccumSequence) obj;
+
+      boolean result = true;
+      result = result && (hasKey() == other.hasKey());
+      if (hasKey()) {
+        result = result && getKey()
+            .equals(other.getKey());
+      }
+      result = result && (hasPipelineVersion() == other.hasPipelineVersion());
+      if (hasPipelineVersion()) {
+        result = result && getPipelineVersion()
+            .equals(other.getPipelineVersion());
+      }
+      result = result && getAccumsList()
+          .equals(other.getAccumsList());
+      result = result && (hasLowerWindowBoundary() == other.hasLowerWindowBoundary());
+      if (hasLowerWindowBoundary()) {
+        result = result && getLowerWindowBoundary()
+            .equals(other.getLowerWindowBoundary());
+      }
+      result = result && (hasUpperWindowBoundary() == other.hasUpperWindowBoundary());
+      if (hasUpperWindowBoundary()) {
+        result = result && getUpperWindowBoundary()
+            .equals(other.getUpperWindowBoundary());
+      }
+      return result;
+    }
+
+    @java.lang.Override
+    public int hashCode() {
+      if (memoizedHashCode != 0) {
+        return memoizedHashCode;
+      }
+      int hash = 41;
+      hash = (19 * hash) + getDescriptor().hashCode();
+      if (hasKey()) {
+        hash = (37 * hash) + KEY_FIELD_NUMBER;
+        hash = (53 * hash) + getKey().hashCode();
+      }
+      if (hasPipelineVersion()) {
+        hash = (37 * hash) + PIPELINEVERSION_FIELD_NUMBER;
+        hash = (53 * hash) + getPipelineVersion().hashCode();
+      }
+      if (getAccumsCount() > 0) {
+        hash = (37 * hash) + ACCUMS_FIELD_NUMBER;
+        hash = (53 * hash) + getAccumsList().hashCode();
+      }
+      if (hasLowerWindowBoundary()) {
+        hash = (37 * hash) + LOWERWINDOWBOUNDARY_FIELD_NUMBER;
+        hash = (53 * hash) + getLowerWindowBoundary().hashCode();
+      }
+      if (hasUpperWindowBoundary()) {
+        hash = (37 * hash) + UPPERWINDOWBOUNDARY_FIELD_NUMBER;
+        hash = (53 * hash) + getUpperWindowBoundary().hashCode();
+      }
+      hash = (29 * hash) + unknownFields.hashCode();
+      memoizedHashCode = hash;
+      return hash;
+    }
+
+    public static org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSAccumSequence parseFrom(
+        com.google.protobuf.ByteString data)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data);
+    }
+    public static org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSAccumSequence parseFrom(
+        com.google.protobuf.ByteString data,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data, extensionRegistry);
+    }
+    public static org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSAccumSequence parseFrom(byte[] data)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data);
+    }
+    public static org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSAccumSequence parseFrom(
+        byte[] data,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data, extensionRegistry);
+    }
+    public static org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSAccumSequence parseFrom(java.io.InputStream input)
+        throws java.io.IOException {
+      return com.google.protobuf.GeneratedMessageV3
+          .parseWithIOException(PARSER, input);
+    }
+    public static org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSAccumSequence parseFrom(
+        java.io.InputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws java.io.IOException {
+      return com.google.protobuf.GeneratedMessageV3
+          .parseWithIOException(PARSER, input, extensionRegistry);
+    }
+    public static org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSAccumSequence parseDelimitedFrom(java.io.InputStream input)
+        throws java.io.IOException {
+      return com.google.protobuf.GeneratedMessageV3
+          .parseDelimitedWithIOException(PARSER, input);
+    }
+    public static org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSAccumSequence parseDelimitedFrom(
+        java.io.InputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws java.io.IOException {
+      return com.google.protobuf.GeneratedMessageV3
+          .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
+    }
+    public static org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSAccumSequence parseFrom(
+        com.google.protobuf.CodedInputStream input)
+        throws java.io.IOException {
+      return com.google.protobuf.GeneratedMessageV3
+          .parseWithIOException(PARSER, input);
+    }
+    public static org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSAccumSequence parseFrom(
+        com.google.protobuf.CodedInputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws java.io.IOException {
+      return com.google.protobuf.GeneratedMessageV3
+          .parseWithIOException(PARSER, input, extensionRegistry);
+    }
+
+    public Builder newBuilderForType() { return newBuilder(); }
+    public static Builder newBuilder() {
+      return DEFAULT_INSTANCE.toBuilder();
+    }
+    public static Builder newBuilder(org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSAccumSequence prototype) {
+      return DEFAULT_INSTANCE.toBuilder().mergeFrom(prototype);
+    }
+    public Builder toBuilder() {
+      return this == DEFAULT_INSTANCE
+          ? new Builder() : new Builder().mergeFrom(this);
+    }
+
+    @java.lang.Override
+    protected Builder newBuilderForType(
+        com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+      Builder builder = new Builder(parent);
+      return builder;
+    }
+    /**
+     * <pre>
+     * Time Series sequence holder
+     * </pre>
+     *
+     * Protobuf type {@code protos.TSAccumSequence}
+     */
+    public static final class Builder extends
+        com.google.protobuf.GeneratedMessageV3.Builder<Builder> implements
+        // @@protoc_insertion_point(builder_implements:protos.TSAccumSequence)
+        org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSAccumSequenceOrBuilder {
+      public static final com.google.protobuf.Descriptors.Descriptor
+          getDescriptor() {
+        return org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.internal_static_protos_TSAccumSequence_descriptor;
+      }
+
+      protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+          internalGetFieldAccessorTable() {
+        return org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.internal_static_protos_TSAccumSequence_fieldAccessorTable
+            .ensureFieldAccessorsInitialized(
+                org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSAccumSequence.class, org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSAccumSequence.Builder.class);
+      }
+
+      // Construct using org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSAccumSequence.newBuilder()
+      private Builder() {
+        maybeForceBuilderInitialization();
+      }
+
+      private Builder(
+          com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+        super(parent);
+        maybeForceBuilderInitialization();
+      }
+      private void maybeForceBuilderInitialization() {
+        if (com.google.protobuf.GeneratedMessageV3
+                .alwaysUseFieldBuilders) {
+          getAccumsFieldBuilder();
+        }
+      }
+      public Builder clear() {
+        super.clear();
+        if (keyBuilder_ == null) {
+          key_ = null;
+        } else {
+          key_ = null;
+          keyBuilder_ = null;
+        }
+        if (pipelineVersionBuilder_ == null) {
+          pipelineVersion_ = null;
+        } else {
+          pipelineVersion_ = null;
+          pipelineVersionBuilder_ = null;
+        }
+        if (accumsBuilder_ == null) {
+          accums_ = java.util.Collections.emptyList();
+          bitField0_ = (bitField0_ & ~0x00000004);
+        } else {
+          accumsBuilder_.clear();
+        }
+        if (lowerWindowBoundaryBuilder_ == null) {
+          lowerWindowBoundary_ = null;
+        } else {
+          lowerWindowBoundary_ = null;
+          lowerWindowBoundaryBuilder_ = null;
+        }
+        if (upperWindowBoundaryBuilder_ == null) {
+          upperWindowBoundary_ = null;
+        } else {
+          upperWindowBoundary_ = null;
+          upperWindowBoundaryBuilder_ = null;
+        }
+        return this;
+      }
+
+      public com.google.protobuf.Descriptors.Descriptor
+          getDescriptorForType() {
+        return org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.internal_static_protos_TSAccumSequence_descriptor;
+      }
+
+      public org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSAccumSequence getDefaultInstanceForType() {
+        return org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSAccumSequence.getDefaultInstance();
+      }
+
+      public org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSAccumSequence build() {
+        org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSAccumSequence result = buildPartial();
+        if (!result.isInitialized()) {
+          throw newUninitializedMessageException(result);
+        }
+        return result;
+      }
+
+      public org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSAccumSequence buildPartial() {
+        org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSAccumSequence result = new org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSAccumSequence(this);
+        int from_bitField0_ = bitField0_;
+        int to_bitField0_ = 0;
+        if (keyBuilder_ == null) {
+          result.key_ = key_;
+        } else {
+          result.key_ = keyBuilder_.build();
+        }
+        if (pipelineVersionBuilder_ == null) {
+          result.pipelineVersion_ = pipelineVersion_;
+        } else {
+          result.pipelineVersion_ = pipelineVersionBuilder_.build();
+        }
+        if (accumsBuilder_ == null) {
+          if (((bitField0_ & 0x00000004) == 0x00000004)) {
+            accums_ = java.util.Collections.unmodifiableList(accums_);
+            bitField0_ = (bitField0_ & ~0x00000004);
+          }
+          result.accums_ = accums_;
+        } else {
+          result.accums_ = accumsBuilder_.build();
+        }
+        if (lowerWindowBoundaryBuilder_ == null) {
+          result.lowerWindowBoundary_ = lowerWindowBoundary_;
+        } else {
+          result.lowerWindowBoundary_ = lowerWindowBoundaryBuilder_.build();
+        }
+        if (upperWindowBoundaryBuilder_ == null) {
+          result.upperWindowBoundary_ = upperWindowBoundary_;
+        } else {
+          result.upperWindowBoundary_ = upperWindowBoundaryBuilder_.build();
+        }
+        result.bitField0_ = to_bitField0_;
+        onBuilt();
+        return result;
+      }
+
+      public Builder clone() {
+        return (Builder) super.clone();
+      }
+      public Builder setField(
+          com.google.protobuf.Descriptors.FieldDescriptor field,
+          Object value) {
+        return (Builder) super.setField(field, value);
+      }
+      public Builder clearField(
+          com.google.protobuf.Descriptors.FieldDescriptor field) {
+        return (Builder) super.clearField(field);
+      }
+      public Builder clearOneof(
+          com.google.protobuf.Descriptors.OneofDescriptor oneof) {
+        return (Builder) super.clearOneof(oneof);
+      }
+      public Builder setRepeatedField(
+          com.google.protobuf.Descriptors.FieldDescriptor field,
+          int index, Object value) {
+        return (Builder) super.setRepeatedField(field, index, value);
+      }
+      public Builder addRepeatedField(
+          com.google.protobuf.Descriptors.FieldDescriptor field,
+          Object value) {
+        return (Builder) super.addRepeatedField(field, value);
+      }
+      public Builder mergeFrom(com.google.protobuf.Message other) {
+        if (other instanceof org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSAccumSequence) {
+          return mergeFrom((org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSAccumSequence)other);
+        } else {
+          super.mergeFrom(other);
+          return this;
+        }
+      }
+
+      public Builder mergeFrom(org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSAccumSequence other) {
+        if (other == org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSAccumSequence.getDefaultInstance()) return this;
+        if (other.hasKey()) {
+          mergeKey(other.getKey());
+        }
+        if (other.hasPipelineVersion()) {
+          mergePipelineVersion(other.getPipelineVersion());
+        }
+        if (accumsBuilder_ == null) {
+          if (!other.accums_.isEmpty()) {
+            if (accums_.isEmpty()) {
+              accums_ = other.accums_;
+              bitField0_ = (bitField0_ & ~0x00000004);
+            } else {
+              ensureAccumsIsMutable();
+              accums_.addAll(other.accums_);
+            }
+            onChanged();
+          }
+        } else {
+          if (!other.accums_.isEmpty()) {
+            if (accumsBuilder_.isEmpty()) {
+              accumsBuilder_.dispose();
+              accumsBuilder_ = null;
+              accums_ = other.accums_;
+              bitField0_ = (bitField0_ & ~0x00000004);
+              accumsBuilder_ = 
+                com.google.protobuf.GeneratedMessageV3.alwaysUseFieldBuilders ?
+                   getAccumsFieldBuilder() : null;
+            } else {
+              accumsBuilder_.addAllMessages(other.accums_);
+            }
+          }
+        }
+        if (other.hasLowerWindowBoundary()) {
+          mergeLowerWindowBoundary(other.getLowerWindowBoundary());
+        }
+        if (other.hasUpperWindowBoundary()) {
+          mergeUpperWindowBoundary(other.getUpperWindowBoundary());
+        }
+        onChanged();
+        return this;
+      }
+
+      public final boolean isInitialized() {
+        return true;
+      }
+
+      public Builder mergeFrom(
+          com.google.protobuf.CodedInputStream input,
+          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws java.io.IOException {
+        org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSAccumSequence parsedMessage = null;
+        try {
+          parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
+        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+          parsedMessage = (org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSAccumSequence) e.getUnfinishedMessage();
+          throw e.unwrapIOException();
+        } finally {
+          if (parsedMessage != null) {
+            mergeFrom(parsedMessage);
+          }
+        }
+        return this;
+      }
+      private int bitField0_;
+
+      private org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSKey key_ = null;
+      private com.google.protobuf.SingleFieldBuilderV3<
+          org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSKey, org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSKey.Builder, org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSKeyOrBuilder> keyBuilder_;
+      /**
+       * <code>.protos.TSKey key = 1;</code>
+       */
+      public boolean hasKey() {
+        return keyBuilder_ != null || key_ != null;
+      }
+      /**
+       * <code>.protos.TSKey key = 1;</code>
+       */
+      public org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSKey getKey() {
+        if (keyBuilder_ == null) {
+          return key_ == null ? org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSKey.getDefaultInstance() : key_;
+        } else {
+          return keyBuilder_.getMessage();
+        }
+      }
+      /**
+       * <code>.protos.TSKey key = 1;</code>
+       */
+      public Builder setKey(org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSKey value) {
+        if (keyBuilder_ == null) {
+          if (value == null) {
+            throw new NullPointerException();
+          }
+          key_ = value;
+          onChanged();
+        } else {
+          keyBuilder_.setMessage(value);
+        }
+
+        return this;
+      }
+      /**
+       * <code>.protos.TSKey key = 1;</code>
+       */
+      public Builder setKey(
+          org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSKey.Builder builderForValue) {
+        if (keyBuilder_ == null) {
+          key_ = builderForValue.build();
+          onChanged();
+        } else {
+          keyBuilder_.setMessage(builderForValue.build());
+        }
+
+        return this;
+      }
+      /**
+       * <code>.protos.TSKey key = 1;</code>
+       */
+      public Builder mergeKey(org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSKey value) {
+        if (keyBuilder_ == null) {
+          if (key_ != null) {
+            key_ =
+              org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSKey.newBuilder(key_).mergeFrom(value).buildPartial();
+          } else {
+            key_ = value;
+          }
+          onChanged();
+        } else {
+          keyBuilder_.mergeFrom(value);
+        }
+
+        return this;
+      }
+      /**
+       * <code>.protos.TSKey key = 1;</code>
+       */
+      public Builder clearKey() {
+        if (keyBuilder_ == null) {
+          key_ = null;
+          onChanged();
+        } else {
+          key_ = null;
+          keyBuilder_ = null;
+        }
+
+        return this;
+      }
+      /**
+       * <code>.protos.TSKey key = 1;</code>
+       */
+      public org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSKey.Builder getKeyBuilder() {
+        
+        onChanged();
+        return getKeyFieldBuilder().getBuilder();
+      }
+      /**
+       * <code>.protos.TSKey key = 1;</code>
+       */
+      public org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSKeyOrBuilder getKeyOrBuilder() {
+        if (keyBuilder_ != null) {
+          return keyBuilder_.getMessageOrBuilder();
+        } else {
+          return key_ == null ?
+              org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSKey.getDefaultInstance() : key_;
+        }
+      }
+      /**
+       * <code>.protos.TSKey key = 1;</code>
+       */
+      private com.google.protobuf.SingleFieldBuilderV3<
+          org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSKey, org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSKey.Builder, org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSKeyOrBuilder> 
+          getKeyFieldBuilder() {
+        if (keyBuilder_ == null) {
+          keyBuilder_ = new com.google.protobuf.SingleFieldBuilderV3<
+              org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSKey, org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSKey.Builder, org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSKeyOrBuilder>(
+                  getKey(),
+                  getParentForChildren(),
+                  isClean());
+          key_ = null;
+        }
+        return keyBuilder_;
+      }
+
+      private org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.PipelineVersion pipelineVersion_ = null;
+      private com.google.protobuf.SingleFieldBuilderV3<
+          org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.PipelineVersion, org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.PipelineVersion.Builder, org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.PipelineVersionOrBuilder> pipelineVersionBuilder_;
+      /**
+       * <code>.protos.PipelineVersion pipelineVersion = 2;</code>
+       */
+      public boolean hasPipelineVersion() {
+        return pipelineVersionBuilder_ != null || pipelineVersion_ != null;
+      }
+      /**
+       * <code>.protos.PipelineVersion pipelineVersion = 2;</code>
+       */
+      public org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.PipelineVersion getPipelineVersion() {
+        if (pipelineVersionBuilder_ == null) {
+          return pipelineVersion_ == null ? org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.PipelineVersion.getDefaultInstance() : pipelineVersion_;
+        } else {
+          return pipelineVersionBuilder_.getMessage();
+        }
+      }
+      /**
+       * <code>.protos.PipelineVersion pipelineVersion = 2;</code>
+       */
+      public Builder setPipelineVersion(org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.PipelineVersion value) {
+        if (pipelineVersionBuilder_ == null) {
+          if (value == null) {
+            throw new NullPointerException();
+          }
+          pipelineVersion_ = value;
+          onChanged();
+        } else {
+          pipelineVersionBuilder_.setMessage(value);
+        }
+
+        return this;
+      }
+      /**
+       * <code>.protos.PipelineVersion pipelineVersion = 2;</code>
+       */
+      public Builder setPipelineVersion(
+          org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.PipelineVersion.Builder builderForValue) {
+        if (pipelineVersionBuilder_ == null) {
+          pipelineVersion_ = builderForValue.build();
+          onChanged();
+        } else {
+          pipelineVersionBuilder_.setMessage(builderForValue.build());
+        }
+
+        return this;
+      }
+      /**
+       * <code>.protos.PipelineVersion pipelineVersion = 2;</code>
+       */
+      public Builder mergePipelineVersion(org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.PipelineVersion value) {
+        if (pipelineVersionBuilder_ == null) {
+          if (pipelineVersion_ != null) {
+            pipelineVersion_ =
+              org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.PipelineVersion.newBuilder(pipelineVersion_).mergeFrom(value).buildPartial();
+          } else {
+            pipelineVersion_ = value;
+          }
+          onChanged();
+        } else {
+          pipelineVersionBuilder_.mergeFrom(value);
+        }
+
+        return this;
+      }
+      /**
+       * <code>.protos.PipelineVersion pipelineVersion = 2;</code>
+       */
+      public Builder clearPipelineVersion() {
+        if (pipelineVersionBuilder_ == null) {
+          pipelineVersion_ = null;
+          onChanged();
+        } else {
+          pipelineVersion_ = null;
+          pipelineVersionBuilder_ = null;
+        }
+
+        return this;
+      }
+      /**
+       * <code>.protos.PipelineVersion pipelineVersion = 2;</code>
+       */
+      public org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.PipelineVersion.Builder getPipelineVersionBuilder() {
+        
+        onChanged();
+        return getPipelineVersionFieldBuilder().getBuilder();
+      }
+      /**
+       * <code>.protos.PipelineVersion pipelineVersion = 2;</code>
+       */
+      public org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.PipelineVersionOrBuilder getPipelineVersionOrBuilder() {
+        if (pipelineVersionBuilder_ != null) {
+          return pipelineVersionBuilder_.getMessageOrBuilder();
+        } else {
+          return pipelineVersion_ == null ?
+              org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.PipelineVersion.getDefaultInstance() : pipelineVersion_;
+        }
+      }
+      /**
+       * <code>.protos.PipelineVersion pipelineVersion = 2;</code>
+       */
+      private com.google.protobuf.SingleFieldBuilderV3<
+          org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.PipelineVersion, org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.PipelineVersion.Builder, org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.PipelineVersionOrBuilder> 
+          getPipelineVersionFieldBuilder() {
+        if (pipelineVersionBuilder_ == null) {
+          pipelineVersionBuilder_ = new com.google.protobuf.SingleFieldBuilderV3<
+              org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.PipelineVersion, org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.PipelineVersion.Builder, org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.PipelineVersionOrBuilder>(
+                  getPipelineVersion(),
+                  getParentForChildren(),
+                  isClean());
+          pipelineVersion_ = null;
+        }
+        return pipelineVersionBuilder_;
+      }
+
+      private java.util.List<org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSAccum> accums_ =
+        java.util.Collections.emptyList();
+      private void ensureAccumsIsMutable() {
+        if (!((bitField0_ & 0x00000004) == 0x00000004)) {
+          accums_ = new java.util.ArrayList<org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSAccum>(accums_);
+          bitField0_ |= 0x00000004;
+         }
+      }
+
+      private com.google.protobuf.RepeatedFieldBuilderV3<
+          org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSAccum, org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSAccum.Builder, org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSAccumOrBuilder> accumsBuilder_;
+
+      /**
+       * <code>repeated .protos.TSAccum accums = 3;</code>
+       */
+      public java.util.List<org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSAccum> getAccumsList() {
+        if (accumsBuilder_ == null) {
+          return java.util.Collections.unmodifiableList(accums_);
+        } else {
+          return accumsBuilder_.getMessageList();
+        }
+      }
+      /**
+       * <code>repeated .protos.TSAccum accums = 3;</code>
+       */
+      public int getAccumsCount() {
+        if (accumsBuilder_ == null) {
+          return accums_.size();
+        } else {
+          return accumsBuilder_.getCount();
+        }
+      }
+      /**
+       * <code>repeated .protos.TSAccum accums = 3;</code>
+       */
+      public org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSAccum getAccums(int index) {
+        if (accumsBuilder_ == null) {
+          return accums_.get(index);
+        } else {
+          return accumsBuilder_.getMessage(index);
+        }
+      }
+      /**
+       * <code>repeated .protos.TSAccum accums = 3;</code>
+       */
+      public Builder setAccums(
+          int index, org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSAccum value) {
+        if (accumsBuilder_ == null) {
+          if (value == null) {
+            throw new NullPointerException();
+          }
+          ensureAccumsIsMutable();
+          accums_.set(index, value);
+          onChanged();
+        } else {
+          accumsBuilder_.setMessage(index, value);
+        }
+        return this;
+      }
+      /**
+       * <code>repeated .protos.TSAccum accums = 3;</code>
+       */
+      public Builder setAccums(
+          int index, org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSAccum.Builder builderForValue) {
+        if (accumsBuilder_ == null) {
+          ensureAccumsIsMutable();
+          accums_.set(index, builderForValue.build());
+          onChanged();
+        } else {
+          accumsBuilder_.setMessage(index, builderForValue.build());
+        }
+        return this;
+      }
+      /**
+       * <code>repeated .protos.TSAccum accums = 3;</code>
+       */
+      public Builder addAccums(org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSAccum value) {
+        if (accumsBuilder_ == null) {
+          if (value == null) {
+            throw new NullPointerException();
+          }
+          ensureAccumsIsMutable();
+          accums_.add(value);
+          onChanged();
+        } else {
+          accumsBuilder_.addMessage(value);
+        }
+        return this;
+      }
+      /**
+       * <code>repeated .protos.TSAccum accums = 3;</code>
+       */
+      public Builder addAccums(
+          int index, org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSAccum value) {
+        if (accumsBuilder_ == null) {
+          if (value == null) {
+            throw new NullPointerException();
+          }
+          ensureAccumsIsMutable();
+          accums_.add(index, value);
+          onChanged();
+        } else {
+          accumsBuilder_.addMessage(index, value);
+        }
+        return this;
+      }
+      /**
+       * <code>repeated .protos.TSAccum accums = 3;</code>
+       */
+      public Builder addAccums(
+          org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSAccum.Builder builderForValue) {
+        if (accumsBuilder_ == null) {
+          ensureAccumsIsMutable();
+          accums_.add(builderForValue.build());
+          onChanged();
+        } else {
+          accumsBuilder_.addMessage(builderForValue.build());
+        }
+        return this;
+      }
+      /**
+       * <code>repeated .protos.TSAccum accums = 3;</code>
+       */
+      public Builder addAccums(
+          int index, org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSAccum.Builder builderForValue) {
+        if (accumsBuilder_ == null) {
+          ensureAccumsIsMutable();
+          accums_.add(index, builderForValue.build());
+          onChanged();
+        } else {
+          accumsBuilder_.addMessage(index, builderForValue.build());
+        }
+        return this;
+      }
+      /**
+       * <code>repeated .protos.TSAccum accums = 3;</code>
+       */
+      public Builder addAllAccums(
+          java.lang.Iterable<? extends org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSAccum> values) {
+        if (accumsBuilder_ == null) {
+          ensureAccumsIsMutable();
+          com.google.protobuf.AbstractMessageLite.Builder.addAll(
+              values, accums_);
+          onChanged();
+        } else {
+          accumsBuilder_.addAllMessages(values);
+        }
+        return this;
+      }
+      /**
+       * <code>repeated .protos.TSAccum accums = 3;</code>
+       */
+      public Builder clearAccums() {
+        if (accumsBuilder_ == null) {
+          accums_ = java.util.Collections.emptyList();
+          bitField0_ = (bitField0_ & ~0x00000004);
+          onChanged();
+        } else {
+          accumsBuilder_.clear();
+        }
+        return this;
+      }
+      /**
+       * <code>repeated .protos.TSAccum accums = 3;</code>
+       */
+      public Builder removeAccums(int index) {
+        if (accumsBuilder_ == null) {
+          ensureAccumsIsMutable();
+          accums_.remove(index);
+          onChanged();
+        } else {
+          accumsBuilder_.remove(index);
+        }
+        return this;
+      }
+      /**
+       * <code>repeated .protos.TSAccum accums = 3;</code>
+       */
+      public org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSAccum.Builder getAccumsBuilder(
+          int index) {
+        return getAccumsFieldBuilder().getBuilder(index);
+      }
+      /**
+       * <code>repeated .protos.TSAccum accums = 3;</code>
+       */
+      public org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSAccumOrBuilder getAccumsOrBuilder(
+          int index) {
+        if (accumsBuilder_ == null) {
+          return accums_.get(index);  } else {
+          return accumsBuilder_.getMessageOrBuilder(index);
+        }
+      }
+      /**
+       * <code>repeated .protos.TSAccum accums = 3;</code>
+       */
+      public java.util.List<? extends org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSAccumOrBuilder> 
+           getAccumsOrBuilderList() {
+        if (accumsBuilder_ != null) {
+          return accumsBuilder_.getMessageOrBuilderList();
+        } else {
+          return java.util.Collections.unmodifiableList(accums_);
+        }
+      }
+      /**
+       * <code>repeated .protos.TSAccum accums = 3;</code>
+       */
+      public org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSAccum.Builder addAccumsBuilder() {
+        return getAccumsFieldBuilder().addBuilder(
+            org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSAccum.getDefaultInstance());
+      }
+      /**
+       * <code>repeated .protos.TSAccum accums = 3;</code>
+       */
+      public org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSAccum.Builder addAccumsBuilder(
+          int index) {
+        return getAccumsFieldBuilder().addBuilder(
+            index, org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSAccum.getDefaultInstance());
+      }
+      /**
+       * <code>repeated .protos.TSAccum accums = 3;</code>
+       */
+      public java.util.List<org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSAccum.Builder> 
+           getAccumsBuilderList() {
+        return getAccumsFieldBuilder().getBuilderList();
+      }
+      private com.google.protobuf.RepeatedFieldBuilderV3<
+          org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSAccum, org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSAccum.Builder, org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSAccumOrBuilder> 
+          getAccumsFieldBuilder() {
+        if (accumsBuilder_ == null) {
+          accumsBuilder_ = new com.google.protobuf.RepeatedFieldBuilderV3<
+              org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSAccum, org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSAccum.Builder, org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSAccumOrBuilder>(
+                  accums_,
+                  ((bitField0_ & 0x00000004) == 0x00000004),
+                  getParentForChildren(),
+                  isClean());
+          accums_ = null;
+        }
+        return accumsBuilder_;
+      }
+
+      private com.google.protobuf.Timestamp lowerWindowBoundary_ = null;
+      private com.google.protobuf.SingleFieldBuilderV3<
+          com.google.protobuf.Timestamp, com.google.protobuf.Timestamp.Builder, com.google.protobuf.TimestampOrBuilder> lowerWindowBoundaryBuilder_;
+      /**
+       * <code>.google.protobuf.Timestamp lowerWindowBoundary = 4;</code>
+       */
+      public boolean hasLowerWindowBoundary() {
+        return lowerWindowBoundaryBuilder_ != null || lowerWindowBoundary_ != null;
+      }
+      /**
+       * <code>.google.protobuf.Timestamp lowerWindowBoundary = 4;</code>
+       */
+      public com.google.protobuf.Timestamp getLowerWindowBoundary() {
+        if (lowerWindowBoundaryBuilder_ == null) {
+          return lowerWindowBoundary_ == null ? com.google.protobuf.Timestamp.getDefaultInstance() : lowerWindowBoundary_;
+        } else {
+          return lowerWindowBoundaryBuilder_.getMessage();
+        }
+      }
+      /**
+       * <code>.google.protobuf.Timestamp lowerWindowBoundary = 4;</code>
+       */
+      public Builder setLowerWindowBoundary(com.google.protobuf.Timestamp value) {
+        if (lowerWindowBoundaryBuilder_ == null) {
+          if (value == null) {
+            throw new NullPointerException();
+          }
+          lowerWindowBoundary_ = value;
+          onChanged();
+        } else {
+          lowerWindowBoundaryBuilder_.setMessage(value);
+        }
+
+        return this;
+      }
+      /**
+       * <code>.google.protobuf.Timestamp lowerWindowBoundary = 4;</code>
+       */
+      public Builder setLowerWindowBoundary(
+          com.google.protobuf.Timestamp.Builder builderForValue) {
+        if (lowerWindowBoundaryBuilder_ == null) {
+          lowerWindowBoundary_ = builderForValue.build();
+          onChanged();
+        } else {
+          lowerWindowBoundaryBuilder_.setMessage(builderForValue.build());
+        }
+
+        return this;
+      }
+      /**
+       * <code>.google.protobuf.Timestamp lowerWindowBoundary = 4;</code>
+       */
+      public Builder mergeLowerWindowBoundary(com.google.protobuf.Timestamp value) {
+        if (lowerWindowBoundaryBuilder_ == null) {
+          if (lowerWindowBoundary_ != null) {
+            lowerWindowBoundary_ =
+              com.google.protobuf.Timestamp.newBuilder(lowerWindowBoundary_).mergeFrom(value).buildPartial();
+          } else {
+            lowerWindowBoundary_ = value;
+          }
+          onChanged();
+        } else {
+          lowerWindowBoundaryBuilder_.mergeFrom(value);
+        }
+
+        return this;
+      }
+      /**
+       * <code>.google.protobuf.Timestamp lowerWindowBoundary = 4;</code>
+       */
+      public Builder clearLowerWindowBoundary() {
+        if (lowerWindowBoundaryBuilder_ == null) {
+          lowerWindowBoundary_ = null;
+          onChanged();
+        } else {
+          lowerWindowBoundary_ = null;
+          lowerWindowBoundaryBuilder_ = null;
+        }
+
+        return this;
+      }
+      /**
+       * <code>.google.protobuf.Timestamp lowerWindowBoundary = 4;</code>
+       */
+      public com.google.protobuf.Timestamp.Builder getLowerWindowBoundaryBuilder() {
+        
+        onChanged();
+        return getLowerWindowBoundaryFieldBuilder().getBuilder();
+      }
+      /**
+       * <code>.google.protobuf.Timestamp lowerWindowBoundary = 4;</code>
+       */
+      public com.google.protobuf.TimestampOrBuilder getLowerWindowBoundaryOrBuilder() {
+        if (lowerWindowBoundaryBuilder_ != null) {
+          return lowerWindowBoundaryBuilder_.getMessageOrBuilder();
+        } else {
+          return lowerWindowBoundary_ == null ?
+              com.google.protobuf.Timestamp.getDefaultInstance() : lowerWindowBoundary_;
+        }
+      }
+      /**
+       * <code>.google.protobuf.Timestamp lowerWindowBoundary = 4;</code>
+       */
+      private com.google.protobuf.SingleFieldBuilderV3<
+          com.google.protobuf.Timestamp, com.google.protobuf.Timestamp.Builder, com.google.protobuf.TimestampOrBuilder> 
+          getLowerWindowBoundaryFieldBuilder() {
+        if (lowerWindowBoundaryBuilder_ == null) {
+          lowerWindowBoundaryBuilder_ = new com.google.protobuf.SingleFieldBuilderV3<
+              com.google.protobuf.Timestamp, com.google.protobuf.Timestamp.Builder, com.google.protobuf.TimestampOrBuilder>(
+                  getLowerWindowBoundary(),
+                  getParentForChildren(),
+                  isClean());
+          lowerWindowBoundary_ = null;
+        }
+        return lowerWindowBoundaryBuilder_;
+      }
+
+      private com.google.protobuf.Timestamp upperWindowBoundary_ = null;
+      private com.google.protobuf.SingleFieldBuilderV3<
+          com.google.protobuf.Timestamp, com.google.protobuf.Timestamp.Builder, com.google.protobuf.TimestampOrBuilder> upperWindowBoundaryBuilder_;
+      /**
+       * <code>.google.protobuf.Timestamp upperWindowBoundary = 5;</code>
+       */
+      public boolean hasUpperWindowBoundary() {
+        return upperWindowBoundaryBuilder_ != null || upperWindowBoundary_ != null;
+      }
+      /**
+       * <code>.google.protobuf.Timestamp upperWindowBoundary = 5;</code>
+       */
+      public com.google.protobuf.Timestamp getUpperWindowBoundary() {
+        if (upperWindowBoundaryBuilder_ == null) {
+          return upperWindowBoundary_ == null ? com.google.protobuf.Timestamp.getDefaultInstance() : upperWindowBoundary_;
+        } else {
+          return upperWindowBoundaryBuilder_.getMessage();
+        }
+      }
+      /**
+       * <code>.google.protobuf.Timestamp upperWindowBoundary = 5;</code>
+       */
+      public Builder setUpperWindowBoundary(com.google.protobuf.Timestamp value) {
+        if (upperWindowBoundaryBuilder_ == null) {
+          if (value == null) {
+            throw new NullPointerException();
+          }
+          upperWindowBoundary_ = value;
+          onChanged();
+        } else {
+          upperWindowBoundaryBuilder_.setMessage(value);
+        }
+
+        return this;
+      }
+      /**
+       * <code>.google.protobuf.Timestamp upperWindowBoundary = 5;</code>
+       */
+      public Builder setUpperWindowBoundary(
+          com.google.protobuf.Timestamp.Builder builderForValue) {
+        if (upperWindowBoundaryBuilder_ == null) {
+          upperWindowBoundary_ = builderForValue.build();
+          onChanged();
+        } else {
+          upperWindowBoundaryBuilder_.setMessage(builderForValue.build());
+        }
+
+        return this;
+      }
+      /**
+       * <code>.google.protobuf.Timestamp upperWindowBoundary = 5;</code>
+       */
+      public Builder mergeUpperWindowBoundary(com.google.protobuf.Timestamp value) {
+        if (upperWindowBoundaryBuilder_ == null) {
+          if (upperWindowBoundary_ != null) {
+            upperWindowBoundary_ =
+              com.google.protobuf.Timestamp.newBuilder(upperWindowBoundary_).mergeFrom(value).buildPartial();
+          } else {
+            upperWindowBoundary_ = value;
+          }
+          onChanged();
+        } else {
+          upperWindowBoundaryBuilder_.mergeFrom(value);
+        }
+
+        return this;
+      }
+      /**
+       * <code>.google.protobuf.Timestamp upperWindowBoundary = 5;</code>
+       */
+      public Builder clearUpperWindowBoundary() {
+        if (upperWindowBoundaryBuilder_ == null) {
+          upperWindowBoundary_ = null;
+          onChanged();
+        } else {
+          upperWindowBoundary_ = null;
+          upperWindowBoundaryBuilder_ = null;
+        }
+
+        return this;
+      }
+      /**
+       * <code>.google.protobuf.Timestamp upperWindowBoundary = 5;</code>
+       */
+      public com.google.protobuf.Timestamp.Builder getUpperWindowBoundaryBuilder() {
+        
+        onChanged();
+        return getUpperWindowBoundaryFieldBuilder().getBuilder();
+      }
+      /**
+       * <code>.google.protobuf.Timestamp upperWindowBoundary = 5;</code>
+       */
+      public com.google.protobuf.TimestampOrBuilder getUpperWindowBoundaryOrBuilder() {
+        if (upperWindowBoundaryBuilder_ != null) {
+          return upperWindowBoundaryBuilder_.getMessageOrBuilder();
+        } else {
+          return upperWindowBoundary_ == null ?
+              com.google.protobuf.Timestamp.getDefaultInstance() : upperWindowBoundary_;
+        }
+      }
+      /**
+       * <code>.google.protobuf.Timestamp upperWindowBoundary = 5;</code>
+       */
+      private com.google.protobuf.SingleFieldBuilderV3<
+          com.google.protobuf.Timestamp, com.google.protobuf.Timestamp.Builder, com.google.protobuf.TimestampOrBuilder> 
+          getUpperWindowBoundaryFieldBuilder() {
+        if (upperWindowBoundaryBuilder_ == null) {
+          upperWindowBoundaryBuilder_ = new com.google.protobuf.SingleFieldBuilderV3<
+              com.google.protobuf.Timestamp, com.google.protobuf.Timestamp.Builder, com.google.protobuf.TimestampOrBuilder>(
+                  getUpperWindowBoundary(),
+                  getParentForChildren(),
+                  isClean());
+          upperWindowBoundary_ = null;
+        }
+        return upperWindowBoundaryBuilder_;
+      }
+      public final Builder setUnknownFields(
+          final com.google.protobuf.UnknownFieldSet unknownFields) {
+        return this;
+      }
+
+      public final Builder mergeUnknownFields(
+          final com.google.protobuf.UnknownFieldSet unknownFields) {
+        return this;
+      }
+
+
+      // @@protoc_insertion_point(builder_scope:protos.TSAccumSequence)
+    }
+
+    // @@protoc_insertion_point(class_scope:protos.TSAccumSequence)
+    private static final org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSAccumSequence DEFAULT_INSTANCE;
+    static {
+      DEFAULT_INSTANCE = new org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSAccumSequence();
+    }
+
+    public static org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSAccumSequence getDefaultInstance() {
+      return DEFAULT_INSTANCE;
+    }
+
+    private static final com.google.protobuf.Parser<TSAccumSequence>
+        PARSER = new com.google.protobuf.AbstractParser<TSAccumSequence>() {
+      public TSAccumSequence parsePartialFrom(
+          com.google.protobuf.CodedInputStream input,
+          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws com.google.protobuf.InvalidProtocolBufferException {
+          return new TSAccumSequence(input, extensionRegistry);
+      }
+    };
+
+    public static com.google.protobuf.Parser<TSAccumSequence> parser() {
+      return PARSER;
+    }
+
+    @java.lang.Override
+    public com.google.protobuf.Parser<TSAccumSequence> getParserForType() {
+      return PARSER;
+    }
+
+    public org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSAccumSequence getDefaultInstanceForType() {
+      return DEFAULT_INSTANCE;
+    }
+
+  }
+
+  private static final com.google.protobuf.Descriptors.Descriptor
+    internal_static_protos_TSKey_descriptor;
+  private static final 
+    com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+      internal_static_protos_TSKey_fieldAccessorTable;
+  private static final com.google.protobuf.Descriptors.Descriptor
+    internal_static_protos_TSDataPoint_descriptor;
+  private static final 
+    com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+      internal_static_protos_TSDataPoint_fieldAccessorTable;
+  private static final com.google.protobuf.Descriptors.Descriptor
+    internal_static_protos_TSDataPoint_MetadataEntry_descriptor;
+  private static final 
+    com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+      internal_static_protos_TSDataPoint_MetadataEntry_fieldAccessorTable;
+  private static final com.google.protobuf.Descriptors.Descriptor
+    internal_static_protos_TSDataSchema_descriptor;
+  private static final 
+    com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+      internal_static_protos_TSDataSchema_fieldAccessorTable;
+  private static final com.google.protobuf.Descriptors.Descriptor
+    internal_static_protos_TSDataSchema_FeaturesEntry_descriptor;
+  private static final 
+    com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+      internal_static_protos_TSDataSchema_FeaturesEntry_fieldAccessorTable;
+  private static final com.google.protobuf.Descriptors.Descriptor
+    internal_static_protos_TSDataPointSequence_descriptor;
+  private static final 
+    com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+      internal_static_protos_TSDataPointSequence_fieldAccessorTable;
+  private static final com.google.protobuf.Descriptors.Descriptor
+    internal_static_protos_PipelineVersion_descriptor;
+  private static final 
+    com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+      internal_static_protos_PipelineVersion_fieldAccessorTable;
+  private static final com.google.protobuf.Descriptors.Descriptor
+    internal_static_protos_TSAccum_descriptor;
+  private static final 
+    com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+      internal_static_protos_TSAccum_fieldAccessorTable;
+  private static final com.google.protobuf.Descriptors.Descriptor
+    internal_static_protos_TSAccumSequence_descriptor;
+  private static final 
+    com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+      internal_static_protos_TSAccumSequence_fieldAccessorTable;
+
+  public static com.google.protobuf.Descriptors.FileDescriptor
+      getDescriptor() {
+    return descriptor;
+  }
+  private static  com.google.protobuf.Descriptors.FileDescriptor
+      descriptor;
+  static {
+    java.lang.String[] descriptorData = {
+      "\n\020TimeSeries.proto\022\006protos\032\037google/proto" +
+      "buf/timestamp.proto\"\024\n\005TSKey\022\013\n\003key\030\001 \001(" +
+      "\t\"\205\002\n\013TSDataPoint\022\032\n\003key\030\001 \001(\0132\r.protos." +
+      "TSKey\022\023\n\tfloat_val\030\003 \001(\002H\000\022\024\n\ndouble_val" +
+      "\030\004 \001(\001H\000\022\021\n\007int_val\030\005 \001(\005H\000\022-\n\ttimestamp" +
+      "\030\007 \001(\0132\032.google.protobuf.Timestamp\0223\n\010me" +
+      "tadata\030\010 \003(\0132!.protos.TSDataPoint.Metada" +
+      "taEntry\032/\n\rMetadataEntry\022\013\n\003key\030\001 \001(\t\022\r\n" +
+      "\005value\030\002 \001(\t:\0028\001B\007\n\005value\"\240\001\n\014TSDataSche" +
+      "ma\022\032\n\003key\030\001 \001(\0132\r.protos.TSKey\0224\n\010featur",
+      "es\030\002 \003(\0132\".protos.TSDataSchema.FeaturesE" +
+      "ntry\032>\n\rFeaturesEntry\022\013\n\003key\030\001 \001(\t\022\034\n\005va" +
+      "lue\030\002 \001(\0132\r.protos.TSKey:\0028\001\"\302\001\n\023TSDataP" +
+      "ointSequence\022\032\n\003key\030\001 \001(\0132\r.protos.TSKey" +
+      "\0220\n\017pipelineVersion\030\002 \001(\0132\027.protos.Pipel" +
+      "ineVersion\022#\n\006accums\030\003 \003(\0132\023.protos.TSDa" +
+      "taPoint\022\033\n\023lowerWindowBoundary\030\010 \001(\003\022\033\n\023" +
+      "upperWindowBoundary\030\t \001(\003\"j\n\017PipelineVer" +
+      "sion\022\024\n\014majorVersion\030\001 \001(\t\022\024\n\014minorVersi" +
+      "on\030\002 \001(\t\022\024\n\014pipelineName\030\003 \001(\t\022\025\n\rtransf",
+      "ormName\030\004 \001(\t\"\203\002\n\007TSAccum\022\032\n\003key\030\001 \001(\0132\r" +
+      ".protos.TSKey\0220\n\017pipelineVersion\030\002 \001(\0132\027" +
+      ".protos.PipelineVersion\022!\n\004data\030\003 \001(\0132\023." +
+      "protos.TSDataPoint\022\025\n\rdurationMills\030\004 \001(" +
+      "\003\0227\n\023lowerWindowBoundary\030\005 \001(\0132\032.google." +
+      "protobuf.Timestamp\0227\n\023upperWindowBoundar" +
+      "y\030\006 \001(\0132\032.google.protobuf.Timestamp\"\362\001\n\017" +
+      "TSAccumSequence\022\032\n\003key\030\001 \001(\0132\r.protos.TS" +
+      "Key\0220\n\017pipelineVersion\030\002 \001(\0132\027.protos.Pi" +
+      "pelineVersion\022\037\n\006accums\030\003 \003(\0132\017.protos.T",
+      "SAccum\0227\n\023lowerWindowBoundary\030\004 \001(\0132\032.go" +
+      "ogle.protobuf.Timestamp\0227\n\023upperWindowBo" +
+      "undary\030\005 \001(\0132\032.google.protobuf.Timestamp" +
+      "*;\n\016DownSampleType\022\007\n\003SUM\020\000\022\t\n\005FIRST\020\001\022\010" +
+      "\n\004LAST\020\002\022\013\n\007AVERAGE\020\003BI\n3org.apache.beam" +
+      ".examples.advanced.timeseries.protosB\020Ti" +
+      "meSeriesProtosP\000b\006proto3"
+    };
+    com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner =
+        new com.google.protobuf.Descriptors.FileDescriptor.    InternalDescriptorAssigner() {
+          public com.google.protobuf.ExtensionRegistry assignDescriptors(
+              com.google.protobuf.Descriptors.FileDescriptor root) {
+            descriptor = root;
+            return null;
+          }
+        };
+    com.google.protobuf.Descriptors.FileDescriptor
+      .internalBuildGeneratedFileFrom(descriptorData,
+        new com.google.protobuf.Descriptors.FileDescriptor[] {
+          com.google.protobuf.TimestampProto.getDescriptor(),
+        }, assigner);
+    internal_static_protos_TSKey_descriptor =
+      getDescriptor().getMessageTypes().get(0);
+    internal_static_protos_TSKey_fieldAccessorTable = new
+      com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
+        internal_static_protos_TSKey_descriptor,
+        new java.lang.String[] { "Key", });
+    internal_static_protos_TSDataPoint_descriptor =
+      getDescriptor().getMessageTypes().get(1);
+    internal_static_protos_TSDataPoint_fieldAccessorTable = new
+      com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
+        internal_static_protos_TSDataPoint_descriptor,
+        new java.lang.String[] { "Key", "FloatVal", "DoubleVal", "IntVal", "Timestamp", "Metadata", "Value", });
+    internal_static_protos_TSDataPoint_MetadataEntry_descriptor =
+      internal_static_protos_TSDataPoint_descriptor.getNestedTypes().get(0);
+    internal_static_protos_TSDataPoint_MetadataEntry_fieldAccessorTable = new
+      com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
+        internal_static_protos_TSDataPoint_MetadataEntry_descriptor,
+        new java.lang.String[] { "Key", "Value", });
+    internal_static_protos_TSDataSchema_descriptor =
+      getDescriptor().getMessageTypes().get(2);
+    internal_static_protos_TSDataSchema_fieldAccessorTable = new
+      com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
+        internal_static_protos_TSDataSchema_descriptor,
+        new java.lang.String[] { "Key", "Features", });
+    internal_static_protos_TSDataSchema_FeaturesEntry_descriptor =
+      internal_static_protos_TSDataSchema_descriptor.getNestedTypes().get(0);
+    internal_static_protos_TSDataSchema_FeaturesEntry_fieldAccessorTable = new
+      com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
+        internal_static_protos_TSDataSchema_FeaturesEntry_descriptor,
+        new java.lang.String[] { "Key", "Value", });
+    internal_static_protos_TSDataPointSequence_descriptor =
+      getDescriptor().getMessageTypes().get(3);
+    internal_static_protos_TSDataPointSequence_fieldAccessorTable = new
+      com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
+        internal_static_protos_TSDataPointSequence_descriptor,
+        new java.lang.String[] { "Key", "PipelineVersion", "Accums", "LowerWindowBoundary", "UpperWindowBoundary", });
+    internal_static_protos_PipelineVersion_descriptor =
+      getDescriptor().getMessageTypes().get(4);
+    internal_static_protos_PipelineVersion_fieldAccessorTable = new
+      com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
+        internal_static_protos_PipelineVersion_descriptor,
+        new java.lang.String[] { "MajorVersion", "MinorVersion", "PipelineName", "TransformName", });
+    internal_static_protos_TSAccum_descriptor =
+      getDescriptor().getMessageTypes().get(5);
+    internal_static_protos_TSAccum_fieldAccessorTable = new
+      com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
+        internal_static_protos_TSAccum_descriptor,
+        new java.lang.String[] { "Key", "PipelineVersion", "Data", "DurationMills", "LowerWindowBoundary", "UpperWindowBoundary", });
+    internal_static_protos_TSAccumSequence_descriptor =
+      getDescriptor().getMessageTypes().get(6);
+    internal_static_protos_TSAccumSequence_fieldAccessorTable = new
+      com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
+        internal_static_protos_TSAccumSequence_descriptor,
+        new java.lang.String[] { "Key", "PipelineVersion", "Accums", "LowerWindowBoundary", "UpperWindowBoundary", });
+    com.google.protobuf.TimestampProto.getDescriptor();
+  }
+
+  // @@protoc_insertion_point(outer_class_scope)
+}
diff --git a/examples/java8/src/main/java/org/apache/beam/examples/advanced/timeseries/transform/library/BackFillAllWindowsAndKeys.java b/examples/java8/src/main/java/org/apache/beam/examples/advanced/timeseries/transform/library/BackFillAllWindowsAndKeys.java
new file mode 100644
index 00000000000..3f723d44dd1
--- /dev/null
+++ b/examples/java8/src/main/java/org/apache/beam/examples/advanced/timeseries/transform/library/BackFillAllWindowsAndKeys.java
@@ -0,0 +1,141 @@
+/*
+ * 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.beam.examples.advanced.timeseries.transform.library;
+
+import com.google.protobuf.Timestamp;
+
+import java.util.List;
+
+import org.apache.beam.examples.advanced.timeseries.configuration.TSConfiguration;
+import org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSDataPoint;
+import org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSKey;
+import org.apache.beam.sdk.annotations.Experimental;
+import org.apache.beam.sdk.io.GenerateSequence;
+import org.apache.beam.sdk.transforms.DoFn;
+import org.apache.beam.sdk.transforms.Flatten;
+import org.apache.beam.sdk.transforms.PTransform;
+import org.apache.beam.sdk.transforms.ParDo;
+import org.apache.beam.sdk.transforms.SerializableFunction;
+import org.apache.beam.sdk.transforms.windowing.FixedWindows;
+import org.apache.beam.sdk.transforms.windowing.Window;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.PCollectionList;
+import org.apache.beam.sdk.values.PCollectionView;
+import org.joda.time.Duration;
+import org.joda.time.Instant;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * This class generates a Noop element. Streaming mode it will create a tick ever x time duration.
+ * Batch mode it will create a window for every down sample Fixed window duration.
+ */
+@SuppressWarnings("serial")
+@Experimental
+public class BackFillAllWindowsAndKeys
+    extends PTransform<PCollection<TSDataPoint>, PCollection<TSDataPoint>> {
+
+  private static final Logger LOG = LoggerFactory.getLogger(BackFillAllWindowsAndKeys.class);
+
+  TSConfiguration configuration;
+  PCollectionView<List<String>> allKeys;
+
+  /**
+   * Fill any gaps in time series data per Key-Window combination. Window is based on the down
+   * sample interval provided in the configuration All keys are passed as a PCollectioView to this
+   * transform
+   * @param configuration
+   * @param allKeys
+   */
+  public BackFillAllWindowsAndKeys(TSConfiguration configuration,
+      PCollectionView<List<String>> allKeys) {
+    this.configuration = configuration;
+    this.allKeys = allKeys;
+  }
+
+  @Override
+  public PCollection<TSDataPoint> expand(PCollection<TSDataPoint> input) {
+
+    // Window into down sample size
+    PCollection<TSDataPoint> windowedMain =
+        input.apply(Window.into(FixedWindows.of(configuration.downSampleDuration())));
+
+    // Generate ticks within every window
+    PCollection<Long> noop;
+
+    if (configuration.isStreaming()) {
+      noop = input.getPipeline()
+          .apply(GenerateSequence.from(0).withRate(1, configuration.downSampleDuration())
+              .withTimestampFn(new TimeFunction(configuration)));
+    } else {
+      if (configuration.endTime() == null) {
+        throw new IllegalStateException("EndTime must be set in batch mode");
+      }
+      LOG.info(
+          String.format("Creating windows from 0 to %s", Utils.calculateNumWindows(configuration)));
+
+      noop = input.getPipeline()
+          .apply(GenerateSequence.from(0).to(Utils.calculateNumWindows(configuration))
+              .withTimestampFn(new TimeFunction(configuration)));
+    }
+    // In this step we create a dummy data point for each Fixed window
+    // and key combination between starttime and endtime or infinite in stream mode
+    PCollection<TSDataPoint> fillTimeSeries =
+        noop.apply("Create Noop window ticks", ParDo.of(new DoFn<Long, TSDataPoint>() {
+          @ProcessElement
+          public void processElement(ProcessContext c) {
+            for (String s : c.sideInput(allKeys)) {
+              c.output(TSDataPoint.newBuilder().setKey(TSKey.newBuilder().setKey(s))
+                  .setTimestamp(Timestamp.newBuilder().setSeconds(c.timestamp().getMillis() / 1000))
+                  .putMetadata(TSConfiguration.HEARTBEAT, "true").build());
+            }
+            LOG.info(
+                String.format("Creating HeartBeat for all values with generator %s timestamp %s",
+                    c.element(), c.timestamp()));
+          }
+        }).withSideInputs(allKeys))
+            .apply(Window.into(FixedWindows.of(configuration.downSampleDuration())));
+
+    // Flatten the Noop's and the real data together
+    PCollection<TSDataPoint> allValues =
+        PCollectionList.of(windowedMain).and(fillTimeSeries).apply(Flatten.pCollections());
+
+    // Apply Down sample window
+    PCollection<TSDataPoint> allWindowsValues = allValues
+        .apply(Window.<TSDataPoint>into(FixedWindows.of(configuration.downSampleDuration()))
+            .withAllowedLateness(Duration.ZERO));
+    return allWindowsValues;
+  }
+
+  /**
+   * This function provides GenerateSequence with the time stamps it needs to attach to the
+   * elements.
+   */
+  public static class TimeFunction implements SerializableFunction<Long, Instant> {
+
+    Duration downSampleSize;
+    TSConfiguration configuration;
+
+    public TimeFunction(TSConfiguration configuration) {
+      this.configuration = configuration;
+    }
+
+    @Override
+    public Instant apply(Long input) {
+      return configuration.startTime().plus(configuration.downSampleDuration().getMillis() * input);
+    }
+
+  }
+}
diff --git a/examples/java8/src/main/java/org/apache/beam/examples/advanced/timeseries/transform/library/Utils.java b/examples/java8/src/main/java/org/apache/beam/examples/advanced/timeseries/transform/library/Utils.java
new file mode 100644
index 00000000000..64bd1f3be16
--- /dev/null
+++ b/examples/java8/src/main/java/org/apache/beam/examples/advanced/timeseries/transform/library/Utils.java
@@ -0,0 +1,147 @@
+/*
+ * 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.beam.examples.advanced.timeseries.transform.library;
+
+import org.apache.beam.examples.advanced.timeseries.configuration.TSConfiguration;
+import org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos.TSDataPoint;
+import org.apache.beam.sdk.annotations.Experimental;
+import org.apache.beam.sdk.transforms.DoFn;
+import org.apache.beam.sdk.transforms.PTransform;
+import org.apache.beam.sdk.transforms.ParDo;
+import org.apache.beam.sdk.values.KV;
+import org.apache.beam.sdk.values.PCollection;
+import org.joda.time.Instant;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * Utils for use with the timeseries BEAM examples.
+ */
+@SuppressWarnings("serial")
+@Experimental
+public class Utils {
+
+  private static final Logger LOG = LoggerFactory.getLogger(Utils.class);
+
+  /**
+   * Transform to create a Key Value pair from a TSDataPoint object.
+   * @return PTransform
+   */
+  public static PTransform<PCollection<TSDataPoint>,
+    PCollection<KV<String, TSDataPoint>>> createKVTSDataPoint() {
+    return new CreateKVTSDataPoint();
+  }
+
+  /**
+   * Transform to create a Key Value pair from a TSDataPoint object.
+   */
+  static class CreateKVTSDataPoint
+      extends PTransform<PCollection<TSDataPoint>, PCollection<KV<String, TSDataPoint>>> {
+    @Override
+    public PCollection<KV<String, TSDataPoint>> expand(PCollection<TSDataPoint> input) {
+      return input.apply(ParDo.of(new DoFn<TSDataPoint, KV<String, TSDataPoint>>() {
+        @ProcessElement
+        public void processElement(ProcessContext c) {
+          c.output(KV.of(c.element().getKey().getKey(), c.element()));
+        }
+      }));
+    }
+  }
+
+  /**
+   * Attach time stamp to Process Context.
+   * @return PTransform
+   */
+  public static PTransform<PCollection<TSDataPoint>, PCollection<TSDataPoint>> extractTimeStamp() {
+    return new ExtractTimeStamp();
+  }
+
+  /**
+   * Attach time stamp to Process Context.
+   */
+  static class ExtractTimeStamp
+      extends PTransform<PCollection<TSDataPoint>, PCollection<TSDataPoint>> {
+
+    @Override
+    public PCollection<TSDataPoint> expand(PCollection<TSDataPoint> input) {
+
+      return input.apply(ParDo.of(new DoFn<TSDataPoint, TSDataPoint>() {
+        @ProcessElement
+        public void processElement(ProcessContext c) {
+          c.outputWithTimestamp(c.element(),
+              new Instant(c.element().getTimestamp().getSeconds() * 1000));
+        }
+      }));
+    }
+  }
+
+  /**
+   * Transform to test Start and End time are in tolerances.
+   */
+  public static class CheckConfiguration
+      extends PTransform<PCollection<TSDataPoint>, PCollection<TSDataPoint>> {
+
+    TSConfiguration configuration;
+
+    public CheckConfiguration(TSConfiguration configuration) {
+      this.configuration = configuration;
+    }
+
+    @Override
+    public PCollection<TSDataPoint> expand(PCollection<TSDataPoint> input) {
+
+      return input.apply(ParDo.of(new DoFn<TSDataPoint, TSDataPoint>() {
+        @ProcessElement
+        public void processElement(ProcessContext c) {
+          if (!configuration.isStreaming()
+              && (c.timestamp().getMillis() > configuration.endTime().getMillis())) {
+            String err =
+                String.format("Time stamp %s past Configuration EndTime %s found in batch mode",
+                    c.timestamp().getMillis(), configuration.endTime().getMillis());
+            LOG.error(err);
+            throw new IllegalStateException(err);
+          }
+          if (c.timestamp().getMillis() < configuration.startTime().getMillis()) {
+            String err = String.format("Time stamp %s before Configuration StartTime %s found",
+                c.timestamp().getMillis(), configuration.startTime().getMillis());
+            LOG.error(err);
+            throw new IllegalStateException(err);
+          }
+
+          c.output(c.element());
+        }
+      }));
+    }
+  }
+
+  /**
+   * Calculate the number of fixed windows between end and start time.
+   * @param TSConfiguration
+   * @return long
+   */
+  public static long calculateNumWindows(TSConfiguration configuration) {
+
+    LOG.info(
+        String.format("Computing Number of windows using startTime %s, endTime %s and Duration %s",
+            configuration.startTime(), configuration.endTime(),
+            configuration.downSampleDuration().getMillis()));
+
+    double result =
+        Math.ceil((configuration.endTime().getMillis() - configuration.startTime().getMillis())
+            / configuration.downSampleDuration().getMillis());
+    return (long) result;
+  }
+}
diff --git a/examples/java8/src/test/java/org/apache/beam/examples/advanced/timeseries/TimeseriesFillPipelineTest.java b/examples/java8/src/test/java/org/apache/beam/examples/advanced/timeseries/TimeseriesFillPipelineTest.java
new file mode 100644
index 00000000000..815da776f2c
--- /dev/null
+++ b/examples/java8/src/test/java/org/apache/beam/examples/advanced/timeseries/TimeseriesFillPipelineTest.java
@@ -0,0 +1,202 @@
+/*
+ * 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.beam.examples.advanced.timeseries;
+
+import com.google.common.collect.ImmutableList;
+import com.google.protobuf.Timestamp;
+import org.apache.beam.examples.MinimalWordCountJava8;
+import org.apache.beam.examples.advanced.timeseries.configuration.TSConfiguration;
+import org.apache.beam.examples.advanced.timeseries.protos.TimeSeriesProtos;
+import org.apache.beam.examples.advanced.timeseries.transform.library.BackFillAllWindowsAndKeys;
+import org.apache.beam.examples.advanced.timeseries.transform.library.Utils;
+import org.apache.beam.sdk.Pipeline;
+import org.apache.beam.sdk.extensions.gcp.options.GcsOptions;
+import org.apache.beam.sdk.io.TextIO;
+import org.apache.beam.sdk.options.PipelineOptions;
+import org.apache.beam.sdk.options.PipelineOptionsFactory;
+import org.apache.beam.sdk.testing.TestPipeline;
+import org.apache.beam.sdk.transforms.*;
+import org.apache.beam.sdk.transforms.windowing.IntervalWindow;
+import org.apache.beam.sdk.util.GcsUtil;
+import org.apache.beam.sdk.util.gcsfs.GcsPath;
+import org.apache.beam.sdk.values.KV;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.PCollectionView;
+import org.apache.beam.sdk.values.TypeDescriptors;
+import org.joda.time.Duration;
+import org.joda.time.Instant;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.JUnit4;
+import org.mockito.Mockito;
+import org.mockito.invocation.InvocationOnMock;
+import org.mockito.stubbing.Answer;
+
+import java.io.IOException;
+import java.io.Serializable;
+import java.nio.channels.FileChannel;
+import java.nio.channels.SeekableByteChannel;
+import java.nio.file.Files;
+import java.nio.file.StandardOpenOption;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+
+/**
+ * To keep {@link TimeseriesFillPipeline} simple, it is not factored or testable. This test
+ * file should be maintained with a copy of its code for a basic smoke test.
+ */
+@RunWith(JUnit4.class)
+public class TimeseriesFillPipelineTest implements Serializable {
+
+  @Rule
+  public TestPipeline p = TestPipeline.create().enableAbandonedNodeEnforcement(false);
+
+  /**
+   * A basic smoke test that ensures there is no crash at pipeline construction time.
+   */
+  @Test
+  public void testTimeseriesFillPipeline() throws Exception {
+    p.getOptions();
+
+    // Setup start and end time for example Pipeline
+
+    Instant startTime = new Instant("2000-01-01T00:00:00");
+
+    Instant endTime = startTime.plus(Duration.standardSeconds(60));
+
+    Duration downSampleDuration = Duration.standardSeconds(20);
+    // Setup configuration
+    TSConfiguration configuration = TSConfiguration.builder().setStartTime(startTime)
+        .setEndTime(endTime).setDownSampleDuration(downSampleDuration)
+        .setFillOption(TSConfiguration.FillOptions.NONE).setIsStreaming(false).build();
+
+    // Create pipeline
+    PipelineOptions options = PipelineOptionsFactory.create();
+    Pipeline p = Pipeline.create(options);
+
+    // Define a view of all possible keys
+    PCollectionView<List<String>> allKeys =
+        p.apply(Create.of("TS1", "TS2", "TS3")).apply(View.asList());
+
+    // Create 3 mock time series
+    PCollection<TimeSeriesProtos.TSDataPoint> dataPoints = p.apply(Create.of(createMockTimeSeries(configuration)));
+
+    // As this example is not using a unbounded source which will set the
+    // time stamp we will manually set it
+    PCollection<TimeSeriesProtos.TSDataPoint> dataPointsWithTimestamp = dataPoints.apply(Utils.extractTimeStamp());
+
+    // Generate a tick for each window for each key even if there was no
+    // data
+    PCollection<TimeSeriesProtos.TSDataPoint> dataPointsWithbackFill =
+        dataPointsWithTimestamp.apply(new BackFillAllWindowsAndKeys(configuration, allKeys));
+
+    dataPointsWithbackFill.apply(ParDo.of(new DoFn<TimeSeriesProtos.TSDataPoint, KV<String, TimeSeriesProtos.TSDataPoint>>() {
+
+      // Print the output of the back fill
+      // In order to print all data from all collections we need to
+      // re-key and re-window into a Global Window
+
+      @ProcessElement
+      public void process(ProcessContext c) {
+        c.output(KV.of(c.element().getKey().getKey(), c.element()));
+      }
+    })).apply(GroupByKey.create())
+        .apply(ParDo.of(new DoFn<KV<String, Iterable<TimeSeriesProtos.TSDataPoint>>, KV<String, Double>>() {
+
+          @ProcessElement
+          public void process(ProcessContext c, IntervalWindow w) {
+
+            StringBuffer sb = new StringBuffer();
+
+            sb.append(String.format("Key is %s Time Window is %s \n", c.element().getKey(),
+                w.toString()));
+
+            for (TimeSeriesProtos.TSDataPoint ts : c.element().getValue()) {
+
+              TimeSeriesProtos.TSDataPointSequence.Builder list = TimeSeriesProtos.TSDataPointSequence.newBuilder();
+
+              list.addAccums(ts);
+
+              c.output(KV.of(c.element().getKey(), ts.getDoubleVal()));
+
+            }
+
+          }
+        })).apply(Sum.doublesPerKey())
+        .apply(ParDo.of(new DoFn<KV<String, Double>, KV<String, Double>>() {
+
+          // Print the per wind
+          // In order to print all data from all collections we need to
+          // re-key and re-window into a Global Window
+
+          @ProcessElement
+          public void process(ProcessContext c, IntervalWindow w) {
+            c.output(c.element());
+          }
+        }));
+
+  }
+
+
+  /**
+   * Create three mock time series with missing ticks.
+   * @param configuration
+   * @return List
+   */
+  private static List<TimeSeriesProtos.TSDataPoint> createMockTimeSeries(TSConfiguration configuration) {
+
+    // Create Time series
+    List<TimeSeriesProtos.TSDataPoint> ts1 = new ArrayList<>();
+
+    double numElements =
+        Math.floor(configuration.endTime().getMillis() - configuration.startTime().getMillis())
+            / 5000;
+
+    long startTime = configuration.startTime().getMillis();
+
+    // Create a Data point every 5 seconds for TS1
+    // Remove a data point at position 4,5,6,7
+    for (long i = 0; i < numElements; i++) {
+      if (i < 4 || i > 7) {
+        ts1.add(
+            TimeSeriesProtos.TSDataPoint.newBuilder().setKey(TimeSeriesProtos.TSKey.newBuilder().setKey("TS1")).setDoubleVal(i)
+                .setTimestamp(Timestamp.newBuilder().setSeconds((startTime / 1000) + (5 * i))).build());
+      }
+    }
+
+    // Create a Data point every 5 seconds for TS2
+    // Remove data point at min 0
+    for (long i = 0; i < numElements; i++) {
+      if (i != 0) {
+        ts1.add(
+            TimeSeriesProtos.TSDataPoint.newBuilder().setKey(TimeSeriesProtos.TSKey.newBuilder().setKey("TS2"))
+                .setDoubleVal(i + 100)
+                .setTimestamp(Timestamp.newBuilder().setSeconds((startTime / 1000) + (5 * i))).build());
+      }
+    }
+
+    // Add only one value at point 0
+    ts1.add(TimeSeriesProtos.TSDataPoint.newBuilder().setKey(TimeSeriesProtos.TSKey.newBuilder().setKey("TS3")).setDoubleVal(1000)
+        .setTimestamp(Timestamp.newBuilder().setSeconds((startTime / 1000))).build());
+
+    return ts1;
+  }
+
+}


 

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


Issue Time Tracking
-------------------

    Worklog Id:     (was: 98864)
    Time Spent: 20m  (was: 10m)

> Create more advanced Timeseries processing examples using state API
> -------------------------------------------------------------------
>
>                 Key: BEAM-2953
>                 URL: https://issues.apache.org/jira/browse/BEAM-2953
>             Project: Beam
>          Issue Type: Improvement
>          Components: examples-java
>    Affects Versions: 2.1.0
>            Reporter: Reza ardeshir rokni
>            Assignee: Reuven Lax
>            Priority: Minor
>          Time Spent: 20m
>  Remaining Estimate: 0h
>
> As described in the phase 1 portion of this solution outline:
> https://cloud.google.com/solutions/correlating-time-series-dataflow
> BEAM can be used to build out some very interesting pre-processing stages for time series data. Some examples that will be useful:
> - Downsampling time series based on simple AVG, MIN, MAX
> - Creating a value for each time window using generatesequence as a seed 
> - Loading the value of a downsample with the previous value (used in FX with previous close being brought into current open value) 
> This will show some concrete examples of keyed state as well as the use of combiners. 
> The samples can also be used to show how you can create a ordered list of values per key from a unbounded topic which has multiple time series keys. 



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)