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

SAMZA-2289: Samza-sql - Fix udf to work with disabled arg check. #1124

Merged
merged 3 commits into from Aug 1, 2019
Merged
Show file tree
Hide file tree
Changes from 1 commit
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Jump to
Jump to file
Failed to load files.
Diff view
Diff view
@@ -0,0 +1,42 @@
/*
* 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.fn;

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 = "GetNestedField", description = "UDF that extracts a field value from a nested SamzaSqlRelRecord")
public class GetNestedFieldUdf implements ScalarUdf {
@Override
public void init(Config udfConfig, Context context) {
}

@SamzaSqlUdfMethod(params = {SamzaSqlFieldType.ANY, SamzaSqlFieldType.STRING},
returns = SamzaSqlFieldType.ANY)
public Object execute(Object currentFieldOrValue, String fieldName) {
GetSqlFieldUdf udf = new GetSqlFieldUdf();
return udf.getSqlField(currentFieldOrValue, fieldName);
}
}
50 changes: 31 additions & 19 deletions samza-sql/src/main/java/org/apache/samza/sql/fn/GetSqlFieldUdf.java
Expand Up @@ -21,7 +21,8 @@

import java.util.List;
import java.util.Map;
import org.apache.commons.lang.Validate;
import org.apache.avro.util.Utf8;
import org.apache.commons.lang3.Validate;
import org.apache.samza.config.Config;
import org.apache.samza.context.Context;
import org.apache.samza.sql.SamzaSqlRelRecord;
Expand Down Expand Up @@ -53,22 +54,15 @@
* - sessionKey (Scalar)
*
*/
@SamzaSqlUdf(name = "GetSqlField", description = "Get an element from complex Sql field as a String.")
@SamzaSqlUdf(name = "GetSqlField", description = "Deprecated : Please use GetNestedField.")
public class GetSqlFieldUdf implements ScalarUdf {
@Override
public void init(Config udfConfig, Context context) {
}

@SamzaSqlUdfMethod(params = {SamzaSqlFieldType.ANY, SamzaSqlFieldType.STRING})
public String execute(Object field, String fieldName) {
Object currentFieldOrValue = field;
Validate.isTrue(currentFieldOrValue == null
|| currentFieldOrValue instanceof SamzaSqlRelRecord);

String[] fieldNameChain = fieldName.split("\\.");
for (int i = 0; i < fieldNameChain.length && currentFieldOrValue != null; i++) {
currentFieldOrValue = extractField(fieldNameChain[i], currentFieldOrValue);
}
@SamzaSqlUdfMethod(params = {SamzaSqlFieldType.ANY, SamzaSqlFieldType.STRING}, returns = SamzaSqlFieldType.STRING)
public String execute(Object currentFieldOrValue, String fieldName) {
currentFieldOrValue = getSqlField(currentFieldOrValue, fieldName);

if (currentFieldOrValue != null) {
return currentFieldOrValue.toString();
Expand All @@ -77,23 +71,41 @@ public String execute(Object field, String fieldName) {
return null;
}

static Object extractField(String fieldName, Object current) {
public Object getSqlField(Object currentFieldOrValue, String fieldName) {
if (currentFieldOrValue != null) {
String[] fieldNameChain = (fieldName).split("\\.");
for (int i = 0; i < fieldNameChain.length && currentFieldOrValue != null; i++) {
currentFieldOrValue = extractField(fieldNameChain[i], currentFieldOrValue, true);
}
}

return currentFieldOrValue;
}

static Object extractField(String fieldName, Object current, boolean validateField) {
if (current instanceof SamzaSqlRelRecord) {
SamzaSqlRelRecord record = (SamzaSqlRelRecord) current;
Validate.isTrue(record.getFieldNames().contains(fieldName),
String.format("Invalid field %s in %s", fieldName, record));
if (validateField) {
Validate.isTrue(record.getFieldNames().contains(fieldName),
String.format("Invalid field %s in record %s", fieldName, record));
}
return record.getField(fieldName).orElse(null);
} else if (current instanceof Map) {
Map map = (Map) current;
Validate.isTrue(map.containsKey(fieldName), String.format("Invalid field %s in %s", fieldName, map));
return map.get(fieldName);
if (map.containsKey(fieldName)) {
return map.get(fieldName);
} else if (map.containsKey(new Utf8(fieldName))) {
return map.get(new Utf8(fieldName));
} else {
throw new IllegalArgumentException(String.format("Couldn't find the field %s in map %s", fieldName, map));
}
} else if (current instanceof List && fieldName.endsWith("]")) {
List list = (List) current;
int index = Integer.parseInt(fieldName.substring(fieldName.indexOf("[") + 1, fieldName.length() - 1));
return list.get(index);
}

throw new IllegalArgumentException(String.format(
"Unsupported accessing operation for data type: %s with field: %s.", current.getClass(), fieldName));
throw new IllegalArgumentException(
String.format("Unsupported accessing operation for data type: %s with field: %s.", current.getClass(), fieldName));
}
}
Expand Up @@ -87,7 +87,8 @@ public CallImplementor getImplementor() {
// SAMZA: 2230 To allow UDFS to accept Untyped arguments.
// We explicitly Convert the untyped arguments to type that the UDf expects.
for(int index = 0; index < translatedOperands.size(); index++) {
if (translatedOperands.get(index).type == Object.class && udfMethod.getParameters()[index].getType() != Object.class) {
if (!udfMetadata.isDisableArgCheck() && translatedOperands.get(index).type == Object.class
&& udfMethod.getParameters()[index].getType() != Object.class) {
convertedOperands.add(Expressions.convert_(translatedOperands.get(index), udfMethod.getParameters()[index].getType()));
} else {
convertedOperands.add(translatedOperands.get(index));
Expand Down
Expand Up @@ -37,6 +37,7 @@
import org.apache.samza.sql.avro.schemas.SimpleRecord;
import org.apache.samza.sql.fn.BuildOutputRecordUdf;
import org.apache.samza.sql.fn.FlattenUdf;
import org.apache.samza.sql.fn.GetNestedFieldUdf;
import org.apache.samza.sql.fn.RegexMatchUdf;
import org.apache.samza.sql.impl.ConfigBasedIOResolverFactory;
import org.apache.samza.sql.impl.ConfigBasedUdfResolver;
Expand Down Expand Up @@ -99,7 +100,7 @@ public static Map<String, String> fetchStaticConfigsWithFactories(Map<String, St
staticConfigs.put(configUdfResolverDomain + ConfigBasedUdfResolver.CFG_UDF_CLASSES, Joiner.on(",")
.join(MyTestUdf.class.getName(), RegexMatchUdf.class.getName(), FlattenUdf.class.getName(),
MyTestArrayUdf.class.getName(), BuildOutputRecordUdf.class.getName(), MyTestPolyUdf.class.getName(),
MyTestObjUdf.class.getName()));
MyTestObjUdf.class.getName(), GetNestedFieldUdf.class.getName()));

String avroSystemConfigPrefix =
String.format(ConfigBasedIOResolverFactory.CFG_FMT_SAMZA_PREFIX, SAMZA_SYSTEM_TEST_AVRO);
Expand Down Expand Up @@ -187,6 +188,9 @@ public static Map<String, String> fetchStaticConfigsWithFactories(Map<String, St
staticConfigs.put(configAvroRelSchemaProviderDomain + String.format(ConfigBasedAvroRelSchemaProviderFactory.CFG_SOURCE_SCHEMA,
"testavro", "PROFILE"), Profile.SCHEMA$.toString());

staticConfigs.put(configAvroRelSchemaProviderDomain + String.format(ConfigBasedAvroRelSchemaProviderFactory.CFG_SOURCE_SCHEMA,
"testavro", "PROFILE1"), Profile.SCHEMA$.toString());

staticConfigs.put(configAvroRelSchemaProviderDomain + String.format(ConfigBasedAvroRelSchemaProviderFactory.CFG_SOURCE_SCHEMA,
"testavro", "PAGEVIEW"), PageView.SCHEMA$.toString());

Expand Down
Expand Up @@ -504,7 +504,7 @@ public void testEndToEndUdf() throws Exception {
TestAvroSystemFactory.messages.clear();
Map<String, String> staticConfigs = SamzaSqlTestConfig.fetchStaticConfigsWithFactories(configs, numMessages);
String sql1 = "Insert into testavro.outputTopic(id, long_value) "
+ "select id, MyTest(id) as long_value from testavro.SIMPLE1";
+ "select id, BuildOutputRecord('key', 'value') as long_value from testavro.SIMPLE1";
List<String> sqlStmts = Collections.singletonList(sql1);
staticConfigs.put(SamzaSqlApplicationConfig.CFG_SQL_STMTS_JSON, JsonUtil.toJson(sqlStmts));
runApplication(new MapConfig(staticConfigs));
Expand All @@ -522,6 +522,26 @@ public void testEndToEndUdf() throws Exception {
IntStream.range(0, numMessages).map(udf::execute).boxed().collect(Collectors.toList()).equals(outMessages));
}

@Test
public void testEndToEndUdfWithDisabledArgCheck() throws Exception {
int numMessages = 20;
TestAvroSystemFactory.messages.clear();
Map<String, String> staticConfigs = SamzaSqlTestConfig.fetchStaticConfigsWithFactories(configs, numMessages);
String sql1 = "Insert into testavro.PROFILE1(id, address) "
+ "select id, BuildOutputRecord('key', GetNestedField(address, 'zip')) as long_value from testavro.PROFILE";
List<String> sqlStmts = Collections.singletonList(sql1);
staticConfigs.put(SamzaSqlApplicationConfig.CFG_SQL_STMTS_JSON, JsonUtil.toJson(sqlStmts));
runApplication(new MapConfig(staticConfigs));

LOG.info("output Messages " + TestAvroSystemFactory.messages);

List<Integer> outMessages = TestAvroSystemFactory.messages.stream()
.map(x -> Integer.valueOf(((GenericRecord) x.getMessage()).get("id").toString()))
.sorted()
.collect(Collectors.toList());
Assert.assertEquals(outMessages.size(), numMessages);
}

@Test
public void testEndToEndUdfPolymorphism() throws Exception {
int numMessages = 20;
Expand Down