Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

METRICS-4792 : OpenTelemetry Traces Protobuf InputFormat #129

Closed
wants to merge 22 commits into from
Closed
Show file tree
Hide file tree
Changes from 12 commits
Commits
File filter

Filter by extension

Filter by extension


Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
2 changes: 1 addition & 1 deletion distribution/pom.xml
Original file line number Diff line number Diff line change
Expand Up @@ -626,7 +626,7 @@
<argument>-c</argument>
<argument>io.confluent.druid.extensions:confluent-extensions</argument>
<argument>-c</argument>
<argument>org.apache.druid.extensions.contrib:opentelemetry-emitter</argument>
<argument>org.apache.druid.extensions.contrib:druid-opentelemetry-extensions</argument>
kamal-narayan marked this conversation as resolved.
Show resolved Hide resolved
</arguments>
</configuration>
</execution>
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -26,7 +26,7 @@
import org.apache.druid.data.input.MapBasedInputRow;
import org.apache.druid.data.input.impl.ByteEntity;
import org.apache.druid.data.input.impl.DimensionsSpec;
import org.apache.druid.data.input.opentelemetry.protobuf.OpenTelemetryMetricsProtobufReader;
import org.apache.druid.data.input.opentelemetry.protobuf.metrics.OpenTelemetryMetricsProtobufReader;
import org.apache.druid.indexing.seekablestream.SettableByteEntity;
import org.apache.druid.java.util.common.parsers.CloseableIterator;

Expand Down
5 changes: 5 additions & 0 deletions extensions-contrib/opentelemetry-extensions/pom.xml
Original file line number Diff line number Diff line change
Expand Up @@ -85,6 +85,11 @@
<artifactId>junit</artifactId>
<scope>test</scope>
</dependency>
<dependency>
<groupId>nl.jqno.equalsverifier</groupId>
<artifactId>equalsverifier</artifactId>
<scope>test</scope>
</dependency>
Comment on lines +93 to +97
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

adding a whole new dependency for a single test seems a little heavyweight? Can we accomplish that without this additional dependency?

Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

The Equals tests are required to meet the code coverage threshold. (More details in the comment below)

Given there are 10 fields to check, I think it would be better to use this library as it covers all the negative branch cases too.

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

ok, I wonder why we don't need it for metrics input format. It looks like we already use this dependency elsewhere in druid so I'm ok using that here.

<!-- jmh -->
<dependency>
<groupId>org.openjdk.jmh</groupId>
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,48 @@
/*
* 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.opentelemetry.protobuf;

import org.apache.druid.data.input.InputEntity;
import org.apache.druid.data.input.InputFormat;
import org.apache.druid.data.input.impl.ByteEntity;
import org.apache.druid.indexing.seekablestream.SettableByteEntity;

public abstract class OpenTelemetryInputFormat implements InputFormat
{

@Override
public boolean isSplittable()
kamal-narayan marked this conversation as resolved.
Show resolved Hide resolved
{
return false;
}

protected static SettableByteEntity<? extends ByteEntity> getSettableEntity(InputEntity source)
{
// Sampler passes a KafkaRecordEntity directly, while the normal code path wraps the same entity in a
// SettableByteEntity
if (source instanceof SettableByteEntity) {
return (SettableByteEntity<? extends ByteEntity>) source;
} else {
SettableByteEntity<ByteEntity> wrapper = new SettableByteEntity<>();
wrapper.setEntity((ByteEntity) source);
return wrapper;
}
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -23,21 +23,23 @@
import com.fasterxml.jackson.databind.jsontype.NamedType;
import com.fasterxml.jackson.databind.module.SimpleModule;
import com.google.inject.Binder;
import org.apache.druid.data.input.opentelemetry.protobuf.metrics.OpenTelemetryMetricsProtobufInputFormat;
import org.apache.druid.data.input.opentelemetry.protobuf.traces.OpenTelemetryTracesProtobufInputFormat;
import org.apache.druid.initialization.DruidModule;

import java.util.Collections;
import java.util.List;

public class OpenTelemetryProtobufExtensionsModule implements DruidModule
{

@Override
public List<? extends Module> getJacksonModules()
{
return Collections.singletonList(
new SimpleModule("OpenTelemetryProtobufInputFormat")
.registerSubtypes(
new NamedType(OpenTelemetryMetricsProtobufInputFormat.class, "opentelemetry-metrics-protobuf")
new NamedType(OpenTelemetryMetricsProtobufInputFormat.class, "opentelemetry-metrics-protobuf"),
new NamedType(OpenTelemetryTracesProtobufInputFormat.class, "opentelemetry-traces-protobuf")
)
);
}
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,151 @@
/*
* 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.opentelemetry.protobuf;

import com.google.common.base.Supplier;
import com.google.common.base.Suppliers;
import com.google.common.collect.Sets;
import com.google.protobuf.InvalidProtocolBufferException;
import io.opentelemetry.proto.common.v1.AnyValue;
import io.opentelemetry.proto.resource.v1.Resource;
import org.apache.druid.data.input.InputEntityReader;
import org.apache.druid.data.input.InputRow;
import org.apache.druid.data.input.InputRowListPlusRawValues;
import org.apache.druid.data.input.MapBasedInputRow;
import org.apache.druid.data.input.impl.ByteEntity;
import org.apache.druid.data.input.impl.DimensionsSpec;
import org.apache.druid.indexing.seekablestream.SettableByteEntity;
import org.apache.druid.java.util.common.CloseableIterators;
import org.apache.druid.java.util.common.parsers.CloseableIterator;
import org.apache.druid.java.util.common.parsers.ParseException;

import javax.annotation.Nullable;
import java.nio.ByteBuffer;
import java.util.ArrayList;
import java.util.HashMap;
import java.util.Iterator;
import java.util.List;
import java.util.Map;

public abstract class OpenTelemetryProtobufReader implements InputEntityReader
{

protected final String resourceAttributePrefix;
protected final SettableByteEntity<? extends ByteEntity> source;
private final DimensionsSpec dimensionsSpec;

public OpenTelemetryProtobufReader(DimensionsSpec dimensionsSpec,
SettableByteEntity<? extends ByteEntity> source,
String resourceAttributePrefix)
{
this.dimensionsSpec = dimensionsSpec;
this.resourceAttributePrefix = resourceAttributePrefix;
this.source = source;
}

public abstract List<InputRow> parseData(ByteBuffer byteBuffer) throws InvalidProtocolBufferException;
kamal-narayan marked this conversation as resolved.
Show resolved Hide resolved

@Override
public CloseableIterator<InputRow> read()
{
Supplier<Iterator<InputRow>> supplier = Suppliers.memoize(() -> readAsList().iterator());
kamal-narayan marked this conversation as resolved.
Show resolved Hide resolved
return CloseableIterators.withEmptyBaggage(new Iterator<InputRow>() {
@Override
public boolean hasNext()
{
return supplier.get().hasNext();
}
@Override
public InputRow next()
{
return supplier.get().next();
}
});
}

private List<InputRow> readAsList()
{
try {
ByteBuffer buffer = source.getEntity().getBuffer();
List<InputRow> rows = parseData(buffer);
// Explicitly move the position assuming that all the remaining bytes have been consumed because the protobuf
// parser does not update the position itself
buffer.position(buffer.limit());
return rows;
}
catch (InvalidProtocolBufferException e) {
throw new ParseException(null, e, "Protobuf message could not be parsed");
}
}

@Nullable
protected static Object parseAnyValue(AnyValue value)
{
switch (value.getValueCase()) {
case INT_VALUE:
return value.getIntValue();
case BOOL_VALUE:
return value.getBoolValue();
case DOUBLE_VALUE:
return value.getDoubleValue();
case STRING_VALUE:
return value.getStringValue();

// TODO: Support KVLIST_VALUE, ARRAY_VALUE and BYTES_VALUE

default:
// VALUE_NOT_SET
return null;
}
}

protected InputRow createRow(long timeUnixMilli, Map<String, Object> event)
{
final List<String> dimensions;
if (!dimensionsSpec.getDimensionNames().isEmpty()) {
dimensions = dimensionsSpec.getDimensionNames();
} else {
dimensions = new ArrayList<>(Sets.difference(event.keySet(), dimensionsSpec.getDimensionExclusions()));
}
return new MapBasedInputRow(timeUnixMilli, dimensions, event);
}

protected Map<String, Object> getResourceAttributes(Resource resource)
{
return resource.getAttributesList()
.stream()
.collect(
HashMap::new,
(m, kv) -> {
Object value = parseAnyValue(kv.getValue());
if (value != null) {
m.put(resourceAttributePrefix + kv.getKey(), value);
}
},
HashMap::putAll
);
}

@Override
public CloseableIterator<InputRowListPlusRawValues> sample()
{
return read().map(row -> InputRowListPlusRawValues.of(row, ((MapBasedInputRow) row).getEvent()));
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -17,21 +17,19 @@
* under the License.
*/

package org.apache.druid.data.input.opentelemetry.protobuf;
package org.apache.druid.data.input.opentelemetry.protobuf.metrics;

import com.fasterxml.jackson.annotation.JsonProperty;
import org.apache.druid.data.input.InputEntity;
import org.apache.druid.data.input.InputEntityReader;
import org.apache.druid.data.input.InputFormat;
import org.apache.druid.data.input.InputRowSchema;
import org.apache.druid.data.input.impl.ByteEntity;
import org.apache.druid.indexing.seekablestream.SettableByteEntity;
import org.apache.druid.data.input.opentelemetry.protobuf.OpenTelemetryInputFormat;
import org.apache.druid.java.util.common.StringUtils;

import java.io.File;
import java.util.Objects;

public class OpenTelemetryMetricsProtobufInputFormat implements InputFormat
public class OpenTelemetryMetricsProtobufInputFormat extends OpenTelemetryInputFormat
{
private static final String DEFAULT_METRIC_DIMENSION = "metric";
private static final String DEFAULT_VALUE_DIMENSION = "value";
Expand All @@ -55,28 +53,12 @@ public OpenTelemetryMetricsProtobufInputFormat(
this.resourceAttributePrefix = resourceAttributePrefix != null ? resourceAttributePrefix : DEFAULT_RESOURCE_PREFIX;
}

@Override
public boolean isSplittable()
{
return false;
}

@Override
public InputEntityReader createReader(InputRowSchema inputRowSchema, InputEntity source, File temporaryDirectory)
{
// Sampler passes a KafkaRecordEntity directly, while the normal code path wraps the same entity in a
// SettableByteEntity
SettableByteEntity<? extends ByteEntity> settableEntity;
if (source instanceof SettableByteEntity) {
settableEntity = (SettableByteEntity<? extends ByteEntity>) source;
} else {
SettableByteEntity<ByteEntity> wrapper = new SettableByteEntity<>();
wrapper.setEntity((ByteEntity) source);
settableEntity = wrapper;
}
return new OpenTelemetryMetricsProtobufReader(
inputRowSchema.getDimensionsSpec(),
settableEntity,
getSettableEntity(source),
metricDimension,
valueDimension,
metricAttributePrefix,
Expand Down
Loading