-
Notifications
You must be signed in to change notification settings - Fork 0
METRICS-4792 : OpenTelemetry Traces Protobuf InputFormat #129
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
Changes from 19 commits
c53d7c6
9352441
f574fa4
dff447f
874b2bc
ce812c9
c5ba6e9
b3c2c85
3e401df
128c704
94faf8a
d0d5cc7
dc6e3dc
c8e3a77
04a5434
fdfe198
6dffb45
63cd879
e896053
348c0be
c05a185
7f5e5ba
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -67,6 +67,11 @@ | |
<artifactId>jackson-databind</artifactId> | ||
<scope>provided</scope> | ||
</dependency> | ||
<dependency> | ||
<groupId>commons-codec</groupId> | ||
<artifactId>commons-codec</artifactId> | ||
<scope>provided</scope> | ||
</dependency> | ||
<dependency> | ||
<groupId>org.apache.druid</groupId> | ||
<artifactId>druid-core</artifactId> | ||
|
@@ -85,6 +90,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
There was a problem hiding this comment. Choose a reason for hiding this commentThe 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? There was a problem hiding this comment. Choose a reason for hiding this commentThe 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. There was a problem hiding this comment. Choose a reason for hiding this commentThe 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> | ||
|
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,108 @@ | ||
/* | ||
* Licensed to the Apache Software Foundation (ASF) under one | ||
* or more contributor license agreements. See the NOTICE file | ||
* distributed with this work for additional information | ||
* regarding copyright ownership. The ASF licenses this file | ||
* to you under the Apache License, Version 2.0 (the | ||
* "License"); you may not use this file except in compliance | ||
* with the License. You may obtain a copy of the License at | ||
* | ||
* http://www.apache.org/licenses/LICENSE-2.0 | ||
* | ||
* Unless required by applicable law or agreed to in writing, | ||
* software distributed under the License is distributed on an | ||
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY | ||
* KIND, either express or implied. See the License for the | ||
* specific language governing permissions and limitations | ||
* under the License. | ||
*/ | ||
|
||
package org.apache.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 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 java.nio.ByteBuffer; | ||
import java.util.ArrayList; | ||
import java.util.Iterator; | ||
import java.util.List; | ||
import java.util.Map; | ||
|
||
|
||
public abstract class OpenXProtobufReader implements InputEntityReader | ||
kamal-narayan marked this conversation as resolved.
Show resolved
Hide resolved
|
||
{ | ||
|
||
protected final SettableByteEntity<? extends ByteEntity> source; | ||
protected final DimensionsSpec dimensionsSpec; | ||
|
||
public OpenXProtobufReader(DimensionsSpec dimensionsSpec, | ||
SettableByteEntity<? extends ByteEntity> source) | ||
{ | ||
this.dimensionsSpec = dimensionsSpec; | ||
this.source = source; | ||
} | ||
|
||
public abstract List<InputRow> parseData(ByteBuffer byteBuffer) throws InvalidProtocolBufferException; | ||
|
||
@Override | ||
public CloseableIterator<InputRow> read() | ||
{ | ||
Supplier<Iterator<InputRow>> supplier = Suppliers.memoize(() -> readAsList().iterator()); | ||
return CloseableIterators.withEmptyBaggage(new Iterator<InputRow>() { | ||
@Override | ||
public boolean hasNext() | ||
{ | ||
return supplier.get().hasNext(); | ||
} | ||
@Override | ||
public InputRow next() | ||
{ | ||
return supplier.get().next(); | ||
} | ||
}); | ||
} | ||
|
||
public 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"); | ||
} | ||
} | ||
|
||
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); | ||
} | ||
|
||
@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 |
---|---|---|
@@ -0,0 +1,84 @@ | ||
/* | ||
* 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 io.opentelemetry.proto.common.v1.AnyValue; | ||
import io.opentelemetry.proto.resource.v1.Resource; | ||
import org.apache.druid.data.input.InputEntity; | ||
import org.apache.druid.data.input.impl.ByteEntity; | ||
import org.apache.druid.indexing.seekablestream.SettableByteEntity; | ||
|
||
import javax.annotation.Nullable; | ||
import java.util.HashMap; | ||
import java.util.Map; | ||
|
||
public class Utils | ||
{ | ||
public 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; | ||
} | ||
} | ||
|
||
@Nullable | ||
public 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; | ||
} | ||
} | ||
|
||
public static Map<String, Object> getResourceAttributes(Resource resource, String resourceAttributePrefix) | ||
{ | ||
return resource | ||
.getAttributesList() | ||
.stream() | ||
.collect( | ||
HashMap::new, | ||
(m, kv) -> { | ||
Object value = Utils.parseAnyValue(kv.getValue()); | ||
if (value != null) { | ||
m.put(resourceAttributePrefix + kv.getKey(), value); | ||
} | ||
}, | ||
HashMap::putAll | ||
); | ||
} | ||
} |
Uh oh!
There was an error while loading. Please reload this page.