You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@druid.apache.org by xv...@apache.org on 2019/07/26 19:20:54 UTC

[incubator-druid] 02/03: add opencensus extension

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

xvrl pushed a commit to branch opencensus-ext
in repository https://gitbox.apache.org/repos/asf/incubator-druid.git

commit 78fb4e1a6c3508a56823a8d0252f6e82b573f2d0
Author: Sumit Arrawatia <su...@gmail.com>
AuthorDate: Thu May 16 23:23:58 2019 -0700

    add opencensus extension
---
 distribution/pom.xml                               |   2 +
 extensions-contrib/opencensus-extensions/pom.xml   | 132 ++++++++++
 .../OpenCensusProtobufExtensionsModule.java        |  49 ++++
 .../protobuf/OpenCensusProtobufInputRowParser.java | 199 +++++++++++++++
 .../org.apache.druid.initialization.DruidModule    |  16 ++
 .../OpenCensusProtobufInputRowParserTest.java      | 267 +++++++++++++++++++++
 pom.xml                                            |   1 +
 7 files changed, 666 insertions(+)

diff --git a/distribution/pom.xml b/distribution/pom.xml
index 17d0c35..5f5228f 100644
--- a/distribution/pom.xml
+++ b/distribution/pom.xml
@@ -359,6 +359,8 @@
                                         <argument>org.apache.druid.extensions.contrib:druid-moving-average-query</argument>
                                         <argument>-c</argument>
                                         <argument>org.apache.druid.extensions.contrib:druid-tdigestsketch</argument>
+                                        <argument>-c</argument>
+                                        <argument>org.apache.druid.extensions.contrib:druid-opencensus-extensions</argument>
                                     </arguments>
                                 </configuration>
                             </execution>
diff --git a/extensions-contrib/opencensus-extensions/pom.xml b/extensions-contrib/opencensus-extensions/pom.xml
new file mode 100644
index 0000000..33c67ef
--- /dev/null
+++ b/extensions-contrib/opencensus-extensions/pom.xml
@@ -0,0 +1,132 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+  ~ 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.
+  -->
+
+<project xmlns="http://maven.apache.org/POM/4.0.0"
+         xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+         xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+  <modelVersion>4.0.0</modelVersion>
+
+  <groupId>org.apache.druid.extensions.contrib</groupId>
+  <artifactId>druid-opencensus-extensions</artifactId>
+  <name>druid-opencensus-extensions</name>
+  <description>druid-opencensus-extensions</description>
+
+  <parent>
+    <artifactId>druid</artifactId>
+    <groupId>org.apache.druid</groupId>
+    <version>0.16.0-incubating-SNAPSHOT</version>
+    <relativePath>../../pom.xml</relativePath>
+  </parent>
+
+  <properties>
+    <protobuf.version>3.2.0</protobuf.version>
+  </properties>
+
+  <dependencies>
+    <dependency>
+      <groupId>io.opencensus</groupId>
+      <artifactId>opencensus-proto</artifactId>
+      <version>0.2.0</version>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.druid</groupId>
+      <artifactId>druid-core</artifactId>
+      <version>${project.parent.version}</version>
+      <scope>provided</scope>
+    </dependency>
+    <dependency>
+      <groupId>com.google.protobuf</groupId>
+      <artifactId>protobuf-java</artifactId>
+      <version>${protobuf.version}</version>
+    </dependency>
+    <dependency>
+      <groupId>com.google.protobuf</groupId>
+      <artifactId>protobuf-java-util</artifactId>
+      <version>${protobuf.version}</version>
+      <exclusions>
+        <exclusion>
+          <groupId>com.google.guava</groupId>
+          <artifactId>guava</artifactId>
+        </exclusion>
+      </exclusions>
+    </dependency>
+    <!-- test -->
+    <dependency>
+      <groupId>junit</groupId>
+      <artifactId>junit</artifactId>
+      <scope>test</scope>
+    </dependency>
+  </dependencies>
+  <build>
+    <plugins>
+      <plugin>
+        <groupId>org.apache.maven.plugins</groupId>
+        <artifactId>maven-shade-plugin</artifactId>
+        <version>3.0.0</version>
+        <configuration>
+          <createDependencyReducedPom>false</createDependencyReducedPom>
+          <relocations>
+            <relocation>
+              <pattern>com.google.protobuf</pattern>
+              <shadedPattern>shaded.com.google.protobuf</shadedPattern>
+            </relocation>
+          </relocations>
+        </configuration>
+        <executions>
+          <execution>
+            <phase>package</phase>
+            <goals>
+              <goal>shade</goal>
+            </goals>
+          </execution>
+        </executions>
+      </plugin>
+      <plugin>
+        <groupId>org.apache.maven.plugins</groupId>
+        <artifactId>maven-resources-plugin</artifactId>
+        <version>3.0.2</version>
+        <configuration>
+          <nonFilteredFileExtensions>
+            <nonFilteredFileExtension>desc</nonFilteredFileExtension>
+          </nonFilteredFileExtensions>
+        </configuration>
+      </plugin>
+    </plugins>
+  </build>
+  <profiles>
+    <profile>
+      <id>strict</id>
+      <build>
+        <plugins>
+          <plugin>
+            <groupId>org.apache.maven.plugins</groupId>
+            <artifactId>maven-compiler-plugin</artifactId>
+            <configuration>
+              <compilerArgs>
+                <!-- protobuf compiler generated classes miss @Override, that is not easy to fix -->
+                <arg>-Xep:MissingOverride:WARN</arg>
+              </compilerArgs>
+            </configuration>
+          </plugin>
+        </plugins>
+      </build>
+    </profile>
+  </profiles>
+</project>
diff --git a/extensions-contrib/opencensus-extensions/src/main/java/org/apache/druid/data/input/opencensus/protobuf/OpenCensusProtobufExtensionsModule.java b/extensions-contrib/opencensus-extensions/src/main/java/org/apache/druid/data/input/opencensus/protobuf/OpenCensusProtobufExtensionsModule.java
new file mode 100644
index 0000000..39576e4
--- /dev/null
+++ b/extensions-contrib/opencensus-extensions/src/main/java/org/apache/druid/data/input/opencensus/protobuf/OpenCensusProtobufExtensionsModule.java
@@ -0,0 +1,49 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.druid.data.input.opencensus.protobuf;
+
+import com.fasterxml.jackson.databind.Module;
+import com.fasterxml.jackson.databind.jsontype.NamedType;
+import com.fasterxml.jackson.databind.module.SimpleModule;
+import com.google.inject.Binder;
+import org.apache.druid.initialization.DruidModule;
+
+import java.util.Collections;
+import java.util.List;
+
+public class OpenCensusProtobufExtensionsModule implements DruidModule
+{
+
+  @Override
+  public List<? extends Module> getJacksonModules()
+  {
+    return Collections.singletonList(
+        new SimpleModule("OpenCensusProtobufInputRowParserModule")
+            .registerSubtypes(
+                new NamedType(OpenCensusProtobufInputRowParser.class, "opencensus-protobuf")
+            )
+    );
+  }
+
+  @Override
+  public void configure(Binder binder)
+  {
+  }
+}
diff --git a/extensions-contrib/opencensus-extensions/src/main/java/org/apache/druid/data/input/opencensus/protobuf/OpenCensusProtobufInputRowParser.java b/extensions-contrib/opencensus-extensions/src/main/java/org/apache/druid/data/input/opencensus/protobuf/OpenCensusProtobufInputRowParser.java
new file mode 100644
index 0000000..53b447f
--- /dev/null
+++ b/extensions-contrib/opencensus-extensions/src/main/java/org/apache/druid/data/input/opencensus/protobuf/OpenCensusProtobufInputRowParser.java
@@ -0,0 +1,199 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.druid.data.input.opencensus.protobuf;
+
+import com.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import com.google.common.collect.Lists;
+import com.google.common.collect.Sets;
+import com.google.protobuf.ByteString;
+import com.google.protobuf.InvalidProtocolBufferException;
+import io.opencensus.proto.metrics.v1.Metric;
+import io.opencensus.proto.metrics.v1.Point;
+import io.opencensus.proto.metrics.v1.TimeSeries;
+import org.apache.druid.data.input.ByteBufferInputRowParser;
+import org.apache.druid.data.input.InputRow;
+import org.apache.druid.data.input.MapBasedInputRow;
+import org.apache.druid.data.input.impl.ParseSpec;
+import org.apache.druid.java.util.common.logger.Logger;
+import org.apache.druid.java.util.common.parsers.ParseException;
+
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+public class OpenCensusProtobufInputRowParser implements ByteBufferInputRowParser
+{
+  private static final Logger LOG = new Logger(OpenCensusProtobufInputRowParser.class);
+
+  private static final String SEPARATOR = "-";
+  public static final String NAME = "name";
+  public static final String VALUE = "value";
+  public static final String TIMESTAMP_COLUMN = "timestamp";
+  private final ParseSpec parseSpec;
+  private final List<String> dimensions;
+
+  @JsonCreator
+  public OpenCensusProtobufInputRowParser(
+      @JsonProperty("parseSpec") ParseSpec parseSpec
+  )
+  {
+    this.parseSpec = parseSpec;
+    this.dimensions = parseSpec.getDimensionsSpec().getDimensionNames();
+    LOG.info("Creating Open Census Protobuf parser with spec:" + parseSpec);
+  }
+
+  @Override
+  public ParseSpec getParseSpec()
+  {
+    return parseSpec;
+  }
+
+  @Override
+  public OpenCensusProtobufInputRowParser withParseSpec(ParseSpec parseSpec)
+  {
+    return new OpenCensusProtobufInputRowParser(parseSpec);
+  }
+
+  @Override
+  public List<InputRow> parseBatch(ByteBuffer input)
+  {
+
+    Metric metric;
+    try {
+      metric = Metric.parseFrom(ByteString.copyFrom(input));
+    }
+    catch (InvalidProtocolBufferException e) {
+      throw new ParseException(e, "Protobuf message could not be parsed");
+    }
+
+    final List<String> dimensions;
+
+    if (!this.dimensions.isEmpty()) {
+      dimensions = this.dimensions;
+    } else {
+      Set<String> recordDimensions = metric.getMetricDescriptor().getLabelKeysList().stream()
+          .map(s -> s.getKey())
+          .collect(Collectors.toSet());
+      recordDimensions.add(NAME);
+      recordDimensions.add(VALUE);
+
+
+      dimensions = Lists.newArrayList(
+          Sets.difference(recordDimensions, parseSpec.getDimensionsSpec().getDimensionExclusions())
+      );
+    }
+
+    // Flatten out the OpenCensus record into druid rows.
+    List<InputRow> rows = new ArrayList<>();
+    for (TimeSeries ts : metric.getTimeseriesList()) {
+
+      HashMap<String, Object> labels = new HashMap<>();
+
+      // Add labels to record.
+      for (int i = 0; i < metric.getMetricDescriptor().getLabelKeysCount(); i++) {
+        labels.put(metric.getMetricDescriptor().getLabelKeys(i).getKey(), ts.getLabelValues(i).getValue());
+      }
+
+      // One row per timeseries- point.
+      for (Point point : ts.getPointsList()) {
+        // Time in millis
+        labels.put(TIMESTAMP_COLUMN, point.getTimestamp().getSeconds() * 1000);
+
+        switch (point.getValueCase()) {
+          case DOUBLE_VALUE:
+            HashMap<String, Object> doubleGauge = new HashMap<>();
+            doubleGauge.putAll(labels);
+            doubleGauge.put(NAME, metric.getMetricDescriptor().getName());
+            doubleGauge.put(VALUE, point.getDoubleValue());
+            rows.add(new MapBasedInputRow(
+                parseSpec.getTimestampSpec().extractTimestamp(doubleGauge),
+                dimensions,
+                doubleGauge
+            ));
+            break;
+          case INT64_VALUE:
+            HashMap<String, Object> intGauge = new HashMap<>();
+            intGauge.putAll(labels);
+            intGauge.put(VALUE, point.getInt64Value());
+            intGauge.put(NAME, metric.getMetricDescriptor().getName());
+            rows.add(new MapBasedInputRow(
+                parseSpec.getTimestampSpec().extractTimestamp(intGauge),
+                dimensions,
+                intGauge
+            ));
+            break;
+          case SUMMARY_VALUE:
+            // count
+            HashMap<String, Object> summaryCount = new HashMap<>();
+            summaryCount.putAll(labels);
+            summaryCount.put(NAME, metric.getMetricDescriptor().getName() + SEPARATOR + "count");
+            summaryCount.put(VALUE, point.getSummaryValue().getCount().getValue());
+            rows.add(new MapBasedInputRow(
+                parseSpec.getTimestampSpec().extractTimestamp(summaryCount),
+                dimensions,
+                summaryCount
+            ));
+
+            // sum
+            HashMap<String, Object> summarySum = new HashMap<>();
+            summarySum.putAll(labels);
+            summarySum.put(NAME, metric.getMetricDescriptor().getName() + SEPARATOR + "sum");
+            summarySum.put(VALUE, point.getSummaryValue().getSnapshot().getSum().getValue());
+            rows.add(new MapBasedInputRow(
+                parseSpec.getTimestampSpec().extractTimestamp(summarySum),
+                dimensions,
+                summarySum
+            ));
+
+            // TODO : Do we put percentiles into druid ?
+            break;
+          case DISTRIBUTION_VALUE:
+            // count
+            HashMap<String, Object> distCount = new HashMap<>();
+            distCount.put(NAME, metric.getMetricDescriptor().getName() + SEPARATOR + "count");
+            distCount.put(VALUE, point.getDistributionValue().getCount());
+            rows.add(new MapBasedInputRow(
+                parseSpec.getTimestampSpec().extractTimestamp(distCount),
+                dimensions,
+                distCount
+            ));
+
+            // sum
+            HashMap<String, Object> distSum = new HashMap<>();
+            distSum.put(NAME, metric.getMetricDescriptor().getName() + SEPARATOR + "sum");
+            distSum.put(VALUE, point.getDistributionValue().getSum());
+            rows.add(new MapBasedInputRow(
+                parseSpec.getTimestampSpec().extractTimestamp(distSum),
+                dimensions,
+                distSum
+            ));
+            // TODO: How to handle buckets ?
+            break;
+        }
+      }
+    }
+    return rows;
+  }
+
+}
diff --git a/extensions-contrib/opencensus-extensions/src/main/resources/META-INF/services/org.apache.druid.initialization.DruidModule b/extensions-contrib/opencensus-extensions/src/main/resources/META-INF/services/org.apache.druid.initialization.DruidModule
new file mode 100755
index 0000000..54b4400
--- /dev/null
+++ b/extensions-contrib/opencensus-extensions/src/main/resources/META-INF/services/org.apache.druid.initialization.DruidModule
@@ -0,0 +1,16 @@
+# 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.
+
+org.apache.druid.data.input.opencensus.protobuf.OpenCensusProtobufExtensionsModule
\ No newline at end of file
diff --git a/extensions-contrib/opencensus-extensions/src/test/java/org/apache/druid/data/input/opencensus/protobuf/OpenCensusProtobufInputRowParserTest.java b/extensions-contrib/opencensus-extensions/src/test/java/org/apache/druid/data/input/opencensus/protobuf/OpenCensusProtobufInputRowParserTest.java
new file mode 100644
index 0000000..8e55755
--- /dev/null
+++ b/extensions-contrib/opencensus-extensions/src/test/java/org/apache/druid/data/input/opencensus/protobuf/OpenCensusProtobufInputRowParserTest.java
@@ -0,0 +1,267 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.druid.data.input.opencensus.protobuf;
+
+import com.google.common.collect.Lists;
+import com.google.protobuf.DoubleValue;
+import com.google.protobuf.Int64Value;
+import com.google.protobuf.Timestamp;
+import io.opencensus.proto.metrics.v1.LabelKey;
+import io.opencensus.proto.metrics.v1.LabelValue;
+import io.opencensus.proto.metrics.v1.Metric;
+import io.opencensus.proto.metrics.v1.MetricDescriptor;
+import io.opencensus.proto.metrics.v1.Point;
+import io.opencensus.proto.metrics.v1.SummaryValue;
+import io.opencensus.proto.metrics.v1.TimeSeries;
+import io.opencensus.proto.resource.v1.Resource;
+import org.apache.druid.data.input.InputRow;
+import org.apache.druid.data.input.impl.DimensionsSpec;
+import org.apache.druid.data.input.impl.JSONParseSpec;
+import org.apache.druid.data.input.impl.ParseSpec;
+import org.apache.druid.data.input.impl.TimestampSpec;
+import org.apache.druid.java.util.common.parsers.JSONPathFieldSpec;
+import org.apache.druid.java.util.common.parsers.JSONPathFieldType;
+import org.apache.druid.java.util.common.parsers.JSONPathSpec;
+import org.joda.time.DateTime;
+import org.joda.time.chrono.ISOChronology;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.ExpectedException;
+
+import java.io.ByteArrayOutputStream;
+import java.nio.ByteBuffer;
+import java.util.Collections;
+import java.util.List;
+
+import static org.junit.Assert.assertEquals;
+
+public class OpenCensusProtobufInputRowParserTest
+{
+  @Rule
+  public ExpectedException expectedException = ExpectedException.none();
+
+  private ParseSpec parseSpec;
+
+  @Before
+  public void setUp()
+  {
+    parseSpec = new JSONParseSpec(
+        new TimestampSpec("timestamp", "millis", null),
+        new DimensionsSpec(null, null, null),
+        new JSONPathSpec(
+            true,
+            Lists.newArrayList(
+                new JSONPathFieldSpec(JSONPathFieldType.ROOT, "name", ""),
+                new JSONPathFieldSpec(JSONPathFieldType.ROOT, "value", ""),
+                new JSONPathFieldSpec(JSONPathFieldType.ROOT, "foo_key", "")
+            )
+        ), null
+    );
+
+  }
+
+
+  @Test
+  public void testGaugeParse() throws Exception
+  {
+
+    //configure parser with desc file
+    OpenCensusProtobufInputRowParser parser = new OpenCensusProtobufInputRowParser(parseSpec);
+
+    DateTime dateTime = new DateTime(2019, 07, 12, 9, 30, ISOChronology.getInstanceUTC());
+
+    Timestamp timestamp = Timestamp.newBuilder().setSeconds(dateTime.getMillis() / 1000)
+        .setNanos((int) ((dateTime.getMillis() % 1000) * 1000000)).build();
+
+    System.out.println(timestamp.getSeconds() * 1000);
+
+    Metric d = doubleGaugeMetric(timestamp);
+    ByteArrayOutputStream out = new ByteArrayOutputStream();
+    d.writeTo(out);
+
+    InputRow row = parser.parseBatch(ByteBuffer.wrap(out.toByteArray())).get(0);
+    assertEquals(dateTime.getMillis(), row.getTimestampFromEpoch());
+
+    assertDimensionEquals(row, "name", "metric_gauge_double");
+    assertDimensionEquals(row, "foo_key", "foo_value");
+
+
+    assertEquals(2000, row.getMetric("value").doubleValue(), 0.0);
+  }
+
+  @Test
+  public void testSummaryParse() throws Exception
+  {
+    //configure parser with desc file
+    OpenCensusProtobufInputRowParser parser = new OpenCensusProtobufInputRowParser(parseSpec);
+
+    DateTime dateTime = new DateTime(2019, 07, 12, 9, 30, ISOChronology.getInstanceUTC());
+
+    Timestamp timestamp = Timestamp.newBuilder().setSeconds(dateTime.getMillis() / 1000)
+        .setNanos((int) ((dateTime.getMillis() % 1000) * 1000000)).build();
+
+    Metric d = summaryMetric(timestamp);
+    ByteArrayOutputStream out = new ByteArrayOutputStream();
+    d.writeTo(out);
+
+    List<InputRow> rows = parser.parseBatch(ByteBuffer.wrap(out.toByteArray()));
+
+    assertEquals(2, rows.size());
+
+    InputRow row = rows.get(0);
+    assertEquals(dateTime.getMillis(), row.getTimestampFromEpoch());
+    assertDimensionEquals(row, "name", "metric_summary-count");
+    assertDimensionEquals(row, "foo_key", "foo_value");
+    assertEquals(40, row.getMetric("value").doubleValue(), 0.0);
+
+    row = rows.get(1);
+    assertEquals(dateTime.getMillis(), row.getTimestampFromEpoch());
+    assertDimensionEquals(row, "name", "metric_summary-sum");
+    assertDimensionEquals(row, "foo_key", "foo_value");
+    assertEquals(10, row.getMetric("value").doubleValue(), 0.0);
+
+  }
+
+  private void assertDimensionEquals(InputRow row, String dimension, Object expected)
+  {
+    List<String> values = row.getDimension(dimension);
+
+    assertEquals(1, values.size());
+    assertEquals(expected, values.get(0));
+  }
+
+  private Metric doubleGaugeMetric(Timestamp timestamp)
+  {
+    Metric dist = Metric.newBuilder()
+        .setMetricDescriptor(
+            MetricDescriptor.newBuilder()
+                .setName("metric_gauge_double")
+                .setDescription("metric_gauge_double_description")
+                .setUnit("ms")
+                .setType(
+                    MetricDescriptor.Type.GAUGE_DOUBLE)
+                .addLabelKeys(
+                    LabelKey.newBuilder()
+                        .setKey("foo_key")
+                        .build())
+                .build())
+        .setResource(
+            Resource.newBuilder()
+                .setType("env")
+                .putAllLabels(Collections.singletonMap("env_key", "env_val"))
+                .build())
+        .addTimeseries(
+            TimeSeries.newBuilder()
+                .setStartTimestamp(timestamp)
+                .addLabelValues(
+                    LabelValue.newBuilder()
+                        .setHasValue(true)
+                        .setValue("foo_value")
+                        .build())
+                .addPoints(
+                    Point.newBuilder()
+                        .setTimestamp(timestamp)
+                        .setDoubleValue(2000)
+                        .build())
+                .build())
+        .build();
+
+    return dist;
+  }
+
+
+  private Metric summaryMetric(Timestamp timestamp)
+  {
+
+    SummaryValue.Snapshot snapshot = SummaryValue.Snapshot.newBuilder()
+        .setSum(DoubleValue.newBuilder().setValue(10).build())
+        .addPercentileValues(SummaryValue.Snapshot.ValueAtPercentile.newBuilder()
+            .setPercentile(50.0)
+            .setValue(10)
+            .build())
+        .addPercentileValues(SummaryValue.Snapshot.ValueAtPercentile.newBuilder()
+            .setPercentile(75.0)
+            .setValue(20)
+            .build())
+        .addPercentileValues(SummaryValue.Snapshot.ValueAtPercentile.newBuilder()
+            .setPercentile(95.0)
+            .setValue(30)
+            .build())
+        .addPercentileValues(SummaryValue.Snapshot.ValueAtPercentile.newBuilder()
+            .setPercentile(98.0)
+            .setValue(40)
+            .build())
+        .addPercentileValues(SummaryValue.Snapshot.ValueAtPercentile.newBuilder()
+            .setPercentile(99.0)
+            .setValue(50)
+            .build())
+        .addPercentileValues(SummaryValue.Snapshot.ValueAtPercentile.newBuilder()
+            .setPercentile(99.9)
+            .setValue(60)
+            .build())
+        .build();
+
+
+    SummaryValue summaryValue = SummaryValue.newBuilder()
+        .setCount(Int64Value.newBuilder().setValue(40).build())
+        .setSnapshot(snapshot)
+        .build();
+
+
+    Metric dist = Metric.newBuilder()
+        .setMetricDescriptor(
+            MetricDescriptor.newBuilder()
+                .setName("metric_summary")
+                .setDescription("metric_summary_description")
+                .setUnit("ms")
+                .setType(
+                    MetricDescriptor.Type.SUMMARY)
+                .addLabelKeys(
+                    LabelKey.newBuilder()
+                        .setKey("foo_key")
+                        .build())
+                .build())
+        .setResource(
+            Resource.newBuilder()
+                .setType("env")
+                .putAllLabels(Collections.singletonMap("env_key", "env_val"))
+                .build())
+        .addTimeseries(
+            TimeSeries.newBuilder()
+                .setStartTimestamp(timestamp)
+                .addLabelValues(
+                    LabelValue.newBuilder()
+                        .setHasValue(true)
+                        .setValue("foo_value")
+                        .build())
+                .addPoints(
+                    Point.newBuilder()
+                        .setTimestamp(timestamp)
+                        .setSummaryValue(summaryValue)
+                        .build())
+                .build())
+        .build();
+
+    return dist;
+  }
+
+
+}
diff --git a/pom.xml b/pom.xml
index 1a802c9..d767383 100644
--- a/pom.xml
+++ b/pom.xml
@@ -172,6 +172,7 @@
         <module>extensions-contrib/moving-average-query</module>
         <module>extensions-contrib/tdigestsketch</module>
         <module>extensions-contrib/influxdb-emitter</module>
+        <module>extensions-contrib/opencensus-extensions</module>
         <!-- distribution packaging -->
         <module>distribution</module>
     </modules>


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