forked from apache/druid
-
Notifications
You must be signed in to change notification settings - Fork 0
Commit
This commit does not belong to any branch on this repository, and may belong to a fork outside of the repository.
Hybrid OpenCensusProtobufInputFormat in opencensus-extensions (#69)
* Support OpenTelemetry payloads in OpenCensusProtobufInputFormat Support reading mixed OpenTelemetry and OpenCensus topics based on Kafka version header * workaround classloader isolation Workaround classloader isolation by using method handles to get access to KafkaRecordEntity related methods and check record headers Co-authored-by: Xavier Léauté <[email protected]>
- Loading branch information
1 parent
4b93e7c
commit 788b94d
Showing
9 changed files
with
613 additions
and
8 deletions.
There are no files selected for viewing
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
107 changes: 107 additions & 0 deletions
107
...s-contrib/opencensus-extensions/src/main/java/org/apache/druid/data/input/KafkaUtils.java
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,107 @@ | ||
/* | ||
* 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; | ||
|
||
import java.lang.invoke.MethodHandle; | ||
import java.lang.invoke.MethodHandles; | ||
import java.lang.invoke.MethodType; | ||
import java.util.Objects; | ||
|
||
public class KafkaUtils | ||
{ | ||
/** | ||
* Creates a MethodHandle that – when invoked on a KafkaRecordEntity - returns the given header value | ||
* for the underlying KafkaRecordEntity | ||
* | ||
* The method handle is roughly equivalent to the following function | ||
* | ||
* (KafkaRecordEntity input) -> { | ||
* Header h = input.getRecord().headers().lastHeader(header) | ||
* if (h != null) { | ||
* return h.value(); | ||
* } else { | ||
* return null; | ||
* } | ||
* } | ||
* | ||
* Since KafkaRecordEntity only exists in the kafka-indexing-service plugin classloader, | ||
* we need to look up the relevant classes in the classloader where the InputEntity was instantiated. | ||
* | ||
* The handle returned by this method should be cached for the classloader it was invoked with. | ||
* | ||
* If the lookup fails for whatever reason, the method handle will always return null; | ||
* | ||
* @param classLoader the kafka-indexing-service classloader | ||
* @param header the header value to look up | ||
* @return a MethodHandle | ||
*/ | ||
public static MethodHandle lookupGetHeaderMethod(ClassLoader classLoader, String header) | ||
{ | ||
try { | ||
Class entityType = Class.forName("org.apache.druid.data.input.kafka.KafkaRecordEntity", true, classLoader); | ||
Class recordType = Class.forName("org.apache.kafka.clients.consumer.ConsumerRecord", true, classLoader); | ||
Class headersType = Class.forName("org.apache.kafka.common.header.Headers", true, classLoader); | ||
Class headerType = Class.forName("org.apache.kafka.common.header.Header", true, classLoader); | ||
|
||
final MethodHandles.Lookup lookup = MethodHandles.lookup(); | ||
MethodHandle nonNullTest = lookup.findStatic(Objects.class, "nonNull", | ||
MethodType.methodType(boolean.class, Object.class) | ||
).asType(MethodType.methodType(boolean.class, headerType)); | ||
|
||
final MethodHandle getRecordMethod = lookup.findVirtual( | ||
entityType, | ||
"getRecord", | ||
MethodType.methodType(recordType) | ||
); | ||
final MethodHandle headersMethod = lookup.findVirtual(recordType, "headers", MethodType.methodType(headersType)); | ||
final MethodHandle lastHeaderMethod = lookup.findVirtual( | ||
headersType, | ||
"lastHeader", | ||
MethodType.methodType(headerType, String.class) | ||
); | ||
final MethodHandle valueMethod = lookup.findVirtual(headerType, "value", MethodType.methodType(byte[].class)); | ||
|
||
return MethodHandles.filterReturnValue( | ||
MethodHandles.filterReturnValue( | ||
MethodHandles.filterReturnValue(getRecordMethod, headersMethod), | ||
MethodHandles.insertArguments(lastHeaderMethod, 1, header) | ||
), | ||
// return null byte array if header is not present | ||
MethodHandles.guardWithTest( | ||
nonNullTest, | ||
valueMethod, | ||
// match valueMethod signature by dropping the header instance argument | ||
MethodHandles.dropArguments(MethodHandles.constant(byte[].class, null), 0, headerType) | ||
) | ||
); | ||
} | ||
catch (ReflectiveOperationException e) { | ||
// if lookup fails in the classloader where the InputEntity is defined, then the source may not be | ||
// the kafka-indexing-service classloader, or method signatures did not match. | ||
// In that case we return a method handle always returning null | ||
return noopMethodHandle(); | ||
} | ||
} | ||
|
||
static MethodHandle noopMethodHandle() | ||
{ | ||
return MethodHandles.dropArguments(MethodHandles.constant(byte[].class, null), 0, InputEntity.class); | ||
} | ||
} |
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
90 changes: 90 additions & 0 deletions
90
...ntrib/opencensus-extensions/src/test/java/org/apache/druid/data/input/KafkaUtilsTest.java
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -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. | ||
*/ | ||
|
||
package org.apache.druid.data.input; | ||
|
||
|
||
import com.google.common.collect.ImmutableList; | ||
import org.apache.druid.data.input.impl.ByteEntity; | ||
import org.apache.druid.data.input.kafka.KafkaRecordEntity; | ||
import org.apache.kafka.clients.consumer.ConsumerRecord; | ||
import org.apache.kafka.common.header.Header; | ||
import org.apache.kafka.common.header.internals.RecordHeaders; | ||
import org.apache.kafka.common.record.TimestampType; | ||
import org.junit.Assert; | ||
import org.junit.Test; | ||
|
||
import java.lang.invoke.MethodHandle; | ||
import java.nio.ByteBuffer; | ||
|
||
public class KafkaUtilsTest | ||
{ | ||
|
||
private static final byte[] BYTES = ByteBuffer.allocate(Integer.BYTES).putInt(42).array(); | ||
|
||
@Test | ||
public void testNoopMethodHandle() throws Throwable | ||
{ | ||
Assert.assertNull( | ||
KafkaUtils.noopMethodHandle().invoke(new ByteEntity(new byte[]{})) | ||
); | ||
} | ||
|
||
@Test | ||
public void testKafkaRecordEntity() throws Throwable | ||
{ | ||
final MethodHandle handle = KafkaUtils.lookupGetHeaderMethod(KafkaUtilsTest.class.getClassLoader(), "version"); | ||
KafkaRecordEntity input = new KafkaRecordEntity( | ||
new ConsumerRecord<>( | ||
"test", | ||
0, | ||
0, | ||
0, | ||
TimestampType.CREATE_TIME, | ||
-1L, | ||
-1, | ||
-1, | ||
null, | ||
new byte[]{}, | ||
new RecordHeaders(ImmutableList.of(new Header() | ||
{ | ||
@Override | ||
public String key() | ||
{ | ||
return "version"; | ||
} | ||
|
||
@Override | ||
public byte[] value() | ||
{ | ||
return BYTES; | ||
} | ||
})) | ||
) | ||
); | ||
Assert.assertArrayEquals(BYTES, (byte[]) handle.invoke(input)); | ||
} | ||
|
||
@Test(expected = ClassCastException.class) | ||
public void testNonKafkaEntity() throws Throwable | ||
{ | ||
final MethodHandle handle = KafkaUtils.lookupGetHeaderMethod(KafkaUtilsTest.class.getClassLoader(), "version"); | ||
handle.invoke(new ByteEntity(new byte[]{})); | ||
} | ||
} |
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Oops, something went wrong.