-
Notifications
You must be signed in to change notification settings - Fork 334
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
SAMZA-2354: Improve UDF discovery in samza-sql. #1192
Changes from all commits
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 |
---|---|---|
@@ -0,0 +1,96 @@ | ||
/* | ||
* 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.samza.sql.udf; | ||
|
||
import java.lang.reflect.Method; | ||
import java.util.Arrays; | ||
import java.util.Collection; | ||
import java.util.HashSet; | ||
import java.util.List; | ||
import java.util.Set; | ||
import org.apache.commons.lang3.reflect.MethodUtils; | ||
import org.apache.samza.SamzaException; | ||
import org.apache.samza.config.Config; | ||
import org.apache.samza.sql.interfaces.UdfMetadata; | ||
import org.apache.samza.sql.interfaces.UdfResolver; | ||
import org.apache.samza.sql.schema.SamzaSqlFieldType; | ||
import org.apache.samza.sql.udfs.SamzaSqlUdf; | ||
import org.apache.samza.sql.udfs.SamzaSqlUdfMethod; | ||
import org.reflections.Reflections; | ||
import org.reflections.util.ConfigurationBuilder; | ||
import org.slf4j.Logger; | ||
import org.slf4j.LoggerFactory; | ||
|
||
/** | ||
* An UDF resolver implementation that uses reflection to discover the subtypes | ||
* of the {@link SamzaSqlUdf} from the classpath. Performs the validation to | ||
* ensure that all subtypes of {@link SamzaSqlUdf} extend and implement the | ||
* method annotated with {@link SamzaSqlUdfMethod}. | ||
*/ | ||
public class ReflectionBasedUdfResolver implements UdfResolver { | ||
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. Just curious, how much of a hit are we taking with OSS jar to load the UDFs via reflection ? 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. By selectively searching for UDF's only in selected package-prefixes from class-path for UDF's, it's a very minimal over-head. |
||
|
||
private static final Logger LOG = LoggerFactory.getLogger(ReflectionBasedUdfResolver.class); | ||
|
||
private static final String CONFIG_PACKAGE_PREFIX = "samza.sql.udf.resolver.package.prefix"; | ||
|
||
private final Set<UdfMetadata> udfs = new HashSet<>(); | ||
|
||
public ReflectionBasedUdfResolver(Config udfConfig) { | ||
// Searching the entire classpath to discover the subtypes of SamzaSqlUdf is expensive. To reduce the search space, | ||
// the search is limited to the set of package prefixes defined in the configuration. | ||
String samzaSqlUdfPackagePrefix = udfConfig.getOrDefault(CONFIG_PACKAGE_PREFIX, "org.apache.samza"); | ||
|
||
// 1. Build the reflections instance with appropriate configuration. | ||
ConfigurationBuilder configurationBuilder = new ConfigurationBuilder(); | ||
configurationBuilder.forPackages(samzaSqlUdfPackagePrefix.split(",")); | ||
configurationBuilder.addClassLoader(Thread.currentThread().getContextClassLoader()); | ||
Reflections reflections = new Reflections(configurationBuilder); | ||
|
||
// 2. Get all the sub-types of SamzaSqlUdf. | ||
Set<Class<?>> typesAnnotatedWithSamzaSqlUdf = reflections.getTypesAnnotatedWith(SamzaSqlUdf.class); | ||
|
||
for (Class<?> udfClass : typesAnnotatedWithSamzaSqlUdf) { | ||
// 3. Get all the methods that are annotated with SamzaSqlUdfMethod | ||
List<Method> methodsAnnotatedWithSamzaSqlMethod = MethodUtils.getMethodsListWithAnnotation(udfClass, SamzaSqlUdfMethod.class); | ||
|
||
if (methodsAnnotatedWithSamzaSqlMethod.isEmpty()) { | ||
String msg = String.format("Udf class: %s doesn't have any methods annotated with: %s", udfClass.getName(), SamzaSqlUdfMethod.class.getName()); | ||
LOG.error(msg); | ||
throw new SamzaException(msg); | ||
} | ||
|
||
SamzaSqlUdf sqlUdf = udfClass.getAnnotation(SamzaSqlUdf.class); | ||
// 4. If the udf is enabled, then add the udf information of the methods to the udfs list. | ||
if (sqlUdf.enabled()) { | ||
String udfName = sqlUdf.name(); | ||
methodsAnnotatedWithSamzaSqlMethod.forEach(method -> { | ||
SamzaSqlUdfMethod samzaSqlUdfMethod = method.getAnnotation(SamzaSqlUdfMethod.class); | ||
List<SamzaSqlFieldType> params = Arrays.asList(samzaSqlUdfMethod.params()); | ||
udfs.add(new UdfMetadata(udfName, sqlUdf.description(), method, udfConfig.subset(udfName + "."), params, | ||
samzaSqlUdfMethod.returns(), samzaSqlUdfMethod.disableArgumentCheck())); | ||
}); | ||
} | ||
} | ||
} | ||
|
||
@Override | ||
public Collection<UdfMetadata> getUdfs() { | ||
return udfs; | ||
} | ||
} |
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,60 @@ | ||
/* | ||
* 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.samza.sql.udf.impl; | ||
|
||
import com.google.common.collect.ImmutableList; | ||
import com.google.common.collect.ImmutableMap; | ||
import org.apache.samza.config.Config; | ||
import org.apache.samza.config.MapConfig; | ||
import org.apache.samza.sql.interfaces.UdfMetadata; | ||
import org.apache.samza.sql.schema.SamzaSqlFieldType; | ||
import org.apache.samza.sql.udf.ReflectionBasedUdfResolver; | ||
import org.junit.Assert; | ||
import org.junit.Test; | ||
|
||
import java.lang.reflect.Method; | ||
import java.util.Collection; | ||
|
||
public class TestReflectionBasedUdfResolver { | ||
|
||
@Test | ||
public void testShouldReturnNothingWhenNoUDFIsInPackagePrefix() { | ||
Config config = new MapConfig(ImmutableMap.of("samza.sql.udf.resolver.package.prefix", "org.apache.samza.udf.blah.blah")); | ||
ReflectionBasedUdfResolver reflectionBasedUdfResolver = new ReflectionBasedUdfResolver(config); | ||
Collection<UdfMetadata> udfMetadataList = reflectionBasedUdfResolver.getUdfs(); | ||
|
||
Assert.assertEquals(0, udfMetadataList.size()); | ||
} | ||
|
||
@Test | ||
public void testUDfResolverShouldReturnAllUDFInClassPath() throws NoSuchMethodException { | ||
Config config = new MapConfig(ImmutableMap.of("samza.sql.udf.resolver.package.prefix", "org.apache.samza.sql.udf.impl")); | ||
ReflectionBasedUdfResolver reflectionBasedUdfResolver = new ReflectionBasedUdfResolver(config); | ||
Collection<UdfMetadata> udfMetadataList = reflectionBasedUdfResolver.getUdfs(); | ||
|
||
Method method = TestSamzaSqlUdf.class.getMethod("execute", String.class); | ||
UdfMetadata udfMetadata = new UdfMetadata("TESTSAMZASQLUDF", | ||
"Test samza sql udf implementation", method, new MapConfig(), ImmutableList.of(SamzaSqlFieldType.STRING), | ||
SamzaSqlFieldType.STRING, true); | ||
|
||
Assert.assertFalse(udfMetadataList.isEmpty()); | ||
Assert.assertTrue(udfMetadataList.contains(udfMetadata)); | ||
} | ||
} |
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,40 @@ | ||
/* | ||
* 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.samza.sql.udf.impl; | ||
|
||
import org.apache.samza.config.Config; | ||
import org.apache.samza.context.Context; | ||
import org.apache.samza.sql.schema.SamzaSqlFieldType; | ||
import org.apache.samza.sql.udfs.SamzaSqlUdf; | ||
import org.apache.samza.sql.udfs.SamzaSqlUdfMethod; | ||
import org.apache.samza.sql.udfs.ScalarUdf; | ||
|
||
@SamzaSqlUdf(name = "TestSamzaSqlUdf", description = "Test samza sql udf implementation") | ||
public class TestSamzaSqlUdf implements ScalarUdf { | ||
|
||
@Override | ||
public void init(Config udfConfig, Context context) { | ||
|
||
} | ||
|
||
@SamzaSqlUdfMethod(params = {SamzaSqlFieldType.STRING}, returns = SamzaSqlFieldType.STRING) | ||
public String execute(String fieldName) { | ||
return "testResponse"; | ||
} | ||
} |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Sorry, I didn't realize that this is hard-coded here. Could we make this pluggable ? I think calling initializePlugin like above (createIOResolver method) should do it with the udf resolver based arguments.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
I agree that this is a useful thing to do and discussed it offline. Created a follow-up ticket(SAMZA-2355) for making the UDFResolver pluggable in samza-sql. It will be done in an immediate follow-up patch.