diff --git a/.github/workflows/python_release.yml b/.github/workflows/python_release.yml new file mode 100644 index 000000000..3721e2368 --- /dev/null +++ b/.github/workflows/python_release.yml @@ -0,0 +1,25 @@ +name: Python Package +on: + release: + types: [created] + +jobs: + publishPythonZip: + runs-on: ubuntu-latest + steps: + - uses: actions/checkout@v2 + - name: Zip Python Udf + run: | + cd dagger-py-functions + zip -r python_udfs.zip udfs -x "*/__init__.py" + zip -jr data.zip data + zip -r dagger-py-functions.zip requirements.txt data.zip python_udfs.zip + - name: Upload Release + uses: ncipollo/release-action@v1 + with: + artifacts: dagger-py-functions/dagger-py-functions.zip + allowUpdates: true + omitNameDuringUpdate: true + omitBodyDuringUpdate: true + omitPrereleaseDuringUpdate: true + token: ${{ secrets.GITHUB_TOKEN }} \ No newline at end of file diff --git a/.github/workflows/python_validation.yml b/.github/workflows/python_validation.yml new file mode 100644 index 000000000..0a8b04c40 --- /dev/null +++ b/.github/workflows/python_validation.yml @@ -0,0 +1,27 @@ +name: Python Validation + +on: push + +jobs: + pythonValidation: + runs-on: ubuntu-latest + steps: + - uses: actions/checkout@v3 + - name: Set up Python 3.8 + uses: actions/setup-python@v3 + with: + python-version: '3.8' + - name: Install dependencies + run: | + python -m pip install --upgrade pip + pip install apache-flink==1.14.3 + cd dagger-py-functions + if [ -f requirements.txt ]; then pip install -r requirements.txt; fi + - name: Lint with flake8 + run: | + # stop the build if there are Python syntax errors or undefined names + flake8 . --count --select=E9,F63,F7,F82 --show-source --statistics + - name: Test with pytest + run: | + cd dagger-py-functions + pytest --disable-warnings \ No newline at end of file diff --git a/dagger-common/build.gradle b/dagger-common/build.gradle index a41ff942f..765a38dfb 100644 --- a/dagger-common/build.gradle +++ b/dagger-common/build.gradle @@ -61,12 +61,13 @@ dependencies { dependenciesCommonJar ('org.apache.hadoop:hadoop-client:2.8.3') { exclude module:"commons-cli" + exclude module:"commons-compress" } dependenciesCommonJar 'com.google.cloud.bigdataoss:gcs-connector:1.9.0-hadoop2' dependenciesCommonJar 'org.apache.flink:flink-metrics-dropwizard:' + flinkVersion dependenciesCommonJar 'org.apache.flink:flink-json:' + flinkVersion dependenciesCommonJar 'com.jayway.jsonpath:json-path:2.4.0' - dependenciesCommonJar 'io.odpf:stencil:0.1.6' + dependenciesCommonJar 'io.odpf:stencil:0.2.1' dependenciesCommonJar 'com.google.code.gson:gson:2.8.2' dependenciesCommonJar 'org.apache.parquet:parquet-column:1.12.2' diff --git a/dagger-core/env/local.properties b/dagger-core/env/local.properties index 6127bbae2..8d4372bf8 100644 --- a/dagger-core/env/local.properties +++ b/dagger-core/env/local.properties @@ -26,3 +26,7 @@ METRIC_TELEMETRY_ENABLE=true # == Others == FUNCTION_FACTORY_CLASSES=io.odpf.dagger.functions.udfs.factories.FunctionFactory FLINK_ROWTIME_ATTRIBUTE_NAME=rowtime + +# == Python Udf == +PYTHON_UDF_ENABLE=false +PYTHON_UDF_CONFIG={"PYTHON_FILES":"/path/to/files.zip", "PYTHON_REQUIREMENTS": "requirements.txt", "PYTHON_FN_EXECUTION_BUNDLE_SIZE": "1000"} \ No newline at end of file diff --git a/dagger-core/src/main/java/io/odpf/dagger/core/StreamManager.java b/dagger-core/src/main/java/io/odpf/dagger/core/StreamManager.java index e502c975c..e10c6aa4e 100644 --- a/dagger-core/src/main/java/io/odpf/dagger/core/StreamManager.java +++ b/dagger-core/src/main/java/io/odpf/dagger/core/StreamManager.java @@ -28,13 +28,18 @@ import io.odpf.dagger.core.sink.SinkOrchestrator; import io.odpf.dagger.core.source.StreamsFactory; import io.odpf.dagger.core.utils.Constants; +import io.odpf.dagger.functions.udfs.python.PythonUdfConfig; +import io.odpf.dagger.functions.udfs.python.PythonUdfManager; +import java.io.IOException; import java.lang.reflect.Constructor; import java.lang.reflect.InvocationTargetException; import java.time.Duration; import java.util.List; import static io.odpf.dagger.core.utils.Constants.*; +import static io.odpf.dagger.functions.common.Constants.PYTHON_UDF_ENABLE_DEFAULT; +import static io.odpf.dagger.functions.common.Constants.PYTHON_UDF_ENABLE_KEY; import static org.apache.flink.table.api.Expressions.$; /** @@ -138,7 +143,13 @@ private ApiExpression[] getApiExpressions(StreamInfo streamInfo) { * * @return the stream manager */ - public StreamManager registerFunctions() { + public StreamManager registerFunctions() throws IOException { + if (configuration.getBoolean(PYTHON_UDF_ENABLE_KEY, PYTHON_UDF_ENABLE_DEFAULT)) { + PythonUdfConfig pythonUdfConfig = PythonUdfConfig.parse(configuration); + PythonUdfManager pythonUdfManager = new PythonUdfManager(tableEnvironment, pythonUdfConfig); + pythonUdfManager.registerPythonFunctions(); + } + String[] functionFactoryClasses = configuration .getString(Constants.FUNCTION_FACTORY_CLASSES_KEY, Constants.FUNCTION_FACTORY_CLASSES_DEFAULT) .split(","); diff --git a/dagger-core/src/main/java/io/odpf/dagger/core/processors/common/EndpointHandler.java b/dagger-core/src/main/java/io/odpf/dagger/core/processors/common/EndpointHandler.java index 22694b3c8..6e818467e 100644 --- a/dagger-core/src/main/java/io/odpf/dagger/core/processors/common/EndpointHandler.java +++ b/dagger-core/src/main/java/io/odpf/dagger/core/processors/common/EndpointHandler.java @@ -1,7 +1,7 @@ package io.odpf.dagger.core.processors.common; import io.odpf.dagger.core.processors.ColumnNameManager; -import io.odpf.dagger.core.processors.types.SourceConfig; +import io.odpf.dagger.core.utils.Constants.ExternalPostProcessorVariableType; import org.apache.flink.streaming.api.functions.async.ResultFuture; import org.apache.flink.types.Row; @@ -29,7 +29,6 @@ */ public class EndpointHandler { private static final Logger LOGGER = LoggerFactory.getLogger(EndpointHandler.class.getName()); - private SourceConfig sourceConfig; private MeterStatsManager meterStatsManager; private ErrorReporter errorReporter; private String[] inputProtoClasses; @@ -48,13 +47,11 @@ public class EndpointHandler { * @param columnNameManager the column name manager * @param descriptorManager the descriptor manager */ - public EndpointHandler(SourceConfig sourceConfig, - MeterStatsManager meterStatsManager, + public EndpointHandler(MeterStatsManager meterStatsManager, ErrorReporter errorReporter, String[] inputProtoClasses, ColumnNameManager columnNameManager, DescriptorManager descriptorManager) { - this.sourceConfig = sourceConfig; this.meterStatsManager = meterStatsManager; this.errorReporter = errorReporter; this.inputProtoClasses = inputProtoClasses; @@ -63,19 +60,20 @@ public EndpointHandler(SourceConfig sourceConfig, } /** - * Get endpoint or query variables values. + * Get external post processor variables values. * * @param rowManager the row manager + * @param variableType the variable type + * @parm variables the variable list * @param resultFuture the result future * @return the array object */ - public Object[] getEndpointOrQueryVariablesValues(RowManager rowManager, ResultFuture resultFuture) { - String queryVariables = sourceConfig.getVariables(); - if (StringUtils.isEmpty(queryVariables)) { + public Object[] getVariablesValue(RowManager rowManager, ExternalPostProcessorVariableType variableType, String variables, ResultFuture resultFuture) { + if (StringUtils.isEmpty(variables)) { return new Object[0]; } - String[] requiredInputColumns = queryVariables.split(","); + String[] requiredInputColumns = variables.split(","); ArrayList inputColumnValues = new ArrayList<>(); if (descriptorMap == null) { descriptorMap = createDescriptorMap(requiredInputColumns, inputProtoClasses, resultFuture); @@ -84,7 +82,7 @@ public Object[] getEndpointOrQueryVariablesValues(RowManager rowManager, ResultF for (String inputColumnName : requiredInputColumns) { int inputColumnIndex = columnNameManager.getInputIndex(inputColumnName); if (inputColumnIndex == -1) { - throw new InvalidConfigurationException(String.format("Column '%s' not found as configured in the endpoint/query variable", inputColumnName)); + throw new InvalidConfigurationException(String.format("Column '%s' not found as configured in the '%s' variable", inputColumnName, variableType)); } Descriptors.FieldDescriptor fieldDescriptor = descriptorMap.get(inputColumnName); @@ -105,11 +103,12 @@ public Object[] getEndpointOrQueryVariablesValues(RowManager rowManager, ResultF * * @param resultFuture the result future * @param rowManager the row manager - * @param endpointVariablesValues the endpoint variables values + * @param variables the request/header variables + * @param variablesValue the variables value * @return the boolean */ - public boolean isQueryInvalid(ResultFuture resultFuture, RowManager rowManager, Object[] endpointVariablesValues) { - if (!StringUtils.isEmpty(sourceConfig.getVariables()) && (Arrays.asList(endpointVariablesValues).isEmpty() || Arrays.stream(endpointVariablesValues).allMatch(""::equals))) { + public boolean isQueryInvalid(ResultFuture resultFuture, RowManager rowManager, String variables, Object[] variablesValue) { + if (!StringUtils.isEmpty(variables) && (Arrays.asList(variablesValue).isEmpty() || Arrays.stream(variablesValue).allMatch(""::equals))) { LOGGER.warn("Could not populate any request variable. Skipping external calls"); meterStatsManager.markEvent(ExternalSourceAspects.EMPTY_INPUT); resultFuture.complete(singleton(rowManager.getAll())); diff --git a/dagger-core/src/main/java/io/odpf/dagger/core/processors/external/AsyncConnector.java b/dagger-core/src/main/java/io/odpf/dagger/core/processors/external/AsyncConnector.java index eb5622e79..50144d98d 100644 --- a/dagger-core/src/main/java/io/odpf/dagger/core/processors/external/AsyncConnector.java +++ b/dagger-core/src/main/java/io/odpf/dagger/core/processors/external/AsyncConnector.java @@ -161,7 +161,7 @@ public void open(Configuration configuration) throws Exception { meterStatsManager = new MeterStatsManager(getRuntimeContext().getMetricGroup(), true); } if (endpointHandler == null) { - endpointHandler = new EndpointHandler(sourceConfig, meterStatsManager, errorReporter, + endpointHandler = new EndpointHandler(meterStatsManager, errorReporter, schemaConfig.getInputProtoClasses(), schemaConfig.getColumnNameManager(), descriptorManager); } diff --git a/dagger-core/src/main/java/io/odpf/dagger/core/processors/external/es/EsAsyncConnector.java b/dagger-core/src/main/java/io/odpf/dagger/core/processors/external/es/EsAsyncConnector.java index 93e76a4bb..97c3a3c3f 100644 --- a/dagger-core/src/main/java/io/odpf/dagger/core/processors/external/es/EsAsyncConnector.java +++ b/dagger-core/src/main/java/io/odpf/dagger/core/processors/external/es/EsAsyncConnector.java @@ -9,8 +9,8 @@ import io.odpf.dagger.core.processors.external.AsyncConnector; import org.apache.flink.streaming.api.functions.async.ResultFuture; import org.apache.flink.types.Row; - import io.odpf.dagger.core.utils.Constants; +import io.odpf.dagger.core.utils.Constants.ExternalPostProcessorVariableType; import org.apache.http.HttpHost; import org.apache.http.auth.AuthScope; import org.apache.http.auth.UsernamePasswordCredentials; @@ -79,8 +79,8 @@ protected void createClient() { protected void process(Row input, ResultFuture resultFuture) { RowManager rowManager = new RowManager(input); Object[] endpointVariablesValues = getEndpointHandler() - .getEndpointOrQueryVariablesValues(rowManager, resultFuture); - if (getEndpointHandler().isQueryInvalid(resultFuture, rowManager, endpointVariablesValues)) { + .getVariablesValue(rowManager, ExternalPostProcessorVariableType.ENDPOINT_VARIABLE, esSourceConfig.getVariables(), resultFuture); + if (getEndpointHandler().isQueryInvalid(resultFuture, rowManager, esSourceConfig.getVariables(), endpointVariablesValues)) { return; } String esEndpoint = String.format(esSourceConfig.getPattern(), endpointVariablesValues); diff --git a/dagger-core/src/main/java/io/odpf/dagger/core/processors/external/grpc/GrpcAsyncConnector.java b/dagger-core/src/main/java/io/odpf/dagger/core/processors/external/grpc/GrpcAsyncConnector.java index 173be65f2..db28e5b23 100644 --- a/dagger-core/src/main/java/io/odpf/dagger/core/processors/external/grpc/GrpcAsyncConnector.java +++ b/dagger-core/src/main/java/io/odpf/dagger/core/processors/external/grpc/GrpcAsyncConnector.java @@ -20,6 +20,7 @@ import com.google.protobuf.Descriptors; import com.google.protobuf.DynamicMessage; import io.odpf.dagger.core.utils.Constants; +import io.odpf.dagger.core.utils.Constants.ExternalPostProcessorVariableType; import org.apache.commons.lang3.StringUtils; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -90,8 +91,8 @@ protected void process(Row input, ResultFuture resultFuture) throws Excepti RowManager rowManager = new RowManager(input); Object[] requestVariablesValues = getEndpointHandler() - .getEndpointOrQueryVariablesValues(rowManager, resultFuture); - if (getEndpointHandler().isQueryInvalid(resultFuture, rowManager, requestVariablesValues)) { + .getVariablesValue(rowManager, ExternalPostProcessorVariableType.REQUEST_VARIABLES, grpcSourceConfig.getVariables(), resultFuture); + if (getEndpointHandler().isQueryInvalid(resultFuture, rowManager, grpcSourceConfig.getVariables(), requestVariablesValues)) { return; } diff --git a/dagger-core/src/main/java/io/odpf/dagger/core/processors/external/http/HttpAsyncConnector.java b/dagger-core/src/main/java/io/odpf/dagger/core/processors/external/http/HttpAsyncConnector.java index fcb401904..5e6b786a3 100644 --- a/dagger-core/src/main/java/io/odpf/dagger/core/processors/external/http/HttpAsyncConnector.java +++ b/dagger-core/src/main/java/io/odpf/dagger/core/processors/external/http/HttpAsyncConnector.java @@ -15,6 +15,7 @@ import org.apache.flink.types.Row; import io.odpf.dagger.core.utils.Constants; +import io.odpf.dagger.core.utils.Constants.ExternalPostProcessorVariableType; import org.asynchttpclient.AsyncHttpClient; import org.asynchttpclient.BoundRequestBuilder; import org.slf4j.Logger; @@ -94,12 +95,14 @@ protected void process(Row input, ResultFuture resultFuture) { RowManager rowManager = new RowManager(input); Object[] requestVariablesValues = getEndpointHandler() - .getEndpointOrQueryVariablesValues(rowManager, resultFuture); - if (getEndpointHandler().isQueryInvalid(resultFuture, rowManager, requestVariablesValues)) { + .getVariablesValue(rowManager, ExternalPostProcessorVariableType.REQUEST_VARIABLES, httpSourceConfig.getRequestVariables(), resultFuture); + Object[] dynamicHeaderVariablesValues = getEndpointHandler() + .getVariablesValue(rowManager, ExternalPostProcessorVariableType.HEADER_VARIABLES, httpSourceConfig.getHeaderVariables(), resultFuture); + if (getEndpointHandler().isQueryInvalid(resultFuture, rowManager, httpSourceConfig.getRequestVariables(), requestVariablesValues) || getEndpointHandler().isQueryInvalid(resultFuture, rowManager, httpSourceConfig.getHeaderVariables(), dynamicHeaderVariablesValues)) { return; } - BoundRequestBuilder request = HttpRequestFactory.createRequest(httpSourceConfig, httpClient, requestVariablesValues); + BoundRequestBuilder request = HttpRequestFactory.createRequest(httpSourceConfig, httpClient, requestVariablesValues, dynamicHeaderVariablesValues); HttpResponseHandler httpResponseHandler = new HttpResponseHandler(httpSourceConfig, getMeterStatsManager(), rowManager, getColumnNameManager(), getOutputDescriptor(resultFuture), resultFuture, getErrorReporter(), new PostResponseTelemetry()); httpResponseHandler.startTimer(); diff --git a/dagger-core/src/main/java/io/odpf/dagger/core/processors/external/http/HttpSourceConfig.java b/dagger-core/src/main/java/io/odpf/dagger/core/processors/external/http/HttpSourceConfig.java index 60ac14d82..d30efc2ad 100644 --- a/dagger-core/src/main/java/io/odpf/dagger/core/processors/external/http/HttpSourceConfig.java +++ b/dagger-core/src/main/java/io/odpf/dagger/core/processors/external/http/HttpSourceConfig.java @@ -20,6 +20,8 @@ public class HttpSourceConfig implements Serializable, SourceConfig { private String verb; private String requestPattern; private String requestVariables; + private String headerPattern; + private String headerVariables; private String streamTimeout; private String connectTimeout; private boolean failOnErrors; @@ -40,21 +42,25 @@ public class HttpSourceConfig implements Serializable, SourceConfig { * @param verb the verb * @param requestPattern the request pattern * @param requestVariables the request variables + * @param headerPattern the dynamic header pattern + * @param headerVariables the header variables * @param streamTimeout the stream timeout * @param connectTimeout the connect timeout * @param failOnErrors the fail on errors * @param type the type * @param capacity the capacity - * @param headers the headers + * @param headers the static headers * @param outputMapping the output mapping * @param metricId the metric id * @param retainResponseType the retain response type */ - public HttpSourceConfig(String endpoint, String verb, String requestPattern, String requestVariables, String streamTimeout, String connectTimeout, boolean failOnErrors, String type, String capacity, Map headers, Map outputMapping, String metricId, boolean retainResponseType) { + public HttpSourceConfig(String endpoint, String verb, String requestPattern, String requestVariables, String headerPattern, String headerVariables, String streamTimeout, String connectTimeout, boolean failOnErrors, String type, String capacity, Map headers, Map outputMapping, String metricId, boolean retainResponseType) { this.endpoint = endpoint; this.verb = verb; this.requestPattern = requestPattern; this.requestVariables = requestVariables; + this.headerPattern = headerPattern; + this.headerVariables = headerVariables; this.streamTimeout = streamTimeout; this.connectTimeout = connectTimeout; this.failOnErrors = failOnErrors; @@ -102,6 +108,24 @@ public String getRequestVariables() { return requestVariables; } + /** + * Gets header pattern. + * + * @return the header pattern + */ + public String getHeaderPattern() { + return headerPattern; + } + + /** + * Gets header Variable. + * + * @return the header Variable + */ + public String getHeaderVariables() { + return headerVariables; + } + @Override public String getPattern() { return requestPattern; @@ -208,11 +232,11 @@ public boolean equals(Object o) { return false; } HttpSourceConfig that = (HttpSourceConfig) o; - return failOnErrors == that.failOnErrors && retainResponseType == that.retainResponseType && Objects.equals(endpoint, that.endpoint) && Objects.equals(verb, that.verb) && Objects.equals(requestPattern, that.requestPattern) && Objects.equals(requestVariables, that.requestVariables) && Objects.equals(streamTimeout, that.streamTimeout) && Objects.equals(connectTimeout, that.connectTimeout) && Objects.equals(type, that.type) && Objects.equals(capacity, that.capacity) && Objects.equals(headers, that.headers) && Objects.equals(outputMapping, that.outputMapping) && Objects.equals(metricId, that.metricId); + return failOnErrors == that.failOnErrors && retainResponseType == that.retainResponseType && Objects.equals(endpoint, that.endpoint) && Objects.equals(verb, that.verb) && Objects.equals(requestPattern, that.requestPattern) && Objects.equals(requestVariables, that.requestVariables) && Objects.equals(headerPattern, that.headerPattern) && Objects.equals(headerVariables, that.headerVariables) && Objects.equals(streamTimeout, that.streamTimeout) && Objects.equals(connectTimeout, that.connectTimeout) && Objects.equals(type, that.type) && Objects.equals(capacity, that.capacity) && Objects.equals(headers, that.headers) && Objects.equals(outputMapping, that.outputMapping) && Objects.equals(metricId, that.metricId); } @Override public int hashCode() { - return Objects.hash(endpoint, verb, requestPattern, requestVariables, streamTimeout, connectTimeout, failOnErrors, type, capacity, headers, outputMapping, metricId, retainResponseType); + return Objects.hash(endpoint, verb, requestPattern, requestVariables, headerPattern, headerVariables, streamTimeout, connectTimeout, failOnErrors, type, capacity, headers, outputMapping, metricId, retainResponseType); } } diff --git a/dagger-core/src/main/java/io/odpf/dagger/core/processors/external/http/request/HttpGetRequestHandler.java b/dagger-core/src/main/java/io/odpf/dagger/core/processors/external/http/request/HttpGetRequestHandler.java index 58c033d78..cd7a697fe 100644 --- a/dagger-core/src/main/java/io/odpf/dagger/core/processors/external/http/request/HttpGetRequestHandler.java +++ b/dagger-core/src/main/java/io/odpf/dagger/core/processors/external/http/request/HttpGetRequestHandler.java @@ -1,9 +1,16 @@ package io.odpf.dagger.core.processors.external.http.request; +import com.google.gson.Gson; +import io.netty.util.internal.StringUtil; +import io.odpf.dagger.core.exception.InvalidConfigurationException; import io.odpf.dagger.core.processors.external.http.HttpSourceConfig; import org.asynchttpclient.AsyncHttpClient; import org.asynchttpclient.BoundRequestBuilder; +import java.util.HashMap; +import java.util.Map; +import java.util.UnknownFormatConversionException; + /** * The Http get request handler. */ @@ -11,6 +18,7 @@ public class HttpGetRequestHandler implements HttpRequestHandler { private HttpSourceConfig httpSourceConfig; private AsyncHttpClient httpClient; private Object[] requestVariablesValues; + private Object[] dynamicHeaderVariablesValues; /** * Instantiates a new Http get request handler. @@ -19,10 +27,11 @@ public class HttpGetRequestHandler implements HttpRequestHandler { * @param httpClient the http client * @param requestVariablesValues the request variables values */ - public HttpGetRequestHandler(HttpSourceConfig httpSourceConfig, AsyncHttpClient httpClient, Object[] requestVariablesValues) { + public HttpGetRequestHandler(HttpSourceConfig httpSourceConfig, AsyncHttpClient httpClient, Object[] requestVariablesValues, Object[] dynamicHeaderVariablesValues) { this.httpSourceConfig = httpSourceConfig; this.httpClient = httpClient; this.requestVariablesValues = requestVariablesValues; + this.dynamicHeaderVariablesValues = dynamicHeaderVariablesValues; } @Override @@ -31,7 +40,18 @@ public BoundRequestBuilder create() { String endpoint = httpSourceConfig.getEndpoint(); String requestEndpoint = endpoint + endpointPath; BoundRequestBuilder getRequest = httpClient.prepareGet(requestEndpoint); - return addHeaders(getRequest, httpSourceConfig.getHeaders()); + Map headers = httpSourceConfig.getHeaders(); + if (!StringUtil.isNullOrEmpty(httpSourceConfig.getHeaderPattern())) { + try { + String dynamicHeader = String.format(httpSourceConfig.getHeaderPattern(), dynamicHeaderVariablesValues); + headers.putAll(new Gson().fromJson(dynamicHeader, HashMap.class)); + } catch (UnknownFormatConversionException e) { + throw new InvalidConfigurationException(String.format("pattern config '%s' is invalid", httpSourceConfig.getHeaderPattern())); + } catch (IllegalArgumentException e) { + throw new InvalidConfigurationException(String.format("pattern config '%s' is incompatible with the variable config '%s'", httpSourceConfig.getHeaderPattern(), httpSourceConfig.getHeaderVariables())); + } + } + return addHeaders(getRequest, headers); } @Override diff --git a/dagger-core/src/main/java/io/odpf/dagger/core/processors/external/http/request/HttpPostRequestHandler.java b/dagger-core/src/main/java/io/odpf/dagger/core/processors/external/http/request/HttpPostRequestHandler.java index 5dc7b9540..5ca1c1ad3 100644 --- a/dagger-core/src/main/java/io/odpf/dagger/core/processors/external/http/request/HttpPostRequestHandler.java +++ b/dagger-core/src/main/java/io/odpf/dagger/core/processors/external/http/request/HttpPostRequestHandler.java @@ -1,9 +1,16 @@ package io.odpf.dagger.core.processors.external.http.request; +import com.google.gson.Gson; +import io.netty.util.internal.StringUtil; +import io.odpf.dagger.core.exception.InvalidConfigurationException; import io.odpf.dagger.core.processors.external.http.HttpSourceConfig; import org.asynchttpclient.AsyncHttpClient; import org.asynchttpclient.BoundRequestBuilder; +import java.util.HashMap; +import java.util.Map; +import java.util.UnknownFormatConversionException; + /** * The Http post request handler. */ @@ -11,7 +18,7 @@ public class HttpPostRequestHandler implements HttpRequestHandler { private HttpSourceConfig httpSourceConfig; private AsyncHttpClient httpClient; private Object[] requestVariablesValues; - + private Object[] dynamicHeaderVariablesValues; /** * Instantiates a new Http post request handler. * @@ -19,10 +26,11 @@ public class HttpPostRequestHandler implements HttpRequestHandler { * @param httpClient the http client * @param requestVariablesValues the request variables values */ - public HttpPostRequestHandler(HttpSourceConfig httpSourceConfig, AsyncHttpClient httpClient, Object[] requestVariablesValues) { + public HttpPostRequestHandler(HttpSourceConfig httpSourceConfig, AsyncHttpClient httpClient, Object[] requestVariablesValues, Object[] dynamicHeaderVariablesValues) { this.httpSourceConfig = httpSourceConfig; this.httpClient = httpClient; this.requestVariablesValues = requestVariablesValues; + this.dynamicHeaderVariablesValues = dynamicHeaderVariablesValues; } @Override @@ -32,7 +40,18 @@ public BoundRequestBuilder create() { BoundRequestBuilder postRequest = httpClient .preparePost(endpoint) .setBody(requestBody); - return addHeaders(postRequest, httpSourceConfig.getHeaders()); + Map headers = httpSourceConfig.getHeaders(); + if (!StringUtil.isNullOrEmpty(httpSourceConfig.getHeaderPattern())) { + try { + String dynamicHeader = String.format(httpSourceConfig.getHeaderPattern(), dynamicHeaderVariablesValues); + headers.putAll(new Gson().fromJson(dynamicHeader, HashMap.class)); + } catch (UnknownFormatConversionException e) { + throw new InvalidConfigurationException(String.format("pattern config '%s' is invalid", httpSourceConfig.getHeaderPattern())); + } catch (IllegalArgumentException e) { + throw new InvalidConfigurationException(String.format("pattern config '%s' is incompatible with the variable config '%s'", httpSourceConfig.getHeaderPattern(), httpSourceConfig.getHeaderVariables())); + } + } + return addHeaders(postRequest, headers); } @Override diff --git a/dagger-core/src/main/java/io/odpf/dagger/core/processors/external/http/request/HttpRequestFactory.java b/dagger-core/src/main/java/io/odpf/dagger/core/processors/external/http/request/HttpRequestFactory.java index d45023a4c..1e61129ea 100644 --- a/dagger-core/src/main/java/io/odpf/dagger/core/processors/external/http/request/HttpRequestFactory.java +++ b/dagger-core/src/main/java/io/odpf/dagger/core/processors/external/http/request/HttpRequestFactory.java @@ -19,11 +19,11 @@ public class HttpRequestFactory { * @param requestVariablesValues the request variables values * @return the bound request builder */ - public static BoundRequestBuilder createRequest(HttpSourceConfig httpSourceConfig, AsyncHttpClient httpClient, Object[] requestVariablesValues) { + public static BoundRequestBuilder createRequest(HttpSourceConfig httpSourceConfig, AsyncHttpClient httpClient, Object[] requestVariablesValues, Object[] headerVariablesValues) { ArrayList httpRequestHandlers = new ArrayList<>(); - httpRequestHandlers.add(new HttpPostRequestHandler(httpSourceConfig, httpClient, requestVariablesValues)); - httpRequestHandlers.add(new HttpGetRequestHandler(httpSourceConfig, httpClient, requestVariablesValues)); + httpRequestHandlers.add(new HttpPostRequestHandler(httpSourceConfig, httpClient, requestVariablesValues, headerVariablesValues)); + httpRequestHandlers.add(new HttpGetRequestHandler(httpSourceConfig, httpClient, requestVariablesValues, headerVariablesValues)); HttpRequestHandler httpRequestHandler = httpRequestHandlers .stream() diff --git a/dagger-core/src/main/java/io/odpf/dagger/core/processors/external/pg/PgAsyncConnector.java b/dagger-core/src/main/java/io/odpf/dagger/core/processors/external/pg/PgAsyncConnector.java index 46595164e..c1b83d502 100644 --- a/dagger-core/src/main/java/io/odpf/dagger/core/processors/external/pg/PgAsyncConnector.java +++ b/dagger-core/src/main/java/io/odpf/dagger/core/processors/external/pg/PgAsyncConnector.java @@ -10,6 +10,7 @@ import io.odpf.dagger.core.processors.external.AsyncConnector; import io.odpf.dagger.core.metrics.aspects.ExternalSourceAspects; import io.odpf.dagger.core.utils.Constants; +import io.odpf.dagger.core.utils.Constants.ExternalPostProcessorVariableType; import io.vertx.core.Vertx; import io.vertx.core.VertxOptions; import io.vertx.pgclient.PgConnectOptions; @@ -89,8 +90,8 @@ public void process(Row input, ResultFuture resultFuture) { RowManager rowManager = new RowManager(input); Object[] queryVariablesValues = getEndpointHandler() - .getEndpointOrQueryVariablesValues(rowManager, resultFuture); - if (getEndpointHandler().isQueryInvalid(resultFuture, rowManager, queryVariablesValues)) { + .getVariablesValue(rowManager, ExternalPostProcessorVariableType.QUERY_VARIABLES, pgSourceConfig.getVariables(), resultFuture); + if (getEndpointHandler().isQueryInvalid(resultFuture, rowManager, pgSourceConfig.getVariables(), queryVariablesValues)) { return; } diff --git a/dagger-core/src/main/java/io/odpf/dagger/core/utils/Constants.java b/dagger-core/src/main/java/io/odpf/dagger/core/utils/Constants.java index 2bc2b7bed..892aed002 100644 --- a/dagger-core/src/main/java/io/odpf/dagger/core/utils/Constants.java +++ b/dagger-core/src/main/java/io/odpf/dagger/core/utils/Constants.java @@ -165,4 +165,5 @@ public class Constants { public static final long MAX_EVENT_LOOP_EXECUTE_TIME_DEFAULT = 10000; public static final int LONGBOW_OUTPUT_ADDITIONAL_ARITY = 3; + public enum ExternalPostProcessorVariableType { REQUEST_VARIABLES, HEADER_VARIABLES, QUERY_VARIABLES, ENDPOINT_VARIABLE }; } diff --git a/dagger-core/src/test/java/io/odpf/dagger/core/processors/PostProcessorConfigTest.java b/dagger-core/src/test/java/io/odpf/dagger/core/processors/PostProcessorConfigTest.java index a34e757d5..8c5ab8fa8 100644 --- a/dagger-core/src/test/java/io/odpf/dagger/core/processors/PostProcessorConfigTest.java +++ b/dagger-core/src/test/java/io/odpf/dagger/core/processors/PostProcessorConfigTest.java @@ -64,7 +64,7 @@ public void shouldReturnHttpExternalSourceConfig() { outputMapping = new OutputMapping("$.data.tensor.values[0]"); outputMappings.put("surge_factor", outputMapping); - HttpSourceConfig httpSourceConfig = new HttpSourceConfig("http://localhost:8000", "post", null, null, "5000", "5000", true, null, null, headerMap, outputMappings, null, false); + HttpSourceConfig httpSourceConfig = new HttpSourceConfig("http://localhost:8000", "post", null, null, null, null, "5000", "5000", true, null, null, headerMap, outputMappings, null, false); assertEquals(httpSourceConfig, defaultPostProcessorConfig.getExternalSource().getHttpConfig().get(0)); } @@ -120,7 +120,7 @@ public void shouldBeEmptyWhenNoneOfTheConfigsExist() { @Test public void shouldNotBeEmptyWhenExternalSourceHasHttpConfigExist() { ArrayList http = new ArrayList<>(); - http.add(new HttpSourceConfig("", "", "", "", "", "", false, "", "", new HashMap<>(), new HashMap<>(), "metricId_01", false)); + http.add(new HttpSourceConfig("", "", "", "", "", "", "", "", false, "", "", new HashMap<>(), new HashMap<>(), "metricId_01", false)); ArrayList es = new ArrayList<>(); ArrayList pg = new ArrayList<>(); ExternalSourceConfig externalSourceConfig = new ExternalSourceConfig(http, es, pg, new ArrayList<>()); diff --git a/dagger-core/src/test/java/io/odpf/dagger/core/processors/common/EndpointHandlerTest.java b/dagger-core/src/test/java/io/odpf/dagger/core/processors/common/EndpointHandlerTest.java index e103815c6..7cb27945e 100644 --- a/dagger-core/src/test/java/io/odpf/dagger/core/processors/common/EndpointHandlerTest.java +++ b/dagger-core/src/test/java/io/odpf/dagger/core/processors/common/EndpointHandlerTest.java @@ -1,5 +1,6 @@ package io.odpf.dagger.core.processors.common; +import io.odpf.dagger.core.utils.Constants.ExternalPostProcessorVariableType; import io.odpf.stencil.StencilClientFactory; import io.odpf.stencil.client.StencilClient; import io.odpf.dagger.common.core.StencilClientOrchestrator; @@ -69,10 +70,10 @@ public void shouldReturnEndpointQueryVariableValuesForPrimitiveDataFromDescripto row.setField(1, new Row(1)); RowManager rowManager = new RowManager(row); - endpointHandler = new EndpointHandler(sourceConfig, meterStatsManager, errorReporter, + endpointHandler = new EndpointHandler(meterStatsManager, errorReporter, inputProtoClasses, getColumnNameManager(new String[] {"order_number", "customer_id"}), descriptorManager); Object[] endpointOrQueryVariablesValues = endpointHandler - .getEndpointOrQueryVariablesValues(rowManager, resultFuture); + .getVariablesValue(rowManager, ExternalPostProcessorVariableType.REQUEST_VARIABLES, sourceConfig.getVariables(), resultFuture); assertArrayEquals(endpointOrQueryVariablesValues, new Object[] {"123456"}); } @@ -88,10 +89,10 @@ public void shouldReturnEndpointQueryVariableValuesForPrimitiveDataIfInputColumn row.setField(1, new Row(1)); RowManager rowManager = new RowManager(row); - endpointHandler = new EndpointHandler(sourceConfig, meterStatsManager, errorReporter, + endpointHandler = new EndpointHandler(meterStatsManager, errorReporter, inputProtoClasses, getColumnNameManager(new String[] {"order_number", "id"}), descriptorManager); Object[] endpointOrQueryVariablesValues = endpointHandler - .getEndpointOrQueryVariablesValues(rowManager, resultFuture); + .getVariablesValue(rowManager, ExternalPostProcessorVariableType.REQUEST_VARIABLES, sourceConfig.getVariables(), resultFuture); assertArrayEquals(endpointOrQueryVariablesValues, new Object[] {"123456"}); } @@ -116,10 +117,10 @@ public void shouldReturnJsonValueOfEndpointQueryValuesInCaseOfArray() { row.setField(1, new Row(1)); RowManager rowManager = new RowManager(row); - endpointHandler = new EndpointHandler(sourceConfig, meterStatsManager, errorReporter, + endpointHandler = new EndpointHandler(meterStatsManager, errorReporter, inputProtoClasses, getColumnNameManager(new String[] {"order_number", "test_enums"}), descriptorManager); Object[] endpointOrQueryVariablesValues = endpointHandler - .getEndpointOrQueryVariablesValues(rowManager, resultFuture); + .getVariablesValue(rowManager, ExternalPostProcessorVariableType.REQUEST_VARIABLES, sourceConfig.getVariables(), resultFuture); assertArrayEquals(new Object[] {"[\"+I[UNKNOWN]\",\"+I[TYPE1]\"]"}, endpointOrQueryVariablesValues); } @@ -141,10 +142,10 @@ public void shouldReturnJsonValueOfEndpointQueryValuesIncaseOfComplexDatatype() row.setField(1, new Row(1)); RowManager rowManager = new RowManager(row); - endpointHandler = new EndpointHandler(sourceConfig, meterStatsManager, errorReporter, + endpointHandler = new EndpointHandler(meterStatsManager, errorReporter, inputProtoClasses, getColumnNameManager(new String[] {"order_number", "driver_pickup_location"}), descriptorManager); Object[] endpointOrQueryVariablesValues = endpointHandler - .getEndpointOrQueryVariablesValues(rowManager, resultFuture); + .getVariablesValue(rowManager, ExternalPostProcessorVariableType.REQUEST_VARIABLES, "driver_pickup_location", resultFuture); assertArrayEquals(endpointOrQueryVariablesValues, new Object[] {"{\"name\":\"test_driver\",\"address\":null,\"latitude\":172.5,\"longitude\":175.5,\"type\":null,\"note\":null,\"place_id\":null,\"accuracy_meter\":null,\"gate_id\":null}"}); } @@ -161,10 +162,10 @@ public void shouldReturnEndpointQueryVariableValuesForPrimitiveDataFromDescripto row.setField(1, new Row(1)); RowManager rowManager = new RowManager(row); - endpointHandler = new EndpointHandler(sourceConfig, meterStatsManager, errorReporter, + endpointHandler = new EndpointHandler(meterStatsManager, errorReporter, inputProtoClasses, getColumnNameManager(new String[] {"order_number", "customer_id"}), descriptorManager); Object[] endpointOrQueryVariablesValues = endpointHandler - .getEndpointOrQueryVariablesValues(rowManager, resultFuture); + .getVariablesValue(rowManager, ExternalPostProcessorVariableType.REQUEST_VARIABLES, sourceConfig.getVariables(), resultFuture); assertArrayEquals(endpointOrQueryVariablesValues, new Object[] {"123456"}); } @@ -182,10 +183,10 @@ public void shouldInferEndpointVariablesFromTheCorrectStreams() { row.setField(1, new Row(1)); RowManager rowManager = new RowManager(row); - endpointHandler = new EndpointHandler(sourceConfig, meterStatsManager, errorReporter, + endpointHandler = new EndpointHandler(meterStatsManager, errorReporter, inputProtoClasses, getColumnNameManager(new String[] {"order_number", "customer_url"}), descriptorManager); Object[] endpointOrQueryVariablesValues = endpointHandler - .getEndpointOrQueryVariablesValues(rowManager, resultFuture); + .getVariablesValue(rowManager, ExternalPostProcessorVariableType.REQUEST_VARIABLES, sourceConfig.getVariables(), resultFuture); assertArrayEquals(endpointOrQueryVariablesValues, new Object[] {"test_order_number", "customer_url_test"}); } @@ -203,16 +204,16 @@ public void shouldReturnEmptyObjectIfNoQueryVariables() { row.setField(1, new Row(1)); RowManager rowManager = new RowManager(row); - endpointHandler = new EndpointHandler(sourceConfig, meterStatsManager, errorReporter, + endpointHandler = new EndpointHandler(meterStatsManager, errorReporter, inputProtoClasses, getColumnNameManager(new String[] {"order_number", "customer_url"}), descriptorManager); Object[] endpointOrQueryVariablesValues = endpointHandler - .getEndpointOrQueryVariablesValues(rowManager, resultFuture); + .getVariablesValue(rowManager, ExternalPostProcessorVariableType.REQUEST_VARIABLES, sourceConfig.getVariables(), resultFuture); assertArrayEquals(endpointOrQueryVariablesValues, new Object[] {}); } @Test - public void shouldThrowErrorIfVariablesAreNotProperlyConfigures() { + public void shouldThrowErrorIfRequestVariablesAreNotProperlyConfigures() { when(sourceConfig.getVariables()).thenReturn("czx"); inputProtoClasses = new String[] {"io.odpf.dagger.consumer.TestBookingLogMessage", "io.odpf.dagger.consumer.TestBookingLogMessage"}; @@ -224,11 +225,11 @@ public void shouldThrowErrorIfVariablesAreNotProperlyConfigures() { row.setField(1, new Row(1)); RowManager rowManager = new RowManager(row); - endpointHandler = new EndpointHandler(sourceConfig, meterStatsManager, errorReporter, + endpointHandler = new EndpointHandler(meterStatsManager, errorReporter, inputProtoClasses, getColumnNameManager(new String[] {"order_number", "customer_url"}), descriptorManager); InvalidConfigurationException exception = assertThrows(InvalidConfigurationException.class, () -> endpointHandler - .getEndpointOrQueryVariablesValues(rowManager, resultFuture)); - assertEquals("Column 'czx' not found as configured in the endpoint/query variable", exception.getMessage()); + .getVariablesValue(rowManager, ExternalPostProcessorVariableType.REQUEST_VARIABLES, sourceConfig.getVariables(), resultFuture)); + assertEquals("Column 'czx' not found as configured in the 'REQUEST_VARIABLES' variable", exception.getMessage()); } @Test @@ -249,11 +250,11 @@ public void shouldThrowErrorIfInputProtoNotFound() { row.setField(1, new Row(1)); RowManager rowManager = new RowManager(row); - endpointHandler = new EndpointHandler(sourceConfig, meterStatsManager, errorReporter, + endpointHandler = new EndpointHandler(meterStatsManager, errorReporter, inputProtoClasses, getColumnNameManager(new String[] {"order_number", "driver_pickup_location"}), descriptorManager); assertThrows(NullPointerException.class, - () -> endpointHandler.getEndpointOrQueryVariablesValues(rowManager, resultFuture)); + () -> endpointHandler.getVariablesValue(rowManager, ExternalPostProcessorVariableType.REQUEST_VARIABLES, sourceConfig.getVariables(), resultFuture)); verify(errorReporter, times(1)).reportFatalException(any(DescriptorNotFoundException.class)); verify(resultFuture, times(1)).completeExceptionally(any(DescriptorNotFoundException.class)); } @@ -270,12 +271,12 @@ public void shouldCheckIfQueryIsValid() { row.setField(1, new Row(1)); RowManager rowManager = new RowManager(row); - endpointHandler = new EndpointHandler(sourceConfig, meterStatsManager, errorReporter, + endpointHandler = new EndpointHandler(meterStatsManager, errorReporter, inputProtoClasses, getColumnNameManager(new String[] {"order_number", "customer_id"}), descriptorManager); Object[] endpointOrQueryVariablesValues = endpointHandler - .getEndpointOrQueryVariablesValues(rowManager, resultFuture); + .getVariablesValue(rowManager, ExternalPostProcessorVariableType.REQUEST_VARIABLES, sourceConfig.getVariables(), resultFuture); - boolean queryInvalid = endpointHandler.isQueryInvalid(resultFuture, rowManager, endpointOrQueryVariablesValues); + boolean queryInvalid = endpointHandler.isQueryInvalid(resultFuture, rowManager, sourceConfig.getVariables(), endpointOrQueryVariablesValues); assertFalse(queryInvalid); } @@ -291,12 +292,12 @@ public void shouldCheckIfQueryIsInValidInCaseOfSingeEmptyVariableValueForSingleF row.setField(1, new Row(1)); RowManager rowManager = new RowManager(row); - endpointHandler = new EndpointHandler(sourceConfig, meterStatsManager, errorReporter, + endpointHandler = new EndpointHandler(meterStatsManager, errorReporter, inputProtoClasses, getColumnNameManager(new String[] {"order_number", "customer_id"}), descriptorManager); Object[] endpointOrQueryVariablesValues = endpointHandler - .getEndpointOrQueryVariablesValues(rowManager, resultFuture); + .getVariablesValue(rowManager, ExternalPostProcessorVariableType.REQUEST_VARIABLES, sourceConfig.getVariables(), resultFuture); - boolean queryInvalid = endpointHandler.isQueryInvalid(resultFuture, rowManager, endpointOrQueryVariablesValues); + boolean queryInvalid = endpointHandler.isQueryInvalid(resultFuture, rowManager, sourceConfig.getVariables(), endpointOrQueryVariablesValues); assertTrue(queryInvalid); verify(resultFuture, times(1)).complete(any()); verify(meterStatsManager, times(1)).markEvent(ExternalSourceAspects.EMPTY_INPUT); @@ -315,12 +316,12 @@ public void shouldCheckIfQueryIsValidInCaseOfSomeVariableValue() { row.setField(1, new Row(1)); RowManager rowManager = new RowManager(row); - endpointHandler = new EndpointHandler(sourceConfig, meterStatsManager, errorReporter, + endpointHandler = new EndpointHandler(meterStatsManager, errorReporter, inputProtoClasses, getColumnNameManager(new String[] {"order_number", "customer_id"}), descriptorManager); Object[] endpointOrQueryVariablesValues = endpointHandler - .getEndpointOrQueryVariablesValues(rowManager, resultFuture); + .getVariablesValue(rowManager, ExternalPostProcessorVariableType.REQUEST_VARIABLES, sourceConfig.getVariables(), resultFuture); - boolean queryInvalid = endpointHandler.isQueryInvalid(resultFuture, rowManager, endpointOrQueryVariablesValues); + boolean queryInvalid = endpointHandler.isQueryInvalid(resultFuture, rowManager, sourceConfig.getVariables(), endpointOrQueryVariablesValues); assertFalse(queryInvalid); } @@ -337,12 +338,12 @@ public void shouldCheckIfQueryIsInvalidInCaseOfAllVariableValues() { row.setField(1, new Row(1)); RowManager rowManager = new RowManager(row); - endpointHandler = new EndpointHandler(sourceConfig, meterStatsManager, errorReporter, + endpointHandler = new EndpointHandler(meterStatsManager, errorReporter, inputProtoClasses, getColumnNameManager(new String[] {"order_number", "customer_id"}), descriptorManager); Object[] endpointOrQueryVariablesValues = endpointHandler - .getEndpointOrQueryVariablesValues(rowManager, resultFuture); + .getVariablesValue(rowManager, ExternalPostProcessorVariableType.REQUEST_VARIABLES, sourceConfig.getVariables(), resultFuture); - boolean queryInvalid = endpointHandler.isQueryInvalid(resultFuture, rowManager, endpointOrQueryVariablesValues); + boolean queryInvalid = endpointHandler.isQueryInvalid(resultFuture, rowManager, sourceConfig.getVariables(), endpointOrQueryVariablesValues); assertTrue(queryInvalid); verify(meterStatsManager, times(1)).markEvent(ExternalSourceAspects.EMPTY_INPUT); } diff --git a/dagger-core/src/test/java/io/odpf/dagger/core/processors/external/ExternalPostProcessorTest.java b/dagger-core/src/test/java/io/odpf/dagger/core/processors/external/ExternalPostProcessorTest.java index 0205252ed..f31f3c568 100644 --- a/dagger-core/src/test/java/io/odpf/dagger/core/processors/external/ExternalPostProcessorTest.java +++ b/dagger-core/src/test/java/io/odpf/dagger/core/processors/external/ExternalPostProcessorTest.java @@ -72,7 +72,7 @@ public void setup() { HashMap httpColumnNames = new HashMap<>(); httpColumnNames.put("http_field_1", new OutputMapping("")); httpColumnNames.put("http_field_2", new OutputMapping("")); - HttpSourceConfig httpSourceConfig = new HttpSourceConfig("endpoint", "POST", "/some/patttern/%s", "variable", "123", "234", false, "type", "20", new HashMap<>(), httpColumnNames, "metricId_01", false); + HttpSourceConfig httpSourceConfig = new HttpSourceConfig("endpoint", "POST", "/some/patttern/%s", "variable", "", "", "123", "234", false, "type", "20", new HashMap<>(), httpColumnNames, "metricId_01", false); HashMap esOutputMapping = new HashMap<>(); esOutputMapping.put("es_field_1", new OutputMapping("")); EsSourceConfig esSourceConfig = new EsSourceConfig("host", "port", "", "", "endpointPattern", @@ -135,7 +135,7 @@ public void shouldProcessWithRightConfiguration() { outputMapping.put("order_id", new OutputMapping("path")); List httpSourceConfigs = new ArrayList<>(); - HttpSourceConfig httpSourceConfig = new HttpSourceConfig("endpoint", "POST", "/some/patttern/%s", "variable", "123", "234", false, "type", "20", new HashMap<>(), outputMapping, "metricId_01", false); + HttpSourceConfig httpSourceConfig = new HttpSourceConfig("endpoint", "POST", "/some/patttern/%s", "variable", "", "", "123", "234", false, "type", "20", new HashMap<>(), outputMapping, "metricId_01", false); httpSourceConfigs.add(httpSourceConfig); List esSourceConfigs = new ArrayList<>(); diff --git a/dagger-core/src/test/java/io/odpf/dagger/core/processors/external/ExternalSourceConfigTest.java b/dagger-core/src/test/java/io/odpf/dagger/core/processors/external/ExternalSourceConfigTest.java index c3adf236f..177859b60 100644 --- a/dagger-core/src/test/java/io/odpf/dagger/core/processors/external/ExternalSourceConfigTest.java +++ b/dagger-core/src/test/java/io/odpf/dagger/core/processors/external/ExternalSourceConfigTest.java @@ -29,7 +29,7 @@ public void setUp() { HashMap httpOutputMapping = new HashMap<>(); httpOutputMapping.put("http_field_1", new OutputMapping("")); httpOutputMapping.put("http_field_2", new OutputMapping("")); - HttpSourceConfig httpSourceConfig = new HttpSourceConfig("endpoint", "POST", "/some/patttern/%s", "variable", "123", "234", false, "type", "20", new HashMap<>(), httpOutputMapping, "metricId_01", false); + HttpSourceConfig httpSourceConfig = new HttpSourceConfig("endpoint", "POST", "/some/patttern/%s", "variable", "", "", "123", "234", false, "type", "20", new HashMap<>(), httpOutputMapping, "metricId_01", false); http = new ArrayList<>(); http.add(httpSourceConfig); es = new ArrayList<>(); diff --git a/dagger-core/src/test/java/io/odpf/dagger/core/processors/external/es/EsAsyncConnectorTest.java b/dagger-core/src/test/java/io/odpf/dagger/core/processors/external/es/EsAsyncConnectorTest.java index ade3dd94c..e8360543d 100644 --- a/dagger-core/src/test/java/io/odpf/dagger/core/processors/external/es/EsAsyncConnectorTest.java +++ b/dagger-core/src/test/java/io/odpf/dagger/core/processors/external/es/EsAsyncConnectorTest.java @@ -184,7 +184,7 @@ public void shouldNotEnrichOutputWhenEndpointVariableIsInvalid() throws Exceptio ArgumentCaptor invalidConfigurationExceptionCaptor = ArgumentCaptor.forClass(InvalidConfigurationException.class); ArgumentCaptor reportExceptionCaptor = ArgumentCaptor.forClass(InvalidConfigurationException.class); - String expectedExceptionMessage = "Column 'invalid_variable' not found as configured in the endpoint/query variable"; + String expectedExceptionMessage = "Column 'invalid_variable' not found as configured in the 'ENDPOINT_VARIABLE' variable"; verify(resultFuture, times(1)).completeExceptionally(invalidConfigurationExceptionCaptor.capture()); assertEquals(expectedExceptionMessage, invalidConfigurationExceptionCaptor.getValue().getMessage()); diff --git a/dagger-core/src/test/java/io/odpf/dagger/core/processors/external/grpc/GrpcAsyncConnectorTest.java b/dagger-core/src/test/java/io/odpf/dagger/core/processors/external/grpc/GrpcAsyncConnectorTest.java index 7b9c04857..032c2c34f 100644 --- a/dagger-core/src/test/java/io/odpf/dagger/core/processors/external/grpc/GrpcAsyncConnectorTest.java +++ b/dagger-core/src/test/java/io/odpf/dagger/core/processors/external/grpc/GrpcAsyncConnectorTest.java @@ -206,12 +206,12 @@ public void shouldCompleteExceptionallyWhenEndpointVariableIsInvalid() throws Ex verify(meterStatsManager, times(1)).markEvent(INVALID_CONFIGURATION); ArgumentCaptor reportInvalidConfigCaptor = ArgumentCaptor.forClass(InvalidConfigurationException.class); verify(errorReporter, times(1)).reportFatalException(reportInvalidConfigCaptor.capture()); - assertEquals("Column 'invalid_variable' not found as configured in the endpoint/query variable", + assertEquals("Column 'invalid_variable' not found as configured in the 'REQUEST_VARIABLES' variable", reportInvalidConfigCaptor.getValue().getMessage()); ArgumentCaptor invalidConfigurationExceptionArgumentCaptor = ArgumentCaptor.forClass(InvalidConfigurationException.class); verify(resultFuture, times(1)).completeExceptionally(invalidConfigurationExceptionArgumentCaptor.capture()); - assertEquals("Column 'invalid_variable' not found as configured in the endpoint/query variable", + assertEquals("Column 'invalid_variable' not found as configured in the 'REQUEST_VARIABLES' variable", invalidConfigurationExceptionArgumentCaptor.getValue().getMessage()); } diff --git a/dagger-core/src/test/java/io/odpf/dagger/core/processors/external/grpc/GrpcSourceConfigTest.java b/dagger-core/src/test/java/io/odpf/dagger/core/processors/external/grpc/GrpcSourceConfigTest.java index fbed065c6..8b7331ecf 100644 --- a/dagger-core/src/test/java/io/odpf/dagger/core/processors/external/grpc/GrpcSourceConfigTest.java +++ b/dagger-core/src/test/java/io/odpf/dagger/core/processors/external/grpc/GrpcSourceConfigTest.java @@ -119,13 +119,13 @@ public void hasTypeShouldBeTrueWhenTypeIsPresent() { @Test public void hasTypeShouldBeFalseWhenTypeIsNull() { - HttpSourceConfig httpSourceConfig = new HttpSourceConfig("", "", "", "", null, "", false, null, "", new HashMap<>(), new HashMap<>(), metricId, false); + HttpSourceConfig httpSourceConfig = new HttpSourceConfig("", "", "", "", "", "", null, "", false, null, "", new HashMap<>(), new HashMap<>(), metricId, false); assertFalse(httpSourceConfig.hasType()); } @Test public void hasTypeShouldBeFalseWhenTypeIsEmpty() { - HttpSourceConfig httpSourceConfig = new HttpSourceConfig("", "", "", "", "", "", false, "", "", new HashMap<>(), new HashMap<>(), metricId, false); + HttpSourceConfig httpSourceConfig = new HttpSourceConfig("", "", "", "", "", "", "", "", false, "", "", new HashMap<>(), new HashMap<>(), metricId, false); assertFalse(httpSourceConfig.hasType()); } diff --git a/dagger-core/src/test/java/io/odpf/dagger/core/processors/external/http/HttpAsyncConnectorTest.java b/dagger-core/src/test/java/io/odpf/dagger/core/processors/external/http/HttpAsyncConnectorTest.java index 2f401b59d..0d049484c 100644 --- a/dagger-core/src/test/java/io/odpf/dagger/core/processors/external/http/HttpAsyncConnectorTest.java +++ b/dagger-core/src/test/java/io/odpf/dagger/core/processors/external/http/HttpAsyncConnectorTest.java @@ -97,7 +97,7 @@ public void setUp() { externalMetricConfig = new ExternalMetricConfig("metricId-http-01", shutDownPeriod, telemetryEnabled); defaultHttpSourceConfig = new HttpSourceConfig("http://localhost:8080/test", "POST", "{\"key\": \"%s\"}", - "customer_id", "123", "234", false, httpConfigType, "345", + "customer_id", "", "", "123", "234", false, httpConfigType, "345", headers, outputMapping, "metricId_02", false); } @@ -184,10 +184,10 @@ public void shouldCompleteExceptionallyIfOutputDescriptorNotFound() throws Excep } @Test - public void shouldCompleteExceptionallyWhenEndpointVariableIsInvalid() throws Exception { + public void shouldCompleteExceptionallyWhenRequestVariableIsInvalid() throws Exception { when(defaultDescriptorManager.getDescriptor(inputProtoClasses[0])).thenReturn(TestBookingLogMessage.getDescriptor()); String invalidRequestVariable = "invalid_variable"; - defaultHttpSourceConfig = new HttpSourceConfig("http://localhost:8080/test", "POST", "{\"key\": \"%s\"}", invalidRequestVariable, "123", "234", false, httpConfigType, "345", headers, outputMapping, "metricId_02", false); + defaultHttpSourceConfig = new HttpSourceConfig("http://localhost:8080/test", "POST", "{\"key\": \"%s\"}", invalidRequestVariable, "", "", "123", "234", false, httpConfigType, "345", headers, outputMapping, "metricId_02", false); when(httpClient.preparePost("http://localhost:8080/test")).thenReturn(boundRequestBuilder); when(boundRequestBuilder.setBody("{\"key\": \"123456\"}")).thenReturn(boundRequestBuilder); HttpAsyncConnector httpAsyncConnector = new HttpAsyncConnector(defaultHttpSourceConfig, externalMetricConfig, schemaConfig, httpClient, errorReporter, meterStatsManager, defaultDescriptorManager); @@ -198,19 +198,19 @@ public void shouldCompleteExceptionallyWhenEndpointVariableIsInvalid() throws Ex verify(meterStatsManager, times(1)).markEvent(INVALID_CONFIGURATION); ArgumentCaptor exceptionArgumentCaptor = ArgumentCaptor.forClass(InvalidConfigurationException.class); verify(errorReporter, times(1)).reportFatalException(exceptionArgumentCaptor.capture()); - assertEquals("Column 'invalid_variable' not found as configured in the endpoint/query variable", exceptionArgumentCaptor.getValue().getMessage()); + assertEquals("Column 'invalid_variable' not found as configured in the 'REQUEST_VARIABLES' variable", exceptionArgumentCaptor.getValue().getMessage()); ArgumentCaptor futureCaptor = ArgumentCaptor.forClass(InvalidConfigurationException.class); verify(resultFuture, times(1)).completeExceptionally(futureCaptor.capture()); - assertEquals("Column 'invalid_variable' not found as configured in the endpoint/query variable", + assertEquals("Column 'invalid_variable' not found as configured in the 'REQUEST_VARIABLES' variable", futureCaptor.getValue().getMessage()); } @Test public void shouldCompleteExceptionallyWhenEndpointVariableIsEmptyAndRequiredInPattern() throws Exception { String emptyRequestVariable = ""; - defaultHttpSourceConfig = new HttpSourceConfig("http://localhost:8080/test", "POST", "{\"key\": \"%s\"}", emptyRequestVariable, "123", "234", false, httpConfigType, "345", headers, outputMapping, "metricId_02", false); + defaultHttpSourceConfig = new HttpSourceConfig("http://localhost:8080/test", "POST", "{\"key\": \"%s\"}", emptyRequestVariable, "", "", "123", "234", false, httpConfigType, "345", headers, outputMapping, "metricId_02", false); when(defaultDescriptorManager.getDescriptor(inputProtoClasses[0])).thenReturn(TestBookingLogMessage.getDescriptor()); when(httpClient.preparePost("http://localhost:8080/test")).thenReturn(boundRequestBuilder); when(boundRequestBuilder.setBody("{\"key\": \"123456\"}")).thenReturn(boundRequestBuilder); @@ -227,7 +227,7 @@ public void shouldCompleteExceptionallyWhenEndpointVariableIsEmptyAndRequiredInP @Test public void shouldEnrichWhenEndpointVariableIsEmptyAndNotRequiredInPattern() throws Exception { String emptyRequestVariable = ""; - defaultHttpSourceConfig = new HttpSourceConfig("http://localhost:8080/test", "POST", "{\"key\": \"static\"}", emptyRequestVariable, "123", "234", false, httpConfigType, "345", headers, outputMapping, "metricId_02", false); + defaultHttpSourceConfig = new HttpSourceConfig("http://localhost:8080/test", "POST", "{\"key\": \"static\"}", emptyRequestVariable, "", "", "123", "234", false, httpConfigType, "345", headers, outputMapping, "metricId_02", false); HttpAsyncConnector httpAsyncConnector = new HttpAsyncConnector(defaultHttpSourceConfig, externalMetricConfig, schemaConfig, httpClient, errorReporter, meterStatsManager, defaultDescriptorManager); when(httpClient.preparePost("http://localhost:8080/test")).thenReturn(boundRequestBuilder); @@ -247,7 +247,7 @@ public void shouldEnrichWhenEndpointVariableIsEmptyAndNotRequiredInPattern() thr @Test public void shouldCompleteExceptionallyWhenEndpointPatternIsInvalid() throws Exception { String invalidRequestPattern = "{\"key\": \"%\"}"; - defaultHttpSourceConfig = new HttpSourceConfig("http://localhost:8080/test", "POST", invalidRequestPattern, "customer_id", "123", "234", false, httpConfigType, "345", headers, outputMapping, "metricId_02", false); + defaultHttpSourceConfig = new HttpSourceConfig("http://localhost:8080/test", "POST", invalidRequestPattern, "customer_id", "", "", "123", "234", false, httpConfigType, "345", headers, outputMapping, "metricId_02", false); when(defaultDescriptorManager.getDescriptor(inputProtoClasses[0])).thenReturn(TestBookingLogMessage.getDescriptor()); when(httpClient.preparePost("http://localhost:8080/test")).thenReturn(boundRequestBuilder); when(boundRequestBuilder.setBody("{\"key\": \"123456\"}")).thenReturn(boundRequestBuilder); @@ -262,7 +262,7 @@ public void shouldCompleteExceptionallyWhenEndpointPatternIsInvalid() throws Exc @Test public void shouldGetDescriptorFromOutputProtoIfTypeNotGiven() throws Exception { - defaultHttpSourceConfig = new HttpSourceConfig("http://localhost:8080/test", "POST", "{\"key\": \"%s\"}", "customer_id", "123", "234", false, null, "345", headers, outputMapping, "metricId_02", false); + defaultHttpSourceConfig = new HttpSourceConfig("http://localhost:8080/test", "POST", "{\"key\": \"%s\"}", "customer_id", "", "", "123", "234", false, null, "345", headers, outputMapping, "metricId_02", false); when(httpClient.preparePost("http://localhost:8080/test")).thenReturn(boundRequestBuilder); when(boundRequestBuilder.setBody("{\"key\": \"123456\"}")).thenReturn(boundRequestBuilder); when(stencilClientOrchestrator.getStencilClient()).thenReturn(stencilClient); @@ -277,7 +277,7 @@ public void shouldGetDescriptorFromOutputProtoIfTypeNotGiven() throws Exception @Test public void shouldGetDescriptorFromTypeIfGiven() throws Exception { - defaultHttpSourceConfig = new HttpSourceConfig("http://localhost:8080/test", "POST", "{\"key\": \"%s\"}", "customer_id", "123", "234", false, "TestBookingLogMessage", "345", headers, outputMapping, "metricId_02", false); + defaultHttpSourceConfig = new HttpSourceConfig("http://localhost:8080/test", "POST", "{\"key\": \"%s\"}", "customer_id", "", "", "123", "234", false, "TestBookingLogMessage", "345", headers, outputMapping, "metricId_02", false); when(httpClient.preparePost("http://localhost:8080/test")).thenReturn(boundRequestBuilder); when(boundRequestBuilder.setBody("{\"key\": \"123456\"}")).thenReturn(boundRequestBuilder); when(stencilClientOrchestrator.getStencilClient()).thenReturn(stencilClient); @@ -293,7 +293,7 @@ public void shouldGetDescriptorFromTypeIfGiven() throws Exception { @Test public void shouldCompleteExceptionallyWhenEndpointPatternIsIncompatible() throws Exception { String invalidRequestPattern = "{\"key\": \"%d\"}"; - defaultHttpSourceConfig = new HttpSourceConfig("http://localhost:8080/test", "POST", invalidRequestPattern, "customer_id", "123", "234", false, httpConfigType, "345", headers, outputMapping, "metricId_02", false); + defaultHttpSourceConfig = new HttpSourceConfig("http://localhost:8080/test", "POST", invalidRequestPattern, "customer_id", "", "", "123", "234", false, httpConfigType, "345", headers, outputMapping, "metricId_02", false); when(defaultDescriptorManager.getDescriptor(inputProtoClasses[0])).thenReturn(TestBookingLogMessage.getDescriptor()); when(httpClient.preparePost("http://localhost:8080/test")).thenReturn(boundRequestBuilder); when(boundRequestBuilder.setBody("{\"key\": \"123456\"}")).thenReturn(boundRequestBuilder); @@ -337,9 +337,85 @@ public void shouldAddCustomHeaders() throws Exception { verify(boundRequestBuilder, times(1)).addHeader("content-type", "application/json"); } + @Test + public void shouldAddDynamicHeaders() throws Exception { + when(httpClient.preparePost("http://localhost:8080/test")).thenReturn(boundRequestBuilder); + when(boundRequestBuilder.setBody("{\"key\": \"123456\"}")).thenReturn(boundRequestBuilder); + when(stencilClientOrchestrator.getStencilClient()).thenReturn(stencilClient); + when(defaultDescriptorManager.getDescriptor(inputProtoClasses[0])).thenReturn(TestBookingLogMessage.getDescriptor()); + HttpSourceConfig dynamicHeaderHttpSourceConfig = new HttpSourceConfig("http://localhost:8080/test", "POST", "{\"key\": \"%s\"}", + "customer_id", "{\"X_KEY\": \"%s\"}", "customer_id", "123", "234", false, httpConfigType, "345", + headers, outputMapping, "metricId_02", false); + HttpAsyncConnector httpAsyncConnector = new HttpAsyncConnector(dynamicHeaderHttpSourceConfig, externalMetricConfig, schemaConfig, httpClient, errorReporter, meterStatsManager, defaultDescriptorManager); + + httpAsyncConnector.open(flinkConfiguration); + httpAsyncConnector.asyncInvoke(streamData, resultFuture); + verify(boundRequestBuilder, times(2)).addHeader(anyString(), anyString()); + verify(boundRequestBuilder, times(1)).addHeader("content-type", "application/json"); + verify(boundRequestBuilder, times(1)).addHeader("X_KEY", "123456"); + } + + @Test + public void shouldNotAddDynamicHeaders() throws Exception { + when(httpClient.preparePost("http://localhost:8080/test")).thenReturn(boundRequestBuilder); + when(boundRequestBuilder.setBody("{\"key\": \"123456\"}")).thenReturn(boundRequestBuilder); + when(stencilClientOrchestrator.getStencilClient()).thenReturn(stencilClient); + when(defaultDescriptorManager.getDescriptor(inputProtoClasses[0])).thenReturn(TestBookingLogMessage.getDescriptor()); + HttpSourceConfig dynamicHeaderHttpSourceConfig = new HttpSourceConfig("http://localhost:8080/test", "POST", "{\"key\": \"%s\"}", + "customer_id", "{\"X_KEY\": \"%s\"}", "customer_ids", "123", "234", false, httpConfigType, "345", + headers, outputMapping, "metricId_02", false); + HttpAsyncConnector httpAsyncConnector = new HttpAsyncConnector(dynamicHeaderHttpSourceConfig, externalMetricConfig, schemaConfig, httpClient, errorReporter, meterStatsManager, defaultDescriptorManager); + httpAsyncConnector.open(flinkConfiguration); + httpAsyncConnector.asyncInvoke(streamData, resultFuture); + + verify(meterStatsManager, times(1)).markEvent(INVALID_CONFIGURATION); + verify(errorReporter, times(1)).reportFatalException(any(InvalidConfigurationException.class)); + verify(resultFuture, times(1)).completeExceptionally(any(InvalidConfigurationException.class)); + } + + @Test + public void shouldCompleteExceptionallyWhenHeaderVariableIsInvalid() throws Exception { + when(defaultDescriptorManager.getDescriptor(inputProtoClasses[0])).thenReturn(TestBookingLogMessage.getDescriptor()); + String invalidHeaderVariable = "invalid_variable"; + defaultHttpSourceConfig = new HttpSourceConfig("http://localhost:8080/test", "POST", "{\"key\": \"%s\"}", "customer_id", "{\"key\": \"%s\"}", invalidHeaderVariable, "123", "234", false, httpConfigType, "345", headers, outputMapping, "metricId_02", false); + when(httpClient.preparePost("http://localhost:8080/test")).thenReturn(boundRequestBuilder); + when(boundRequestBuilder.setBody("{\"key\": \"123456\"}")).thenReturn(boundRequestBuilder); + HttpAsyncConnector httpAsyncConnector = new HttpAsyncConnector(defaultHttpSourceConfig, externalMetricConfig, schemaConfig, httpClient, errorReporter, meterStatsManager, defaultDescriptorManager); + + httpAsyncConnector.open(flinkConfiguration); + httpAsyncConnector.asyncInvoke(streamData, resultFuture); + + verify(meterStatsManager, times(1)).markEvent(INVALID_CONFIGURATION); + ArgumentCaptor exceptionArgumentCaptor = ArgumentCaptor.forClass(InvalidConfigurationException.class); + verify(errorReporter, times(1)).reportFatalException(exceptionArgumentCaptor.capture()); + assertEquals("Column 'invalid_variable' not found as configured in the 'HEADER_VARIABLES' variable", exceptionArgumentCaptor.getValue().getMessage()); + + ArgumentCaptor futureCaptor = ArgumentCaptor.forClass(InvalidConfigurationException.class); + verify(resultFuture, times(1)).completeExceptionally(futureCaptor.capture()); + + assertEquals("Column 'invalid_variable' not found as configured in the 'HEADER_VARIABLES' variable", + futureCaptor.getValue().getMessage()); + } + + @Test + public void shouldCompleteExceptionallyWhenHeaderPatternIsIncompatible() throws Exception { + String invalidHeaderPattern = "{\"key\": \"%d\"}"; + defaultHttpSourceConfig = new HttpSourceConfig("http://localhost:8080/test", "POST", "{\"key\": \"%s\"}", "customer_id", invalidHeaderPattern, "customer_id", "123", "234", false, httpConfigType, "345", headers, outputMapping, "metricId_02", false); + when(defaultDescriptorManager.getDescriptor(inputProtoClasses[0])).thenReturn(TestBookingLogMessage.getDescriptor()); + when(httpClient.preparePost("http://localhost:8080/test")).thenReturn(boundRequestBuilder); + when(boundRequestBuilder.setBody("{\"key\": \"123456\"}")).thenReturn(boundRequestBuilder); + HttpAsyncConnector httpAsyncConnector = new HttpAsyncConnector(defaultHttpSourceConfig, externalMetricConfig, schemaConfig, httpClient, errorReporter, meterStatsManager, defaultDescriptorManager); + httpAsyncConnector.open(flinkConfiguration); + httpAsyncConnector.asyncInvoke(streamData, resultFuture); + + verify(meterStatsManager, times(1)).markEvent(INVALID_CONFIGURATION); + verify(errorReporter, times(1)).reportFatalException(any(InvalidConfigurationException.class)); + verify(resultFuture, times(1)).completeExceptionally(any(InvalidConfigurationException.class)); + } + @Test public void shouldThrowExceptionInTimeoutIfFailOnErrorIsTrue() throws Exception { - HttpSourceConfig httpSourceConfig = new HttpSourceConfig("http://localhost:8080/test", "POST", "{\"key\": \"%s\"}", "customer_id", "123", "234", true, httpConfigType, "345", headers, outputMapping, "metricId_02", false); + HttpSourceConfig httpSourceConfig = new HttpSourceConfig("http://localhost:8080/test", "POST", "{\"key\": \"%s\"}", "customer_id", "", "", "123", "234", true, httpConfigType, "345", headers, outputMapping, "metricId_02", false); HttpAsyncConnector httpAsyncConnector = new HttpAsyncConnector(httpSourceConfig, externalMetricConfig, schemaConfig, httpClient, errorReporter, meterStatsManager, defaultDescriptorManager); httpAsyncConnector.timeout(streamData, resultFuture); @@ -349,7 +425,7 @@ public void shouldThrowExceptionInTimeoutIfFailOnErrorIsTrue() throws Exception @Test public void shouldReportFatalInTimeoutIfFailOnErrorIsTrue() throws Exception { - HttpSourceConfig httpSourceConfig = new HttpSourceConfig("http://localhost:8080/test", "POST", "{\"key\": \"%s\"}", "customer_id", "123", "234", true, httpConfigType, "345", headers, outputMapping, "metricId_02", false); + HttpSourceConfig httpSourceConfig = new HttpSourceConfig("http://localhost:8080/test", "POST", "{\"key\": \"%s\"}", "customer_id", "", "", "123", "234", true, httpConfigType, "345", headers, outputMapping, "metricId_02", false); HttpAsyncConnector httpAsyncConnector = new HttpAsyncConnector(httpSourceConfig, externalMetricConfig, schemaConfig, httpClient, errorReporter, meterStatsManager, defaultDescriptorManager); httpAsyncConnector.timeout(streamData, resultFuture); @@ -359,7 +435,7 @@ public void shouldReportFatalInTimeoutIfFailOnErrorIsTrue() throws Exception { @Test public void shouldReportNonFatalInTimeoutIfFailOnErrorIsFalse() { - HttpSourceConfig httpSourceConfig = new HttpSourceConfig("http://localhost:8080/test", "POST", "{\"key\": \"%s\"}", "customer_id", "123", "234", false, httpConfigType, "345", headers, outputMapping, "metricId_02", false); + HttpSourceConfig httpSourceConfig = new HttpSourceConfig("http://localhost:8080/test", "POST", "{\"key\": \"%s\"}", "customer_id", "", "", "123", "234", false, httpConfigType, "345", headers, outputMapping, "metricId_02", false); HttpAsyncConnector httpAsyncConnector = new HttpAsyncConnector(httpSourceConfig, externalMetricConfig, schemaConfig, httpClient, errorReporter, meterStatsManager, defaultDescriptorManager); httpAsyncConnector.timeout(streamData, resultFuture); @@ -379,7 +455,7 @@ public void shouldPassTheInputWithRowSizeCorrespondingToColumnNamesInTimeoutIfFa public void shouldThrowExceptionIfUnsupportedHttpVerbProvided() throws Exception { when(defaultDescriptorManager.getDescriptor(inputProtoClasses[0])).thenReturn(TestBookingLogMessage.getDescriptor()); - HttpSourceConfig httpSourceConfig = new HttpSourceConfig("http://localhost:8080/test", "PATCH", "{\"key\": \"%s\"}", "customer_id", "123", "234", true, httpConfigType, "345", headers, outputMapping, "metricId_02", false); + HttpSourceConfig httpSourceConfig = new HttpSourceConfig("http://localhost:8080/test", "PATCH", "{\"key\": \"%s\"}", "customer_id", "", "", "123", "234", true, httpConfigType, "345", headers, outputMapping, "metricId_02", false); HttpAsyncConnector httpAsyncConnector = new HttpAsyncConnector(httpSourceConfig, externalMetricConfig, schemaConfig, httpClient, errorReporter, meterStatsManager, defaultDescriptorManager); httpAsyncConnector.open(flinkConfiguration); @@ -413,4 +489,5 @@ public void shouldThrowIfRuntimeContextNotInitialized() throws Exception { HttpAsyncConnector httpAsyncConnector = new HttpAsyncConnector(defaultHttpSourceConfig, externalMetricConfig, schemaConfig, httpClient, null, meterStatsManager, defaultDescriptorManager); httpAsyncConnector.open(flinkConfiguration); } + } diff --git a/dagger-core/src/test/java/io/odpf/dagger/core/processors/external/http/HttpResponseHandlerTest.java b/dagger-core/src/test/java/io/odpf/dagger/core/processors/external/http/HttpResponseHandlerTest.java index c5fc30f84..7fa9548a9 100644 --- a/dagger-core/src/test/java/io/odpf/dagger/core/processors/external/http/HttpResponseHandlerTest.java +++ b/dagger-core/src/test/java/io/odpf/dagger/core/processors/external/http/HttpResponseHandlerTest.java @@ -74,7 +74,7 @@ public void setup() { streamData.setField(1, new Row(2)); rowManager = new RowManager(streamData); columnNameManager = new ColumnNameManager(inputColumnNames, outputColumnNames); - httpSourceConfig = new HttpSourceConfig("http://localhost:8080/test", "POST", "{\"key\": \"%s\"}", "customer_id", "123", "234", false, httpConfigType, "345", headers, outputMapping, "metricId_02", false); + httpSourceConfig = new HttpSourceConfig("http://localhost:8080/test", "POST", "{\"key\": \"%s\"}", "customer_id", "", "", "123", "234", false, httpConfigType, "345", headers, outputMapping, "metricId_02", false); } @Test @@ -148,7 +148,7 @@ public void shouldPassInputIfFailOnErrorFalseAndStatusCodeIsOtherThan5XXAnd4XX() @Test public void shouldThrowErrorIfFailOnErrorTrueAndStatusCodeIs404() { - httpSourceConfig = new HttpSourceConfig("http://localhost:8080/test", "POST", "{\"key\": \"%s\"}", "customer_id", "123", "234", true, httpConfigType, "345", headers, outputMapping, "metricId_02", false); + httpSourceConfig = new HttpSourceConfig("http://localhost:8080/test", "POST", "{\"key\": \"%s\"}", "customer_id", "", "", "123", "234", true, httpConfigType, "345", headers, outputMapping, "metricId_02", false); HttpResponseHandler httpResponseHandler = new HttpResponseHandler(httpSourceConfig, meterStatsManager, rowManager, columnNameManager, descriptor, resultFuture, errorReporter, new PostResponseTelemetry()); when(response.getStatusCode()).thenReturn(404); @@ -167,7 +167,7 @@ public void shouldThrowErrorIfFailOnErrorTrueAndStatusCodeIs404() { @Test public void shouldThrowErrorIfFailOnErrorTrueAndStatusCodeIs4XXOtherThan404() { - httpSourceConfig = new HttpSourceConfig("http://localhost:8080/test", "POST", "{\"key\": \"%s\"}", "customer_id", "123", "234", true, httpConfigType, "345", headers, outputMapping, "metricId_02", false); + httpSourceConfig = new HttpSourceConfig("http://localhost:8080/test", "POST", "{\"key\": \"%s\"}", "customer_id", "", "", "123", "234", true, httpConfigType, "345", headers, outputMapping, "metricId_02", false); HttpResponseHandler httpResponseHandler = new HttpResponseHandler(httpSourceConfig, meterStatsManager, rowManager, columnNameManager, descriptor, resultFuture, errorReporter, new PostResponseTelemetry()); when(response.getStatusCode()).thenReturn(400); @@ -185,7 +185,7 @@ public void shouldThrowErrorIfFailOnErrorTrueAndStatusCodeIs4XXOtherThan404() { @Test public void shouldThrowErrorIfFailOnErrorTrueAndStatusCodeIs5XX() { - httpSourceConfig = new HttpSourceConfig("http://localhost:8080/test", "POST", "{\"key\": \"%s\"}", "customer_id", "123", "234", true, httpConfigType, "345", headers, outputMapping, "metricId_02", false); + httpSourceConfig = new HttpSourceConfig("http://localhost:8080/test", "POST", "{\"key\": \"%s\"}", "customer_id", "", "", "123", "234", true, httpConfigType, "345", headers, outputMapping, "metricId_02", false); HttpResponseHandler httpResponseHandler = new HttpResponseHandler(httpSourceConfig, meterStatsManager, rowManager, columnNameManager, descriptor, resultFuture, errorReporter, new PostResponseTelemetry()); when(response.getStatusCode()).thenReturn(502); @@ -203,7 +203,7 @@ public void shouldThrowErrorIfFailOnErrorTrueAndStatusCodeIs5XX() { @Test public void shouldThrowErrorIfFailOnErrorTrueAndStatusCodeIsOtherThan5XXAnd4XX() { - httpSourceConfig = new HttpSourceConfig("http://localhost:8080/test", "POST", "{\"key\": \"%s\"}", "customer_id", "123", "234", true, httpConfigType, "345", headers, outputMapping, "metricId_02", false); + httpSourceConfig = new HttpSourceConfig("http://localhost:8080/test", "POST", "{\"key\": \"%s\"}", "customer_id", "", "", "123", "234", true, httpConfigType, "345", headers, outputMapping, "metricId_02", false); HttpResponseHandler httpResponseHandler = new HttpResponseHandler(httpSourceConfig, meterStatsManager, rowManager, columnNameManager, descriptor, resultFuture, errorReporter, new PostResponseTelemetry()); when(response.getStatusCode()).thenReturn(302); @@ -237,7 +237,7 @@ public void shouldPassInputIfFailOnErrorFalseAndOnThrowable() { @Test public void shouldThrowErrorIfFailOnErrorTrueAndOnThrowable() { - httpSourceConfig = new HttpSourceConfig("http://localhost:8080/test", "POST", "{\"key\": \"%s\"}", "customer_id", "123", "234", true, httpConfigType, "345", headers, outputMapping, "metricId_02", false); + httpSourceConfig = new HttpSourceConfig("http://localhost:8080/test", "POST", "{\"key\": \"%s\"}", "customer_id", "", "", "123", "234", true, httpConfigType, "345", headers, outputMapping, "metricId_02", false); HttpResponseHandler httpResponseHandler = new HttpResponseHandler(httpSourceConfig, meterStatsManager, rowManager, columnNameManager, descriptor, resultFuture, errorReporter, new PostResponseTelemetry()); Throwable throwable = new Throwable("throwable message"); @@ -258,7 +258,7 @@ public void shouldPopulateSingleResultFromHttpCallInInputRow() { outputMapping.put("surge_factor", new OutputMapping("$.surge")); outputColumnNames = Collections.singletonList("surge_factor"); columnNameManager = new ColumnNameManager(inputColumnNames, outputColumnNames); - httpSourceConfig = new HttpSourceConfig("http://localhost:8080/test", "POST", "{\"key\": \"%s\"}", "customer_id", "123", "234", false, httpConfigType, "345", headers, outputMapping, "metricId_02", false); + httpSourceConfig = new HttpSourceConfig("http://localhost:8080/test", "POST", "{\"key\": \"%s\"}", "customer_id", "", "", "123", "234", false, httpConfigType, "345", headers, outputMapping, "metricId_02", false); HttpResponseHandler httpResponseHandler = new HttpResponseHandler(httpSourceConfig, meterStatsManager, rowManager, columnNameManager, descriptor, resultFuture, errorReporter, new PostResponseTelemetry()); Row resultStreamData = new Row(2); Row outputData = new Row(2); @@ -284,7 +284,7 @@ public void shouldPopulateMultipleResultsFromHttpCallInInputRow() { outputMapping.put("s2_id_level", new OutputMapping("$.prediction")); outputColumnNames = Arrays.asList("surge_factor", "s2_id_level"); columnNameManager = new ColumnNameManager(inputColumnNames, outputColumnNames); - httpSourceConfig = new HttpSourceConfig("http://localhost:8080/test", "POST", "{\"key\": \"%s\"}", "customer_id", "123", "234", false, httpConfigType, "345", headers, outputMapping, "metricId_02", false); + httpSourceConfig = new HttpSourceConfig("http://localhost:8080/test", "POST", "{\"key\": \"%s\"}", "customer_id", "", "", "123", "234", false, httpConfigType, "345", headers, outputMapping, "metricId_02", false); HttpResponseHandler httpResponseHandler = new HttpResponseHandler(httpSourceConfig, meterStatsManager, rowManager, columnNameManager, descriptor, resultFuture, errorReporter, new PostResponseTelemetry()); Row resultStreamData = new Row(2); Row outputData = new Row(2); @@ -313,7 +313,7 @@ public void shouldThrowExceptionIfFieldNotFoundInFieldDescriptorWhenTypeIsPassed outputMapping.put("surge_factor", new OutputMapping("$.surge")); outputColumnNames = Collections.singletonList("surge_factor"); columnNameManager = new ColumnNameManager(inputColumnNames, outputColumnNames); - httpSourceConfig = new HttpSourceConfig("http://localhost:8080/test", "POST", "{\"key\": \"%s\"}", "customer_id", "123", "234", false, httpConfigType, "345", headers, outputMapping, "metricId_02", false); + httpSourceConfig = new HttpSourceConfig("http://localhost:8080/test", "POST", "{\"key\": \"%s\"}", "customer_id", "", "", "123", "234", false, httpConfigType, "345", headers, outputMapping, "metricId_02", false); when(response.getStatusCode()).thenReturn(200); when(response.getResponseBody()).thenReturn("{\n" + " \"surge\": 0.732\n" @@ -331,7 +331,7 @@ public void shouldThrowExceptionIfPathIsWrongIfFailOnErrorsTrue() { outputMapping.put("surge_factor", new OutputMapping("invalidPath")); outputColumnNames = Collections.singletonList("surge_factor"); columnNameManager = new ColumnNameManager(inputColumnNames, outputColumnNames); - httpSourceConfig = new HttpSourceConfig("http://localhost:8080/test", "POST", "{\"key\": \"%s\"}", "customer_id", "123", "234", true, httpConfigType, "345", headers, outputMapping, "metricId_02", false); + httpSourceConfig = new HttpSourceConfig("http://localhost:8080/test", "POST", "{\"key\": \"%s\"}", "customer_id", "", "", "123", "234", true, httpConfigType, "345", headers, outputMapping, "metricId_02", false); HttpResponseHandler httpResponseHandler = new HttpResponseHandler(httpSourceConfig, meterStatsManager, rowManager, columnNameManager, descriptor, resultFuture, errorReporter, new PostResponseTelemetry()); Row resultStreamData = new Row(2); Row outputData = new Row(2); @@ -356,7 +356,7 @@ public void shouldPopulateResultAsObjectIfTypeIsNotPassedAndRetainResponseTypeIs outputMapping.put("surge_factor", new OutputMapping("$.surge")); outputColumnNames = Collections.singletonList("surge_factor"); columnNameManager = new ColumnNameManager(inputColumnNames, outputColumnNames); - httpSourceConfig = new HttpSourceConfig("http://localhost:8080/test", "POST", "{\"key\": \"%s\"}", "customer_id", "123", "234", false, null, "345", headers, outputMapping, "metricId_02", true); + httpSourceConfig = new HttpSourceConfig("http://localhost:8080/test", "POST", "{\"key\": \"%s\"}", "customer_id", "", "", "123", "234", false, null, "345", headers, outputMapping, "metricId_02", true); HttpResponseHandler httpResponseHandler = new HttpResponseHandler(httpSourceConfig, meterStatsManager, rowManager, columnNameManager, descriptor, resultFuture, errorReporter, new PostResponseTelemetry()); Row resultStreamData = new Row(2); Row outputData = new Row(2); @@ -381,7 +381,7 @@ public void shouldNotPopulateResultAsObjectIfTypeIsNotPassedAndRetainResponseTyp outputMapping.put("surge_factor", new OutputMapping("$.surge")); outputColumnNames = Collections.singletonList("surge_factor"); columnNameManager = new ColumnNameManager(inputColumnNames, outputColumnNames); - httpSourceConfig = new HttpSourceConfig("http://localhost:8080/test", "POST", "{\"key\": \"%s\"}", "customer_id", "123", "234", false, null, "345", headers, outputMapping, "metricId_02", false); + httpSourceConfig = new HttpSourceConfig("http://localhost:8080/test", "POST", "{\"key\": \"%s\"}", "customer_id", "", "", "123", "234", false, null, "345", headers, outputMapping, "metricId_02", false); HttpResponseHandler httpResponseHandler = new HttpResponseHandler(httpSourceConfig, meterStatsManager, rowManager, columnNameManager, descriptor, resultFuture, errorReporter, new PostResponseTelemetry()); Row resultStreamData = new Row(2); Row outputData = new Row(2); diff --git a/dagger-core/src/test/java/io/odpf/dagger/core/processors/external/http/HttpSourceConfigTest.java b/dagger-core/src/test/java/io/odpf/dagger/core/processors/external/http/HttpSourceConfigTest.java index 96220922b..6ee87cc1f 100644 --- a/dagger-core/src/test/java/io/odpf/dagger/core/processors/external/http/HttpSourceConfigTest.java +++ b/dagger-core/src/test/java/io/odpf/dagger/core/processors/external/http/HttpSourceConfigTest.java @@ -22,6 +22,8 @@ public class HttpSourceConfigTest { private String verb; private String requestPattern; private String requestVariables; + private String headerPattern; + private String headerVariables; private String connectTimeout; private boolean failOnErrors; private String type; @@ -41,13 +43,15 @@ public void setup() { verb = "POST"; requestPattern = "/customers/customer/%s"; requestVariables = "customer_id"; + headerPattern = "{\"X_KEY\":\"%s\"}"; + headerVariables = "customer_id"; connectTimeout = "234"; failOnErrors = false; type = "InputProtoMessage"; capacity = "345"; metricId = "metricId-http-01"; retainResponseType = false; - defaultHttpSourceConfig = new HttpSourceConfig(endpoint, verb, requestPattern, requestVariables, streamTimeout, connectTimeout, failOnErrors, type, capacity, headerMap, outputMappings, metricId, retainResponseType); + defaultHttpSourceConfig = new HttpSourceConfig(endpoint, verb, requestPattern, requestVariables, headerPattern, headerVariables, streamTimeout, connectTimeout, failOnErrors, type, capacity, headerMap, outputMappings, metricId, retainResponseType); } @Test @@ -107,13 +111,13 @@ public void hasTypeShouldBeTrueWhenTypeIsPresent() { @Test public void hasTypeShouldBeFalseWhenTypeIsNull() { - HttpSourceConfig httpSourceConfig = new HttpSourceConfig("", "", "", "", null, "", false, null, "", new HashMap<>(), new HashMap<>(), metricId, false); + HttpSourceConfig httpSourceConfig = new HttpSourceConfig("", "", "", "", "", "", null, "", false, null, "", new HashMap<>(), new HashMap<>(), metricId, false); assertFalse(httpSourceConfig.hasType()); } @Test public void hasTypeShouldBeFalseWhenTypeIsEmpty() { - HttpSourceConfig httpSourceConfig = new HttpSourceConfig("", "", "", "", "", "", false, "", "", new HashMap<>(), new HashMap<>(), metricId, false); + HttpSourceConfig httpSourceConfig = new HttpSourceConfig("", "", "", "", "", "", "", "", false, "", "", new HashMap<>(), new HashMap<>(), metricId, false); assertFalse(httpSourceConfig.hasType()); } @@ -147,7 +151,7 @@ public void shouldValidate() { @Test public void shouldThrowExceptionIfAllFieldsMissing() { - HttpSourceConfig httpSourceConfig = new HttpSourceConfig(null, null, null, requestVariables, null, null, false, null, capacity, null, null, metricId, retainResponseType); + HttpSourceConfig httpSourceConfig = new HttpSourceConfig(null, null, null, requestVariables, null, null, null, null, false, null, capacity, null, null, metricId, retainResponseType); IllegalArgumentException exception = assertThrows(IllegalArgumentException.class, () -> httpSourceConfig.validateFields()); assertEquals("Missing required fields: [endpoint, streamTimeout, requestPattern, verb, connectTimeout, outputMapping]", exception.getMessage()); @@ -156,7 +160,7 @@ public void shouldThrowExceptionIfAllFieldsMissing() { @Test public void shouldThrowExceptionIfSomeFieldsMissing() { - HttpSourceConfig httpSourceConfig = new HttpSourceConfig("localhost", "post", "body", requestVariables, null, null, false, null, capacity, null, null, "metricId_01", retainResponseType); + HttpSourceConfig httpSourceConfig = new HttpSourceConfig("localhost", "post", "body", requestVariables, null, null, null, null, false, null, capacity, null, null, "metricId_01", retainResponseType); IllegalArgumentException exception = assertThrows(IllegalArgumentException.class, () -> httpSourceConfig.validateFields()); assertEquals("Missing required fields: [streamTimeout, connectTimeout, outputMapping]", exception.getMessage()); @@ -170,7 +174,7 @@ public void shouldThrowExceptionIfFieldsOfNestedObjectsAreMissing() { outputMappings.put("field", outputMappingWithNullField); defaultHttpSourceConfig = new HttpSourceConfig("http://localhost", - "post", "request_body", requestVariables, "4000", "1000", false, "", capacity, headerMap, outputMappings, "metricId_01", retainResponseType); + "post", "request_body", requestVariables, "", "", "4000", "1000", false, "", capacity, headerMap, outputMappings, "metricId_01", retainResponseType); IllegalArgumentException exception = assertThrows(IllegalArgumentException.class, () -> defaultHttpSourceConfig.validateFields()); assertEquals("Missing required fields: [path]", exception.getMessage()); @@ -185,7 +189,7 @@ public void shouldThrowExceptionIfRequestPatternIsEmpty() { outputMappings.put("field", outputMappingWithNullField); defaultHttpSourceConfig = new HttpSourceConfig("http://localhost", - "post", "", requestVariables, "4000", "1000", false, "", capacity, headerMap, outputMappings, "metricId_01", retainResponseType); + "post", "", requestVariables, "", "", "4000", "1000", false, "", capacity, headerMap, outputMappings, "metricId_01", retainResponseType); IllegalArgumentException exception = assertThrows(IllegalArgumentException.class, () -> defaultHttpSourceConfig.validateFields()); assertEquals("Missing required fields: [requestPattern]", exception.getMessage()); @@ -220,7 +224,7 @@ public void shouldReturnMandatoryFields() { @Test public void shouldValidateWhenOutputMappingIsEmpty() { - defaultHttpSourceConfig = new HttpSourceConfig(endpoint, verb, requestPattern, requestVariables, streamTimeout, connectTimeout, failOnErrors, type, capacity, headerMap, new HashMap<>(), "metricId_01", retainResponseType); + defaultHttpSourceConfig = new HttpSourceConfig(endpoint, verb, requestPattern, requestVariables, headerPattern, headerVariables, streamTimeout, connectTimeout, failOnErrors, type, capacity, headerMap, new HashMap<>(), "metricId_01", retainResponseType); IllegalArgumentException exception = assertThrows(IllegalArgumentException.class, () -> defaultHttpSourceConfig.validateFields()); diff --git a/dagger-core/src/test/java/io/odpf/dagger/core/processors/external/http/request/HttpGetRequestHandlerTest.java b/dagger-core/src/test/java/io/odpf/dagger/core/processors/external/http/request/HttpGetRequestHandlerTest.java index b2479d306..afe6d3579 100644 --- a/dagger-core/src/test/java/io/odpf/dagger/core/processors/external/http/request/HttpGetRequestHandlerTest.java +++ b/dagger-core/src/test/java/io/odpf/dagger/core/processors/external/http/request/HttpGetRequestHandlerTest.java @@ -1,5 +1,6 @@ package io.odpf.dagger.core.processors.external.http.request; +import io.odpf.dagger.core.exception.InvalidConfigurationException; import io.odpf.dagger.core.processors.external.http.HttpSourceConfig; import org.asynchttpclient.AsyncHttpClient; import org.asynchttpclient.BoundRequestBuilder; @@ -11,7 +12,7 @@ import java.util.HashMap; import static org.junit.Assert.*; -import static org.mockito.Mockito.when; +import static org.mockito.Mockito.*; import static org.mockito.MockitoAnnotations.initMocks; public class HttpGetRequestHandlerTest { @@ -23,33 +24,101 @@ public class HttpGetRequestHandlerTest { private HttpSourceConfig httpSourceConfig; private ArrayList requestVariablesValues; + private ArrayList headerVariablesValues; @Before public void setup() { initMocks(this); requestVariablesValues = new ArrayList<>(); requestVariablesValues.add(1); + headerVariablesValues = new ArrayList<>(); + headerVariablesValues.add("1"); + headerVariablesValues.add("2"); } @Test public void shouldReturnTrueForGetVerbOnCanCreate() { - httpSourceConfig = new HttpSourceConfig("http://localhost:8080/test", "GET", "{\"key\": \"%s\"}", "1", "123", "234", false, "type", "345", new HashMap<>(), null, "metricId_01", false); - HttpGetRequestHandler httpGetRequestBuilder = new HttpGetRequestHandler(httpSourceConfig, httpClient, requestVariablesValues.toArray()); + httpSourceConfig = new HttpSourceConfig("http://localhost:8080/test", "GET", "{\"key\": \"%s\"}", "1", "", "", "123", "234", false, "type", "345", new HashMap<>(), null, "metricId_01", false); + HttpGetRequestHandler httpGetRequestBuilder = new HttpGetRequestHandler(httpSourceConfig, httpClient, requestVariablesValues.toArray(), headerVariablesValues.toArray()); assertTrue(httpGetRequestBuilder.canCreate()); } @Test public void shouldReturnFalseForVerbOtherThanGetOnCanBuild() { - httpSourceConfig = new HttpSourceConfig("http://localhost:8080/test", "POST", "{\"key\": \"%s\"}", "1", "123", "234", false, "type", "345", new HashMap<>(), null, "metricId_01", false); - HttpGetRequestHandler httpGetRequestBuilder = new HttpGetRequestHandler(httpSourceConfig, httpClient, requestVariablesValues.toArray()); + httpSourceConfig = new HttpSourceConfig("http://localhost:8080/test", "POST", "{\"key\": \"%s\"}", "1", "", "", "123", "234", false, "type", "345", new HashMap<>(), null, "metricId_01", false); + HttpGetRequestHandler httpGetRequestBuilder = new HttpGetRequestHandler(httpSourceConfig, httpClient, requestVariablesValues.toArray(), headerVariablesValues.toArray()); assertFalse(httpGetRequestBuilder.canCreate()); } @Test public void shouldBuildGetRequest() { when(httpClient.prepareGet("http://localhost:8080/test/key/1")).thenReturn(request); - httpSourceConfig = new HttpSourceConfig("http://localhost:8080/test", "GET", "/key/%s", "1", "123", "234", false, "type", "345", new HashMap<>(), null, "metricId_01", false); - HttpGetRequestHandler httpGetRequestBuilder = new HttpGetRequestHandler(httpSourceConfig, httpClient, requestVariablesValues.toArray()); + httpSourceConfig = new HttpSourceConfig("http://localhost:8080/test", "GET", "/key/%s", "1", "", "", "123", "234", false, "type", "345", new HashMap<>(), null, "metricId_01", false); + HttpGetRequestHandler httpGetRequestBuilder = new HttpGetRequestHandler(httpSourceConfig, httpClient, requestVariablesValues.toArray(), headerVariablesValues.toArray()); assertEquals(request, httpGetRequestBuilder.create()); } + + @Test + public void shouldBuildGetRequestWithOnlyDynamicHeader() { + when(httpClient.prepareGet("http://localhost:8080/test/key/1")).thenReturn(request); + when(request.addHeader("header_key", "1")).thenReturn(request); + httpSourceConfig = new HttpSourceConfig("http://localhost:8080/test", "GET", "/key/%s", "1", "{\"header_key\": \"%s\"}", "1", "123", "234", false, "type", "345", new HashMap<>(), null, "metricId_01", false); + HttpGetRequestHandler httpGetRequestBuilder = new HttpGetRequestHandler(httpSourceConfig, httpClient, requestVariablesValues.toArray(), headerVariablesValues.toArray()); + httpGetRequestBuilder.create(); + verify(request, times(1)).addHeader(anyString(), anyString()); + verify(request, times(1)).addHeader("header_key", "1"); + } + + @Test + public void shouldBuildGetRequestWithDynamicAndStaticHeader() { + when(httpClient.prepareGet("http://localhost:8080/test/key/1")).thenReturn(request); + when(request.addHeader("header_key", "1")).thenReturn(request); + HashMap staticHeader = new HashMap(); + staticHeader.put("static", "2"); + httpSourceConfig = new HttpSourceConfig("http://localhost:8080/test", "GET", "/key/%s", "1", "{\"header_key\": \"%s\"}", "1", "123", "234", false, "type", "345", staticHeader, null, "metricId_01", false); + HttpGetRequestHandler httpGetRequestBuilder = new HttpGetRequestHandler(httpSourceConfig, httpClient, requestVariablesValues.toArray(), headerVariablesValues.toArray()); + httpGetRequestBuilder.create(); + verify(request, times(2)).addHeader(anyString(), anyString()); + verify(request, times(1)).addHeader("header_key", "1"); + verify(request, times(1)).addHeader("static", "2"); + } + + @Test + public void shouldBuildGetRequestWithMultipleDynamicAndStaticHeaders() { + when(httpClient.prepareGet("http://localhost:8080/test/key/1")).thenReturn(request); + HashMap staticHeader = new HashMap(); + staticHeader.put("static", "3"); + httpSourceConfig = new HttpSourceConfig("http://localhost:8080/test", "GET", "/key/%s", "1", "{\"header_key_1\": \"%s\",\"header_key_2\": \"%s\"}", "1,2", "123", "234", false, "type", "345", staticHeader, null, "metricId_01", false); + HttpGetRequestHandler httpGetRequestBuilder = new HttpGetRequestHandler(httpSourceConfig, httpClient, requestVariablesValues.toArray(), headerVariablesValues.toArray()); + httpGetRequestBuilder.create(); + verify(request, times(3)).addHeader(anyString(), anyString()); + verify(request, times(1)).addHeader("header_key_1", "1"); + verify(request, times(1)).addHeader("header_key_2", "2"); + verify(request, times(1)).addHeader("static", "3"); + } + + @Test + public void shouldThrowErrorIfHeaderVariablesAreIncompatible() { + when(httpClient.prepareGet("http://localhost:8080/test/key/1")).thenReturn(request); + HashMap staticHeader = new HashMap(); + staticHeader.put("static", "3"); + ArrayList incompatibleHeaderVariablesValues = new ArrayList<>(); + incompatibleHeaderVariablesValues.add("test1"); + incompatibleHeaderVariablesValues.add("test12"); + httpSourceConfig = new HttpSourceConfig("http://localhost:8080/test", "GET", "/key/%s", "1", "{\"header_key_1\": \"%s\",\"header_key_2\": \"%d\"}", "1,2", "123", "234", false, "type", "345", staticHeader, null, "metricId_01", false); + HttpGetRequestHandler httpGetRequestBuilder = new HttpGetRequestHandler(httpSourceConfig, httpClient, requestVariablesValues.toArray(), incompatibleHeaderVariablesValues.toArray()); + InvalidConfigurationException exception = assertThrows(InvalidConfigurationException.class, () -> httpGetRequestBuilder.create()); + assertEquals("pattern config '{\"header_key_1\": \"%s\",\"header_key_2\": \"%d\"}' is incompatible with the variable config '1,2'", exception.getMessage()); + } + + @Test + public void shouldThrowErrorIfHeaderHeaderPatternIsInvalid() { + when(httpClient.prepareGet("http://localhost:8080/test/key/1")).thenReturn(request); + HashMap staticHeader = new HashMap(); + staticHeader.put("static", "3"); + httpSourceConfig = new HttpSourceConfig("http://localhost:8080/test", "GET", "/key/%s", "1", "{\"header_key_1\": \"%s\",\"header_key_2\": \"%p\"}", "1,2", "123", "234", false, "type", "345", staticHeader, null, "metricId_01", false); + HttpGetRequestHandler httpGetRequestBuilder = new HttpGetRequestHandler(httpSourceConfig, httpClient, requestVariablesValues.toArray(), headerVariablesValues.toArray()); + InvalidConfigurationException exception = assertThrows(InvalidConfigurationException.class, () -> httpGetRequestBuilder.create()); + assertEquals("pattern config '{\"header_key_1\": \"%s\",\"header_key_2\": \"%p\"}' is invalid", exception.getMessage()); + } } diff --git a/dagger-core/src/test/java/io/odpf/dagger/core/processors/external/http/request/HttpPostRequestHandlerTest.java b/dagger-core/src/test/java/io/odpf/dagger/core/processors/external/http/request/HttpPostRequestHandlerTest.java index 99a4493ba..a5c2f89ca 100644 --- a/dagger-core/src/test/java/io/odpf/dagger/core/processors/external/http/request/HttpPostRequestHandlerTest.java +++ b/dagger-core/src/test/java/io/odpf/dagger/core/processors/external/http/request/HttpPostRequestHandlerTest.java @@ -1,5 +1,6 @@ package io.odpf.dagger.core.processors.external.http.request; +import io.odpf.dagger.core.exception.InvalidConfigurationException; import io.odpf.dagger.core.processors.external.http.HttpSourceConfig; import org.asynchttpclient.AsyncHttpClient; import org.asynchttpclient.BoundRequestBuilder; @@ -11,7 +12,7 @@ import java.util.HashMap; import static org.junit.Assert.*; -import static org.mockito.Mockito.when; +import static org.mockito.Mockito.*; import static org.mockito.MockitoAnnotations.initMocks; public class HttpPostRequestHandlerTest { @@ -24,35 +25,105 @@ public class HttpPostRequestHandlerTest { private HttpSourceConfig httpSourceConfig; private ArrayList requestVariablesValues; + private ArrayList dynamicHeaderVariablesValues; @Before public void setup() { initMocks(this); requestVariablesValues = new ArrayList<>(); requestVariablesValues.add(1); + dynamicHeaderVariablesValues = new ArrayList<>(); + dynamicHeaderVariablesValues.add("1"); + dynamicHeaderVariablesValues.add("2"); } @Test public void shouldReturnTrueForPostVerbOnCanCreate() { - httpSourceConfig = new HttpSourceConfig("http://localhost:8080/test", "POST", "{\"key\": \"%s\"}", "1", "123", "234", false, "type", "345", new HashMap<>(), null, "metricId_01", false); - HttpPostRequestHandler httpPostRequestBuilder = new HttpPostRequestHandler(httpSourceConfig, httpClient, requestVariablesValues.toArray()); + httpSourceConfig = new HttpSourceConfig("http://localhost:8080/test", "POST", "{\"key\": \"%s\"}", "1", "", "", "123", "234", false, "type", "345", new HashMap<>(), null, "metricId_01", false); + HttpPostRequestHandler httpPostRequestBuilder = new HttpPostRequestHandler(httpSourceConfig, httpClient, requestVariablesValues.toArray(), dynamicHeaderVariablesValues.toArray()); assertTrue(httpPostRequestBuilder.canCreate()); } @Test public void shouldReturnFalseForVerbOtherThanPostOnCanBuild() { - httpSourceConfig = new HttpSourceConfig("http://localhost:8080/test", "GET", "{\"key\": \"%s\"}", "1", "123", "234", false, "type", "345", new HashMap<>(), null, "metricId_01", false); - HttpPostRequestHandler httpPostRequestBuilder = new HttpPostRequestHandler(httpSourceConfig, httpClient, requestVariablesValues.toArray()); + httpSourceConfig = new HttpSourceConfig("http://localhost:8080/test", "GET", "{\"key\": \"%s\"}", "1", "", "", "123", "234", false, "type", "345", new HashMap<>(), null, "metricId_01", false); + HttpPostRequestHandler httpPostRequestBuilder = new HttpPostRequestHandler(httpSourceConfig, httpClient, requestVariablesValues.toArray(), dynamicHeaderVariablesValues.toArray()); assertFalse(httpPostRequestBuilder.canCreate()); } @Test - public void shouldBuildPostRequest() { + public void shouldBuildPostRequestWithoutHeader() { when(httpClient.preparePost("http://localhost:8080/test")).thenReturn(request); when(request.setBody("{\"key\": \"1\"}")).thenReturn(request); - httpSourceConfig = new HttpSourceConfig("http://localhost:8080/test", "POST", "{\"key\": \"%s\"}", "1", "123", "234", false, "type", "345", new HashMap<>(), null, "metricId_01", false); - HttpPostRequestHandler httpPostRequestBuilder = new HttpPostRequestHandler(httpSourceConfig, httpClient, requestVariablesValues.toArray()); + httpSourceConfig = new HttpSourceConfig("http://localhost:8080/test", "POST", "{\"key\": \"%s\"}", "1", "", "", "123", "234", false, "type", "345", new HashMap<>(), null, "metricId_01", false); + HttpPostRequestHandler httpPostRequestBuilder = new HttpPostRequestHandler(httpSourceConfig, httpClient, requestVariablesValues.toArray(), dynamicHeaderVariablesValues.toArray()); assertEquals(request, httpPostRequestBuilder.create()); } + @Test + public void shouldBuildPostRequestWithOnlyDynamicHeader() { + when(httpClient.preparePost("http://localhost:8080/test")).thenReturn(request); + when(request.setBody("{\"key\": \"1\"}")).thenReturn(request); + httpSourceConfig = new HttpSourceConfig("http://localhost:8080/test", "POST", "{\"key\": \"%s\"}", "1", "{\"header_key\": \"%s\"}", "1", "123", "234", false, "type", "345", new HashMap<>(), null, "metricId_01", false); + HttpPostRequestHandler httpPostRequestBuilder = new HttpPostRequestHandler(httpSourceConfig, httpClient, requestVariablesValues.toArray(), dynamicHeaderVariablesValues.toArray()); + httpPostRequestBuilder.create(); + verify(request, times(1)).addHeader(anyString(), anyString()); + verify(request, times(1)).addHeader("header_key", "1"); + } + + @Test + public void shouldBuildPostRequestWithDynamicAndStaticHeader() { + when(httpClient.preparePost("http://localhost:8080/test")).thenReturn(request); + when(request.setBody("{\"key\": \"1\"}")).thenReturn(request); + HashMap staticHeader = new HashMap(); + staticHeader.put("static", "2"); + httpSourceConfig = new HttpSourceConfig("http://localhost:8080/test", "POST", "{\"key\": \"%s\"}", "1", "{\"header_key\": \"%s\"}", "1", "123", "234", false, "type", "345", staticHeader, null, "metricId_01", false); + HttpPostRequestHandler httpPostRequestBuilder = new HttpPostRequestHandler(httpSourceConfig, httpClient, requestVariablesValues.toArray(), dynamicHeaderVariablesValues.toArray()); + httpPostRequestBuilder.create(); + verify(request, times(2)).addHeader(anyString(), anyString()); + verify(request, times(1)).addHeader("header_key", "1"); + verify(request, times(1)).addHeader("static", "2"); + } + + @Test + public void shouldBuildPostRequestWithMultipleDynamicAndStaticHeaders() { + when(httpClient.preparePost("http://localhost:8080/test")).thenReturn(request); + when(request.setBody("{\"key\": \"1\"}")).thenReturn(request); + HashMap staticHeader = new HashMap(); + staticHeader.put("static", "3"); + httpSourceConfig = new HttpSourceConfig("http://localhost:8080/test", "POST", "{\"key\": \"%s\"}", "1", "{\"header_key_1\": \"%s\",\"header_key_2\": \"%s\"}", "1,2", "123", "234", false, "type", "345", staticHeader, null, "metricId_01", false); + HttpPostRequestHandler httpPostRequestBuilder = new HttpPostRequestHandler(httpSourceConfig, httpClient, requestVariablesValues.toArray(), dynamicHeaderVariablesValues.toArray()); + httpPostRequestBuilder.create(); + verify(request, times(3)).addHeader(anyString(), anyString()); + verify(request, times(1)).addHeader("header_key_1", "1"); + verify(request, times(1)).addHeader("header_key_2", "2"); + verify(request, times(1)).addHeader("static", "3"); + } + + @Test + public void shouldThrowErrorIfHeaderVariablesAreIncompatible() { + when(httpClient.preparePost("http://localhost:8080/test")).thenReturn(request); + when(request.setBody("{\"key\": \"1\"}")).thenReturn(request); + HashMap staticHeader = new HashMap(); + staticHeader.put("static", "3"); + ArrayList incompatibleHeaderVariablesValues = new ArrayList<>(); + incompatibleHeaderVariablesValues.add("test1"); + incompatibleHeaderVariablesValues.add("test12"); + httpSourceConfig = new HttpSourceConfig("http://localhost:8080/test", "POST", "{\"key\": \"%s\"}", "1", "{\"header_key_1\": \"%s\",\"header_key_2\": \"%d\"}", "1,2", "123", "234", false, "type", "345", staticHeader, null, "metricId_01", false); + HttpPostRequestHandler httpPostRequestBuilder = new HttpPostRequestHandler(httpSourceConfig, httpClient, requestVariablesValues.toArray(), incompatibleHeaderVariablesValues.toArray()); + InvalidConfigurationException exception = assertThrows(InvalidConfigurationException.class, () -> httpPostRequestBuilder.create()); + assertEquals("pattern config '{\"header_key_1\": \"%s\",\"header_key_2\": \"%d\"}' is incompatible with the variable config '1,2'", exception.getMessage()); + } + + @Test + public void shouldThrowErrorIfHeaderHeaderPatternIsInvalid() { + when(httpClient.preparePost("http://localhost:8080/test")).thenReturn(request); + when(request.setBody("{\"key\": \"1\"}")).thenReturn(request); + HashMap staticHeader = new HashMap(); + staticHeader.put("static", "3"); + httpSourceConfig = new HttpSourceConfig("http://localhost:8080/test", "POST", "{\"key\": \"%s\"}", "1", "{\"header_key_1\": \"%s\",\"header_key_2\": \"%p\"}", "1,2", "123", "234", false, "type", "345", staticHeader, null, "metricId_01", false); + HttpPostRequestHandler httpPostRequestBuilder = new HttpPostRequestHandler(httpSourceConfig, httpClient, requestVariablesValues.toArray(), dynamicHeaderVariablesValues.toArray()); + InvalidConfigurationException exception = assertThrows(InvalidConfigurationException.class, () -> httpPostRequestBuilder.create()); + assertEquals("pattern config '{\"header_key_1\": \"%s\",\"header_key_2\": \"%p\"}' is invalid", exception.getMessage()); + } } diff --git a/dagger-core/src/test/java/io/odpf/dagger/core/processors/external/http/request/HttpRequestFactoryTest.java b/dagger-core/src/test/java/io/odpf/dagger/core/processors/external/http/request/HttpRequestFactoryTest.java index 9e76146f6..0a6144954 100644 --- a/dagger-core/src/test/java/io/odpf/dagger/core/processors/external/http/request/HttpRequestFactoryTest.java +++ b/dagger-core/src/test/java/io/odpf/dagger/core/processors/external/http/request/HttpRequestFactoryTest.java @@ -25,22 +25,24 @@ public class HttpRequestFactoryTest { private HttpSourceConfig httpSourceConfig; private ArrayList requestVariablesValues; + private ArrayList headerVariablesValues; private boolean retainResponseType; @Before public void setup() { initMocks(this); requestVariablesValues = new ArrayList<>(); + headerVariablesValues = new ArrayList<>(); requestVariablesValues.add(1); retainResponseType = false; } @Test public void shouldReturnPostRequestOnTheBasisOfConfiguration() { - httpSourceConfig = new HttpSourceConfig("http://localhost:8080/test", "POST", "{\"key\": \"%s\"}", "1", "123", "234", false, "type", "345", new HashMap<>(), null, "metricId_01", retainResponseType); + httpSourceConfig = new HttpSourceConfig("http://localhost:8080/test", "POST", "{\"key\": \"%s\"}", "1", "", "", "123", "234", false, "type", "345", new HashMap<>(), null, "metricId_01", retainResponseType); when(httpClient.preparePost("http://localhost:8080/test")).thenReturn(request); when(request.setBody("{\"key\": \"123456\"}")).thenReturn(request); - HttpRequestFactory.createRequest(httpSourceConfig, httpClient, requestVariablesValues.toArray()); + HttpRequestFactory.createRequest(httpSourceConfig, httpClient, requestVariablesValues.toArray(), headerVariablesValues.toArray()); verify(httpClient, times(1)).preparePost("http://localhost:8080/test"); verify(httpClient, times(0)).prepareGet(any(String.class)); @@ -48,9 +50,9 @@ public void shouldReturnPostRequestOnTheBasisOfConfiguration() { @Test public void shouldReturnGetRequestOnTheBasisOfConfiguration() { - httpSourceConfig = new HttpSourceConfig("http://localhost:8080/test", "GET", "/key/%s", "1", "123", "234", false, "type", "345", new HashMap<>(), null, "metricId_01", retainResponseType); + httpSourceConfig = new HttpSourceConfig("http://localhost:8080/test", "GET", "/key/%s", "1", "", "", "123", "234", false, "type", "345", new HashMap<>(), null, "metricId_01", retainResponseType); when(httpClient.prepareGet("http://localhost:8080/test/key/1")).thenReturn(request); - HttpRequestFactory.createRequest(httpSourceConfig, httpClient, requestVariablesValues.toArray()); + HttpRequestFactory.createRequest(httpSourceConfig, httpClient, requestVariablesValues.toArray(), headerVariablesValues.toArray()); verify(httpClient, times(1)).prepareGet("http://localhost:8080/test/key/1"); verify(httpClient, times(0)).preparePost(any(String.class)); @@ -58,8 +60,8 @@ public void shouldReturnGetRequestOnTheBasisOfConfiguration() { @Test public void shouldThrowExceptionForUnsupportedHttpVerb() { - httpSourceConfig = new HttpSourceConfig("http://localhost:8080/test", "PATCH", "/key/%s", "1", "123", "234", false, "type", "345", new HashMap<>(), null, "metricId_01", retainResponseType); - assertThrows(InvalidHttpVerbException.class, () -> HttpRequestFactory.createRequest(httpSourceConfig, httpClient, requestVariablesValues.toArray())); + httpSourceConfig = new HttpSourceConfig("http://localhost:8080/test", "PATCH", "/key/%s", "1", "", "", "123", "234", false, "type", "345", new HashMap<>(), null, "metricId_01", retainResponseType); + assertThrows(InvalidHttpVerbException.class, () -> HttpRequestFactory.createRequest(httpSourceConfig, httpClient, requestVariablesValues.toArray(), headerVariablesValues.toArray())); } } diff --git a/dagger-core/src/test/java/io/odpf/dagger/core/processors/external/pg/PgAsyncConnectorTest.java b/dagger-core/src/test/java/io/odpf/dagger/core/processors/external/pg/PgAsyncConnectorTest.java index ec71872d5..21909f98e 100644 --- a/dagger-core/src/test/java/io/odpf/dagger/core/processors/external/pg/PgAsyncConnectorTest.java +++ b/dagger-core/src/test/java/io/odpf/dagger/core/processors/external/pg/PgAsyncConnectorTest.java @@ -158,12 +158,12 @@ public void shouldNotEnrichOutputWhenQueryVariableIsInvalid() throws Exception { ArgumentCaptor invalidConfigCaptor = ArgumentCaptor.forClass(InvalidConfigurationException.class); verify(resultFuture, times(1)).completeExceptionally(invalidConfigCaptor.capture()); - assertEquals("Column 'invalid_variable' not found as configured in the endpoint/query variable", invalidConfigCaptor.getValue().getMessage()); + assertEquals("Column 'invalid_variable' not found as configured in the 'QUERY_VARIABLES' variable", invalidConfigCaptor.getValue().getMessage()); verify(meterStatsManager, times(1)).markEvent(ExternalSourceAspects.INVALID_CONFIGURATION); ArgumentCaptor reportExceptionCaptor = ArgumentCaptor.forClass(InvalidConfigurationException.class); verify(errorReporter, times(1)).reportFatalException(reportExceptionCaptor.capture()); - assertEquals("Column 'invalid_variable' not found as configured in the endpoint/query variable", reportExceptionCaptor.getValue().getMessage()); + assertEquals("Column 'invalid_variable' not found as configured in the 'QUERY_VARIABLES' variable", reportExceptionCaptor.getValue().getMessage()); verify(pgClient, never()).query(any(String.class)); } diff --git a/dagger-functions/build.gradle b/dagger-functions/build.gradle index 476331d14..1303f9494 100644 --- a/dagger-functions/build.gradle +++ b/dagger-functions/build.gradle @@ -44,6 +44,9 @@ sourceSets { } dependencies { + compileOnly 'org.projectlombok:lombok:1.18.8' + annotationProcessor 'org.projectlombok:lombok:1.18.8' + compileOnly project(path: ':dagger-common', configuration: 'minimalCommonJar') compileOnly project(path: ':dagger-common', configuration: 'dependenciesCommonJar') compileOnly 'org.apache.flink:flink-streaming-java_2.11:' + flinkVersion @@ -56,10 +59,11 @@ dependencies { compileOnly group: 'org.apache.flink', name: 'flink-metrics-dropwizard', version: flinkVersion dependenciesFunctionsJar 'com.github.davidmoten:geo:0.7.6' + dependenciesFunctionsJar 'org.apache.flink:flink-python_2.11:' + flinkVersion dependenciesFunctionsJar group: 'org.apache.commons', name: 'commons-jexl3', version: '3.1' dependenciesFunctionsJar group: 'org.isuper', name: 's2-geometry-library-java', version: '0.0.1' dependenciesFunctionsJar group: 'com.google.cloud', name: 'google-cloud-storage', version: '1.67.0' - + testImplementation project(':dagger-common').sourceSets.test.output testImplementation group: 'junit', name: 'junit', version: '4.12' testImplementation 'org.mockito:mockito-core:2.0.99-beta' diff --git a/dagger-functions/src/main/java/io/odpf/dagger/functions/common/Constants.java b/dagger-functions/src/main/java/io/odpf/dagger/functions/common/Constants.java index 485bcd275..fff13666c 100644 --- a/dagger-functions/src/main/java/io/odpf/dagger/functions/common/Constants.java +++ b/dagger-functions/src/main/java/io/odpf/dagger/functions/common/Constants.java @@ -7,4 +7,17 @@ public class Constants { public static final String UDF_DART_GCS_PROJECT_ID_DEFAULT = ""; public static final String UDF_DART_GCS_BUCKET_ID_KEY = "UDF_DART_GCS_BUCKET_ID"; public static final String UDF_DART_GCS_BUCKET_ID_DEFAULT = ""; + + public static final String PYTHON_UDF_CONFIG = "PYTHON_UDF_CONFIG"; + public static final String PYTHON_UDF_ENABLE_KEY = "PYTHON_UDF_ENABLE"; + public static final boolean PYTHON_UDF_ENABLE_DEFAULT = false; + public static final String PYTHON_FILES_KEY = "PYTHON_FILES"; + public static final String PYTHON_REQUIREMENTS_KEY = "PYTHON_REQUIREMENTS"; + public static final String PYTHON_ARCHIVES_KEY = "PYTHON_ARCHIVES"; + public static final String PYTHON_FN_EXECUTION_ARROW_BATCH_SIZE_KEY = "PYTHON_FN_EXECUTION_ARROW_BATCH_SIZE"; + public static final Integer PYTHON_FN_EXECUTION_ARROW_BATCH_SIZE_DEFAULT = 10000; + public static final String PYTHON_FN_EXECUTION_BUNDLE_SIZE_KEY = "PYTHON_FN_EXECUTION_BUNDLE_SIZE"; + public static final Integer PYTHON_FN_EXECUTION_BUNDLE_SIZE_DEFAULT = 100000; + public static final String PYTHON_FN_EXECUTION_BUNDLE_TIME_KEY = "PYTHON_FN_EXECUTION_BUNDLE_TIME"; + public static final long PYTHON_FN_EXECUTION_BUNDLE_TIME_DEFAULT = 1000; } diff --git a/dagger-functions/src/main/java/io/odpf/dagger/functions/exceptions/PythonFilesEmptyException.java b/dagger-functions/src/main/java/io/odpf/dagger/functions/exceptions/PythonFilesEmptyException.java new file mode 100644 index 000000000..e849dc208 --- /dev/null +++ b/dagger-functions/src/main/java/io/odpf/dagger/functions/exceptions/PythonFilesEmptyException.java @@ -0,0 +1,17 @@ +package io.odpf.dagger.functions.exceptions; + +/** + * The type Python files empty exception. + */ +public class PythonFilesEmptyException extends RuntimeException { + + /** + * Instantiates a new Python files empty exception. + * + * @param message the message + */ + public PythonFilesEmptyException(String message) { + super(message); + } + +} diff --git a/dagger-functions/src/main/java/io/odpf/dagger/functions/exceptions/PythonFilesFormatException.java b/dagger-functions/src/main/java/io/odpf/dagger/functions/exceptions/PythonFilesFormatException.java new file mode 100644 index 000000000..02771fda6 --- /dev/null +++ b/dagger-functions/src/main/java/io/odpf/dagger/functions/exceptions/PythonFilesFormatException.java @@ -0,0 +1,17 @@ +package io.odpf.dagger.functions.exceptions; + +/** + * The type Python files format exception. + */ +public class PythonFilesFormatException extends RuntimeException { + + /** + * Instantiates a new Python files format exception. + * + * @param message the message + */ + public PythonFilesFormatException(String message) { + super(message); + } + +} diff --git a/dagger-functions/src/main/java/io/odpf/dagger/functions/udfs/python/PythonUdfConfig.java b/dagger-functions/src/main/java/io/odpf/dagger/functions/udfs/python/PythonUdfConfig.java new file mode 100644 index 000000000..2f33f75c8 --- /dev/null +++ b/dagger-functions/src/main/java/io/odpf/dagger/functions/udfs/python/PythonUdfConfig.java @@ -0,0 +1,110 @@ +package io.odpf.dagger.functions.udfs.python; + +import com.google.gson.Gson; +import com.google.gson.GsonBuilder; +import com.google.gson.annotations.SerializedName; +import io.odpf.dagger.common.configuration.Configuration; +import lombok.Getter; + +import static io.odpf.dagger.functions.common.Constants.*; + +/** + * The type Python udf config. + */ +public class PythonUdfConfig { + private static final Gson GSON = new GsonBuilder() + .enableComplexMapKeySerialization() + .setPrettyPrinting() + .create(); + + @SerializedName(PYTHON_FILES_KEY) + private String pythonFiles; + + @SerializedName(PYTHON_REQUIREMENTS_KEY) + @Getter + private String pythonRequirements; + + @SerializedName(PYTHON_ARCHIVES_KEY) + private String pythonArchives; + + @SerializedName(PYTHON_FN_EXECUTION_ARROW_BATCH_SIZE_KEY) + private Integer pythonArrowBatchSize; + + @SerializedName(PYTHON_FN_EXECUTION_BUNDLE_SIZE_KEY) + private Integer pythonBundleSize; + + @SerializedName(PYTHON_FN_EXECUTION_BUNDLE_TIME_KEY) + private Long pythonBundleTime; + + /** + * Gets python files. + * + * @return the python files + */ + public String getPythonFiles() { + if (pythonFiles != null) { + return pythonFiles.replaceAll("\\s+", ""); + } + return null; + } + + /** + * Gets python archives. + * + * @return the python archives + */ + public String getPythonArchives() { + if (pythonArchives != null) { + return pythonArchives.replaceAll("\\s+", ""); + } + return null; + } + + /** + * Gets python arrow batch size. + * + * @return the python arrow batch size + */ + public int getPythonArrowBatchSize() { + if (pythonArrowBatchSize == null) { + return PYTHON_FN_EXECUTION_ARROW_BATCH_SIZE_DEFAULT; + } + return pythonArrowBatchSize; + } + + /** + * Gets python bundle size. + * + * @return the python bundle size + */ + public int getPythonBundleSize() { + if (pythonBundleSize == null) { + return PYTHON_FN_EXECUTION_BUNDLE_SIZE_DEFAULT; + } + return pythonBundleSize; + } + + /** + * Gets python bundle time. + * + * @return the python bundle time + */ + public long getPythonBundleTime() { + if (pythonBundleTime == null) { + return PYTHON_FN_EXECUTION_BUNDLE_TIME_DEFAULT; + } + return pythonBundleTime; + } + + /** + * Parse python udf config. + * + * @param configuration the configuration + * @return the python udf config + */ + public static PythonUdfConfig parse(Configuration configuration) { + String jsonString = configuration.getString(PYTHON_UDF_CONFIG, ""); + + return GSON.fromJson(jsonString, PythonUdfConfig.class); + } +} diff --git a/dagger-functions/src/main/java/io/odpf/dagger/functions/udfs/python/PythonUdfManager.java b/dagger-functions/src/main/java/io/odpf/dagger/functions/udfs/python/PythonUdfManager.java new file mode 100644 index 000000000..59fb99d2a --- /dev/null +++ b/dagger-functions/src/main/java/io/odpf/dagger/functions/udfs/python/PythonUdfManager.java @@ -0,0 +1,81 @@ +package io.odpf.dagger.functions.udfs.python; + +import io.odpf.dagger.functions.exceptions.PythonFilesEmptyException; +import io.odpf.dagger.functions.udfs.python.file.type.FileType; +import io.odpf.dagger.functions.udfs.python.file.type.FileTypeFactory; +import org.apache.flink.table.api.bridge.java.StreamTableEnvironment; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; + +/** + * The type Python udf manager. + */ +public class PythonUdfManager { + + private StreamTableEnvironment tableEnvironment; + private PythonUdfConfig pythonUdfConfig; + + /** + * Instantiates a new Python udf manager. + * + * @param tableEnvironment the table environment + * @param pythonUdfConfig the python udf config + */ + public PythonUdfManager(StreamTableEnvironment tableEnvironment, PythonUdfConfig pythonUdfConfig) { + this.tableEnvironment = tableEnvironment; + this.pythonUdfConfig = pythonUdfConfig; + } + + /** + * Register python functions. + */ + public void registerPythonFunctions() throws IOException { + String inputFiles = pythonUdfConfig.getPythonFiles(); + String[] pythonFiles; + if (inputFiles != null) { + registerPythonConfig(); + pythonFiles = inputFiles.split(","); + } else { + throw new PythonFilesEmptyException("Python files can not be null"); + } + + for (String pythonFile : pythonFiles) { + FileType fileType = FileTypeFactory.getFileType(pythonFile); + List fileNames = fileType.getFileNames(); + List sqlQueries = createQuery(fileNames); + executeSql(sqlQueries); + } + } + + private void registerPythonConfig() { + if (pythonUdfConfig.getPythonRequirements() != null) { + tableEnvironment.getConfig().getConfiguration().setString("python.requirements", pythonUdfConfig.getPythonRequirements()); + } + if (pythonUdfConfig.getPythonArchives() != null) { + tableEnvironment.getConfig().getConfiguration().setString("python.archives", pythonUdfConfig.getPythonArchives()); + } + tableEnvironment.getConfig().getConfiguration().setString("python.files", pythonUdfConfig.getPythonFiles()); + tableEnvironment.getConfig().getConfiguration().setInteger("python.fn-execution.arrow.batch.size", pythonUdfConfig.getPythonArrowBatchSize()); + tableEnvironment.getConfig().getConfiguration().setInteger("python.fn-execution.bundle.size", pythonUdfConfig.getPythonBundleSize()); + tableEnvironment.getConfig().getConfiguration().setLong("python.fn-execution.bundle.time", pythonUdfConfig.getPythonBundleTime()); + } + + private void executeSql(List sqlQueries) { + for (String query : sqlQueries) { + tableEnvironment.executeSql(query); + } + } + + private List createQuery(List fileNames) { + List sqlQueries = new ArrayList<>(); + for (String fileName : fileNames) { + fileName = fileName.replace(".py", "").replace("/", "."); + String functionName = fileName.substring(fileName.lastIndexOf(".") + 1); + String query = "CREATE TEMPORARY FUNCTION " + functionName.toUpperCase() + " AS '" + fileName + "." + functionName + "' LANGUAGE PYTHON"; + sqlQueries.add(query); + } + return sqlQueries; + } +} diff --git a/dagger-functions/src/main/java/io/odpf/dagger/functions/udfs/python/file/source/FileSource.java b/dagger-functions/src/main/java/io/odpf/dagger/functions/udfs/python/file/source/FileSource.java new file mode 100644 index 000000000..915f81d32 --- /dev/null +++ b/dagger-functions/src/main/java/io/odpf/dagger/functions/udfs/python/file/source/FileSource.java @@ -0,0 +1,16 @@ +package io.odpf.dagger.functions.udfs.python.file.source; + +import java.io.IOException; + +/** + * The interface File source. + */ +public interface FileSource { + + /** + * Get object file byte [ ]. + * + * @return the byte [ ] + */ + byte[] getObjectFile() throws IOException; +} diff --git a/dagger-functions/src/main/java/io/odpf/dagger/functions/udfs/python/file/source/FileSourceFactory.java b/dagger-functions/src/main/java/io/odpf/dagger/functions/udfs/python/file/source/FileSourceFactory.java new file mode 100644 index 000000000..8053f19fd --- /dev/null +++ b/dagger-functions/src/main/java/io/odpf/dagger/functions/udfs/python/file/source/FileSourceFactory.java @@ -0,0 +1,29 @@ +package io.odpf.dagger.functions.udfs.python.file.source; + +import io.odpf.dagger.functions.udfs.python.file.source.gcs.GcsFileSource; +import io.odpf.dagger.functions.udfs.python.file.source.local.LocalFileSource; + +/** + * The type File source factory. + */ +public class FileSourceFactory { + + /** + * Gets file source. + * + * @param pythonFile the python file + * @return the file source + */ + public static FileSource getFileSource(String pythonFile) { + if ("GS".equals(getFileSourcePrefix(pythonFile))) { + return new GcsFileSource(pythonFile); + } else { + return new LocalFileSource(pythonFile); + } + } + + private static String getFileSourcePrefix(String pythonFile) { + String[] files = pythonFile.split("://"); + return files[0].toUpperCase(); + } +} diff --git a/dagger-functions/src/main/java/io/odpf/dagger/functions/udfs/python/file/source/gcs/GcsClient.java b/dagger-functions/src/main/java/io/odpf/dagger/functions/udfs/python/file/source/gcs/GcsClient.java new file mode 100644 index 000000000..e3bd69a3f --- /dev/null +++ b/dagger-functions/src/main/java/io/odpf/dagger/functions/udfs/python/file/source/gcs/GcsClient.java @@ -0,0 +1,56 @@ +package io.odpf.dagger.functions.udfs.python.file.source.gcs; + +import com.google.cloud.storage.Blob; +import com.google.cloud.storage.BlobId; +import com.google.cloud.storage.Storage; +import com.google.cloud.storage.StorageOptions; + +import java.util.Arrays; +import java.util.List; +import java.util.stream.Collectors; + +/** + * The type Gcs client. + */ +public class GcsClient { + + private Storage storage; + + /** + * Instantiates a new Gcs client. + */ + public GcsClient() { + + if (storage == null) { + storage = StorageOptions.newBuilder() + .build().getService(); + } + } + + /** + * Instantiates a new Gcs client. + * This constructor used for unit test purposes. + * + * @param storage the storage + */ + public GcsClient(Storage storage) { + this.storage = storage; + } + + /** + * Get file byte [ ]. + * + * @param pythonFile the python file + * @return the byte [ ] + */ + public byte[] getFile(String pythonFile) { + List file = Arrays.asList(pythonFile.replace("gs://", "").split("/")); + + String bucketName = file.get(0); + String objectName = file.stream().skip(1).collect(Collectors.joining("/")); + + Blob blob = storage.get(BlobId.of(bucketName, objectName)); + + return blob.getContent(); + } +} diff --git a/dagger-functions/src/main/java/io/odpf/dagger/functions/udfs/python/file/source/gcs/GcsFileSource.java b/dagger-functions/src/main/java/io/odpf/dagger/functions/udfs/python/file/source/gcs/GcsFileSource.java new file mode 100644 index 000000000..d7b7a5bef --- /dev/null +++ b/dagger-functions/src/main/java/io/odpf/dagger/functions/udfs/python/file/source/gcs/GcsFileSource.java @@ -0,0 +1,51 @@ +package io.odpf.dagger.functions.udfs.python.file.source.gcs; + +import io.odpf.dagger.functions.udfs.python.file.source.FileSource; + + +/** + * The type Gcs file source. + */ +public class GcsFileSource implements FileSource { + + private GcsClient gcsClient; + private String pythonFile; + + /** + * Instantiates a new Gcs file source. + * + * @param pythonFile the python file + */ + public GcsFileSource(String pythonFile) { + this.pythonFile = pythonFile; + } + + /** + * Instantiates a new Gcs file source. + * This constructor used for unit test purposes. + * + * @param pythonFile the python file + * @param gcsClient the gcs client + */ + public GcsFileSource(String pythonFile, GcsClient gcsClient) { + this.pythonFile = pythonFile; + this.gcsClient = gcsClient; + } + + @Override + public byte[] getObjectFile() { + return getGcsClient().getFile(pythonFile); + } + + /** + * Gets gcs client. + * + * @return the gcs client + */ + private GcsClient getGcsClient() { + if (this.gcsClient == null) { + this.gcsClient = new GcsClient(); + } + return this.gcsClient; + } +} diff --git a/dagger-functions/src/main/java/io/odpf/dagger/functions/udfs/python/file/source/local/LocalFileSource.java b/dagger-functions/src/main/java/io/odpf/dagger/functions/udfs/python/file/source/local/LocalFileSource.java new file mode 100644 index 000000000..3f3aff624 --- /dev/null +++ b/dagger-functions/src/main/java/io/odpf/dagger/functions/udfs/python/file/source/local/LocalFileSource.java @@ -0,0 +1,29 @@ +package io.odpf.dagger.functions.udfs.python.file.source.local; + +import io.odpf.dagger.functions.udfs.python.file.source.FileSource; + +import java.io.IOException; +import java.nio.file.Files; +import java.nio.file.Paths; + +/** + * The type Local file source. + */ +public class LocalFileSource implements FileSource { + + private String pythonFile; + + /** + * Instantiates a new Local file source. + * + * @param pythonFile the python file + */ + public LocalFileSource(String pythonFile) { + this.pythonFile = pythonFile; + } + + @Override + public byte[] getObjectFile() throws IOException { + return Files.readAllBytes(Paths.get(pythonFile)); + } +} diff --git a/dagger-functions/src/main/java/io/odpf/dagger/functions/udfs/python/file/type/FileType.java b/dagger-functions/src/main/java/io/odpf/dagger/functions/udfs/python/file/type/FileType.java new file mode 100644 index 000000000..1123b42f1 --- /dev/null +++ b/dagger-functions/src/main/java/io/odpf/dagger/functions/udfs/python/file/type/FileType.java @@ -0,0 +1,17 @@ +package io.odpf.dagger.functions.udfs.python.file.type; + +import java.io.IOException; +import java.util.List; + +/** + * The interface File type. + */ +public interface FileType { + + /** + * Gets file names. + * + * @return the file names + */ + List getFileNames() throws IOException; +} diff --git a/dagger-functions/src/main/java/io/odpf/dagger/functions/udfs/python/file/type/FileTypeFactory.java b/dagger-functions/src/main/java/io/odpf/dagger/functions/udfs/python/file/type/FileTypeFactory.java new file mode 100644 index 000000000..83543ad19 --- /dev/null +++ b/dagger-functions/src/main/java/io/odpf/dagger/functions/udfs/python/file/type/FileTypeFactory.java @@ -0,0 +1,34 @@ +package io.odpf.dagger.functions.udfs.python.file.type; + +import io.odpf.dagger.functions.exceptions.PythonFilesFormatException; +import io.odpf.dagger.functions.udfs.python.file.source.FileSource; +import io.odpf.dagger.functions.udfs.python.file.source.FileSourceFactory; + +/** + * The type File type factory. + */ +public class FileTypeFactory { + + /** + * Gets file type. + * + * @param pythonFile the python file + * @return the file type + */ + public static FileType getFileType(String pythonFile) { + FileSource fileSource = FileSourceFactory.getFileSource(pythonFile); + switch (getFileTypeFormat(pythonFile)) { + case "PY": + return new PythonFileType(pythonFile); + case "ZIP": + return new ZipFileType(fileSource); + default: + throw new PythonFilesFormatException("Python files should be in .py or .zip format"); + } + } + + private static String getFileTypeFormat(String pythonFile) { + String[] files = pythonFile.split("\\."); + return files[files.length - 1].toUpperCase(); + } +} diff --git a/dagger-functions/src/main/java/io/odpf/dagger/functions/udfs/python/file/type/PythonFileType.java b/dagger-functions/src/main/java/io/odpf/dagger/functions/udfs/python/file/type/PythonFileType.java new file mode 100644 index 000000000..2bc993e04 --- /dev/null +++ b/dagger-functions/src/main/java/io/odpf/dagger/functions/udfs/python/file/type/PythonFileType.java @@ -0,0 +1,33 @@ +package io.odpf.dagger.functions.udfs.python.file.type; + +import io.odpf.dagger.functions.exceptions.PythonFilesEmptyException; + +import java.util.Collections; +import java.util.List; + +/** + * The type Python file type. + */ +public class PythonFileType implements FileType { + + private String pythonFile; + + /** + * Instantiates a new Python file type. + * + * @param pythonFile the python file + */ + public PythonFileType(String pythonFile) { + this.pythonFile = pythonFile; + } + + @Override + public List getFileNames() { + if (pythonFile == null) { + throw new PythonFilesEmptyException("Python files can not be null"); + } + String name = pythonFile.substring(pythonFile.lastIndexOf('/') + 1); + + return Collections.singletonList(name); + } +} diff --git a/dagger-functions/src/main/java/io/odpf/dagger/functions/udfs/python/file/type/ZipFileType.java b/dagger-functions/src/main/java/io/odpf/dagger/functions/udfs/python/file/type/ZipFileType.java new file mode 100644 index 000000000..31fc00cd0 --- /dev/null +++ b/dagger-functions/src/main/java/io/odpf/dagger/functions/udfs/python/file/type/ZipFileType.java @@ -0,0 +1,47 @@ +package io.odpf.dagger.functions.udfs.python.file.type; + +import io.odpf.dagger.functions.udfs.python.file.source.FileSource; + +import java.io.ByteArrayInputStream; +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; +import java.util.zip.ZipEntry; +import java.util.zip.ZipInputStream; + +/** + * The type Zip file type. + */ +public class ZipFileType implements FileType { + + private FileSource fileSource; + + public ZipFileType(FileSource fileSource) { + this.fileSource = fileSource; + } + + @Override + public List getFileNames() throws IOException { + byte[] object = fileSource.getObjectFile(); + + ZipInputStream zi = new ZipInputStream(new ByteArrayInputStream(object)); + ZipEntry zipEntry; + List entries = new ArrayList<>(); + while ((zipEntry = zi.getNextEntry()) != null) { + entries.add(zipEntry); + } + + List fileNames = new ArrayList<>(); + for (ZipEntry entry : entries) { + String name = entry.getName(); + if (isPythonFile(name)) { + fileNames.add(name); + } + } + return fileNames; + } + + private boolean isPythonFile(String fileName) { + return fileName.endsWith(".py"); + } +} diff --git a/dagger-functions/src/test/java/io/odpf/dagger/functions/udfs/python/PythonUdfConfigTest.java b/dagger-functions/src/test/java/io/odpf/dagger/functions/udfs/python/PythonUdfConfigTest.java new file mode 100644 index 000000000..01be1b7af --- /dev/null +++ b/dagger-functions/src/test/java/io/odpf/dagger/functions/udfs/python/PythonUdfConfigTest.java @@ -0,0 +1,82 @@ +package io.odpf.dagger.functions.udfs.python; + +import io.odpf.dagger.common.configuration.Configuration; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; +import org.mockito.Mock; + +import static io.odpf.dagger.functions.common.Constants.*; +import static org.mockito.Mockito.when; +import static org.mockito.MockitoAnnotations.initMocks; + +public class PythonUdfConfigTest { + + @Mock + private Configuration configuration; + + @Before + public void setup() { + initMocks(this); + } + + @Test + public void shouldParseConfig() { + String pythonJsonConfig = "{ \"PYTHON_FILES\": \"/path/to/function.zip\", \"PYTHON_ARCHIVES\": \"/path/to/file.txt\", \"PYTHON_REQUIREMENTS\": \"requirements.txt\", \"PYTHON_FN_EXECUTION_ARROW_BATCH_SIZE\": \"10000\", \"PYTHON_FN_EXECUTION_BUNDLE_SIZE\": \"100000\", \"PYTHON_FN_EXECUTION_BUNDLE_TIME\": \"1000\" }"; + + when(configuration.getString(PYTHON_UDF_CONFIG, "")).thenReturn(pythonJsonConfig); + PythonUdfConfig pythonUdfConfig = PythonUdfConfig.parse(configuration); + + Assert.assertNotNull(pythonUdfConfig); + Assert.assertEquals(pythonUdfConfig.getPythonFiles(), "/path/to/function.zip"); + Assert.assertEquals(pythonUdfConfig.getPythonArchives(), "/path/to/file.txt"); + Assert.assertEquals(pythonUdfConfig.getPythonRequirements(), "requirements.txt"); + Assert.assertEquals(pythonUdfConfig.getPythonArrowBatchSize(), 10000); + Assert.assertEquals(pythonUdfConfig.getPythonBundleSize(), 100000); + Assert.assertEquals(pythonUdfConfig.getPythonBundleTime(), 1000); + } + + @Test + public void shouldUseDefaultValueIfConfigIsNotGiven() { + String pythonJsonConfig = "{ \"PYTHON_FILES\": \"/path/to/function.zip\", \"PYTHON_ARCHIVES\": \"/path/to/file.txt\", \"PYTHON_REQUIREMENTS\": \"requirements.txt\" }"; + + when(configuration.getString(PYTHON_UDF_CONFIG, "")).thenReturn(pythonJsonConfig); + PythonUdfConfig pythonUdfConfig = PythonUdfConfig.parse(configuration); + + Assert.assertEquals(pythonUdfConfig.getPythonArrowBatchSize(), 10000); + Assert.assertEquals(pythonUdfConfig.getPythonBundleSize(), 100000); + Assert.assertEquals(pythonUdfConfig.getPythonBundleTime(), 1000); + } + + @Test + public void shouldReturnNullIfPythonFilesConfigIsNotGiven() { + String pythonJsonConfig = "{\"PYTHON_FN_EXECUTION_ARROW_BATCH_SIZE\": \"10000\", \"PYTHON_FN_EXECUTION_BUNDLE_SIZE\": \"100000\", \"PYTHON_FN_EXECUTION_BUNDLE_TIME\": \"1000\"}"; + + when(configuration.getString(PYTHON_UDF_CONFIG, "")).thenReturn(pythonJsonConfig); + PythonUdfConfig pythonUdfConfig = PythonUdfConfig.parse(configuration); + + Assert.assertNull(pythonUdfConfig.getPythonFiles()); + Assert.assertNull(pythonUdfConfig.getPythonArchives()); + Assert.assertNull(pythonUdfConfig.getPythonRequirements()); + } + + @Test + public void shouldRemoveWhitespaceInPythonFilesConfig() { + String pythonJsonConfig = "{ \"PYTHON_FILES\": \" /path/to/function.zip, /path/to/files/test.py \"}"; + + when(configuration.getString(PYTHON_UDF_CONFIG, "")).thenReturn(pythonJsonConfig); + PythonUdfConfig pythonUdfConfig = PythonUdfConfig.parse(configuration); + + Assert.assertEquals(pythonUdfConfig.getPythonFiles(), "/path/to/function.zip,/path/to/files/test.py"); + } + + @Test + public void shouldRemoveWhitespaceInPythonArchivesConfig() { + String pythonJsonConfig = "{ \"PYTHON_FILES\": \"/path/to/function.zip\", \"PYTHON_ARCHIVES\": \" /path/to/data.zip, /path/to/files/second_data.zip \"}"; + + when(configuration.getString(PYTHON_UDF_CONFIG, "")).thenReturn(pythonJsonConfig); + PythonUdfConfig pythonUdfConfig = PythonUdfConfig.parse(configuration); + + Assert.assertEquals(pythonUdfConfig.getPythonArchives(), "/path/to/data.zip,/path/to/files/second_data.zip"); + } +} diff --git a/dagger-functions/src/test/java/io/odpf/dagger/functions/udfs/python/PythonUdfManagerTest.java b/dagger-functions/src/test/java/io/odpf/dagger/functions/udfs/python/PythonUdfManagerTest.java new file mode 100644 index 000000000..d7a210db6 --- /dev/null +++ b/dagger-functions/src/test/java/io/odpf/dagger/functions/udfs/python/PythonUdfManagerTest.java @@ -0,0 +1,190 @@ +package io.odpf.dagger.functions.udfs.python; + +import io.odpf.dagger.functions.exceptions.PythonFilesFormatException; +import io.odpf.dagger.functions.exceptions.PythonFilesEmptyException; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.table.api.TableConfig; +import org.apache.flink.table.api.bridge.java.StreamTableEnvironment; +import org.junit.Before; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.ExpectedException; +import org.mockito.Mock; + +import java.io.File; +import java.io.IOException; + +import static org.mockito.Mockito.times; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.when; + +import static org.mockito.MockitoAnnotations.initMocks; + +public class PythonUdfManagerTest { + + @Rule + public ExpectedException expectedEx = ExpectedException.none(); + + @Mock + private StreamTableEnvironment tableEnvironment; + + @Mock + private PythonUdfConfig pythonUdfConfig; + + @Mock + private TableConfig tableConfig; + + @Mock + private Configuration configuration; + + @Before + public void setup() { + initMocks(this); + } + + @Test + public void shouldRegisterPythonUdfConfig() throws IOException { + String pathFile = getPath("python_udf.zip"); + + when(tableEnvironment.getConfig()).thenReturn(tableConfig); + when(tableConfig.getConfiguration()).thenReturn(configuration); + when(pythonUdfConfig.getPythonFiles()).thenReturn(pathFile); + when(pythonUdfConfig.getPythonArchives()).thenReturn("/path/to/file.txt"); + when(pythonUdfConfig.getPythonRequirements()).thenReturn("requirements.txt"); + when(pythonUdfConfig.getPythonArrowBatchSize()).thenReturn(10000); + when(pythonUdfConfig.getPythonBundleSize()).thenReturn(100000); + when(pythonUdfConfig.getPythonBundleTime()).thenReturn(1000L); + + PythonUdfManager pythonUdfManager = new PythonUdfManager(tableEnvironment, pythonUdfConfig); + pythonUdfManager.registerPythonFunctions(); + + verify(configuration, times(1)).setString("python.files", pathFile); + verify(configuration, times(1)).setString("python.archives", "/path/to/file.txt"); + verify(configuration, times(1)).setString("python.requirements", "requirements.txt"); + verify(configuration, times(1)).setInteger("python.fn-execution.arrow.batch.size", 10000); + verify(configuration, times(1)).setInteger("python.fn-execution.bundle.size", 100000); + verify(configuration, times(1)).setLong("python.fn-execution.bundle.time", 1000); + } + + @Test + public void shouldNotRegisterConfigIfNotSet() throws IOException { + String pathFile = getPath("python_udf.zip"); + + when(tableEnvironment.getConfig()).thenReturn(tableConfig); + when(tableConfig.getConfiguration()).thenReturn(configuration); + when(pythonUdfConfig.getPythonFiles()).thenReturn(pathFile); + + PythonUdfManager pythonUdfManager = new PythonUdfManager(tableEnvironment, pythonUdfConfig); + pythonUdfManager.registerPythonFunctions(); + + verify(configuration, times(1)).setString("python.files", pathFile); + verify(configuration, times(0)).setString("python.archives", "/path/to/file.txt"); + verify(configuration, times(0)).setString("python.requirements", "requirements.txt"); + } + + @Test + public void shouldRegisterPythonUdfFromPyFile() throws IOException { + String pathFile = getPath("test_udf.py"); + String sqlRegisterUdf = "CREATE TEMPORARY FUNCTION TEST_UDF AS 'test_udf.test_udf' LANGUAGE PYTHON"; + + when(tableEnvironment.getConfig()).thenReturn(tableConfig); + when(tableConfig.getConfiguration()).thenReturn(configuration); + when(pythonUdfConfig.getPythonFiles()).thenReturn(pathFile); + + PythonUdfManager pythonUdfManager = new PythonUdfManager(tableEnvironment, pythonUdfConfig); + pythonUdfManager.registerPythonFunctions(); + + verify(configuration, times(1)).setString("python.files", pathFile); + verify(tableEnvironment, times(1)).executeSql(sqlRegisterUdf); + } + + @Test + public void shouldOnlyExecutePyFormatInsideZipFile() throws IOException { + String pathFile = getPath("python_udf.zip"); + + String sqlRegisterFirstUdf = "CREATE TEMPORARY FUNCTION MULTIPLY AS 'python_udf.scalar.multiply.multiply' LANGUAGE PYTHON"; + String sqlRegisterSecondUdf = "CREATE TEMPORARY FUNCTION ADD AS 'python_udf.scalar.add.add' LANGUAGE PYTHON"; + String sqlRegisterThirdUdf = "CREATE TEMPORARY FUNCTION SUBSTRACT AS 'python_udf.vectorized.substract.substract' LANGUAGE PYTHON"; + + when(tableEnvironment.getConfig()).thenReturn(tableConfig); + when(tableConfig.getConfiguration()).thenReturn(configuration); + when(pythonUdfConfig.getPythonFiles()).thenReturn(pathFile); + + PythonUdfManager pythonUdfManager = new PythonUdfManager(tableEnvironment, pythonUdfConfig); + pythonUdfManager.registerPythonFunctions(); + + verify(configuration, times(1)).setString("python.files", pathFile); + verify(tableEnvironment, times(0)).executeSql(sqlRegisterFirstUdf); + verify(tableEnvironment, times(1)).executeSql(sqlRegisterSecondUdf); + verify(tableEnvironment, times(1)).executeSql(sqlRegisterThirdUdf); + } + + @Test + public void shouldRegisterPythonUdfFromPyAndZipFile() throws IOException { + String zipPathFile = getPath("python_udf.zip"); + String pyPathFile = getPath("test_udf.py"); + + String sqlRegisterFirstUdf = "CREATE TEMPORARY FUNCTION ADD AS 'python_udf.scalar.add.add' LANGUAGE PYTHON"; + String sqlRegisterSecondUdf = "CREATE TEMPORARY FUNCTION SUBSTRACT AS 'python_udf.vectorized.substract.substract' LANGUAGE PYTHON"; + String sqlRegisterThirdUdf = "CREATE TEMPORARY FUNCTION TEST_UDF AS 'test_udf.test_udf' LANGUAGE PYTHON"; + + when(tableEnvironment.getConfig()).thenReturn(tableConfig); + when(tableConfig.getConfiguration()).thenReturn(configuration); + when(pythonUdfConfig.getPythonFiles()).thenReturn(zipPathFile + "," + pyPathFile); + + PythonUdfManager pythonUdfManager = new PythonUdfManager(tableEnvironment, pythonUdfConfig); + pythonUdfManager.registerPythonFunctions(); + + verify(configuration, times(1)).setString("python.files", zipPathFile + "," + pyPathFile); + verify(tableEnvironment, times(1)).executeSql(sqlRegisterFirstUdf); + verify(tableEnvironment, times(1)).executeSql(sqlRegisterSecondUdf); + verify(tableEnvironment, times(1)).executeSql(sqlRegisterThirdUdf); + } + + @Test + public void shouldThrowExceptionIfPythonFilesNotInZipOrPyFormat() throws IOException { + expectedEx.expect(PythonFilesFormatException.class); + expectedEx.expectMessage("Python files should be in .py or .zip format"); + + File file = File.createTempFile("test_file", ".txt"); + file.deleteOnExit(); + + when(tableEnvironment.getConfig()).thenReturn(tableConfig); + when(tableConfig.getConfiguration()).thenReturn(configuration); + when(pythonUdfConfig.getPythonFiles()).thenReturn("test_file.txt"); + + PythonUdfManager pythonUdfManager = new PythonUdfManager(tableEnvironment, pythonUdfConfig); + pythonUdfManager.registerPythonFunctions(); + } + + @Test + public void shouldThrowExceptionIfPythonFilesIsEmpty() throws IOException { + expectedEx.expect(PythonFilesFormatException.class); + expectedEx.expectMessage("Python files should be in .py or .zip format"); + + when(tableEnvironment.getConfig()).thenReturn(tableConfig); + when(tableConfig.getConfiguration()).thenReturn(configuration); + when(pythonUdfConfig.getPythonFiles()).thenReturn(""); + + PythonUdfManager pythonUdfManager = new PythonUdfManager(tableEnvironment, pythonUdfConfig); + pythonUdfManager.registerPythonFunctions(); + } + + @Test + public void shouldThrowExceptionIfPythonFilesIsNull() throws IOException { + expectedEx.expect(PythonFilesEmptyException.class); + expectedEx.expectMessage("Python files can not be null"); + + when(tableEnvironment.getConfig()).thenReturn(tableConfig); + when(tableConfig.getConfiguration()).thenReturn(configuration); + + PythonUdfManager pythonUdfManager = new PythonUdfManager(tableEnvironment, pythonUdfConfig); + pythonUdfManager.registerPythonFunctions(); + } + + private String getPath(String filename) { + ClassLoader classLoader = getClass().getClassLoader(); + + return classLoader.getResource(filename).getPath(); + } +} diff --git a/dagger-functions/src/test/java/io/odpf/dagger/functions/udfs/python/file/source/FileSourceFactoryTest.java b/dagger-functions/src/test/java/io/odpf/dagger/functions/udfs/python/file/source/FileSourceFactoryTest.java new file mode 100644 index 000000000..2cd5a472f --- /dev/null +++ b/dagger-functions/src/test/java/io/odpf/dagger/functions/udfs/python/file/source/FileSourceFactoryTest.java @@ -0,0 +1,27 @@ +package io.odpf.dagger.functions.udfs.python.file.source; + +import io.odpf.dagger.functions.udfs.python.file.source.gcs.GcsFileSource; +import io.odpf.dagger.functions.udfs.python.file.source.local.LocalFileSource; +import org.junit.Assert; +import org.junit.Test; + +public class FileSourceFactoryTest { + + @Test + public void shouldGetLocalFileSource() { + String pythonFile = "/path/to/file/test_function.py"; + + FileSource fileSource = FileSourceFactory.getFileSource(pythonFile); + + Assert.assertTrue(fileSource instanceof LocalFileSource); + } + + @Test + public void shouldGetGcsFileSource() { + String pythonFile = "gs://bucket-name/path/to/file/test_function.py"; + + FileSource fileSource = FileSourceFactory.getFileSource(pythonFile); + + Assert.assertTrue(fileSource instanceof GcsFileSource); + } +} diff --git a/dagger-functions/src/test/java/io/odpf/dagger/functions/udfs/python/file/source/gcs/GcsClientTest.java b/dagger-functions/src/test/java/io/odpf/dagger/functions/udfs/python/file/source/gcs/GcsClientTest.java new file mode 100644 index 000000000..5c0c7950e --- /dev/null +++ b/dagger-functions/src/test/java/io/odpf/dagger/functions/udfs/python/file/source/gcs/GcsClientTest.java @@ -0,0 +1,47 @@ +package io.odpf.dagger.functions.udfs.python.file.source.gcs; + +import com.google.cloud.storage.Blob; +import com.google.cloud.storage.BlobId; +import com.google.cloud.storage.Storage; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; +import org.mockito.Mock; + +import java.util.Arrays; + +import static org.mockito.Mockito.*; +import static org.mockito.MockitoAnnotations.initMocks; + +public class GcsClientTest { + + @Mock + private Storage storage; + + @Mock + private Blob blob; + + @Before + public void setup() { + initMocks(this); + } + + @Test + public void shouldGetObjectFile() { + + String pythonFile = "gs://bucket_name/path/to/file/python_udf.zip"; + String bucketName = "bucket_name"; + String objectName = "path/to/file/python_udf.zip"; + String expectedValue = Arrays.toString("objectFile".getBytes()); + + when(storage.get(BlobId.of(bucketName, objectName))).thenReturn(blob); + when(blob.getContent()).thenReturn("objectFile".getBytes()); + + GcsClient gcsClient = new GcsClient(storage); + byte[] actualValue = gcsClient.getFile(pythonFile); + + verify(storage, times(1)).get(BlobId.of(bucketName, objectName)); + verify(blob, times(1)).getContent(); + Assert.assertEquals(expectedValue, Arrays.toString(actualValue)); + } +} diff --git a/dagger-functions/src/test/java/io/odpf/dagger/functions/udfs/python/file/source/gcs/GcsFileSourceTest.java b/dagger-functions/src/test/java/io/odpf/dagger/functions/udfs/python/file/source/gcs/GcsFileSourceTest.java new file mode 100644 index 000000000..d72d6a8f2 --- /dev/null +++ b/dagger-functions/src/test/java/io/odpf/dagger/functions/udfs/python/file/source/gcs/GcsFileSourceTest.java @@ -0,0 +1,38 @@ +package io.odpf.dagger.functions.udfs.python.file.source.gcs; + +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; +import org.mockito.Mock; + +import java.io.IOException; +import java.nio.file.Files; +import java.nio.file.Paths; + +import static org.mockito.Mockito.when; +import static org.mockito.MockitoAnnotations.initMocks; + +public class GcsFileSourceTest { + + @Mock + private GcsClient gcsClient; + + @Before + public void setup() { + initMocks(this); + } + + @Test + public void shouldGetObjectFile() throws IOException { + ClassLoader classLoader = getClass().getClassLoader(); + String pythonFile = classLoader.getResource("python_udf.zip").getFile(); + byte[] expectedObject = Files.readAllBytes(Paths.get(pythonFile)); + + when(gcsClient.getFile(pythonFile)).thenReturn(expectedObject); + GcsFileSource gcsFileSource = new GcsFileSource(pythonFile, gcsClient); + + byte[] actualObject = gcsFileSource.getObjectFile(); + + Assert.assertEquals(expectedObject, actualObject); + } +} diff --git a/dagger-functions/src/test/java/io/odpf/dagger/functions/udfs/python/file/source/local/LocalFileSourceTest.java b/dagger-functions/src/test/java/io/odpf/dagger/functions/udfs/python/file/source/local/LocalFileSourceTest.java new file mode 100644 index 000000000..4094e186a --- /dev/null +++ b/dagger-functions/src/test/java/io/odpf/dagger/functions/udfs/python/file/source/local/LocalFileSourceTest.java @@ -0,0 +1,28 @@ +package io.odpf.dagger.functions.udfs.python.file.source.local; + +import org.junit.Assert; +import org.junit.Test; + +import java.io.IOException; +import java.nio.charset.StandardCharsets; +import java.nio.file.Files; +import java.nio.file.Paths; + +public class LocalFileSourceTest { + + @Test + public void shouldGetObjectFile() throws IOException { + ClassLoader classLoader = getClass().getClassLoader(); + + String pythonFile = classLoader.getResource("python_udf.zip").getPath(); + + byte[] object = Files.readAllBytes(Paths.get(pythonFile)); + String stringObject = new String(object, StandardCharsets.UTF_8); + + LocalFileSource localFileSource = new LocalFileSource(pythonFile); + byte[] actualObject = localFileSource.getObjectFile(); + + String actualStringObject = new String(actualObject, StandardCharsets.UTF_8); + Assert.assertEquals(stringObject, actualStringObject); + } +} diff --git a/dagger-functions/src/test/java/io/odpf/dagger/functions/udfs/python/file/type/FileTypeFactoryTest.java b/dagger-functions/src/test/java/io/odpf/dagger/functions/udfs/python/file/type/FileTypeFactoryTest.java new file mode 100644 index 000000000..67f58cf63 --- /dev/null +++ b/dagger-functions/src/test/java/io/odpf/dagger/functions/udfs/python/file/type/FileTypeFactoryTest.java @@ -0,0 +1,41 @@ +package io.odpf.dagger.functions.udfs.python.file.type; + +import io.odpf.dagger.functions.exceptions.PythonFilesFormatException; +import org.junit.Assert; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.ExpectedException; + +public class FileTypeFactoryTest { + + @Rule + public ExpectedException expectedEx = ExpectedException.none(); + + @Test + public void shouldGetPythonFileType() { + String pythonFile = "/path/to/file/test_udf.py"; + + FileType fileType = FileTypeFactory.getFileType(pythonFile); + + Assert.assertTrue(fileType instanceof PythonFileType); + } + + @Test + public void shouldGetZipFileType() { + String pythonFile = "/path/to/file/python_udf.zip"; + + FileType fileType = FileTypeFactory.getFileType(pythonFile); + + Assert.assertTrue(fileType instanceof ZipFileType); + } + + @Test + public void shouldThrowExceptionIfPythonFilesNotInZipOrPyFormat() { + expectedEx.expect(PythonFilesFormatException.class); + expectedEx.expectMessage("Python files should be in .py or .zip format"); + + String pythonFile = "/path/to/file/test_file.txt"; + + FileTypeFactory.getFileType(pythonFile); + } +} diff --git a/dagger-functions/src/test/java/io/odpf/dagger/functions/udfs/python/file/type/PythonFileTypeTest.java b/dagger-functions/src/test/java/io/odpf/dagger/functions/udfs/python/file/type/PythonFileTypeTest.java new file mode 100644 index 000000000..7ff4e9c69 --- /dev/null +++ b/dagger-functions/src/test/java/io/odpf/dagger/functions/udfs/python/file/type/PythonFileTypeTest.java @@ -0,0 +1,45 @@ +package io.odpf.dagger.functions.udfs.python.file.type; + +import io.odpf.dagger.functions.exceptions.PythonFilesEmptyException; +import org.junit.Assert; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.ExpectedException; + +import java.util.List; + +public class PythonFileTypeTest { + + @Rule + public ExpectedException expectedEx = ExpectedException.none(); + + @Test + public void shouldGetFileNames() { + ClassLoader classLoader = getClass().getClassLoader(); + String pythonFile = classLoader.getResource("test_udf.py").getPath(); + + PythonFileType pythonFileType = new PythonFileType(pythonFile); + List fileNames = pythonFileType.getFileNames(); + + Assert.assertEquals("[test_udf.py]", fileNames.toString()); + } + + @Test + public void shouldGetEmptyFileNamesIfPythonFilesIsEmpty() { + String pythonFile = ""; + + PythonFileType pythonFileType = new PythonFileType(pythonFile); + List fileNames = pythonFileType.getFileNames(); + + Assert.assertEquals("[]", fileNames.toString()); + } + + @Test + public void shouldThrowNullPointerExceptionIfPythonFilesIsNull() { + expectedEx.expect(PythonFilesEmptyException.class); + expectedEx.expectMessage("Python files can not be null"); + + PythonFileType pythonFileType = new PythonFileType(null); + pythonFileType.getFileNames(); + } +} diff --git a/dagger-functions/src/test/java/io/odpf/dagger/functions/udfs/python/file/type/ZipFileTypeTest.java b/dagger-functions/src/test/java/io/odpf/dagger/functions/udfs/python/file/type/ZipFileTypeTest.java new file mode 100644 index 000000000..99acd37ae --- /dev/null +++ b/dagger-functions/src/test/java/io/odpf/dagger/functions/udfs/python/file/type/ZipFileTypeTest.java @@ -0,0 +1,72 @@ +package io.odpf.dagger.functions.udfs.python.file.type; + +import io.odpf.dagger.functions.udfs.python.file.source.gcs.GcsFileSource; +import io.odpf.dagger.functions.udfs.python.file.source.local.LocalFileSource; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; +import org.mockito.Mock; + +import java.io.File; +import java.io.IOException; +import java.nio.file.Files; +import java.util.*; + +import static org.mockito.Mockito.when; +import static org.mockito.MockitoAnnotations.initMocks; + +public class ZipFileTypeTest { + + @Mock + private GcsFileSource gcsFileSource; + + @Mock + private LocalFileSource localFileSource; + + private byte[] zipInBytes; + + @Before + public void setup() throws IOException { + initMocks(this); + ClassLoader classLoader = getClass().getClassLoader(); + File file = new File(Objects.requireNonNull(classLoader.getResource("python_udf.zip")).getFile()); + zipInBytes = Files.readAllBytes(file.toPath()); + } + + @Test + public void shouldGetFileNamesFromLocalZip() throws IOException { + + when(localFileSource.getObjectFile()).thenReturn(zipInBytes); + + ZipFileType zipFileType = new ZipFileType(localFileSource); + List fileNames = zipFileType.getFileNames(); + + Assert.assertEquals("[python_udf/scalar/add.py, python_udf/vectorized/substract.py]", fileNames.toString()); + } + + @Test + public void shouldGetFileNamesFromGcsZip() throws IOException { + + when(gcsFileSource.getObjectFile()).thenReturn(zipInBytes); + + ZipFileType zipFileType = new ZipFileType(gcsFileSource); + List fileNames = zipFileType.getFileNames(); + + Assert.assertEquals("[python_udf/scalar/add.py, python_udf/vectorized/substract.py]", fileNames.toString()); + } + + @Test + public void shouldGetEmptyFileNamesIfZipFileNotContainPyFile() throws IOException { + + ClassLoader classLoader = getClass().getClassLoader(); + File file = new File(Objects.requireNonNull(classLoader.getResource("test_no_py.zip")).getFile()); + zipInBytes = Files.readAllBytes(file.toPath()); + + when(gcsFileSource.getObjectFile()).thenReturn(zipInBytes); + + ZipFileType zipFileType = new ZipFileType(gcsFileSource); + List fileNames = zipFileType.getFileNames(); + + Assert.assertEquals("[]", fileNames.toString()); + } +} diff --git a/dagger-functions/src/test/resources/python_udf.zip b/dagger-functions/src/test/resources/python_udf.zip new file mode 100644 index 000000000..d55b8d1e2 Binary files /dev/null and b/dagger-functions/src/test/resources/python_udf.zip differ diff --git a/dagger-functions/src/test/resources/test_no_py.zip b/dagger-functions/src/test/resources/test_no_py.zip new file mode 100644 index 000000000..ffcfef40f Binary files /dev/null and b/dagger-functions/src/test/resources/test_no_py.zip differ diff --git a/dagger-functions/src/test/resources/test_udf.py b/dagger-functions/src/test/resources/test_udf.py new file mode 100644 index 000000000..39e64f8be --- /dev/null +++ b/dagger-functions/src/test/resources/test_udf.py @@ -0,0 +1,7 @@ +from pyflink.table import DataTypes +from pyflink.table.udf import udf + + +@udf(result_type=DataTypes.STRING()) +def test_udf(text: str): + return text + "_added_text" diff --git a/dagger-py-functions/data/sample_data.txt b/dagger-py-functions/data/sample_data.txt new file mode 100644 index 000000000..157f4b043 --- /dev/null +++ b/dagger-py-functions/data/sample_data.txt @@ -0,0 +1 @@ +sample_text \ No newline at end of file diff --git a/dagger-py-functions/requirements.txt b/dagger-py-functions/requirements.txt new file mode 100644 index 000000000..64e47665c --- /dev/null +++ b/dagger-py-functions/requirements.txt @@ -0,0 +1,2 @@ +pytest==7.1.2 +flake8==4.0.1 \ No newline at end of file diff --git a/dagger-py-functions/tests/__init__.py b/dagger-py-functions/tests/__init__.py new file mode 100644 index 000000000..e69de29bb diff --git a/dagger-py-functions/tests/udfs/__init__.py b/dagger-py-functions/tests/udfs/__init__.py new file mode 100644 index 000000000..e69de29bb diff --git a/dagger-py-functions/tests/udfs/scalar/__init__.py b/dagger-py-functions/tests/udfs/scalar/__init__.py new file mode 100644 index 000000000..e69de29bb diff --git a/dagger-py-functions/tests/udfs/scalar/multiply_test.py b/dagger-py-functions/tests/udfs/scalar/multiply_test.py new file mode 100644 index 000000000..12aced4ef --- /dev/null +++ b/dagger-py-functions/tests/udfs/scalar/multiply_test.py @@ -0,0 +1,6 @@ +from udfs.scalar.multiply import multiply + + +def testMultiply(): + value = multiply._func + assert value(5,10) == 50 diff --git a/dagger-py-functions/tests/udfs/scalar/sample_test.py b/dagger-py-functions/tests/udfs/scalar/sample_test.py new file mode 100644 index 000000000..e6c9dc7d9 --- /dev/null +++ b/dagger-py-functions/tests/udfs/scalar/sample_test.py @@ -0,0 +1,6 @@ +from udfs.scalar.sample import sample + + +def testSample(): + value = sample._func + assert value("input_text_") == "input_text_sample_text" diff --git a/dagger-py-functions/udfs/scalar/multiply.py b/dagger-py-functions/udfs/scalar/multiply.py new file mode 100644 index 000000000..4078293c5 --- /dev/null +++ b/dagger-py-functions/udfs/scalar/multiply.py @@ -0,0 +1,7 @@ +from pyflink.table import DataTypes +from pyflink.table.udf import udf + + +@udf(result_type=DataTypes.FLOAT()) +def multiply(i, j): + return i * j \ No newline at end of file diff --git a/dagger-py-functions/udfs/scalar/sample.py b/dagger-py-functions/udfs/scalar/sample.py new file mode 100644 index 000000000..99103d4bc --- /dev/null +++ b/dagger-py-functions/udfs/scalar/sample.py @@ -0,0 +1,9 @@ +from pyflink.table import DataTypes +from pyflink.table.udf import udf + + +@udf(result_type=DataTypes.STRING()) +def sample(text): + file = open("data/sample_data.txt", "r") + data = file.read() + return text + data diff --git a/docs/docs/advance/post_processor.md b/docs/docs/advance/post_processor.md index f8f02cb24..564da0a7a 100644 --- a/docs/docs/advance/post_processor.md +++ b/docs/docs/advance/post_processor.md @@ -247,6 +247,20 @@ List of comma-separated parameters to be replaced in request_pattern, these vari - Example value: `customer_id` - Type: `optional` +##### `header_pattern` + +Template for the dynamic headers. + +- Example value: `{"key": "%s"}` +- Type: `optional` + +##### `header_variables` + +List of comma-separated parameters to be replaced in header_pattern, these variables must be present in the input proto and selected via the SQL query. + +- Example value: `customer_id` +- Type: `optional` + ##### `stream_timeout` The timeout value for the stream in ms. @@ -325,6 +339,8 @@ PROCESSOR_POSTPROCESSOR_CONFIG = { "verb": "get", "request_pattern": "/customers/customer/%s", "request_variables": "customer_id", + "header_pattern": "{\"Header_Key\": \"%s\"}", + "header_variables": "wallet_id", "stream_timeout": "5000", "connect_timeout": "5000", "fail_on_errors": "false", diff --git a/docs/docs/contribute/add_transformer.md b/docs/docs/contribute/add_transformer.md index 9368f9588..8c7463d6e 100644 --- a/docs/docs/contribute/add_transformer.md +++ b/docs/docs/contribute/add_transformer.md @@ -12,7 +12,7 @@ For adding custom Transformers follow these steps - To define a new Transformer implement Transformer interface. The contract of Transformers is defined [here](https://github.com/odpf/dagger/blob/main/dagger-common/src/main/java/io/odpf/dagger/common/core/Transformer.java). -- Since an input DataStream is available in Transformer, all the Flink supported operators which transform `DataStream -> DataStream` can be applied/used by default for the transformations. Operators are how Flink exposes classic Map-reduce type functionalities. Read more about Flink Operators [here](https://ci.apache.org/projects/flink/flink-docs-release-1.9/dev/stream/operators/). +- Since an input DataStream is available in Transformer, all the Flink supported operators which transform `DataStream -> DataStream` can be applied/used by default for the transformations. Operators are how Flink exposes classic Map-reduce type functionalities. Read more about Flink Operators [here](https://ci.apache.org/projects/flink/flink-docs-release-1.14/dev/stream/operators/). - In the case of single Operator Transformation you can extend the desired Operator in the Transformer class itself. For example, follow this code of [HashTransformer](https://github.com/odpf/dagger/blob/main/dagger-functions/src/main/java/io/odpf/dagger/functions/transformers/HashTransformer.java). You can also define multiple chaining operators to Transform Data. diff --git a/docs/docs/contribute/add_udf.md b/docs/docs/contribute/add_udf.md index 419f9e93c..04f30da37 100644 --- a/docs/docs/contribute/add_udf.md +++ b/docs/docs/contribute/add_udf.md @@ -4,21 +4,29 @@ Want a function to use in SQL which is not supported both by Flink and one of th `Note`: _Please go through the [Contribution guide](../contribute/contribution.md) to know about all the conventions and practices we tend to follow and to know about the contribution process to the dagger._ -For adding custom UDFs follow these steps +For adding custom UDFs follow these steps: -- Ensure none of the [built-in functions](https://ci.apache.org/projects/flink/flink-docs-master/docs/dev/table/functions/systemfunctions) or [existing UDF](../reference/udfs.md) suits your requirement. -- For adding a UDF, figure out which type of UDF you required. Flink supports three types of [User defined function](https://ci.apache.org/projects/flink/flink-docs-release-1.9/dev/table/udfs.html). Choose one of them according to the requirement. +- Ensure none of the [built-in functions](https://nightlies.apache.org/flink/flink-docs-release-1.14/docs/dev/table/functions/systemfunctions/) or [existing UDF](../reference/udfs.md) suits your requirement. -- For getting more insights on writing your UDF, follow [this](https://ci.apache.org/projects/flink/flink-docs-release-1.9/dev/table/udfs.html) to create a UDF. It needs to be written in Java/Scala. +- For adding a UDF, figure out which type of UDF you required. Flink supports three types of [User defined function](https://nightlies.apache.org/flink/flink-docs-release-1.14/docs/dev/table/functions/udfs/). Choose one of them according to the requirement. -- UDF need to be the `function-type` directory inside [this](https://github.com/odpf/dagger/tree/main/dagger-functions/src/main/java/io/odpf/dagger/functions/udfs) on `dagger-functions` subproject. +- There are options for programming language you can choose for adding UDF, which is using Java, Scala and Python. -- Extend either of [ScalarUdf](https://github.com/odpf/dagger/blob/main/dagger-common/src/main/java/io/odpf/dagger/common/udfs/ScalarUdf.java), [TableUdf](https://github.com/odpf/dagger/blob/main/dagger-common/src/main/java/io/odpf/dagger/common/udfs/TableUdf.java) or [AggregateUdf](https://github.com/odpf/dagger/blob/main/dagger-common/src/main/java/io/odpf/dagger/common/udfs/AggregateUdf.java) from `dagger-common`. They are boilerplate contracts extending Flink UDF classes. These classes do some more preprocessing(like exposing some metrics) in the `open` method behind the scene. - -- Register the UDF in [this](https://github.com/odpf/dagger/blob/main/dagger-functions/src/main/java/io/odpf/dagger/functions/udfs/factories/FunctionFactory.java) class. This is required to let Flink know about your function. +- For adding UDF with Java/Scala: + - Follow [this](https://nightlies.apache.org/flink/flink-docs-release-1.14/docs/dev/table/functions/udfs/) for more insights on writing your UDF. + - UDF needs to be added in the `function-type` folder inside [this](https://github.com/odpf/dagger/tree/main/dagger-functions/src/main/java/io/odpf/dagger/functions/udfs) on `dagger-functions` subproject. + - Extend either of [ScalarUdf](https://github.com/odpf/dagger/blob/main/dagger-common/src/main/java/io/odpf/dagger/common/udfs/ScalarUdf.java), [TableUdf](https://github.com/odpf/dagger/blob/main/dagger-common/src/main/java/io/odpf/dagger/common/udfs/TableUdf.java) or [AggregateUdf](https://github.com/odpf/dagger/blob/main/dagger-common/src/main/java/io/odpf/dagger/common/udfs/AggregateUdf.java) from `dagger-common`. They are boilerplate contracts extending Flink UDF classes. These classes do some more preprocessing(like exposing some metrics) in the `open` method behind the scene. + - Register the UDF in [this](https://github.com/odpf/dagger/blob/main/dagger-functions/src/main/java/io/odpf/dagger/functions/udfs/factories/FunctionFactory.java) class. This is required to let Flink know about your function. + - If you have some business-specific use-cases and you don't want to add UDFs to the open-sourced repo, you can have a separate local codebase for those UDFs. Those UDFs need to be registered in a similar class like the [`UDFFactory`](https://github.com/odpf/dagger/blob/main/dagger-common/src/main/java/io/odpf/dagger/common/udfs/UdfFactory.java). Keep both the UDF classes and the factory class in the classpath of Dagger. Configure the fully qualified Factory class in the `FUNCTION_FACTORY_CLASSES` parameter and you will be able to use the desired UDF in your query. +- For adding UDF with Python: + - Follow [this](https://nightlies.apache.org/flink/flink-docs-release-1.14/docs/dev/python/table/udfs/overview/) for more insights on writing your UDF. + - UDF need to be added inside [this](https://github.com/odpf/dagger/tree/main/dagger-py-functions/udfs) on `dagger-py-functions` directory. + - Ensure that the filename and method name on the python functions is the same. This name will be registered by dagger as a function name which later can be used on the query. + - Ensure to add dependency needed for the python function on the [requirements.txt](https://github.com/odpf/dagger/tree/main/dagger-py-functions/requirements.txt) file. + - Add python unit test and the make sure the test is passed. + - If you have some business-specific use-cases and you don't want to add UDFs to the open-sourced repo, you can have a separate local codebase for those UDFs and specify that file on the python configuration. + - Bump up the version and raise a PR for the same. Also please add the registered function to the [list of udfs doc](../reference/udfs.md). -- If you have some business-specific use-cases and you don't want to add UDFs to the open-sourced repo, you can have a separate local codebase for those UDFs. Those UDFs need to be registered in a similar class like the [`UDFFactory`](https://github.com/odpf/dagger/blob/main/dagger-common/src/main/java/io/odpf/dagger/common/udfs/UdfFactory.java). Keep both the UDF classes and the factory class in the classpath of Dagger. Configure the fully qualified Factory class in the `FUNCTION_FACTORY_CLASSES` parameter and you will be able to use the desired UDF in your query. - In the subsequent release of the dagger, your functions should be useable in the query. diff --git a/docs/docs/guides/use_udf.md b/docs/docs/guides/use_udf.md index e8ac5a8e7..a8e049513 100644 --- a/docs/docs/guides/use_udf.md +++ b/docs/docs/guides/use_udf.md @@ -26,6 +26,24 @@ Some of the use-cases can not be solved using Flink SQL & the Apache Calcite fun Maps zero or more values to multiple rows and each row may have multiple columns. -All the supported udfs present in the `dagger-functions` subproject in [this](https://github.com/odpf/dagger/tree/main/dagger-functions/src/main/java/io/odpf/dagger/functions/udfs) directory. Follow this to find more details about the already supported UDFs in the dagger. +All the supported java udfs present in the `dagger-functions` subproject in [this](https://github.com/odpf/dagger/tree/main/dagger-functions/src/main/java/io/odpf/dagger/functions/udfs) directory. + +All the supported python udfs present in the [dagger-py-functions](https://github.com/odpf/dagger/tree/main/dagger-py-functions/udfs/) directory. + +Follow [this](../reference/udfs.md) to find more details about the already supported UDFs in the dagger. If any of the predefined functions do not meet your requirement you can create your custom UDFs by extending some implementation. Follow [this](../contribute/add_udf.md) to add your custom UDFs in the dagger. + +## Python Environment Setup + +Python UDF execution requires Python version (3.6, 3.7 or 3.8) with PyFlink installed. + +PyFlink is available in PyPi and can be installed as follows: +``` +$ python -m pip install apache-flink==1.14.3 +``` + +To satisfy the PyFlink requirement regarding the Python environment version, you need to soft link python to point to your python3 interpreter: +``` +ln -s /usr/bin/python3 python +``` diff --git a/docs/docs/reference/configuration.md b/docs/docs/reference/configuration.md index 6454723f2..acefca31a 100644 --- a/docs/docs/reference/configuration.md +++ b/docs/docs/reference/configuration.md @@ -14,6 +14,8 @@ This page contains references for all the application configurations for Dagger. * [PreProcessor](configuration.md#preprocessor) * [PostProcessor](configuration.md#postprocessor) * [Telemetry](configuration.md#telemetry) +* [Python Udfs](configuration.md#python-udfs) + ### Generic @@ -594,4 +596,96 @@ Shutdown period of metric telemetry in milliseconds. * Example value: `10000` * Type: `optional` -* Default value: `10000` \ No newline at end of file +* Default value: `10000` + +### Python Udfs + +#### `PYTHON_UDF_ENABLE` + +Enable/Disable using python udf. + +* Example value: `10000` +* Type: `optional` +* Default value: `10000` + +#### `PYTHON_UDF_CONFIG` + +All the configuration need to use python udf. + +These following variables need to be configured: + +##### `PYTHON_FILES` + +Defines the path of python udf files. Currently only support for `.py` and `.zip` data type. Comma (',') could be used as the separator to specify multiple files. + +* Example value: `/path/to/files.zip` +* Type: `required` + +##### `PYTHON_ARCHIVES` + +Defines the path of files that used on the python udf. Only support for `.zip` data type. Comma (',') could be used as the separator to specify multiple archive files. +The archive files will be extracted to the working directory of python UDF worker. For each archive file, a target directory is specified. If the target directory name is specified, the archive file will be extracted to a directory with the specified name. Otherwise, the archive file will be extracted to a directory with the same name of the archive file. '#' could be used as the separator of the archive file path and the target directory name. + +Example: +* PYTHON_ARCHIVES=/path/to/data.zip + + You should set the path name to `data.zip/data/sample.txt` on the udf to be able open the files. + +* PYTHON_ARCHIVES=/path/to/data.zip#data + + You should set the path name to `data/sample.txt` on the udf to be able open the files. + +* Example how to use this, can be found in this [udf](https://github.com/odpf/dagger/tree/main/dagger-py-functions/udfs/scalar/sample.py) + +* Type: `optional` +* Default value: `(none)` + +##### `PYTHON_REQUIREMENTS` + +Defines the path of python dependency files. + +* Example value: `/path/to/requirements.txt` +* Type: `optional` +* Default value: `(none)` + +##### `PYTHON_FN_EXECUTION_ARROW_BATCH_SIZE` + +The maximum number of elements to include in an arrow batch for python user-defined function execution. + +* Example value: `10000` +* Type: `optional` +* Default value: `10000` + +##### `PYTHON_FN_EXECUTION_BUNDLE_SIZE` + +The maximum number of elements to include in a bundle for python user-defined function execution. + +* Example value: `100000` +* Type: `optional` +* Default value: `100000` + +##### `PYTHON_FN_EXECUTION_BUNDLE_TIME` + +Sets the waiting timeout(in milliseconds) before processing a bundle for Python user-defined function execution. The timeout defines how long the elements of a bundle will be buffered before being processed. + +* Example value: `1000` +* Type: `optional` +* Default value: `1000` + +##### Sample Configuration +``` +PYTHON_UDF_CONFIG = [ + { + "PYTHON_FILES": "/path/to/files.py", + "PYTHON_ARCHIVES": "/path/to/data.zip", + "PYTHON_REQUIREMENTS": "/path/to/requirements.txt", + "PYTHON_FN_EXECUTION_ARROW_BATCH_SIZE": "10000", + "PYTHON_FN_EXECUTION_BUNDLE_SIZE": "100000", + "PYTHON_FN_EXECUTION_BUNDLE_TIME": "1000" + } +] +``` + +Find more details on python udf config [here](https://nightlies.apache.org/flink/flink-docs-release-1.14/docs/dev/python/python_config/#python-options). + + diff --git a/docs/docs/reference/udfs.md b/docs/docs/reference/udfs.md index 529cb2ee2..bc0e26b1d 100644 --- a/docs/docs/reference/udfs.md +++ b/docs/docs/reference/udfs.md @@ -1,8 +1,12 @@ # Udfs -This page contains references for all the custom udfs available on Dagger. +This page contains references for all the custom udfs available on Dagger. -## List of Udfs +The udfs on Dagger divided into two parts: +* [Java Udfs](udfs.md#list-of-java-udfs) +* [Python Udfs](udfs.md#list-of-python-udfs) + +## List of Java Udfs - [Scalar Functions](udfs.md#scalar-functions) - [ArrayAggregate](udfs.md#ArrayAggregate) @@ -795,3 +799,45 @@ LATERAL TABLE( `isOutlier` ) ``` + +## List of Python Udfs + +- [Scalar Functions](udfs.md#python-scalar-functions) + - [Sample](udfs.md#Sample) + - [Multiply](udfs.md#Multiply) +- [Aggregate Functions](udfs.md#python-aggregate-functions) +- [Table Functions](udfs.md#python-table-functions) + +### Python Scalar Functions + +#### Sample +* Contract: + * **String** `Sample(input_string)`. +* Functionality: + * This is one of sample python udfs. + * Adding extra string from [data/sample.txt](../../../dagger-py-functions/data/sample_data.txt) to the input_string. +* Example: + ``` + SELECT + sample(inputString) as input_with_additonal_text + FROM + data_stream + ``` + +#### Multiply +* Contract: + * **Float** `Multiply(input_number1, input_number2)`. +* Functionality: + * This is one of sample python udfs. + * Multiply two input numbers specified. +* Example: + ``` + SELECT + multiply(input_number1, input_number2) as multiply_result + FROM + data_stream + ``` + +### Python Aggregate Functions + +### Python Table Functions \ No newline at end of file diff --git a/docs/docs/rfcs/20220504_python_udf.md b/docs/docs/rfcs/20220504_python_udf.md new file mode 100644 index 000000000..7d2ff34c8 --- /dev/null +++ b/docs/docs/rfcs/20220504_python_udf.md @@ -0,0 +1,71 @@ +## Motivation +Dagger users include developers, analysts, data scientists, etc. For users to use Dagger, they can add new capabilities by defining their own functions commonly referred to as UDFs. Currently, Dagger only supports java as the language for the UDFs. To democratize the process of creating and maintaining the UDFs we want to add support for python. + +## Requirement +Support for adding Python UDF on Dagger +End-to-end flow on adding and using Python UDF on Dagger. + + +## Python User Defined Function +There are two kinds of Python UDF that can be registered on Dagger: +* General Python UDF +* Vectorized Python UDF + + +It shares a similar way as the general user-defined functions on how to define vectorized user-defined functions. Users only need to add an extra parameter func_type="pandas" in the decorator udf or udaf to mark it as a vectorized user-defined function. + + +Type | General Python UDF | Vectorized Python UDF +--- | --- | --- | +Data Processing Method | One piece of data is processed each time a UDF is called | Multiple pieces of data are processed each time a UDF is called +Serialization/Deserialization | Serialization and Deserialization are required for each piece of data on the Java side and Python side| The data transmission format between Java and Python is based on Apache Arrow:
  • Pandas supports Apache Arrow natively, so serialization and deserialization are not required on Python side
  • On the Java side, vectorized optimization is possible, and serialization/deserialization can be avoided as much as possible
| +|Exection Performance|Poor|Good
  • Vectorized execution is of high efficiency
  • High-performance python UDF can be implemented based on high performance libraries such as pandas and numpy
+ +Note: + +When using vectorized udf, Flink will convert the messages to pandas.series, and the udf will use that as an input and the output also pandas.series. The pandas.series size for input and output should be the same. + +## Configuration +There are a few configurations that required for using python UDF, and also options we can adjust for optimization. + +Configuration that will be added on Dagger codebase: +| Key | Default | Type | Example +| --- | --- | --- | ---- | +|PYTHON_UDF_ENABLE|false|Boolean|false| +|PYTHON_UDF_CONFIG|(none)|String|{"PYTHON_FILES":"/path/to/files.zip", "PYTHON_REQUIREMENTS": "requirements.txt", "PYTHON_FN_EXECUTION_BUNDLE_SIZE": "1000"}| + +The following variables than can be configurable on `PYTHON_UDF_CONFIG`: +| Key | Default | Type | Example +| --- | --- | --- | ---- | +|PYTHON_ARCHIVES|(none)|String|/path/to/data.zip| +|PYTHON_FILES|(none)|String|/path/to/files.zip| +|PYTHON_REQUIREMENTS|(none)|String|/path/to/requirements.txt| +|PYTHON_FN_EXECUTION_ARROW_BATCH_SIZE|10000|Integer|10000| +|PYTHON_FN_EXECUTION_BUNDLE_SIZE|100000|Integer|100000| +|PYTHON_FN_EXECUTION_BUNDLE_TIME|1000|Long|1000| + + +## Registering the Udf +Dagger will automatically register the python udf as long as the files meets the following criteria: +* Python file names should be the same with its function method + Example: + + sample.py + ``` + from pyflink.table import DataTypes + from pyflink.table.udf import udf + + + @udf(result_type=DataTypes.STRING()) + def sample(word: str): + return word + "_test" + ``` +* Avoid adding duplicate `.py` filenames. e.g: `__init__.py` + + +## Release the Udf +List of udfs for dagger, will be added on directory `dagger-py-functions` include with its test, data files that are used on the udf, and the udf dependency(requirements.txt). +All of these files will be bundled to single zip file and uploaded to assets on release. + +## Reference +[Flink Python User Defined Functions](https://nightlies.apache.org/flink/flink-docs-release-1.14/docs/dev/python/table/udfs/overview/) diff --git a/version.txt b/version.txt index 967b33ffb..9325c3ccd 100644 --- a/version.txt +++ b/version.txt @@ -1 +1 @@ -0.2.7 \ No newline at end of file +0.3.0 \ No newline at end of file